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/06/02 01:36:39 UTC

[01/50] [abbrv] tez git commit: TEZ-2450. support async http clients in ordered & unordered inputs (rbalamohan)

Repository: tez
Updated Branches:
  refs/heads/TEZ-2003 9b483ba01 -> 044e4fce8 (forced update)


http://git-wip-us.apache.org/repos/asf/tez/blob/9dabf947/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/FetcherOrderedGrouped.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/FetcherOrderedGrouped.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/FetcherOrderedGrouped.java
index 0248f13..a4d38ce 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/FetcherOrderedGrouped.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/FetcherOrderedGrouped.java
@@ -27,6 +27,8 @@ import java.util.LinkedList;
 import java.util.List;
 import java.util.concurrent.atomic.AtomicInteger;
 
+import org.apache.tez.http.BaseHttpConnection;
+import org.apache.tez.http.HttpConnectionParams;
 import org.apache.tez.common.CallableWithNdc;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -43,8 +45,6 @@ import org.apache.tez.runtime.library.common.shuffle.orderedgrouped.MapOutput.Ty
 import org.apache.tez.runtime.library.common.sort.impl.TezIndexRecord;
 import org.apache.tez.runtime.library.common.sort.impl.TezSpillRecord;
 import org.apache.tez.runtime.library.exceptions.FetcherReadTimeoutException;
-import org.apache.tez.runtime.library.common.shuffle.HttpConnection;
-import org.apache.tez.runtime.library.common.shuffle.HttpConnection.HttpConnectionParams;
 import org.apache.tez.runtime.library.common.shuffle.ShuffleUtils;
 
 import com.google.common.annotations.VisibleForTesting;
@@ -86,11 +86,13 @@ class FetcherOrderedGrouped extends CallableWithNdc<Void> {
 
   private final boolean ifileReadAhead;
   private final int ifileReadAheadLength;
-  private LinkedList<InputAttemptIdentifier> remaining;
+  @VisibleForTesting
+  LinkedList<InputAttemptIdentifier> remaining;
 
   volatile DataInputStream input;
 
-  volatile HttpConnection httpConnection;
+  volatile BaseHttpConnection httpConnection;
+  private final boolean asyncHttp;
 
 
   // Initiative value is 0, which means it hasn't retried yet.
@@ -114,7 +116,8 @@ class FetcherOrderedGrouped extends CallableWithNdc<Void> {
                                TezCounter badIdErrsCounter,
                                TezCounter wrongMapErrsCounter,
                                TezCounter connectionErrsCounter,
-                               TezCounter wrongReduceErrsCounter) {
+                               TezCounter wrongReduceErrsCounter,
+                               boolean asyncHttp) {
     this.scheduler = scheduler;
     this.allocator = allocator;
     this.metrics = metrics;
@@ -134,6 +137,7 @@ class FetcherOrderedGrouped extends CallableWithNdc<Void> {
     this.ifileReadAhead = ifileReadAhead;
     this.ifileReadAheadLength = ifileReadAheadLength;
     this.httpConnectionParams = httpConnectionParams;
+    this.asyncHttp = asyncHttp;
     if (codec != null) {
       this.codec = codec;
     } else {
@@ -311,8 +315,8 @@ class FetcherOrderedGrouped extends CallableWithNdc<Void> {
     boolean connectSucceeded = false;
     try {
       URL url = ShuffleUtils.constructInputURL(host.getBaseUrl(), attempts,
-          httpConnectionParams.getKeepAlive());
-      httpConnection = new HttpConnection(url, httpConnectionParams,
+          httpConnectionParams.isKeepAlive());
+      httpConnection = ShuffleUtils.getHttpConnection(asyncHttp, url, httpConnectionParams,
           logIdentifier, jobTokenSecretManager);
       connectSucceeded = httpConnection.connect();
 
@@ -323,7 +327,10 @@ class FetcherOrderedGrouped extends CallableWithNdc<Void> {
       input = httpConnection.getInputStream();
       httpConnection.validate();
       return true;
-    } catch (IOException ie) {
+    } catch (IOException | InterruptedException ie) {
+      if (ie instanceof InterruptedException) {
+        Thread.currentThread().interrupt(); //reset status
+      }
       if (stopped) {
         LOG.info("Not reporting fetch failure, since an Exception was caught after shutdown");
         return false;

http://git-wip-us.apache.org/repos/asf/tez/blob/9dabf947/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleScheduler.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleScheduler.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleScheduler.java
index 85c3a30..75dca64 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleScheduler.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleScheduler.java
@@ -54,10 +54,10 @@ import com.google.common.util.concurrent.MoreExecutors;
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
 import org.apache.hadoop.io.compress.CompressionCodec;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.tez.http.HttpConnectionParams;
 import org.apache.tez.common.CallableWithNdc;
 import org.apache.tez.common.security.JobTokenSecretManager;
 import org.apache.tez.dag.api.TezConstants;
-import org.apache.tez.runtime.library.common.shuffle.HttpConnection;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -144,7 +144,7 @@ class ShuffleScheduler {
 
   private final ListeningExecutorService fetcherExecutor;
 
-  private final HttpConnection.HttpConnectionParams httpConnectionParams;
+  private final HttpConnectionParams httpConnectionParams;
   private final FetchedInputAllocatorOrderedGrouped allocator;
   private final ShuffleClientMetrics shuffleMetrics;
   private final Shuffle shuffle;
@@ -157,6 +157,7 @@ class ShuffleScheduler {
   private final boolean localDiskFetchEnabled;
   private final String localHostname;
   private final int shufflePort;
+  private final boolean asyncHttp;
 
   private final TezCounter ioErrsCounter;
   private final TezCounter wrongLengthErrsCounter;
@@ -245,8 +246,8 @@ class ShuffleScheduler {
     this.startTime = startTime;
     this.lastProgressTime = startTime;
 
-    this.httpConnectionParams =
-        ShuffleUtils.constructHttpShuffleConnectionParams(conf);
+    this.asyncHttp = conf.getBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_USE_ASYNC_HTTP, false);
+    this.httpConnectionParams = ShuffleUtils.getHttpConnectionParams(conf);
     this.shuffleMetrics = new ShuffleClientMetrics(inputContext.getDAGName(),
         inputContext.getTaskVertexName(), inputContext.getTaskIndex(),
         this.conf, UserGroupInformation.getCurrentUser().getShortUserName());
@@ -1016,7 +1017,7 @@ class ShuffleScheduler {
         shuffleMetrics, shuffle, jobTokenSecretManager, ifileReadAhead, ifileReadAheadLength,
         codec, conf, localDiskFetchEnabled, localHostname, shufflePort, srcNameTrimmed, mapHost,
         ioErrsCounter, wrongLengthErrsCounter, badIdErrsCounter, wrongMapErrsCounter,
-        connectionErrsCounter, wrongReduceErrsCounter);
+        connectionErrsCounter, wrongReduceErrsCounter, asyncHttp);
   }
 
   private class FetchFutureCallback implements FutureCallback<Void> {

http://git-wip-us.apache.org/repos/asf/tez/blob/9dabf947/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/OrderedGroupedKVInput.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/OrderedGroupedKVInput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/OrderedGroupedKVInput.java
index 12a5955..7399359 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/OrderedGroupedKVInput.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/OrderedGroupedKVInput.java
@@ -330,6 +330,7 @@ public class OrderedGroupedKVInput extends AbstractLogicalInput {
     confKeys.add(TezRuntimeConfiguration.TEZ_RUNTIME_IO_SORT_FACTOR);
     confKeys.add(TezRuntimeConfiguration.TEZ_RUNTIME_COMBINE_MIN_SPILLS);
     confKeys.add(TezRuntimeConfiguration.TEZ_RUNTIME_COMBINER_CLASS);
+    confKeys.add(TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_USE_ASYNC_HTTP);
     confKeys.add(TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_PARALLEL_COPIES);
     confKeys.add(TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_FETCH_FAILURES_LIMIT);
     confKeys.add(TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_FETCH_MAX_TASK_OUTPUT_AT_ONCE);

http://git-wip-us.apache.org/repos/asf/tez/blob/9dabf947/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/UnorderedKVInput.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/UnorderedKVInput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/UnorderedKVInput.java
index 7fc9317..1016263 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/UnorderedKVInput.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/UnorderedKVInput.java
@@ -242,6 +242,7 @@ public class UnorderedKVInput extends AbstractLogicalInput {
     confKeys.add(TezRuntimeConfiguration.TEZ_RUNTIME_IFILE_READAHEAD_BYTES);
     confKeys.add(TezRuntimeConfiguration.TEZ_RUNTIME_IO_FILE_BUFFER_SIZE);
     confKeys.add(TezRuntimeConfiguration.TEZ_RUNTIME_IO_SORT_FACTOR);
+    confKeys.add(TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_USE_ASYNC_HTTP);
     confKeys.add(TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_PARALLEL_COPIES);
     confKeys.add(TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_FETCH_FAILURES_LIMIT);
     confKeys.add(TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_FETCH_MAX_TASK_OUTPUT_AT_ONCE);

http://git-wip-us.apache.org/repos/asf/tez/blob/9dabf947/tez-runtime-library/src/test/java/org/apache/tez/http/TestHttpConnection.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/test/java/org/apache/tez/http/TestHttpConnection.java b/tez-runtime-library/src/test/java/org/apache/tez/http/TestHttpConnection.java
new file mode 100644
index 0000000..8f0a7ad
--- /dev/null
+++ b/tez-runtime-library/src/test/java/org/apache/tez/http/TestHttpConnection.java
@@ -0,0 +1,202 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.http;
+
+import com.google.common.base.Throwables;
+import org.apache.tez.http.async.netty.AsyncHttpConnection;
+import org.apache.tez.common.security.JobTokenSecretManager;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+import java.io.IOException;
+import java.net.URISyntaxException;
+import java.net.URL;
+import java.nio.channels.ClosedByInterruptException;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.ThreadFactory;
+
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.when;
+
+public class TestHttpConnection {
+
+  private static int connTimeout = 5000;
+  private static int readTimeout = 5000;
+
+  private static final String NOT_HOSTED_URL = "http://10.255.255.255:10221";
+
+  private static ExecutorService executorService;
+  private static URL url;
+  private static JobTokenSecretManager tokenSecretManager;
+
+  private Thread currentThread;
+
+  @BeforeClass
+  public static void setup() throws IOException, URISyntaxException {
+    executorService = Executors.newFixedThreadPool(1,
+        new ThreadFactory() {
+          public Thread newThread(Runnable r) {
+            Thread t = Executors.defaultThreadFactory().newThread(r);
+            t.setDaemon(true);
+            return t;
+          }
+        });
+    url = new URL(NOT_HOSTED_URL);
+    tokenSecretManager = mock(JobTokenSecretManager.class);
+  }
+
+  @AfterClass
+  public static void cleanup() throws Exception {
+    executorService.shutdownNow();
+  }
+
+  public void baseTest(Callable<Void> worker, CountDownLatch latch, String message) throws
+      InterruptedException {
+    long startTime = System.currentTimeMillis();
+    try {
+      Future future = executorService.submit(worker);
+      future.get();
+    } catch (ExecutionException e) {
+      assertTrue(e.getCause().getCause() instanceof IOException);
+      assertTrue(e.getMessage(), e.getMessage().contains(message));
+      long elapsedTime = System.currentTimeMillis() - startTime;
+      assertTrue("elapasedTime=" + elapsedTime + " should be greater than " + connTimeout,
+          elapsedTime > connTimeout);
+    }
+    assertTrue(latch.getCount() == 0);
+  }
+
+  @Test(timeout = 20000)
+  public void testConnectionTimeout() throws IOException, InterruptedException {
+    HttpConnectionParams params = getConnectionParams();
+
+    //For http
+    CountDownLatch latch = new CountDownLatch(1);
+    HttpConnection httpConn = getHttpConnection(params);
+    baseTest(new Worker(latch, httpConn, false), latch, "Failed to connect");
+
+    //For async http
+    latch = new CountDownLatch(1);
+    AsyncHttpConnection asyncHttpConn = getAsyncHttpConnection(params);
+    baseTest(new Worker(latch, asyncHttpConn, false), latch, "connection timed out");
+  }
+
+  @Test(timeout = 20000)
+  @SuppressWarnings("unchecked")
+  //Should be interruptible
+  public void testAsyncHttpConnectionInterrupt()
+      throws IOException, InterruptedException, ExecutionException {
+    CountDownLatch latch = new CountDownLatch(1);
+    HttpConnectionParams params = getConnectionParams();
+    AsyncHttpConnection asyncHttpConn = getAsyncHttpConnection(params);
+    Future future = executorService.submit(new Worker(latch, asyncHttpConn, true));
+
+    while(currentThread == null) {
+      synchronized (this) {
+        wait(100);
+      }
+    }
+
+    assertTrue("currentThread is still null", currentThread != null);
+
+    //Try interrupting the thread (exception verification happens in the worker itself)
+    currentThread.interrupt();
+
+    future.get();
+    assertTrue(latch.getCount() == 0);
+  }
+
+  HttpConnectionParams getConnectionParams() {
+    HttpConnectionParams params = mock(HttpConnectionParams.class);
+    when(params.getBufferSize()).thenReturn(8192);
+    when(params.getKeepAliveMaxConnections()).thenReturn(1);
+    when(params.getConnectionTimeout()).thenReturn(connTimeout);
+    when(params.getReadTimeout()).thenReturn(readTimeout);
+    return params;
+  }
+
+  HttpConnection getHttpConnection(HttpConnectionParams params) throws IOException {
+    HttpConnection realConn = new HttpConnection(url, params, "log", tokenSecretManager);
+    HttpConnection connection = spy(realConn);
+
+    doAnswer(new Answer() {
+      public Void answer(InvocationOnMock invocation) {
+        return null;
+      }
+    }).when(connection).computeEncHash();
+    return connection;
+  }
+
+  AsyncHttpConnection getAsyncHttpConnection(HttpConnectionParams params) throws IOException {
+    AsyncHttpConnection realConn = new AsyncHttpConnection(url, params, "log", tokenSecretManager);
+    AsyncHttpConnection connection = spy(realConn);
+
+    doAnswer(new Answer() {
+      public Void answer(InvocationOnMock invocation) {
+        return null;
+      }
+    }).when(connection).computeEncHash();
+    return connection;
+  }
+
+  class Worker implements Callable<Void> {
+    private CountDownLatch latch;
+    private BaseHttpConnection connection;
+    private boolean expectingInterrupt;
+
+    public Worker(CountDownLatch latch, BaseHttpConnection connection, boolean expectingInterrupt) {
+      this.latch = latch;
+      this.connection = connection;
+      this.expectingInterrupt = expectingInterrupt;
+    }
+
+    @Override
+    public Void call() throws Exception {
+      try {
+        currentThread = Thread.currentThread();
+        connection.connect();
+        fail();
+      } catch(Throwable t) {
+        if (expectingInterrupt) {
+          //ClosedByInterruptException normally; InterruptedException if
+          // TezBodyDeferringAsyncHandler quits otherwise
+          assertTrue((t instanceof InterruptedException) || (t instanceof ClosedByInterruptException));
+        }
+      } finally {
+        latch.countDown();
+        if (connection != null) {
+          connection.cleanup(true);
+        }
+      }
+      return null;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/9dabf947/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 4ef187d..34c2ca7 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
@@ -70,7 +70,8 @@ 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, PORT);
+        ApplicationId.newInstance(0, 1), null, "fetcherTest", conf, ENABLE_LOCAL_FETCH, HOST,
+        PORT, false);
     builder.assignWork(HOST, PORT, 0, Arrays.asList(srcAttempts));
     Fetcher fetcher = spy(builder.build());
 
@@ -88,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,
-        PORT);
+        PORT, false);
     builder.assignWork(HOST + "_OTHER", PORT, 0, Arrays.asList(srcAttempts));
     fetcher = spy(builder.build());
 
@@ -103,7 +104,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);
+        ApplicationId.newInstance(0, 1), null, "fetcherTest", conf, ENABLE_LOCAL_FETCH, HOST, PORT, false);
     builder.assignWork(HOST, PORT + 1, 0, Arrays.asList(srcAttempts));
     fetcher = spy(builder.build());
 
@@ -119,7 +120,8 @@ 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, PORT);
+        ApplicationId.newInstance(0, 1), null, "fetcherTest", conf, DISABLE_LOCAL_FETCH, HOST,
+        PORT, false);
     builder.assignWork(HOST, PORT, 0, Arrays.asList(srcAttempts));
     fetcher = spy(builder.build());
 
@@ -152,7 +154,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);
+        ApplicationId.newInstance(0, 1), null, "fetcherTest", conf, true, HOST, PORT, false);
     builder.assignWork(HOST, PORT, partition, Arrays.asList(srcAttempts));
     Fetcher fetcher = spy(builder.build());
 

http://git-wip-us.apache.org/repos/asf/tez/blob/9dabf947/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestFetcher.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestFetcher.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestFetcher.java
index f77e9a6..385b7b0 100644
--- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestFetcher.java
+++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestFetcher.java
@@ -18,6 +18,7 @@
 
 package org.apache.tez.runtime.library.common.shuffle.orderedgrouped;
 
+import static org.junit.Assert.fail;
 import static org.mockito.Matchers.any;
 import static org.mockito.Matchers.anyBoolean;
 import static org.mockito.Matchers.anyInt;
@@ -44,6 +45,8 @@ import java.util.Arrays;
 import java.util.List;
 
 import com.google.common.collect.Lists;
+import org.apache.tez.http.HttpConnection;
+import org.apache.tez.http.HttpConnectionParams;
 import org.apache.tez.common.counters.TezCounter;
 import org.apache.tez.runtime.library.common.InputIdentifier;
 import org.slf4j.Logger;
@@ -59,7 +62,6 @@ import org.apache.tez.runtime.library.common.InputAttemptIdentifier;
 import org.apache.tez.runtime.library.common.security.SecureShuffleUtils;
 import org.apache.tez.runtime.library.common.sort.impl.TezIndexRecord;
 import org.apache.tez.runtime.library.exceptions.FetcherReadTimeoutException;
-import org.apache.tez.runtime.library.common.shuffle.HttpConnection;
 import org.apache.tez.runtime.library.common.shuffle.ShuffleUtils;
 import org.junit.Assert;
 import org.junit.Test;
@@ -118,7 +120,7 @@ public class TestFetcher {
         new FetcherOrderedGrouped(null, scheduler, merger, metrics, shuffle, null, false, 0,
             null, conf, false, HOST, PORT, "src vertex", mapHost, ioErrsCounter,
             wrongLengthErrsCounter, badIdErrsCounter,
-            wrongMapErrsCounter, connectionErrsCounter, wrongReduceErrsCounter);
+            wrongMapErrsCounter, connectionErrsCounter, wrongReduceErrsCounter, false);
 
     fetcher.call();
     verify(scheduler).getMapsForHost(mapHost);
@@ -146,7 +148,7 @@ public class TestFetcher {
         new FetcherOrderedGrouped(null, scheduler, merger, metrics, shuffle, null, false, 0,
             null, conf, ENABLE_LOCAL_FETCH, HOST, PORT, "src vertex", mapHost, ioErrsCounter,
             wrongLengthErrsCounter, badIdErrsCounter,
-            wrongMapErrsCounter, connectionErrsCounter, wrongReduceErrsCounter);
+            wrongMapErrsCounter, connectionErrsCounter, wrongReduceErrsCounter, false);
 
     // when local mode is enabled and host and port matches use local fetch
     FetcherOrderedGrouped spyFetcher = spy(fetcher);
@@ -163,7 +165,7 @@ public class TestFetcher {
         new FetcherOrderedGrouped(null, scheduler, merger, metrics, shuffle, null, false, 0,
             null, conf, ENABLE_LOCAL_FETCH, HOST, PORT, "src vertex", mapHost, ioErrsCounter,
             wrongLengthErrsCounter, badIdErrsCounter,
-            wrongMapErrsCounter, connectionErrsCounter, wrongReduceErrsCounter);
+            wrongMapErrsCounter, connectionErrsCounter, wrongReduceErrsCounter, false);
     spyFetcher = spy(fetcher);
     doNothing().when(spyFetcher).setupLocalDiskFetch(mapHost);
 
@@ -178,7 +180,7 @@ public class TestFetcher {
         new FetcherOrderedGrouped(null, scheduler, merger, metrics, shuffle, null, false, 0,
             null, conf, ENABLE_LOCAL_FETCH, HOST, PORT, "src vertex", mapHost, ioErrsCounter,
             wrongLengthErrsCounter, badIdErrsCounter,
-            wrongMapErrsCounter, connectionErrsCounter, wrongReduceErrsCounter);
+            wrongMapErrsCounter, connectionErrsCounter, wrongReduceErrsCounter, false);
     spyFetcher = spy(fetcher);
     doNothing().when(spyFetcher).setupLocalDiskFetch(mapHost);
 
@@ -192,7 +194,7 @@ public class TestFetcher {
     fetcher = new FetcherOrderedGrouped(null, scheduler, merger, metrics, shuffle, null, false, 0,
         null, conf, DISABLE_LOCAL_FETCH, HOST, PORT, "src vertex", mapHost, ioErrsCounter,
         wrongLengthErrsCounter, badIdErrsCounter,
-        wrongMapErrsCounter, connectionErrsCounter, wrongReduceErrsCounter);
+        wrongMapErrsCounter, connectionErrsCounter, wrongReduceErrsCounter, false);
     spyFetcher = spy(fetcher);
     doNothing().when(spyFetcher).setupLocalDiskFetch(mapHost);
 
@@ -217,7 +219,7 @@ public class TestFetcher {
         "http://" + HOST + ":" + PORT + "/mapOutput?job=job_123&&reduce=1&map=");
     FetcherOrderedGrouped fetcher = new FetcherOrderedGrouped(null, scheduler, merger, metrics, shuffle, null, false, 0,
         null, conf, true, HOST, PORT, "src vertex", host, ioErrsCounter, wrongLengthErrsCounter, badIdErrsCounter,
-        wrongMapErrsCounter, connectionErrsCounter, wrongReduceErrsCounter);
+        wrongMapErrsCounter, connectionErrsCounter, wrongReduceErrsCounter, false);
     FetcherOrderedGrouped spyFetcher = spy(fetcher);
 
 
@@ -342,6 +344,7 @@ public class TestFetcher {
   }
 
   @Test(timeout = 5000)
+  @SuppressWarnings("unchecked")
   public void testWithRetry() throws Exception {
     Configuration conf = new TezConfiguration();
     conf.setInt(TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_READ_TIMEOUT, 3000);
@@ -355,13 +358,12 @@ public class TestFetcher {
     when(inputContext.getCounters()).thenReturn(new TezCounters());
     when(inputContext.getSourceVertexName()).thenReturn("");
 
-    HttpConnection.HttpConnectionParams httpConnectionParams =
-        ShuffleUtils.constructHttpShuffleConnectionParams(conf);
+    HttpConnectionParams httpConnectionParams = ShuffleUtils.getHttpConnectionParams(conf);
     final MapHost host = new MapHost(1, HOST + ":" + PORT,
         "http://" + HOST + ":" + PORT + "/mapOutput?job=job_123&&reduce=1&map=");
     FetcherOrderedGrouped mockFetcher = new FetcherOrderedGrouped(null, scheduler, merger, metrics, shuffle, null, false, 0,
         null, conf, false, HOST, PORT, "src vertex", host, ioErrsCounter, wrongLengthErrsCounter, badIdErrsCounter,
-        wrongMapErrsCounter, connectionErrsCounter, wrongReduceErrsCounter);
+        wrongMapErrsCounter, connectionErrsCounter, wrongReduceErrsCounter, false);
     final FetcherOrderedGrouped fetcher = spy(mockFetcher);
 
 
@@ -426,4 +428,53 @@ public class TestFetcher {
 
   }
 
+  @Test
+  @SuppressWarnings("unchecked")
+  public void testAsyncWithException() throws Exception {
+    Configuration conf = new TezConfiguration();
+    conf.setInt(TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_READ_TIMEOUT, 3000);
+    conf.setInt(TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_CONNECT_TIMEOUT, 3000);
+
+    ShuffleScheduler scheduler = mock(ShuffleScheduler.class);
+    MergeManager merger = mock(MergeManager.class);
+    ShuffleClientMetrics metrics = mock(ShuffleClientMetrics.class);
+    Shuffle shuffle = mock(Shuffle.class);
+
+    TezCounters counters = new TezCounters();
+    InputContext inputContext = mock(InputContext.class);
+    when(inputContext.getCounters()).thenReturn(counters);
+    when(inputContext.getSourceVertexName()).thenReturn("");
+
+    JobTokenSecretManager jobMgr = mock(JobTokenSecretManager.class);
+    doReturn(new byte[10]).when(jobMgr).computeHash(any(byte[].class));
+
+    HttpConnectionParams httpConnectionParams = ShuffleUtils.getHttpConnectionParams(conf);
+    final MapHost host = new MapHost(1, HOST + ":" + PORT,
+        "http://" + HOST + ":" + PORT + "/mapOutput?job=job_123&&reduce=1&map=");
+    FetcherOrderedGrouped mockFetcher =
+        new FetcherOrderedGrouped(httpConnectionParams, scheduler, merger, metrics, shuffle, jobMgr,
+            false, 0,
+            null, conf, false, HOST, PORT, "src vertex", host, ioErrsCounter,
+            wrongLengthErrsCounter, badIdErrsCounter,
+            wrongMapErrsCounter, connectionErrsCounter, wrongReduceErrsCounter, true);
+    final FetcherOrderedGrouped fetcher = spy(mockFetcher);
+    fetcher.remaining = Lists.newLinkedList();
+
+    final List<InputAttemptIdentifier> srcAttempts = Arrays.asList(
+        new InputAttemptIdentifier(0, 1, InputAttemptIdentifier.PATH_PREFIX + "pathComponent_0"),
+        new InputAttemptIdentifier(1, 2, InputAttemptIdentifier.PATH_PREFIX + "pathComponent_1"),
+        new InputAttemptIdentifier(3, 4, InputAttemptIdentifier.PATH_PREFIX + "pathComponent_3")
+    );
+    doReturn(srcAttempts).when(scheduler).getMapsForHost(host);
+
+    try {
+      long currentIOErrors = ioErrsCounter.getValue();
+      boolean connected = fetcher.setupConnection(host, srcAttempts);
+      Assert.assertTrue(connected == false);
+      //Ensure that counters are incremented (i.e it followed the exception codepath)
+      Assert.assertTrue(ioErrsCounter.getValue() > currentIOErrors);
+    } catch (IOException e) {
+      fail();
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/9dabf947/tez-tests/src/test/java/org/apache/tez/test/TestPipelinedShuffle.java
----------------------------------------------------------------------
diff --git a/tez-tests/src/test/java/org/apache/tez/test/TestPipelinedShuffle.java b/tez-tests/src/test/java/org/apache/tez/test/TestPipelinedShuffle.java
index 2a63293..25c149d 100644
--- a/tez-tests/src/test/java/org/apache/tez/test/TestPipelinedShuffle.java
+++ b/tez-tests/src/test/java/org/apache/tez/test/TestPipelinedShuffle.java
@@ -94,6 +94,7 @@ public class TestPipelinedShuffle {
     }
   }
 
+  //TODO: Add support for async http clients
   @Before
   public void setupTezCluster() throws Exception {
     //With 1 MB sort buffer and with good amount of dataset, it would spill records
@@ -126,8 +127,18 @@ public class TestPipelinedShuffle {
 
   @Test
   public void baseTest() throws Exception {
+    Configuration conf = new Configuration(miniTezCluster.getConfig());
+    conf.setBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_USE_ASYNC_HTTP, false);
+    test(conf);
+
+    conf = new Configuration(miniTezCluster.getConfig());
+    conf.setBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_USE_ASYNC_HTTP, true);
+    test(conf);
+  }
+
+  private void test(Configuration conf) throws Exception {
     PipelinedShuffleJob pipelinedShuffle = new PipelinedShuffleJob();
-    pipelinedShuffle.setConf(new Configuration(miniTezCluster.getConfig()));
+    pipelinedShuffle.setConf(conf);
 
     String[] args = new String[] { };
     assertEquals(0, pipelinedShuffle.run(args));

http://git-wip-us.apache.org/repos/asf/tez/blob/9dabf947/tez-tests/src/test/java/org/apache/tez/test/TestSecureShuffle.java
----------------------------------------------------------------------
diff --git a/tez-tests/src/test/java/org/apache/tez/test/TestSecureShuffle.java b/tez-tests/src/test/java/org/apache/tez/test/TestSecureShuffle.java
index 8da3f08..e3e42d3 100644
--- a/tez-tests/src/test/java/org/apache/tez/test/TestSecureShuffle.java
+++ b/tez-tests/src/test/java/org/apache/tez/test/TestSecureShuffle.java
@@ -36,6 +36,7 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.server.namenode.EditLogFileOutputStream;
 import org.apache.hadoop.mapreduce.MRConfig;
 import org.apache.hadoop.security.ssl.KeyStoreTestUtil;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.tez.mapreduce.examples.TestOrderedWordCount;
 import org.apache.tez.runtime.library.api.TezRuntimeConfiguration;
 import org.junit.After;
@@ -65,22 +66,30 @@ public class TestSecureShuffle {
   private boolean enableSSLInCluster; //To set ssl config in cluster
   private int resultWithTezSSL; //expected result with tez ssl setting
   private int resultWithoutTezSSL; //expected result without tez ssl setting
+  private boolean asyncHttp;
 
-  public TestSecureShuffle(boolean sslInCluster, int resultWithTezSSL, int resultWithoutTezSSL) {
+  public TestSecureShuffle(boolean sslInCluster, int resultWithTezSSL, int resultWithoutTezSSL,
+      boolean asyncHttp) {
     this.enableSSLInCluster = sslInCluster;
     this.resultWithTezSSL = resultWithTezSSL;
     this.resultWithoutTezSSL = resultWithoutTezSSL;
+    this.asyncHttp = asyncHttp;
   }
 
-  @Parameterized.Parameters(name = "test[sslInCluster:{0}, resultWithTezSSL:{1}, resultWithoutTezSSL:{2}]")
+  @Parameterized.Parameters(name = "test[sslInCluster:{0}, resultWithTezSSL:{1}, "
+      + "resultWithoutTezSSL:{2}, asyncHttp:{3}]")
   public static Collection<Object[]> getParameters() {
     Collection<Object[]> parameters = new ArrayList<Object[]>();
     //enable ssl in cluster, succeed with tez-ssl enabled, fail with tez-ssl disabled
-    parameters.add(new Object[] { true, 0, 1 });
+    parameters.add(new Object[] { true, 0, 1, false });
+
+    //With asyncHttp
+    parameters.add(new Object[] { true, 0, 1, true });
+    parameters.add(new Object[] { false, 1, 0, true });
 
     //Negative testcase
-    // disable ssl in cluster, fail with tez-ssl enabled, succeed with tez-ssl disabled
-    parameters.add(new Object[] { false, 1, 0 });
+    //disable ssl in cluster, fail with tez-ssl enabled, succeed with tez-ssl disabled
+    parameters.add(new Object[] { false, 1, 0, false });
 
     return parameters;
   }
@@ -151,7 +160,7 @@ public class TestSecureShuffle {
    *
    * @throws Exception
    */
-  @Test(timeout = 240000)
+  @Test(timeout = 500000)
   public void testSecureShuffle() throws Exception {
     //With tez-ssl setting
     miniTezCluster.getConfig().setBoolean(


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

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


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

Branch: refs/heads/TEZ-2003
Commit: aa6c53ff2085c303840f2f08b0148cc16effc112
Parents: 3b36e0d
Author: Siddharth Seth <ss...@apache.org>
Authored: Fri Feb 13 17:24:05 2015 -0800
Committer: Siddharth Seth <ss...@apache.org>
Committed: Mon Jun 1 14:09:26 2015 -0700

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


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

http://git-wip-us.apache.org/repos/asf/tez/blob/aa6c53ff/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index cafba97..4f1cc8b 100644
--- a/pom.xml
+++ b/pom.xml
@@ -172,6 +172,11 @@
         <type>test-jar</type>
       </dependency>
       <dependency>
+        <groupId>org.apache.tez</groupId>
+        <artifactId>tez-ext-service-tests</artifactId>
+        <version>${project.version}</version>
+      </dependency>
+      <dependency>
         <groupId>org.apache.pig</groupId>
         <artifactId>pig</artifactId>
         <version>${pig.version}</version>
@@ -662,6 +667,7 @@
     <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/aa6c53ff/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 828405e..708190e 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
@@ -1179,6 +1179,8 @@ public class TezConfiguration extends Configuration {
   public static final String TEZ_AM_CONTAINER_LAUNCHER_CLASS = TEZ_AM_PREFIX + "container-launcher.class";
   @ConfigurationScope(Scope.VERTEX)
   public static final String TEZ_AM_TASK_SCHEDULER_CLASS = TEZ_AM_PREFIX + "task-scheduler.class";
+  @ConfigurationScope(Scope.VERTEX)
+  public static final String TEZ_AM_TASK_COMMUNICATOR_CLASS = TEZ_AM_PREFIX + "task-communicator.class";
 
 
   // TODO only validate property here, value can also be validated if necessary

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

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

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

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

http://git-wip-us.apache.org/repos/asf/tez/blob/aa6c53ff/tez-ext-service-tests/pom.xml
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/pom.xml b/tez-ext-service-tests/pom.xml
new file mode 100644
index 0000000..37f68b1
--- /dev/null
+++ b/tez-ext-service-tests/pom.xml
@@ -0,0 +1,161 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  ~ Licensed under the Apache License, Version 2.0 (the "License");
+  ~ you may not use this file except in compliance with the License.
+  ~ You may obtain a copy of the License at
+  ~
+  ~     http://www.apache.org/licenses/LICENSE-2.0
+  ~
+  ~ Unless required by applicable law or agreed to in writing, software
+  ~ distributed under the License is distributed on an "AS IS" BASIS,
+  ~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  ~ See the License for the specific language governing permissions and
+  ~ limitations under the License.
+  -->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <artifactId>tez</artifactId>
+    <groupId>org.apache.tez</groupId>
+    <version>0.7.0-SNAPSHOT</version>
+  </parent>
+
+  <!-- TODO TEZ-2003 Merge this into the tez-tests module -->
+  <artifactId>tez-ext-service-tests</artifactId>
+
+  <dependencies>
+    <dependency>
+      <groupId>log4j</groupId>
+      <artifactId>log4j</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>com.google.guava</groupId>
+      <artifactId>guava</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-common</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.mockito</groupId>
+      <artifactId>mockito-all</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.tez</groupId>
+      <artifactId>tez-runtime-internals</artifactId>
+    </dependency>
+    <dependency>
+      <!-- Required for the ShuffleHandler -->
+      <groupId>org.apache.tez</groupId>
+      <artifactId>tez-runtime-library</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.tez</groupId>
+      <artifactId>tez-dag</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.tez</groupId>
+      <artifactId>tez-tests</artifactId>
+      <scope>test</scope>
+      <type>test-jar</type>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-hdfs</artifactId>
+      <scope>test</scope>
+      <type>test-jar</type>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-yarn-server-tests</artifactId>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-common</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-common</artifactId>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-mapreduce-client-jobclient</artifactId>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+
+  </dependencies>
+
+  <build>
+    <!--
+     Include all files in src/main/resources.  By default, do not apply property
+     substitution (filtering=false), but do apply property substitution to
+     version-info.properties (filtering=true).  This will substitute the
+     version information correctly, but prevent Maven from altering other files.
+     -->
+    <resources>
+      <resource>
+        <directory>${basedir}/src/main/resources</directory>
+        <excludes>
+          <exclude>tez-api-version-info.properties</exclude>
+        </excludes>
+        <filtering>false</filtering>
+      </resource>
+      <resource>
+        <directory>${basedir}/src/main/resources</directory>
+        <includes>
+          <include>tez-api-version-info.properties</include>
+        </includes>
+        <filtering>true</filtering>
+      </resource>
+    </resources>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.rat</groupId>
+        <artifactId>apache-rat-plugin</artifactId>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.hadoop</groupId>
+        <artifactId>hadoop-maven-plugins</artifactId>
+        <executions>
+          <execution>
+            <id>compile-protoc</id>
+            <phase>generate-sources</phase>
+            <goals>
+              <goal>protoc</goal>
+            </goals>
+            <configuration>
+              <protocVersion>${protobuf.version}</protocVersion>
+              <protocCommand>${protoc.path}</protocCommand>
+              <imports>
+                <param>${basedir}/src/test/proto</param>
+                <param>${basedir}/../tez-api/src/main/proto</param>
+              </imports>
+              <source>
+                <directory>${basedir}/src/test/proto</directory>
+                <includes>
+                  <include>TezDaemonProtocol.proto</include>
+                </includes>
+              </source>
+              <output>${project.build.directory}/generated-test-sources/java</output>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+    </plugins>
+  </build>
+
+</project>

http://git-wip-us.apache.org/repos/asf/tez/blob/aa6c53ff/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/TezTestServiceCommunicator.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/TezTestServiceCommunicator.java b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/TezTestServiceCommunicator.java
new file mode 100644
index 0000000..ac50878
--- /dev/null
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/TezTestServiceCommunicator.java
@@ -0,0 +1,152 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.dag.app;
+
+
+import java.util.concurrent.Callable;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+import com.google.common.util.concurrent.FutureCallback;
+import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.util.concurrent.ListeningExecutorService;
+import com.google.common.util.concurrent.MoreExecutors;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import com.google.protobuf.Message;
+import org.apache.hadoop.service.AbstractService;
+import org.apache.tez.service.TezTestServiceProtocolBlockingPB;
+import org.apache.tez.service.impl.TezTestServiceProtocolClientImpl;
+import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.RunContainerRequestProto;
+import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.RunContainerResponseProto;
+import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.SubmitWorkRequestProto;
+import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.SubmitWorkResponseProto;
+
+public class TezTestServiceCommunicator extends AbstractService {
+
+  private final ConcurrentMap<String, TezTestServiceProtocolBlockingPB> hostProxies;
+  private final ListeningExecutorService executor;
+
+  // TODO Convert this into a singleton
+  public TezTestServiceCommunicator(int numThreads) {
+    super(TezTestServiceCommunicator.class.getSimpleName());
+    ExecutorService localExecutor = Executors.newFixedThreadPool(numThreads,
+        new ThreadFactoryBuilder().setNameFormat("TezTestServiceCommunicator #%2d").build());
+    this.hostProxies = new ConcurrentHashMap<String, TezTestServiceProtocolBlockingPB>();
+    executor = MoreExecutors.listeningDecorator(localExecutor);
+  }
+
+  @Override
+  public void serviceStop() {
+    executor.shutdownNow();
+  }
+
+
+  public void runContainer(RunContainerRequestProto request, String host, int port,
+                           final ExecuteRequestCallback<RunContainerResponseProto> callback) {
+    ListenableFuture<RunContainerResponseProto> future = executor.submit(new RunContainerCallable(request, host, port));
+    Futures.addCallback(future, new FutureCallback<RunContainerResponseProto>() {
+      @Override
+      public void onSuccess(RunContainerResponseProto result) {
+        callback.setResponse(result);
+      }
+
+      @Override
+      public void onFailure(Throwable t) {
+        callback.indicateError(t);
+      }
+    });
+
+  }
+
+  public void submitWork(SubmitWorkRequestProto request, String host, int port,
+                         final ExecuteRequestCallback<SubmitWorkResponseProto> callback) {
+    ListenableFuture<SubmitWorkResponseProto> future = executor.submit(new SubmitWorkCallable(request, host, port));
+    Futures.addCallback(future, new FutureCallback<SubmitWorkResponseProto>() {
+      @Override
+      public void onSuccess(SubmitWorkResponseProto result) {
+        callback.setResponse(result);
+      }
+
+      @Override
+      public void onFailure(Throwable t) {
+        callback.indicateError(t);
+      }
+    });
+
+  }
+
+
+  private class RunContainerCallable implements Callable<RunContainerResponseProto> {
+
+    final String hostname;
+    final int port;
+    final RunContainerRequestProto request;
+
+    private RunContainerCallable(RunContainerRequestProto request, String hostname, int port) {
+      this.hostname = hostname;
+          this.port = port;
+      this.request = request;
+    }
+
+    @Override
+    public RunContainerResponseProto call() throws Exception {
+      return getProxy(hostname, port).runContainer(null, request);
+    }
+  }
+
+  private class SubmitWorkCallable implements Callable<SubmitWorkResponseProto> {
+    final String hostname;
+    final int port;
+    final SubmitWorkRequestProto request;
+
+    private SubmitWorkCallable(SubmitWorkRequestProto request, String hostname, int port) {
+      this.hostname = hostname;
+      this.port = port;
+      this.request = request;
+    }
+
+    @Override
+    public SubmitWorkResponseProto call() throws Exception {
+      return getProxy(hostname, port).submitWork(null, request);
+    }
+  }
+
+  public interface ExecuteRequestCallback<T extends Message> {
+    void setResponse(T response);
+    void indicateError(Throwable t);
+  }
+
+  private TezTestServiceProtocolBlockingPB getProxy(String hostname, int port) {
+    String hostId = getHostIdentifier(hostname, port);
+
+    TezTestServiceProtocolBlockingPB proxy = hostProxies.get(hostId);
+    if (proxy == null) {
+      proxy = new TezTestServiceProtocolClientImpl(getConfig(), hostname, port);
+      TezTestServiceProtocolBlockingPB proxyOld = hostProxies.putIfAbsent(hostId, proxy);
+      if (proxyOld != null) {
+        // TODO Shutdown the new proxy.
+        proxy = proxyOld;
+      }
+    }
+    return proxy;
+  }
+
+  private String getHostIdentifier(String hostname, int port) {
+    return hostname + ":" + port;
+  }
+}

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

http://git-wip-us.apache.org/repos/asf/tez/blob/aa6c53ff/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceNoOpContainerLauncher.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceNoOpContainerLauncher.java b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceNoOpContainerLauncher.java
new file mode 100644
index 0000000..8c8e486
--- /dev/null
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceNoOpContainerLauncher.java
@@ -0,0 +1,66 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.dag.app.launcher;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.service.AbstractService;
+import org.apache.hadoop.yarn.util.Clock;
+import org.apache.tez.dag.app.AppContext;
+import org.apache.tez.dag.app.TaskAttemptListener;
+import org.apache.tez.dag.app.rm.NMCommunicatorEvent;
+import org.apache.tez.dag.app.rm.NMCommunicatorLaunchRequestEvent;
+import org.apache.tez.dag.app.rm.container.AMContainerEvent;
+import org.apache.tez.dag.app.rm.container.AMContainerEventLaunched;
+import org.apache.tez.dag.app.rm.container.AMContainerEventType;
+import org.apache.tez.dag.history.DAGHistoryEvent;
+import org.apache.tez.dag.history.events.ContainerLaunchedEvent;
+
+public class TezTestServiceNoOpContainerLauncher extends AbstractService implements ContainerLauncher {
+
+  static final Log LOG = LogFactory.getLog(TezTestServiceNoOpContainerLauncher.class);
+
+  private final AppContext context;
+  private final Clock clock;
+
+  public TezTestServiceNoOpContainerLauncher(AppContext appContext, Configuration conf,
+                                         TaskAttemptListener tal) {
+    super(TezTestServiceNoOpContainerLauncher.class.getName());
+    this.context = appContext;
+    this.clock = appContext.getClock();
+  }
+
+  @Override
+  public void handle(NMCommunicatorEvent event) {
+    switch(event.getType()) {
+      case CONTAINER_LAUNCH_REQUEST:
+        final NMCommunicatorLaunchRequestEvent launchEvent = (NMCommunicatorLaunchRequestEvent) event;
+        LOG.info("No-op launch for container: " + launchEvent.getContainerId() + " succeeded on host: " + launchEvent.getNodeId());
+        context.getEventHandler().handle(new AMContainerEventLaunched(launchEvent.getContainerId()));
+        ContainerLaunchedEvent lEvt = new ContainerLaunchedEvent(
+            launchEvent.getContainerId(), clock.getTime(), context.getApplicationAttemptId());
+        context.getHistoryHandler().handle(new DAGHistoryEvent(
+            null, lEvt));
+        break;
+      case CONTAINER_STOP_REQUEST:
+        LOG.info("DEBUG: Ignoring STOP_REQUEST for event: " + event);
+        context.getEventHandler().handle(new AMContainerEvent(event.getContainerId(),
+            AMContainerEventType.C_NM_STOP_SENT));
+        break;
+    }
+
+  }
+}

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

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

http://git-wip-us.apache.org/repos/asf/tez/blob/aa6c53ff/tez-ext-service-tests/src/test/java/org/apache/tez/service/ContainerRunner.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/service/ContainerRunner.java b/tez-ext-service-tests/src/test/java/org/apache/tez/service/ContainerRunner.java
new file mode 100644
index 0000000..2bca4ed
--- /dev/null
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/service/ContainerRunner.java
@@ -0,0 +1,27 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.service;
+
+import java.io.IOException;
+
+import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos;
+import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.RunContainerRequestProto;
+import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.SubmitWorkRequestProto;
+
+public interface ContainerRunner {
+
+  void queueContainer(RunContainerRequestProto request) throws IOException;
+  void submitWork(SubmitWorkRequestProto request) throws IOException;
+}

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

http://git-wip-us.apache.org/repos/asf/tez/blob/aa6c53ff/tez-ext-service-tests/src/test/java/org/apache/tez/service/TezTestServiceConfConstants.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/service/TezTestServiceConfConstants.java b/tez-ext-service-tests/src/test/java/org/apache/tez/service/TezTestServiceConfConstants.java
new file mode 100644
index 0000000..bf4a5bd
--- /dev/null
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/service/TezTestServiceConfConstants.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.service;
+
+public class TezTestServiceConfConstants {
+
+  private static final String TEZ_TEST_SERVICE_PREFIX = "tez.test.service.";
+
+  /** Number of executors per instance - used by the scheduler */
+  public static final String TEZ_TEST_SERVICE_NUM_EXECUTORS_PER_INSTANCE = TEZ_TEST_SERVICE_PREFIX + "num.executors.per-instance";
+
+  /** Memory available per instance - used by the scheduler */
+  public static final String TEZ_TEST_SERVICE_MEMORY_PER_INSTANCE_MB = TEZ_TEST_SERVICE_PREFIX + "memory.per.instance.mb";
+
+  /** CPUs available per instance - used by the scheduler */
+  public static final String TEZ_TEST_SERVICE_VCPUS_PER_INSTANCE = TEZ_TEST_SERVICE_PREFIX + "vcpus.per.instance";
+
+
+  /** Hosts on which the service is running. Currently assuming a single port for all instances */
+  public static final String TEZ_TEST_SERVICE_HOSTS = TEZ_TEST_SERVICE_PREFIX + "hosts";
+
+  /** Port on which the Service(s) listen. Current a single port for all instances */
+  public static final String TEZ_TEST_SERVICE_RPC_PORT = TEZ_TEST_SERVICE_PREFIX + "rpc.port";
+
+  /** Number of threads to use in the AM to communicate with the external service */
+  public static final String TEZ_TEST_SERVICE_AM_COMMUNICATOR_NUM_THREADS = TEZ_TEST_SERVICE_PREFIX + "communicator.num.threads";
+  public static final int TEZ_TEST_SERVICE_AM_COMMUNICATOR_NUM_THREADS_DEFAULT = 2;
+
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/aa6c53ff/tez-ext-service-tests/src/test/java/org/apache/tez/service/TezTestServiceProtocolBlockingPB.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/service/TezTestServiceProtocolBlockingPB.java b/tez-ext-service-tests/src/test/java/org/apache/tez/service/TezTestServiceProtocolBlockingPB.java
new file mode 100644
index 0000000..1108f72
--- /dev/null
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/service/TezTestServiceProtocolBlockingPB.java
@@ -0,0 +1,22 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.service;
+
+import org.apache.hadoop.ipc.ProtocolInfo;
+import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos;
+
+@ProtocolInfo(protocolName = "org.apache.tez.service.TezTestServiceProtocolBlockingPB", protocolVersion = 1)
+public interface TezTestServiceProtocolBlockingPB extends TezTestServiceProtocolProtos.TezTestServiceProtocol.BlockingInterface {
+}
\ No newline at end of file


[08/50] [abbrv] tez git commit: TEZ-2488. Tez AM crashes if a submitted DAG is configured to use invalid resource sizes. (zjffdu)

Posted by ss...@apache.org.
TEZ-2488. Tez AM crashes if a submitted DAG is configured to use invalid resource sizes. (zjffdu)


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

Branch: refs/heads/TEZ-2003
Commit: 17b6aba9c7f85d710eb97a006b9a42fbe0510cbc
Parents: 02aafb5
Author: Jeff Zhang <zj...@apache.org>
Authored: Thu May 28 12:24:39 2015 +0800
Committer: Jeff Zhang <zj...@apache.org>
Committed: Thu May 28 12:24:39 2015 +0800

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../java/org/apache/tez/dag/app/AppContext.java |  2 ++
 .../org/apache/tez/dag/app/DAGAppMaster.java    |  5 +++++
 .../apache/tez/dag/app/dag/impl/DAGImpl.java    | 16 ++++++++++++++++
 .../apache/tez/dag/app/dag/impl/TestCommit.java |  4 +++-
 .../tez/dag/app/dag/impl/TestDAGImpl.java       | 20 ++++++++++++++++++++
 .../tez/dag/app/dag/impl/TestDAGRecovery.java   |  4 ++++
 .../dag/app/dag/impl/TestVertexRecovery.java    |  5 +++++
 8 files changed, 56 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/17b6aba9/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index c9318d2..ef9a839 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -406,6 +406,7 @@ TEZ-UI CHANGES (TEZ-8):
 Release 0.5.4: Unreleased
 
 ALL CHANGES:
+  TEZ-2488. Tez AM crashes if a submitted DAG is configured to use invalid resource sizes.
   TEZ-2080. Localclient should be using tezconf in init instead of yarnconf.
   TEZ-2369. Add a few unit tests for RootInputInitializerManager.
   TEZ-2379. org.apache.hadoop.yarn.state.InvalidStateTransitonException:

http://git-wip-us.apache.org/repos/asf/tez/blob/17b6aba9/tez-dag/src/main/java/org/apache/tez/dag/app/AppContext.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/AppContext.java b/tez-dag/src/main/java/org/apache/tez/dag/app/AppContext.java
index 4781784..c005447 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
@@ -91,6 +91,8 @@ public interface AppContext {
 
   boolean isSession();
 
+  boolean isLocal();
+
   DAGAppMasterState getAMState();
 
   HistoryEventHandler getHistoryHandler();

http://git-wip-us.apache.org/repos/asf/tez/blob/17b6aba9/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 195bc6c..0fadcfa 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
@@ -1423,6 +1423,11 @@ public class DAGAppMaster extends AbstractService {
     }
 
     @Override
+    public boolean isLocal() {
+      return isLocal;
+    }
+
+    @Override
     public DAGAppMasterState getAMState() {
       return state;
     }

http://git-wip-us.apache.org/repos/asf/tez/blob/17b6aba9/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 0a87241..38da302 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
@@ -1413,6 +1413,20 @@ 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()) {
+        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()
+              + ", Cluster MaxContainerCapability=" + appContext.getClusterInfo().getMaxContainerCapability();
+          LOG.error(msg);
+          addDiagnostic(msg);
+          finished(DAGState.FAILED);
+          return DAGState.FAILED;
+        }
+      }
+    }
 
     createDAGEdges(this);
     Map<String,EdgePlan> edgePlans = DagTypeConverters.createEdgePlanMapFromDAGPlan(getJobPlan().getEdgeList());
@@ -1453,6 +1467,7 @@ public class DAGImpl implements org.apache.tez.dag.app.dag.DAG,
         }
       }
     }
+
     return DAGState.INITED;
   }
 
@@ -1678,6 +1693,7 @@ public class DAGImpl implements org.apache.tez.dag.app.dag.DAG,
 
       DAGState state = dag.initializeDAG();
       if (state != DAGState.INITED) {
+        dag.trySetTerminationCause(DAGTerminationCause.INIT_FAILURE);
         return state;
       }
 

http://git-wip-us.apache.org/repos/asf/tez/blob/17b6aba9/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestCommit.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestCommit.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestCommit.java
index 3d3bca4..1ce9570 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestCommit.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestCommit.java
@@ -69,6 +69,7 @@ import org.apache.tez.dag.api.client.VertexStatus;
 import org.apache.tez.dag.api.oldrecords.TaskState;
 import org.apache.tez.dag.api.records.DAGProtos.DAGPlan;
 import org.apache.tez.dag.app.AppContext;
+import org.apache.tez.dag.app.ClusterInfo;
 import org.apache.tez.dag.app.TaskAttemptListener;
 import org.apache.tez.dag.app.TaskHeartbeatHandler;
 import org.apache.tez.dag.app.dag.DAGState;
@@ -319,7 +320,8 @@ public class TestCommit {
         taskAttemptListener, fsTokens, clock, "user", thh, appContext);
     doReturn(dag).when(appContext).getCurrentDAG();
     doReturn(dispatcher.getEventHandler()).when(appContext).getEventHandler();
-
+    ClusterInfo clusterInfo = new ClusterInfo(Resource.newInstance(8192,10));
+    doReturn(clusterInfo).when(appContext).getClusterInfo();
     dispatcher.register(CallableEventType.class, new CallableEventDispatcher());
     taskEventDispatcher = new TaskEventDispatcher();
     dispatcher.register(TaskEventType.class, taskEventDispatcher);

http://git-wip-us.apache.org/repos/asf/tez/blob/17b6aba9/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGImpl.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGImpl.java
index 4787247..e268a99 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGImpl.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGImpl.java
@@ -87,6 +87,7 @@ import org.apache.tez.dag.api.records.DAGProtos.RootInputLeafOutputProto;
 import org.apache.tez.dag.api.records.DAGProtos.TezEntityDescriptorProto;
 import org.apache.tez.dag.api.records.DAGProtos.VertexPlan;
 import org.apache.tez.dag.app.AppContext;
+import org.apache.tez.dag.app.ClusterInfo;
 import org.apache.tez.dag.app.TaskAttemptListener;
 import org.apache.tez.dag.app.TaskHeartbeatHandler;
 import org.apache.tez.dag.app.dag.DAGScheduler;
@@ -182,6 +183,7 @@ public class TestDAGImpl {
   private AppContext dagWithCustomEdgeAppContext;
   private HistoryEventHandler historyEventHandler;
   private TaskAttemptEventDispatcher taskAttemptEventDispatcher;
+  private ClusterInfo clusterInfo = new ClusterInfo(Resource.newInstance(8192,10));
 
   private DAGImpl chooseDAG(TezDAGID curDAGId) {
     if (curDAGId.equals(dagId)) {
@@ -786,6 +788,7 @@ public class TestDAGImpl {
         fsTokens, clock, "user", thh, appContext);
     dag.entityUpdateTracker = new StateChangeNotifierForTest(dag);
     doReturn(dag).when(appContext).getCurrentDAG();
+    doReturn(clusterInfo).when(appContext).getClusterInfo();
     mrrAppContext = mock(AppContext.class);
     doReturn(aclManager).when(mrrAppContext).getAMACLManager();
     doReturn(execService).when(mrrAppContext).getExecService();
@@ -801,6 +804,7 @@ public class TestDAGImpl {
     doReturn(appAttemptId).when(mrrAppContext).getApplicationAttemptId();
     doReturn(appAttemptId.getApplicationId()).when(mrrAppContext).getApplicationID();
     doReturn(historyEventHandler).when(mrrAppContext).getHistoryHandler();
+    doReturn(clusterInfo).when(mrrAppContext).getClusterInfo();
     groupAppContext = mock(AppContext.class);
     doReturn(aclManager).when(groupAppContext).getAMACLManager();
     doReturn(execService).when(groupAppContext).getExecService();
@@ -817,6 +821,7 @@ public class TestDAGImpl {
     doReturn(appAttemptId.getApplicationId())
         .when(groupAppContext).getApplicationID();
     doReturn(historyEventHandler).when(groupAppContext).getHistoryHandler();
+    doReturn(clusterInfo).when(groupAppContext).getClusterInfo();
 
     // reset totalCommitCounter to 0
     TotalCountingOutputCommitter.totalCommitCounter = 0;
@@ -885,6 +890,7 @@ public class TestDAGImpl {
     doReturn(appAttemptId).when(dagWithCustomEdgeAppContext).getApplicationAttemptId();
     doReturn(appAttemptId.getApplicationId()).when(dagWithCustomEdgeAppContext).getApplicationID();
     doReturn(historyEventHandler).when(dagWithCustomEdgeAppContext).getHistoryHandler();
+    doReturn(clusterInfo).when(dagWithCustomEdgeAppContext).getClusterInfo();
     dispatcher.register(TaskAttemptEventType.class, new TaskAttemptEventDisptacher2());
     dispatcher.register(AMSchedulerEventType.class, new AMSchedulerEventHandler());
   }
@@ -922,6 +928,20 @@ public class TestDAGImpl {
   }
 
   @Test(timeout = 5000)
+  public void testDAGInitFailedDuetoInvalidResource() {
+    // cluster maxContainerCapability is less than the vertex resource request
+    ClusterInfo clusterInfo = new ClusterInfo(Resource.newInstance(512,10));
+    doReturn(clusterInfo).when(appContext).getClusterInfo();
+    dag.handle(
+        new DAGEvent(dag.getID(), DAGEventType.DAG_INIT));
+    dispatcher.await();
+    Assert.assertEquals(DAGState.FAILED, dag.getState());
+    Assert.assertEquals(DAGTerminationCause.INIT_FAILURE, dag.getTerminationCause());
+    Assert.assertTrue(StringUtils.join(dag.getDiagnostics(),",")
+        .contains("Vertex's TaskResource is beyond the cluster container capability"));
+  }
+
+  @Test(timeout = 5000)
   public void testDAGStart() {
     initDAG(dag);
     startDAG(dag);

http://git-wip-us.apache.org/repos/asf/tez/blob/17b6aba9/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGRecovery.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGRecovery.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGRecovery.java
index bd4653b..a0d5fb5 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGRecovery.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGRecovery.java
@@ -28,12 +28,14 @@ import java.util.List;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.event.Event;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.util.SystemClock;
 import org.apache.tez.common.counters.TezCounters;
 import org.apache.tez.dag.api.records.DAGProtos.DAGPlan;
 import org.apache.tez.dag.app.AppContext;
+import org.apache.tez.dag.app.ClusterInfo;
 import org.apache.tez.dag.app.TaskAttemptListener;
 import org.apache.tez.dag.app.TaskHeartbeatHandler;
 import org.apache.tez.dag.app.dag.DAGState;
@@ -78,6 +80,8 @@ public class TestDAGRecovery {
   public void setUp() {
     mockAppContext = mock(AppContext.class, RETURNS_DEEP_STUBS);
     when(mockAppContext.getCurrentDAG().getDagUGI()).thenReturn(null);
+    ClusterInfo clusterInfo = new ClusterInfo(Resource.newInstance(8192,10));
+    doReturn(clusterInfo).when(mockAppContext).getClusterInfo();
     mockEventHandler = mock(EventHandler.class);
     tezCounters.findCounter("grp_1", "counter_1").increment(1);
 

http://git-wip-us.apache.org/repos/asf/tez/blob/17b6aba9/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexRecovery.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexRecovery.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexRecovery.java
index ad57ba8..2d03c60 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexRecovery.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexRecovery.java
@@ -20,6 +20,7 @@ package org.apache.tez.dag.app.dag.impl;
 
 import static org.junit.Assert.*;
 import static org.mockito.Mockito.RETURNS_DEEP_STUBS;
+import static org.mockito.Mockito.doReturn;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
@@ -33,6 +34,7 @@ import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.event.DrainDispatcher;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.util.SystemClock;
@@ -51,6 +53,7 @@ import org.apache.tez.dag.api.records.DAGProtos.RootInputLeafOutputProto;
 import org.apache.tez.dag.api.records.DAGProtos.TezEntityDescriptorProto;
 import org.apache.tez.dag.api.records.DAGProtos.VertexPlan;
 import org.apache.tez.dag.app.AppContext;
+import org.apache.tez.dag.app.ClusterInfo;
 import org.apache.tez.dag.app.TaskAttemptListener;
 import org.apache.tez.dag.app.TaskHeartbeatHandler;
 import org.apache.tez.dag.app.dag.VertexState;
@@ -418,6 +421,8 @@ public class TestVertexRecovery {
             new Credentials(), new SystemClock(), user,
             mock(TaskHeartbeatHandler.class), mockAppContext);
     when(mockAppContext.getCurrentDAG()).thenReturn(dag);
+    ClusterInfo clusterInfo = new ClusterInfo(Resource.newInstance(8192,10));
+    doReturn(clusterInfo).when(mockAppContext).getClusterInfo();
 
     dag.handle(new DAGEvent(dagId, DAGEventType.DAG_INIT));
     LOG.info("finish setUp");


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

Branch: refs/heads/TEZ-2003
Commit: 044e4fce868e507abbbe28a48d6d493c844700df
Parents: df757f1
Author: Siddharth Seth <ss...@apache.org>
Authored: Thu May 28 18:29:12 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Mon Jun 1 14:26:41 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/044e4fce/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/044e4fce/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


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

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


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

Branch: refs/heads/TEZ-2003
Commit: 216dcfc50749cad9e9896b3538ea32b32a42d30e
Parents: aa6c53f
Author: Siddharth Seth <ss...@apache.org>
Authored: Wed Feb 18 14:45:34 2015 -0800
Committer: Siddharth Seth <ss...@apache.org>
Committed: Mon Jun 1 14:09:26 2015 -0700

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


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

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

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

http://git-wip-us.apache.org/repos/asf/tez/blob/216dcfc5/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherRouter.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherRouter.java b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherRouter.java
new file mode 100644
index 0000000..34001ed
--- /dev/null
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherRouter.java
@@ -0,0 +1,108 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.dag.app.launcher;
+
+import java.lang.reflect.Constructor;
+import java.lang.reflect.InvocationTargetException;
+import java.net.UnknownHostException;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.service.AbstractService;
+import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.tez.common.ReflectionUtils;
+import org.apache.tez.dag.api.TezConfiguration;
+import org.apache.tez.dag.api.TezUncheckedException;
+import org.apache.tez.dag.app.AppContext;
+import org.apache.tez.dag.app.TaskAttemptListener;
+import org.apache.tez.dag.app.rm.NMCommunicatorEvent;
+
+public class ContainerLauncherRouter extends AbstractService
+    implements EventHandler<NMCommunicatorEvent> {
+
+  static final Log LOG = LogFactory.getLog(ContainerLauncherImpl.class);
+
+  private final ContainerLauncher containerLauncher;
+
+  @VisibleForTesting
+  public ContainerLauncherRouter(ContainerLauncher containerLauncher) {
+    super(ContainerLauncherRouter.class.getName());
+    this.containerLauncher = containerLauncher;
+  }
+
+  // Accepting conf to setup final parameters, if required.
+  public ContainerLauncherRouter(Configuration conf, boolean isLocal, AppContext context,
+                                 TaskAttemptListener taskAttemptListener,
+                                 String workingDirectory) throws UnknownHostException {
+    super(ContainerLauncherRouter.class.getName());
+
+    if (isLocal) {
+      LOG.info("Creating LocalContainerLauncher");
+      containerLauncher =
+          new LocalContainerLauncher(context, taskAttemptListener, workingDirectory);
+    } else {
+      // TODO: Temporary reflection with specific parameters until a clean interface is defined.
+      String containerLauncherClassName =
+          conf.get(TezConfiguration.TEZ_AM_CONTAINER_LAUNCHER_CLASS);
+      if (containerLauncherClassName == null) {
+        LOG.info("Creating Default Container Launcher");
+        containerLauncher = new ContainerLauncherImpl(context);
+      } else {
+        LOG.info("Creating container launcher : " + containerLauncherClassName);
+        Class<? extends ContainerLauncher> containerLauncherClazz =
+            (Class<? extends ContainerLauncher>) ReflectionUtils.getClazz(
+                containerLauncherClassName);
+        try {
+          Constructor<? extends ContainerLauncher> ctor = containerLauncherClazz
+              .getConstructor(AppContext.class, Configuration.class, TaskAttemptListener.class);
+          ctor.setAccessible(true);
+          containerLauncher = ctor.newInstance(context, conf, taskAttemptListener);
+        } catch (NoSuchMethodException e) {
+          throw new TezUncheckedException(e);
+        } catch (InvocationTargetException e) {
+          throw new TezUncheckedException(e);
+        } catch (InstantiationException e) {
+          throw new TezUncheckedException(e);
+        } catch (IllegalAccessException e) {
+          throw new TezUncheckedException(e);
+        }
+      }
+
+    }
+  }
+
+  @Override
+  public void serviceInit(Configuration conf) {
+    ((AbstractService)containerLauncher).init(conf);
+  }
+
+  @Override
+  public void serviceStart() {
+    ((AbstractService)containerLauncher).start();
+  }
+
+  @Override
+  public void serviceStop() {
+    ((AbstractService)containerLauncher).stop();
+  }
+
+
+  @Override
+  public void handle(NMCommunicatorEvent event) {
+    containerLauncher.handle(event);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/216dcfc5/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 9f45284..d9c211a 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
@@ -57,6 +57,7 @@ import org.apache.tez.dag.api.TezConfiguration;
 import org.apache.tez.dag.api.TaskCommunicator;
 import org.apache.tez.dag.api.TezUncheckedException;
 import org.apache.tez.dag.app.launcher.ContainerLauncher;
+import org.apache.tez.dag.app.launcher.ContainerLauncherRouter;
 import org.apache.tez.dag.app.rm.NMCommunicatorEvent;
 import org.apache.tez.dag.app.rm.NMCommunicatorLaunchRequestEvent;
 import org.apache.tez.dag.app.rm.NMCommunicatorStopRequestEvent;
@@ -512,9 +513,9 @@ public class MockDAGAppMaster extends DAGAppMaster {
   
   // use mock container launcher for tests
   @Override
-  protected ContainerLauncher createContainerLauncher(final AppContext context)
+  protected ContainerLauncherRouter createContainerLauncherRouter(final Configuration conf)
       throws UnknownHostException {
-    return containerLauncher;
+    return new ContainerLauncherRouter(containerLauncher);
   }
 
   @Override


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

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


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

Branch: refs/heads/TEZ-2003
Commit: caab5784269a080d70b125aa9266b46d56cfeb9d
Parents: 216dcfc
Author: Siddharth Seth <ss...@apache.org>
Authored: Thu Feb 19 14:59:18 2015 -0800
Committer: Siddharth Seth <ss...@apache.org>
Committed: Mon Jun 1 14:09:26 2015 -0700

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


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

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

http://git-wip-us.apache.org/repos/asf/tez/blob/caab5784/tez-api/src/main/java/org/apache/tez/dag/api/TezConstants.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/TezConstants.java b/tez-api/src/main/java/org/apache/tez/dag/api/TezConstants.java
index bc4208f..3b07c59 100644
--- a/tez-api/src/main/java/org/apache/tez/dag/api/TezConstants.java
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/TezConstants.java
@@ -102,4 +102,7 @@ public class TezConstants {
   /// Version-related Environment variables
   public static final String TEZ_CLIENT_VERSION_ENV = "TEZ_CLIENT_VERSION";
 
+
+  public static final String TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT = "Tez";
+  public static final String TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT = "TezLocal";
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/caab5784/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 1f14887..94e9ba5 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
@@ -111,4 +111,8 @@ public interface AppContext {
   String getAMUser();
 
   Credentials getAppCredentials();
+
+  public Integer getTaskCommunicatorIdentifier(String name);
+  public Integer getTaskScheduerIdentifier(String name);
+  public Integer getContainerLauncherIdentifier(String name);
 }

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

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

http://git-wip-us.apache.org/repos/asf/tez/blob/caab5784/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Vertex.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Vertex.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Vertex.java
index bb3548d..ca11a85 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Vertex.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Vertex.java
@@ -173,4 +173,8 @@ public interface Vertex extends Comparable<Vertex> {
   public int getKilledTaskAttemptCount();
 
   public Configuration getConf();
+
+  public int getTaskSchedulerIdentifier();
+  public int getContainerLauncherIdentifier();
+  public int getTaskCommunicatorIdentifier();
 }

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

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

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

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

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

http://git-wip-us.apache.org/repos/asf/tez/blob/caab5784/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 d9c211a..6683d84 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
@@ -510,10 +510,11 @@ public class MockDAGAppMaster extends DAGAppMaster {
     this.handlerConcurrency = handlerConcurrency;
     this.numConcurrentContainers = numConcurrentContainers;
   }
-  
+
   // use mock container launcher for tests
   @Override
-  protected ContainerLauncherRouter createContainerLauncherRouter(final Configuration conf)
+  protected ContainerLauncherRouter createContainerLauncherRouter(final Configuration conf,
+                                                                  String[] containerLaunchers)
       throws UnknownHostException {
     return new ContainerLauncherRouter(containerLauncher);
   }

http://git-wip-us.apache.org/repos/asf/tez/blob/caab5784/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 f41b51b..62c5b21 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
@@ -32,6 +32,7 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
+import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
@@ -109,18 +110,9 @@ public class TestTaskAttemptListenerImplTezDag {
     doReturn(dag).when(appContext).getCurrentDAG();
     doReturn(appAcls).when(appContext).getApplicationACLs();
     doReturn(amContainerMap).when(appContext).getAllContainers();
-    NodeId nodeId = NodeId.newInstance("localhost", 0);
-    AMContainer amContainer = mock(AMContainer.class);
-    Container container = mock(Container.class);
-    doReturn(nodeId).when(container).getNodeId();
-    doReturn(amContainer).when(amContainerMap).get(any(ContainerId.class));
-    doReturn(container).when(amContainer).getContainer();
-
-    taskAttemptListener =
-        new TaskAttemptListenerImpTezDag(appContext, mock(TaskHeartbeatHandler.class),
-            mock(ContainerHeartbeatHandler.class), null);
-    TezTaskCommunicatorImpl taskCommunicator = (TezTaskCommunicatorImpl)taskAttemptListener.getTaskCommunicator();
-    TezTaskUmbilicalProtocol tezUmbilical = taskCommunicator.getUmbilical();
+
+    taskAttemptListener = new TaskAttemptListenerImplForTest(appContext,
+        mock(TaskHeartbeatHandler.class), mock(ContainerHeartbeatHandler.class), null);
 
     taskSpec = mock(TaskSpec.class);
     doReturn(taskAttemptID).when(taskSpec).getTaskAttemptID();
@@ -131,6 +123,9 @@ public class TestTaskAttemptListenerImplTezDag {
   @Test(timeout = 5000)
   public void testGetTask() throws IOException {
 
+    TezTaskCommunicatorImpl taskCommunicator = (TezTaskCommunicatorImpl)taskAttemptListener.getTaskCommunicator();
+    TezTaskUmbilicalProtocol tezUmbilical = taskCommunicator.getUmbilical();
+
     ContainerId containerId1 = createContainerId(appId, 1);
     ContainerContext containerContext1 = new ContainerContext(containerId1.toString());
     containerTask = tezUmbilical.getTask(containerContext1);

http://git-wip-us.apache.org/repos/asf/tez/blob/caab5784/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestContainerReuse.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestContainerReuse.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestContainerReuse.java
index 89b77a7..54b9adb 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestContainerReuse.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestContainerReuse.java
@@ -1322,7 +1322,7 @@ public class TestContainerReuse {
           InputDescriptor.create("inputClassName"), 1)),
       Collections.singletonList(new OutputSpec("vertexName",
           OutputDescriptor.create("outputClassName"), 1)), null), ta, locationHint,
-      priority.getPriority(), containerContext);
+      priority.getPriority(), containerContext, 0, 0, 0);
     return lr;
   }
 

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

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

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


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

Branch: refs/heads/TEZ-2003
Commit: 54af7bc2976aa11937baa18094b302dc4c7b6f5a
Parents: 323fa5e
Author: Siddharth Seth <ss...@apache.org>
Authored: Wed May 6 17:14:56 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Mon Jun 1 14:21:04 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/54af7bc2/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/54af7bc2/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 {


[05/50] [abbrv] tez git commit: TEZ-2482. Tez UI: Mouse events not working on IE11 (Sreenath Somarajapuram via pramachandran)

Posted by ss...@apache.org.
TEZ-2482. Tez UI: Mouse events not working on IE11 (Sreenath Somarajapuram via pramachandran)


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

Branch: refs/heads/TEZ-2003
Commit: 317d45a5d410f2e6afedbe8563c258f4cc667d67
Parents: 8710df0
Author: Prakash Ramachandran <pr...@hortonworks.com>
Authored: Thu May 28 00:23:08 2015 +0530
Committer: Prakash Ramachandran <pr...@hortonworks.com>
Committed: Thu May 28 00:23:08 2015 +0530

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../scripts/controllers/dag-view-controller.js  | 63 +++++++++++++++-----
 2 files changed, 48 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/317d45a5/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 513285f..4779f83 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -26,6 +26,7 @@ Release 0.7.1: Unreleased
 INCOMPATIBLE CHANGES
 
 ALL CHANGES:
+  TEZ-2482. Tez UI: Mouse events not working on IE11
   TEZ-1529. ATS and TezClient integration in secure kerberos enabled cluster.
   TEZ-2481. Tez UI: graphical view does not render properly on IE11
   TEZ-2474. The old taskNum is logged incorrectly when parallelism is changed

http://git-wip-us.apache.org/repos/asf/tez/blob/317d45a5/tez-ui/src/main/webapp/app/scripts/controllers/dag-view-controller.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/scripts/controllers/dag-view-controller.js b/tez-ui/src/main/webapp/app/scripts/controllers/dag-view-controller.js
index 5241060..aabdb0d 100644
--- a/tez-ui/src/main/webapp/app/scripts/controllers/dag-view-controller.js
+++ b/tez-ui/src/main/webapp/app/scripts/controllers/dag-view-controller.js
@@ -52,24 +52,55 @@ App.DagViewController = App.TablePageController.extend({
     return this._super();
   },
 
+  redirect: function (details) {
+    switch(details.type) {
+      case 'vertex':
+        this.transitionToRoute('vertex', details.d.get('data.id'));
+      break;
+      case 'task':
+        this.transitionToRoute('vertex.tasks', details.d.get('data.id'));
+      break;
+      case 'io':
+        this.transitionToRoute('vertex.additionals', details.d.get('data.id'));
+      break;
+      case 'input':
+        this.transitionToRoute('input.configs', details.d.get('parent.data.id'), details.d.entity);
+      break;
+      case 'output':
+        this.transitionToRoute('output.configs', details.d.get('vertex.data.id'), details.d.entity);
+      break;
+    }
+  },
+
   actions: {
+    modalConfirmed: function () {
+      this.redirect(this.get('redirectionDetails'));
+    },
+    modalCanceled: function () {
+    },
     entityClicked: function (details) {
-      switch(details.type) {
-        case 'vertex':
-          this.transitionToRoute('vertex', details.d.get('data.id'));
-        break;
-        case 'task':
-          this.transitionToRoute('vertex.tasks', details.d.get('data.id'));
-        break;
-        case 'io':
-          this.transitionToRoute('vertex.additionals', details.d.get('data.id'));
-        break;
-        case 'input':
-          this.transitionToRoute('input.configs', details.d.get('parent.data.id'), details.d.entity);
-        break;
-        case 'output':
-          this.transitionToRoute('output.configs', details.d.get('vertex.data.id'), details.d.entity);
-        break;
+
+      /**
+       * In IE 11 under Windows 7, mouse events are not delivered to the page
+       * anymore at all after a SVG use element that was under the mouse is
+       * removed from the DOM in the event listener in response to a mouse click.
+       * See https://connect.microsoft.com/IE/feedback/details/796745
+       *
+       * This condition and related actions must be removed once the bug is fixed
+       * in all supported IE versions
+       */
+      if(App.env.isIE) {
+        this.set('redirectionDetails', details);
+        Bootstrap.ModalManager.confirm(
+          this,
+          'Confirmation Required!',
+          'You will be redirected to %@ page'.fmt(
+            details.type == "io" ? "additionals" : details.type
+          )
+        );
+      }
+      else {
+        this.redirect(details);
       }
     }
   },


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

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


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

Branch: refs/heads/TEZ-2003
Commit: 2b663b1ab8a18e7818ebcc222d2302982b8b1436
Parents: 6a0a452
Author: Siddharth Seth <ss...@apache.org>
Authored: Tue Mar 10 00:47:07 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Mon Jun 1 14:09:27 2015 -0700

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


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

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

http://git-wip-us.apache.org/repos/asf/tez/blob/2b663b1a/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 8346839..b570301 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
@@ -360,7 +360,7 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
     }
     taskCommunicators[taskCommId].registerRunningTaskAttempt(containerId, amContainerTask.getTask(),
         amContainerTask.getAdditionalResources(), amContainerTask.getCredentials(),
-        amContainerTask.haveCredentialsChanged());
+        amContainerTask.haveCredentialsChanged(), amContainerTask.getPriority());
   }
 
   @Override

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

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


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

Branch: refs/heads/TEZ-2003
Commit: ba9d37709c78671bf1203ace56681ae6401b3510
Parents: 1ed50e1
Author: Siddharth Seth <ss...@apache.org>
Authored: Tue May 12 14:27:42 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Mon Jun 1 14:26:41 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/ba9d3770/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/ba9d3770/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/ba9d3770/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/ba9d3770/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/ba9d3770/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/ba9d3770/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();
+          }
         }
       }
     }


[02/50] [abbrv] tez git commit: TEZ-2450. support async http clients in ordered & unordered inputs (rbalamohan)

Posted by ss...@apache.org.
TEZ-2450. support async http clients in ordered & unordered inputs (rbalamohan)


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

Branch: refs/heads/TEZ-2003
Commit: 9dabf94767480750f31d8f3e24d17a89bc036331
Parents: 7be325e
Author: Rajesh Balamohan <rb...@apache.org>
Authored: Wed May 27 05:32:08 2015 +0530
Committer: Rajesh Balamohan <rb...@apache.org>
Committed: Wed May 27 05:32:08 2015 +0530

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 pom.xml                                         |   5 +
 tez-runtime-library/findbugs-exclude.xml        |  12 +
 tez-runtime-library/pom.xml                     |   4 +
 .../org/apache/tez/http/BaseHttpConnection.java |  63 +++
 .../org/apache/tez/http/HttpConnection.java     | 318 ++++++++++++++
 .../apache/tez/http/HttpConnectionParams.java   |  82 ++++
 .../java/org/apache/tez/http/SSLFactory.java    | 238 +++++++++++
 .../http/async/netty/AsyncHttpConnection.java   | 231 ++++++++++
 .../netty/TezBodyDeferringAsyncHandler.java     | 256 +++++++++++
 .../library/api/TezRuntimeConfiguration.java    |   4 +
 .../runtime/library/common/shuffle/Fetcher.java |  34 +-
 .../library/common/shuffle/HttpConnection.java  | 428 -------------------
 .../library/common/shuffle/ShuffleUtils.java    | 117 +++--
 .../common/shuffle/impl/ShuffleManager.java     |  12 +-
 .../orderedgrouped/FetcherOrderedGrouped.java   |  23 +-
 .../orderedgrouped/ShuffleScheduler.java        |  11 +-
 .../library/input/OrderedGroupedKVInput.java    |   1 +
 .../runtime/library/input/UnorderedKVInput.java |   1 +
 .../org/apache/tez/http/TestHttpConnection.java | 202 +++++++++
 .../library/common/shuffle/TestFetcher.java     |  12 +-
 .../shuffle/orderedgrouped/TestFetcher.java     |  71 ++-
 .../apache/tez/test/TestPipelinedShuffle.java   |  13 +-
 .../org/apache/tez/test/TestSecureShuffle.java  |  21 +-
 24 files changed, 1636 insertions(+), 524 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/9dabf947/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index a14e9da..5f5dd48 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -8,6 +8,7 @@ INCOMPATIBLE CHANGES
   TEZ-2468. Change the minimum Java version to Java 7.
 
 ALL CHANGES:
+  TEZ-2450. support async http clients in ordered & unordered inputs.
   TEZ-2454. Change FetcherOrderedGroup to work as Callables instead of blocking threads.
   TEZ-2466. tez-history-parser breaks hadoop 2.2 compatability.
   TEZ-2463. Update site for 0.7.0 release

http://git-wip-us.apache.org/repos/asf/tez/blob/9dabf947/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 44592fa..2922cab 100644
--- a/pom.xml
+++ b/pom.xml
@@ -182,6 +182,11 @@
         <version>1.7.5</version>
       </dependency>
       <dependency>
+	      <groupId>com.ning</groupId>
+	      <artifactId>async-http-client</artifactId>
+	      <version>1.8.16</version>
+      </dependency>
+      <dependency>
         <groupId>org.slf4j</groupId>
         <artifactId>slf4j-log4j12</artifactId>
         <version>1.7.5</version>

http://git-wip-us.apache.org/repos/asf/tez/blob/9dabf947/tez-runtime-library/findbugs-exclude.xml
----------------------------------------------------------------------
diff --git a/tez-runtime-library/findbugs-exclude.xml b/tez-runtime-library/findbugs-exclude.xml
index 489e243..919e1e3 100644
--- a/tez-runtime-library/findbugs-exclude.xml
+++ b/tez-runtime-library/findbugs-exclude.xml
@@ -73,6 +73,18 @@
     <Bug pattern="UWF_UNWRITTEN_PUBLIC_OR_PROTECTED_FIELD"/>
   </Match>
 
+  <Match>
+    <Class name="org.apache.tez.runtime.library.common.shuffle.orderedgrouped.FetcherOrderedGrouped"/>
+    <Method name="setupConnection" params="org.apache.tez.runtime.library.common.shuffle.orderedgrouped.MapHost, java.util.List" returns="boolean"/>
+    <Bug pattern="BC_VACUOUS_INSTANCEOF"/>
+  </Match>
+
+  <Match>
+    <Class name="org.apache.tez.runtime.library.common.shuffle.Fetcher"/>
+    <Method name="setupConnection" params="java.util.List" returns="org.apache.tez.runtime.library.common.shuffle.Fetcher$HostFetchResult"/>
+    <Bug pattern="BC_VACUOUS_INSTANCEOF"/>
+  </Match>
+
   <!-- TODO This needs more looking into -->
   <Match>
     <Class name="org.apache.tez.runtime.library.common.sort.impl.dflt.DefaultSorter"/>

http://git-wip-us.apache.org/repos/asf/tez/blob/9dabf947/tez-runtime-library/pom.xml
----------------------------------------------------------------------
diff --git a/tez-runtime-library/pom.xml b/tez-runtime-library/pom.xml
index 03e0ec3..4433a02 100644
--- a/tez-runtime-library/pom.xml
+++ b/tez-runtime-library/pom.xml
@@ -26,6 +26,10 @@
 
   <dependencies>
     <dependency>
+      <groupId>com.ning</groupId>
+      <artifactId>async-http-client</artifactId>
+    </dependency>
+    <dependency>
       <groupId>org.apache.tez</groupId>
       <artifactId>tez-api</artifactId>
     </dependency>

http://git-wip-us.apache.org/repos/asf/tez/blob/9dabf947/tez-runtime-library/src/main/java/org/apache/tez/http/BaseHttpConnection.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/http/BaseHttpConnection.java b/tez-runtime-library/src/main/java/org/apache/tez/http/BaseHttpConnection.java
new file mode 100644
index 0000000..dd642ae
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/http/BaseHttpConnection.java
@@ -0,0 +1,63 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.http;
+
+import java.io.DataInputStream;
+import java.io.IOException;
+
+public abstract class BaseHttpConnection {
+  /**
+   * Basic/unit connection timeout (in milliseconds)
+   */
+  protected final static int UNIT_CONNECT_TIMEOUT = 60 * 1000;
+
+  /**
+   * Connect to url
+   *
+   * @return boolean
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  public abstract boolean connect() throws IOException, InterruptedException;
+
+  /**
+   * Validate established connection
+   *
+   * @throws IOException
+   */
+  public abstract void validate() throws IOException;
+
+  /**
+   * Get inputstream
+   *
+   * @return DataInputStream
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  public abstract DataInputStream getInputStream() throws IOException, InterruptedException;
+
+  /**
+   * Clean up connection
+   *
+   * @param disconnect
+   * @throws IOException
+   */
+  public abstract void cleanup(boolean disconnect) throws IOException;
+
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/9dabf947/tez-runtime-library/src/main/java/org/apache/tez/http/HttpConnection.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/http/HttpConnection.java b/tez-runtime-library/src/main/java/org/apache/tez/http/HttpConnection.java
new file mode 100644
index 0000000..4732354
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/http/HttpConnection.java
@@ -0,0 +1,318 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.http;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.base.Stopwatch;
+import org.apache.hadoop.io.DataOutputBuffer;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.tez.common.security.JobTokenSecretManager;
+import org.apache.tez.runtime.library.common.security.SecureShuffleUtils;
+import org.apache.tez.runtime.library.common.shuffle.orderedgrouped.ShuffleHeader;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedInputStream;
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.HttpURLConnection;
+import java.net.URL;
+import java.util.concurrent.TimeUnit;
+
+public class HttpConnection extends BaseHttpConnection {
+
+  private static final Logger LOG = LoggerFactory.getLogger(HttpConnection.class);
+
+  private URL url;
+  private final String logIdentifier;
+
+  @VisibleForTesting
+  protected volatile HttpURLConnection connection;
+  private volatile DataInputStream input;
+  private volatile boolean connectionSucceeed;
+  private volatile boolean cleanup;
+
+  private final JobTokenSecretManager jobTokenSecretMgr;
+  private String encHash;
+  private String msgToEncode;
+
+  private final HttpConnectionParams httpConnParams;
+  private final Stopwatch stopWatch;
+
+  /**
+   * HttpConnection
+   *
+   * @param url
+   * @param connParams
+   * @param logIdentifier
+   * @param jobTokenSecretManager
+   * @throws IOException
+   */
+  public HttpConnection(URL url, HttpConnectionParams connParams,
+      String logIdentifier, JobTokenSecretManager jobTokenSecretManager) throws IOException {
+    this.logIdentifier = logIdentifier;
+    this.jobTokenSecretMgr = jobTokenSecretManager;
+    this.httpConnParams = connParams;
+    this.url = url;
+    this.stopWatch = new Stopwatch();
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("MapOutput URL :" + url.toString());
+    }
+  }
+
+  @VisibleForTesting
+  public void computeEncHash() throws IOException {
+    // generate hash of the url
+    msgToEncode = SecureShuffleUtils.buildMsgFrom(url);
+    encHash = SecureShuffleUtils.hashFromString(msgToEncode, jobTokenSecretMgr);
+  }
+
+  private void setupConnection() throws IOException {
+    connection = (HttpURLConnection) url.openConnection();
+    if (httpConnParams.isSslShuffle()) {
+      //Configure for SSL
+      SSLFactory sslFactory = httpConnParams.getSslFactory();
+      Preconditions.checkArgument(sslFactory != null, "SSLFactory can not be null");
+      sslFactory.configure(connection);
+    }
+
+    computeEncHash();
+
+    // put url hash into http header
+    connection.addRequestProperty(SecureShuffleUtils.HTTP_HEADER_URL_HASH, encHash);
+    // set the read timeout
+    connection.setReadTimeout(httpConnParams.getReadTimeout());
+    // put shuffle version into http header
+    connection.addRequestProperty(ShuffleHeader.HTTP_HEADER_NAME,
+        ShuffleHeader.DEFAULT_HTTP_HEADER_NAME);
+    connection.addRequestProperty(ShuffleHeader.HTTP_HEADER_VERSION,
+        ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION);
+  }
+
+  /**
+   * Connect to source
+   *
+   * @return true if connection was successful
+   * false if connection was previously cleaned up
+   * @throws IOException upon connection failure
+   */
+  @Override
+  public boolean connect() throws IOException {
+    return connect(httpConnParams.getConnectionTimeout());
+  }
+
+  /**
+   * Connect to source with specific timeout
+   *
+   * @param connectionTimeout
+   * @return true if connection was successful
+   * false if connection was previously cleaned up
+   * @throws IOException upon connection failure
+   */
+  private boolean connect(int connectionTimeout) throws IOException {
+    stopWatch.reset().start();
+    if (connection == null) {
+      setupConnection();
+    }
+    int unit = 0;
+    if (connectionTimeout < 0) {
+      throw new IOException("Invalid timeout " + "[timeout = " + connectionTimeout + " ms]");
+    } else if (connectionTimeout > 0) {
+      unit = Math.min(UNIT_CONNECT_TIMEOUT, connectionTimeout);
+    }
+    // set the connect timeout to the unit-connect-timeout
+    connection.setConnectTimeout(unit);
+    int connectionFailures = 0;
+    while (true) {
+      long connectStartTime = System.currentTimeMillis();
+      try {
+        connection.connect();
+        connectionSucceeed = true;
+        break;
+      } catch (IOException ioe) {
+        // Don't attempt another connect if already cleanedup.
+        connectionFailures++;
+        if (cleanup) {
+          LOG.info("Cleanup is set to true. Not attempting to"
+              + " connect again. Last exception was: ["
+              + ioe.getClass().getName() + ", " + ioe.getMessage() + "]");
+          return false;
+        }
+        // update the total remaining connect-timeout
+        connectionTimeout -= unit;
+        // throw an exception if we have waited for timeout amount of time
+        // note that the updated value if timeout is used here
+        if (connectionTimeout <= 0) {
+          throw new IOException(
+              "Failed to connect to " + url + ", #connectionFailures=" + connectionFailures, ioe);
+        }
+        long elapsed = System.currentTimeMillis() - connectStartTime;
+        if (elapsed < unit) {
+          try {
+            long sleepTime = unit - elapsed;
+            if (LOG.isDebugEnabled()) {
+              LOG.debug("Sleeping for " + sleepTime + " while establishing connection to " + url +
+                  ", since connectAttempt returned in " + elapsed + " ms");
+            }
+            Thread.sleep(sleepTime);
+          } catch (InterruptedException e) {
+            throw new IOException(
+                "Connection establishment sleep interrupted, #connectionFailures=" +
+                    connectionFailures, e);
+          }
+        }
+
+        // reset the connect timeout for the last try
+        if (connectionTimeout < unit) {
+          unit = connectionTimeout;
+          // reset the connect time out for the final connect
+          connection.setConnectTimeout(unit);
+        }
+
+      }
+    }
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Time taken to connect to " + url.toString() +
+          " " + stopWatch.elapsedTime(TimeUnit.MILLISECONDS) + " ms; connectionFailures="
+          + connectionFailures);
+    }
+    return true;
+  }
+
+  @Override
+  public void validate() throws IOException {
+    stopWatch.reset().start();
+    int rc = connection.getResponseCode();
+    if (rc != HttpURLConnection.HTTP_OK) {
+      throw new IOException("Got invalid response code " + rc + " from " + url
+          + ": " + connection.getResponseMessage());
+    }
+
+    // get the shuffle version
+    if (!ShuffleHeader.DEFAULT_HTTP_HEADER_NAME.equals(connection
+        .getHeaderField(ShuffleHeader.HTTP_HEADER_NAME))
+        || !ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION.equals(connection
+        .getHeaderField(ShuffleHeader.HTTP_HEADER_VERSION))) {
+      throw new IOException("Incompatible shuffle response version");
+    }
+
+    // get the replyHash which is HMac of the encHash we sent to the server
+    String replyHash =
+        connection
+            .getHeaderField(SecureShuffleUtils.HTTP_HEADER_REPLY_URL_HASH);
+    if (replyHash == null) {
+      throw new IOException("security validation of TT Map output failed");
+    }
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("url=" + msgToEncode + ";encHash=" + encHash + ";replyHash="
+          + replyHash);
+    }
+
+    // verify that replyHash is HMac of encHash
+    SecureShuffleUtils.verifyReply(replyHash, encHash, jobTokenSecretMgr);
+    //Following log statement will be used by tez-tool perf-analyzer for mapping attempt to NM host
+    LOG.info("for url=" + url +
+        " sent hash and receievd reply " + stopWatch.elapsedTime(TimeUnit.MILLISECONDS) + " ms");
+  }
+
+  /**
+   * Get the inputstream from the connection
+   *
+   * @return DataInputStream
+   * @throws IOException
+   */
+  @Override
+  public DataInputStream getInputStream() throws IOException {
+    stopWatch.reset().start();
+    if (connectionSucceeed) {
+      input = new DataInputStream(new BufferedInputStream(
+              connection.getInputStream(), httpConnParams.getBufferSize()));
+    }
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Time taken to getInputStream (connect) " + url +
+          " " + stopWatch.elapsedTime(TimeUnit.MILLISECONDS) + " ms");
+    }
+    return input;
+  }
+
+  /**
+   * Cleanup the connection.
+   *
+   * @param disconnect Close the connection if this is true; otherwise respect keepalive
+   * @throws IOException
+   */
+  @Override
+  public void cleanup(boolean disconnect) throws IOException {
+    cleanup = true;
+    stopWatch.reset().start();
+    try {
+      if (input != null) {
+        LOG.info("Closing input on " + logIdentifier);
+        input.close();
+        input = null;
+      }
+      if (httpConnParams.isKeepAlive() && connectionSucceeed) {
+        // Refer:
+        // http://docs.oracle.com/javase/6/docs/technotes/guides/net/http-keepalive.html
+        readErrorStream(connection.getErrorStream());
+      }
+      if (connection != null && (disconnect || !httpConnParams.isKeepAlive())) {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Closing connection on " + logIdentifier);
+        }
+        connection.disconnect();
+        connection = null;
+      }
+    } catch (IOException e) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Exception while shutting down fetcher " + logIdentifier, e);
+      } else {
+        LOG.info("Exception while shutting down fetcher " + logIdentifier
+            + ": " + e.getMessage());
+      }
+    }
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Time taken to cleanup connection to " + url +
+          " " + stopWatch.elapsedTime(TimeUnit.MILLISECONDS) + " ms");
+    }
+  }
+
+  /**
+   * Cleanup the error stream if any, for keepAlive connections
+   *
+   * @param errorStream
+   */
+  private void readErrorStream(InputStream errorStream) {
+    if (errorStream == null) {
+      return;
+    }
+    try {
+      DataOutputBuffer errorBuffer = new DataOutputBuffer();
+      IOUtils.copyBytes(errorStream, errorBuffer, 4096);
+      IOUtils.closeStream(errorBuffer);
+      IOUtils.closeStream(errorStream);
+    } catch (IOException ioe) {
+      // ignore
+    }
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/tez/blob/9dabf947/tez-runtime-library/src/main/java/org/apache/tez/http/HttpConnectionParams.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/http/HttpConnectionParams.java b/tez-runtime-library/src/main/java/org/apache/tez/http/HttpConnectionParams.java
new file mode 100644
index 0000000..aac4bb3
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/http/HttpConnectionParams.java
@@ -0,0 +1,82 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.http;
+
+public class HttpConnectionParams {
+  private final boolean keepAlive;
+  private final int keepAliveMaxConnections;
+  private final int connectionTimeout;
+  private final int readTimeout;
+  private final int bufferSize;
+
+  private final boolean sslShuffle;
+  private final SSLFactory sslFactory;
+
+  public HttpConnectionParams(boolean keepAlive, int keepAliveMaxConnections, int
+      connectionTimeout, int readTimeout, int bufferSize, boolean sslShuffle, SSLFactory
+      sslFactory) {
+    this.keepAlive = keepAlive;
+    this.keepAliveMaxConnections = keepAliveMaxConnections;
+    this.connectionTimeout = connectionTimeout;
+    this.readTimeout = readTimeout;
+    this.bufferSize = bufferSize;
+    this.sslShuffle = sslShuffle;
+    this.sslFactory = sslFactory;
+  }
+
+  public int getBufferSize() {
+    return bufferSize;
+  }
+
+  public int getConnectionTimeout() {
+    return connectionTimeout;
+  }
+
+  public boolean isKeepAlive() {
+    return keepAlive;
+  }
+
+  public int getKeepAliveMaxConnections() {
+    return keepAliveMaxConnections;
+  }
+
+  public int getReadTimeout() {
+    return readTimeout;
+  }
+
+  public boolean isSslShuffle() {
+    return sslShuffle;
+  }
+
+  public SSLFactory getSslFactory() {
+    return sslFactory;
+  }
+
+
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append("keepAlive=").append(keepAlive).append(", ");
+    sb.append("keepAliveMaxConnections=").append(keepAliveMaxConnections).append(", ");
+    sb.append("connectionTimeout=").append(connectionTimeout).append(", ");
+    sb.append("readTimeout=").append(readTimeout).append(", ");
+    sb.append("bufferSize=").append(bufferSize).append(", ");
+    sb.append("bufferSize=").append(bufferSize);
+    return sb.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/9dabf947/tez-runtime-library/src/main/java/org/apache/tez/http/SSLFactory.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/http/SSLFactory.java b/tez-runtime-library/src/main/java/org/apache/tez/http/SSLFactory.java
new file mode 100644
index 0000000..f23739b
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/http/SSLFactory.java
@@ -0,0 +1,238 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.http;
+
+import com.ning.http.client.AsyncHttpClientConfig;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.authentication.client.ConnectionConfigurator;
+import org.apache.hadoop.security.ssl.FileBasedKeyStoresFactory;
+import org.apache.hadoop.security.ssl.KeyStoresFactory;
+import org.apache.hadoop.security.ssl.SSLFactory.Mode;
+import org.apache.hadoop.security.ssl.SSLHostnameVerifier;
+import org.apache.hadoop.util.ReflectionUtils;
+
+import javax.net.ssl.HostnameVerifier;
+import javax.net.ssl.HttpsURLConnection;
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.SSLSocketFactory;
+import java.io.IOException;
+import java.net.HttpURLConnection;
+import java.security.GeneralSecurityException;
+
+import static org.apache.hadoop.security.ssl.SSLFactory.DEFAULT_SSL_ENABLED_PROTOCOLS;
+import static org.apache.hadoop.security.ssl.SSLFactory.DEFAULT_SSL_REQUIRE_CLIENT_CERT;
+import static org.apache.hadoop.security.ssl.SSLFactory.KEYSTORES_FACTORY_CLASS_KEY;
+import static org.apache.hadoop.security.ssl.SSLFactory.SSL_CLIENT_CONF_KEY;
+import static org.apache.hadoop.security.ssl.SSLFactory.SSL_ENABLED_PROTOCOLS;
+import static org.apache.hadoop.security.ssl.SSLFactory.SSL_HOSTNAME_VERIFIER_KEY;
+import static org.apache.hadoop.security.ssl.SSLFactory.SSL_REQUIRE_CLIENT_CERT_KEY;
+import static org.apache.hadoop.security.ssl.SSLFactory.SSL_SERVER_CONF_KEY;
+
+/**
+ * Factory that creates SSLEngine and SSLSocketFactory instances using
+ * Hadoop configuration information.
+ * <p/>
+ * This SSLFactory uses a {@link org.apache.hadoop.security.ssl.ReloadingX509TrustManager} instance,
+ * which reloads public keys if the truststore file changes.
+ * <p/>
+ * This factory is used to configure HTTPS in Hadoop HTTP based endpoints, both
+ * client and server.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class SSLFactory implements ConnectionConfigurator {
+
+  private Configuration conf;
+  private Mode mode;
+  private boolean requireClientCert;
+  private SSLContext context;
+  private HostnameVerifier hostnameVerifier;
+  private KeyStoresFactory keystoresFactory;
+
+  private String[] enabledProtocols = null;
+
+  /**
+   * Creates an SSLFactory.
+   *
+   * @param mode SSLFactory mode, client or server.
+   * @param conf Hadoop configuration from where the SSLFactory configuration
+   *             will be read.
+   */
+  public SSLFactory(Mode mode, Configuration conf) {
+    this.conf = conf;
+    if (mode == null) {
+      throw new IllegalArgumentException("mode cannot be NULL");
+    }
+    this.mode = mode;
+    requireClientCert = conf.getBoolean(SSL_REQUIRE_CLIENT_CERT_KEY,
+        DEFAULT_SSL_REQUIRE_CLIENT_CERT);
+    Configuration sslConf = readSSLConfiguration(mode);
+
+    Class<? extends KeyStoresFactory> klass
+        = conf.getClass(KEYSTORES_FACTORY_CLASS_KEY,
+        FileBasedKeyStoresFactory.class, KeyStoresFactory.class);
+    keystoresFactory = ReflectionUtils.newInstance(klass, sslConf);
+
+    enabledProtocols = conf.getStrings(SSL_ENABLED_PROTOCOLS, DEFAULT_SSL_ENABLED_PROTOCOLS);
+  }
+
+  private Configuration readSSLConfiguration(Mode mode) {
+    Configuration sslConf = new Configuration(false);
+    sslConf.setBoolean(SSL_REQUIRE_CLIENT_CERT_KEY, requireClientCert);
+    String sslConfResource;
+    if (mode == Mode.CLIENT) {
+      sslConfResource = conf.get(SSL_CLIENT_CONF_KEY, "ssl-client.xml");
+    } else {
+      sslConfResource = conf.get(SSL_SERVER_CONF_KEY, "ssl-server.xml");
+    }
+    sslConf.addResource(sslConfResource);
+    return sslConf;
+  }
+
+  /**
+   * Initializes the factory.
+   *
+   * @throws GeneralSecurityException thrown if an SSL initialization error
+   *                                  happened.
+   * @throws IOException              thrown if an IO error happened while reading the SSL
+   *                                  configuration.
+   */
+  public void init() throws GeneralSecurityException, IOException {
+    keystoresFactory.init(mode);
+    context = SSLContext.getInstance("TLS");
+    context.init(keystoresFactory.getKeyManagers(),
+        keystoresFactory.getTrustManagers(), null);
+    context.getDefaultSSLParameters().setProtocols(enabledProtocols);
+    hostnameVerifier = getHostnameVerifier(conf);
+  }
+
+  private HostnameVerifier getHostnameVerifier(Configuration conf)
+      throws GeneralSecurityException, IOException {
+    return getHostnameVerifier(conf.get(SSL_HOSTNAME_VERIFIER_KEY, "DEFAULT").
+        trim().toUpperCase());
+  }
+
+  public static HostnameVerifier getHostnameVerifier(String verifier)
+      throws GeneralSecurityException, IOException {
+    HostnameVerifier hostnameVerifier;
+    if (verifier.equals("DEFAULT")) {
+      hostnameVerifier = SSLHostnameVerifier.DEFAULT;
+    } else if (verifier.equals("DEFAULT_AND_LOCALHOST")) {
+      hostnameVerifier = SSLHostnameVerifier.DEFAULT_AND_LOCALHOST;
+    } else if (verifier.equals("STRICT")) {
+      hostnameVerifier = SSLHostnameVerifier.STRICT;
+    } else if (verifier.equals("STRICT_IE6")) {
+      hostnameVerifier = SSLHostnameVerifier.STRICT_IE6;
+    } else if (verifier.equals("ALLOW_ALL")) {
+      hostnameVerifier = SSLHostnameVerifier.ALLOW_ALL;
+    } else {
+      throw new GeneralSecurityException("Invalid hostname verifier: " +
+          verifier);
+    }
+    return hostnameVerifier;
+  }
+
+  /**
+   * Releases any resources being used.
+   */
+  public void destroy() {
+    keystoresFactory.destroy();
+  }
+
+  /**
+   * Returns the SSLFactory KeyStoresFactory instance.
+   *
+   * @return the SSLFactory KeyStoresFactory instance.
+   */
+  public KeyStoresFactory getKeystoresFactory() {
+    return keystoresFactory;
+  }
+
+
+  /**
+   * Returns a configured SSLSocketFactory.
+   *
+   * @return the configured SSLSocketFactory.
+   * @throws GeneralSecurityException thrown if the SSLSocketFactory could not
+   *                                  be initialized.
+   * @throws IOException              thrown if and IO error occurred while loading
+   *                                  the server keystore.
+   */
+  public SSLSocketFactory createSSLSocketFactory() throws GeneralSecurityException, IOException {
+    if (mode != Mode.CLIENT) {
+      throw new IllegalStateException("Factory is in CLIENT mode");
+    }
+    return context.getSocketFactory();
+  }
+
+  /**
+   * Returns the hostname verifier it should be used in HttpsURLConnections.
+   *
+   * @return the hostname verifier.
+   */
+  public HostnameVerifier getHostnameVerifier() {
+    if (mode != Mode.CLIENT) {
+      throw new IllegalStateException("Factory is in CLIENT mode");
+    }
+    return hostnameVerifier;
+  }
+
+
+
+  /**
+   * If the given {@link HttpURLConnection} is an {@link HttpsURLConnection}
+   * configures the connection with the {@link SSLSocketFactory} and
+   * {@link HostnameVerifier} of this SSLFactory, otherwise does nothing.
+   *
+   * @param conn the {@link HttpURLConnection} instance to configure.
+   * @return the configured {@link HttpURLConnection} instance.
+   * @throws IOException if an IO error occurred.
+   */
+  @Override
+  public HttpURLConnection configure(HttpURLConnection conn) throws IOException {
+    if (conn instanceof HttpsURLConnection) {
+      HttpsURLConnection sslConn = (HttpsURLConnection) conn;
+      try {
+        sslConn.setSSLSocketFactory(createSSLSocketFactory());
+      } catch (GeneralSecurityException ex) {
+        throw new IOException(ex);
+      }
+      sslConn.setHostnameVerifier(getHostnameVerifier());
+      conn = sslConn;
+    }
+    return conn;
+  }
+
+  /**
+   * Set ssl context for {@link com.ning.http.client.AsyncHttpClientConfig.Builder}
+   *
+   * @param asyncNingBuilder {@link com.ning.http.client.AsyncHttpClientConfig.Builder} instance to
+   *                configure.
+   * @throws IOException if an IO error occurred.
+   */
+  public void configure(AsyncHttpClientConfig.Builder asyncNingBuilder) throws IOException {
+    if (asyncNingBuilder != null) {
+      asyncNingBuilder.setSSLContext(context);
+      asyncNingBuilder.setHostnameVerifier(getHostnameVerifier());
+    }
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tez/blob/9dabf947/tez-runtime-library/src/main/java/org/apache/tez/http/async/netty/AsyncHttpConnection.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/http/async/netty/AsyncHttpConnection.java b/tez-runtime-library/src/main/java/org/apache/tez/http/async/netty/AsyncHttpConnection.java
new file mode 100644
index 0000000..f46939d
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/http/async/netty/AsyncHttpConnection.java
@@ -0,0 +1,231 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.http.async.netty;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.base.Stopwatch;
+import com.ning.http.client.AsyncHttpClient;
+import com.ning.http.client.AsyncHttpClientConfig;
+import com.ning.http.client.ListenableFuture;
+import com.ning.http.client.Request;
+import com.ning.http.client.RequestBuilder;
+import com.ning.http.client.Response;
+import org.apache.commons.io.IOUtils;
+import org.apache.tez.http.BaseHttpConnection;
+import org.apache.tez.http.HttpConnectionParams;
+import org.apache.tez.http.SSLFactory;
+import org.apache.tez.common.security.JobTokenSecretManager;
+import org.apache.tez.runtime.library.api.TezRuntimeConfiguration;
+import org.apache.tez.runtime.library.common.security.SecureShuffleUtils;
+import org.apache.tez.runtime.library.common.shuffle.orderedgrouped.ShuffleHeader;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.io.PipedInputStream;
+import java.io.PipedOutputStream;
+import java.net.HttpURLConnection;
+import java.net.URL;
+
+public class AsyncHttpConnection extends BaseHttpConnection {
+
+  private static final Logger LOG = LoggerFactory.getLogger(AsyncHttpConnection.class);
+
+  private final JobTokenSecretManager jobTokenSecretMgr;
+  private String encHash;
+  private String msgToEncode;
+
+  private final HttpConnectionParams httpConnParams;
+  private final Stopwatch stopWatch;
+  private final URL url;
+
+  private static volatile AsyncHttpClient httpAsyncClient;
+
+  private final TezBodyDeferringAsyncHandler handler;
+  private final PipedOutputStream pos; //handler would write to this as and when it receives chunks
+  private final PipedInputStream pis; //connected to pos, which can be used by fetchers
+
+  private Response response;
+  private ListenableFuture<Response> responseFuture;
+  private TezBodyDeferringAsyncHandler.BodyDeferringInputStream dis;
+
+  private void initClient(HttpConnectionParams httpConnParams) throws IOException {
+    if (httpAsyncClient != null) {
+      return;
+    }
+
+    if (httpAsyncClient == null) {
+      synchronized (AsyncHttpConnection.class) {
+        if (httpAsyncClient == null) {
+          LOG.info("Initializing AsyncClient (TezBodyDeferringAsyncHandler)");
+          AsyncHttpClientConfig.Builder builder = new AsyncHttpClientConfig.Builder();
+          if (httpConnParams.isSslShuffle()) {
+            //Configure SSL
+            SSLFactory sslFactory = httpConnParams.getSslFactory();
+            Preconditions.checkArgument(sslFactory != null, "SSLFactory can not be null");
+            sslFactory.configure(builder);
+          }
+
+          /**
+           * TODO : following settings need fine tuning.
+           * Change following config to accept common thread pool later.
+           * Change max connections based on the total inputs (ordered & unordered). Need to tune
+           * setMaxConnections & addRequestFilter.
+           */
+          builder
+              .setAllowPoolingConnection(httpConnParams.isKeepAlive())
+              .setAllowSslConnectionPool(httpConnParams.isKeepAlive())
+              .setCompressionEnabled(false)
+              //.setExecutorService(applicationThreadPool)
+              //.addRequestFilter(new ThrottleRequestFilter())
+              .setMaximumConnectionsPerHost(1)
+              .setConnectionTimeoutInMs(httpConnParams.getConnectionTimeout())
+              .setRequestTimeoutInMs(httpConnParams.getReadTimeout())
+              .setUseRawUrl(true)
+              .build();
+            httpAsyncClient = new AsyncHttpClient(builder.build());
+        }
+      }
+    }
+  }
+
+  public AsyncHttpConnection(URL url, HttpConnectionParams connParams,
+      String logIdentifier, JobTokenSecretManager jobTokenSecretManager) throws IOException {
+    this.jobTokenSecretMgr = jobTokenSecretManager;
+    this.httpConnParams = connParams;
+    this.url = url;
+    this.stopWatch = new Stopwatch();
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("MapOutput URL :" + url.toString());
+    }
+
+    initClient(httpConnParams);
+    pos = new PipedOutputStream();
+    pis = new PipedInputStream(pos, httpConnParams.getBufferSize());
+    handler = new TezBodyDeferringAsyncHandler(pos, url, UNIT_CONNECT_TIMEOUT);
+  }
+
+  @VisibleForTesting
+  public void computeEncHash() throws IOException {
+    // generate hash of the url
+    msgToEncode = SecureShuffleUtils.buildMsgFrom(url);
+    encHash = SecureShuffleUtils.hashFromString(msgToEncode, jobTokenSecretMgr);
+  }
+
+  /**
+   * Connect to source
+   *
+   * @return true if connection was successful
+   * false if connection was previously cleaned up
+   * @throws IOException upon connection failure
+   */
+  public boolean connect() throws IOException, InterruptedException {
+    computeEncHash();
+
+    RequestBuilder rb = new RequestBuilder();
+    rb.setHeader(SecureShuffleUtils.HTTP_HEADER_URL_HASH, encHash);
+    rb.setHeader(ShuffleHeader.HTTP_HEADER_NAME, ShuffleHeader.DEFAULT_HTTP_HEADER_NAME);
+    rb.setHeader(ShuffleHeader.HTTP_HEADER_VERSION, ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION);
+    Request request = rb.setUrl(url.toString()).build();
+
+    //for debugging
+    LOG.debug("Request url={}, encHash={}, id={}", url, encHash);
+
+    try {
+      //Blocks calling thread until it receives headers, but have the option to defer response body
+      responseFuture = httpAsyncClient.executeRequest(request, handler);
+
+      //BodyDeferringAsyncHandler would automatically manage producer and consumer frequency mismatch
+      dis = new TezBodyDeferringAsyncHandler.BodyDeferringInputStream(responseFuture, handler, pis);
+
+      response = dis.getAsapResponse();
+      if (response == null) {
+        throw new IOException("Response is null");
+      }
+    } catch(IOException e) {
+      throw e;
+    }
+
+    //verify the response
+    int rc = response.getStatusCode();
+    if (rc != HttpURLConnection.HTTP_OK) {
+      LOG.debug("Request url={}, id={}", response.getUri());
+      throw new IOException("Got invalid response code " + rc + " from "
+          + url + ": " + response.getStatusText());
+    }
+    return true;
+  }
+
+  public void validate() throws IOException {
+    // get the shuffle version
+    if (!ShuffleHeader.DEFAULT_HTTP_HEADER_NAME
+        .equals(response.getHeader(ShuffleHeader.HTTP_HEADER_NAME))
+        || !ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION
+        .equals(response.getHeader(ShuffleHeader.HTTP_HEADER_VERSION))) {
+      throw new IOException("Incompatible shuffle response version");
+    }
+
+    // get the replyHash which is HMac of the encHash we sent to the server
+    String replyHash = response.getHeader(SecureShuffleUtils.HTTP_HEADER_REPLY_URL_HASH);
+    if (replyHash == null) {
+      throw new IOException("security validation of TT Map output failed");
+    }
+    LOG.debug("url={};encHash={};replyHash={}", msgToEncode, encHash, replyHash);
+
+    // verify that replyHash is HMac of encHash
+    SecureShuffleUtils.verifyReply(replyHash, encHash, jobTokenSecretMgr);
+    //Following log statement will be used by tez-tool perf-analyzer for mapping attempt to NM host
+    LOG.info("for url={} sent hash and receievd reply {} ms", url, stopWatch.elapsedMillis());
+  }
+
+  /**
+   * Get the inputstream from the connection
+   *
+   * @return DataInputStream
+   * @throws IOException
+   */
+  public DataInputStream getInputStream() throws IOException, InterruptedException {
+    Preconditions.checkState(response != null, "Response can not be null");
+    return new DataInputStream(dis);
+  }
+
+  @VisibleForTesting
+  public void close() {
+    httpAsyncClient.close();
+    httpAsyncClient = null;
+  }
+  /**
+   * Cleanup the connection.
+   *
+   * @param disconnect
+   * @throws IOException
+   */
+  public void cleanup(boolean disconnect) throws IOException {
+    // Netty internally has its own connection management and takes care of it.
+    if (response != null) {
+      dis.close();
+    }
+    IOUtils.closeQuietly(pos);
+    IOUtils.closeQuietly(pis);
+    response = null;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/9dabf947/tez-runtime-library/src/main/java/org/apache/tez/http/async/netty/TezBodyDeferringAsyncHandler.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/http/async/netty/TezBodyDeferringAsyncHandler.java b/tez-runtime-library/src/main/java/org/apache/tez/http/async/netty/TezBodyDeferringAsyncHandler.java
new file mode 100644
index 0000000..8e83eac
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/http/async/netty/TezBodyDeferringAsyncHandler.java
@@ -0,0 +1,256 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.tez.http.async.netty;
+
+import com.ning.http.client.AsyncHandler;
+import com.ning.http.client.HttpResponseBodyPart;
+import com.ning.http.client.HttpResponseHeaders;
+import com.ning.http.client.HttpResponseStatus;
+import com.ning.http.client.Response;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.FilterInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.net.URL;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Same as {@link com.ning.http.client.BodyDeferringAsyncHandler} with additional checks handle
+ * errors in getResponse(). Based on testing, at very high load {@link com.ning.http.client
+ * .BodyDeferringAsyncHandler} gets to hung state in getResponse() as it tries to wait
+ * indefinitely for headers to arrive.  This class tries to fix the problem by waiting only for
+ * the connection timeout.
+ */
+@InterfaceAudience.Private
+class TezBodyDeferringAsyncHandler implements AsyncHandler<Response> {
+  private static final Logger LOG = LoggerFactory.getLogger(TezBodyDeferringAsyncHandler.class);
+
+  private final Response.ResponseBuilder responseBuilder = new Response.ResponseBuilder();
+  private final CountDownLatch headersArrived = new CountDownLatch(1);
+  private final OutputStream output;
+
+  private volatile boolean responseSet;
+  private volatile boolean statusReceived;
+  private volatile Response response;
+  private volatile Throwable throwable;
+
+  private final Semaphore semaphore = new Semaphore(1);
+
+  private final URL url;
+  private final int headerReceiveTimeout;
+
+  TezBodyDeferringAsyncHandler(final OutputStream os, final URL url, final int timeout) {
+    this.output = os;
+    this.responseSet = false;
+    this.url = url;
+    this.headerReceiveTimeout = timeout;
+  }
+
+  public void onThrowable(Throwable t) {
+    this.throwable = t;
+    // Counting down to handle error cases too.
+    // In "premature exceptions" cases, the onBodyPartReceived() and
+    // onCompleted()
+    // methods will never be invoked, leaving caller of getResponse() method
+    // blocked forever.
+    try {
+      semaphore.acquire();
+    } catch (InterruptedException e) {
+      // Ignore
+    } finally {
+      LOG.error("Error in asyncHandler ", t);
+      headersArrived.countDown();
+      semaphore.release();
+    }
+    try {
+      closeOut();
+    } catch (IOException e) {
+      // ignore
+    }
+  }
+
+  public AsyncHandler.STATE onStatusReceived(HttpResponseStatus responseStatus) throws Exception {
+    responseBuilder.reset();
+    responseBuilder.accumulate(responseStatus);
+    statusReceived = true;
+    return AsyncHandler.STATE.CONTINUE;
+  }
+
+  public AsyncHandler.STATE onHeadersReceived(HttpResponseHeaders headers) throws Exception {
+    responseBuilder.accumulate(headers);
+    return AsyncHandler.STATE.CONTINUE;
+  }
+
+  public AsyncHandler.STATE onBodyPartReceived(HttpResponseBodyPart bodyPart) throws Exception {
+    // body arrived, flush headers
+    if (!responseSet) {
+      response = responseBuilder.build();
+      responseSet = true;
+      headersArrived.countDown();
+    }
+    bodyPart.writeTo(output);
+    return AsyncHandler.STATE.CONTINUE;
+  }
+
+  protected void closeOut() throws IOException {
+    try {
+      output.flush();
+    } finally {
+      output.close();
+    }
+  }
+
+  public Response onCompleted() throws IOException {
+    if (!responseSet) {
+      response = responseBuilder.build();
+      responseSet = true;
+    }
+    // Counting down to handle error cases too.
+    // In "normal" cases, latch is already at 0 here
+    // But in other cases, for example when because of some error
+    // onBodyPartReceived() is never called, the caller
+    // of getResponse() would remain blocked infinitely.
+    // By contract, onCompleted() is always invoked, even in case of errors
+    headersArrived.countDown();
+    closeOut();
+    try {
+      semaphore.acquire();
+      if (throwable != null) {
+        IOException ioe = new IOException(throwable.getMessage());
+        ioe.initCause(throwable);
+        throw ioe;
+      } else {
+        // sending out current response
+        return responseBuilder.build();
+      }
+    } catch (InterruptedException e) {
+      return null;
+    } finally {
+      semaphore.release();
+    }
+  }
+
+  /**
+   * This method -- unlike Future<Reponse>.get() -- will block only as long,
+   * as headers arrive. This is useful for large transfers, to examine headers
+   * ASAP, and defer body streaming to it's fine destination and prevent
+   * unneeded bandwidth consumption. The response here will contain the very
+   * 1st response from server, so status code and headers, but it might be
+   * incomplete in case of broken servers sending trailing headers. In that
+   * case, the "usual" Future<Response>.get() method will return complete
+   * headers, but multiple invocations of getResponse() will always return the
+   * 1st cached, probably incomplete one. Note: the response returned by this
+   * method will contain everything <em>except</em> the response body itself,
+   * so invoking any method like Response.getResponseBodyXXX() will result in
+   * error! Also, please not that this method might return <code>null</code>
+   * in case of some errors.
+   *
+   * @return a {@link Response}
+   * @throws InterruptedException
+   */
+  public Response getResponse() throws InterruptedException, IOException {
+    /**
+     * Based on testing, it is possible that it is in connected state, but the headers are not
+     * received. Instead of waiting forever, close after timeout for next retry.
+     */
+    boolean result = headersArrived.await(headerReceiveTimeout, TimeUnit.MILLISECONDS);
+    if (!result) {
+      LOG.error("Breaking after timeout={}, url={}, responseSet={} statusReceived={}",
+          headerReceiveTimeout, url, responseSet, statusReceived);
+      return null;
+    }
+    try {
+      semaphore.acquire();
+      if (throwable != null) {
+        IOException ioe = new IOException(throwable.getMessage());
+        ioe.initCause(throwable);
+        throw ioe;
+      } else {
+        return response;
+      }
+    } finally {
+      semaphore.release();
+    }
+  }
+
+  /**
+   * A simple helper class that is used to perform automatic "join" for async
+   * download and the error checking of the Future of the request.
+   */
+  static class BodyDeferringInputStream extends FilterInputStream {
+    private final Future<Response> future;
+    private final TezBodyDeferringAsyncHandler bdah;
+
+    public BodyDeferringInputStream(final Future<Response> future,
+        final TezBodyDeferringAsyncHandler bdah, final InputStream in) {
+      super(in);
+      this.future = future;
+      this.bdah = bdah;
+    }
+
+    /**
+     * Closes the input stream, and "joins" (wait for complete execution
+     * together with potential exception thrown) of the async request.
+     */
+    public void close() throws IOException {
+      // close
+      super.close();
+      // "join" async request
+      try {
+        getLastResponse();
+      } catch (Exception e) {
+        IOException ioe = new IOException(e.getMessage());
+        ioe.initCause(e);
+        throw ioe;
+      }
+    }
+
+    /**
+     * Delegates to {@link TezBodyDeferringAsyncHandler#getResponse()}. Will
+     * blocks as long as headers arrives only. Might return
+     * <code>null</code>. See
+     * {@link TezBodyDeferringAsyncHandler#getResponse()} method for details.
+     *
+     * @return a {@link Response}
+     * @throws InterruptedException
+     */
+    public Response getAsapResponse() throws InterruptedException, IOException {
+      return bdah.getResponse();
+    }
+
+    /**
+     * Delegates to <code>Future<Response>#get()</code> method. Will block
+     * as long as complete response arrives.
+     *
+     * @return a {@link Response}
+     * @throws InterruptedException
+     * @throws java.util.concurrent.ExecutionException
+     */
+    public Response getLastResponse() throws InterruptedException, ExecutionException {
+      return future.get();
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tez/blob/9dabf947/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/TezRuntimeConfiguration.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/TezRuntimeConfiguration.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/TezRuntimeConfiguration.java
index 3d9a701..fc94347 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/TezRuntimeConfiguration.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/TezRuntimeConfiguration.java
@@ -210,6 +210,9 @@ public class TezRuntimeConfiguration {
   public final static int TEZ_RUNTIME_SHUFFLE_BUFFER_SIZE_DEFAULT =
       8 * 1024;
 
+  public static final String TEZ_RUNTIME_SHUFFLE_USE_ASYNC_HTTP = TEZ_RUNTIME_PREFIX +
+      "shuffle.use.async.http";
+  public static final boolean TEZ_RUNTIME_SHUFFLE_USE_ASYNC_HTTP_DEFAULT = false;
 
   public static final String TEZ_RUNTIME_SHUFFLE_ENABLE_SSL = TEZ_RUNTIME_PREFIX +
       "shuffle.ssl.enable";
@@ -352,6 +355,7 @@ public class TezRuntimeConfiguration {
     tezRuntimeKeys.add(TEZ_RUNTIME_UNORDERED_OUTPUT_MAX_PER_BUFFER_SIZE_BYTES);
     tezRuntimeKeys.add(TEZ_RUNTIME_PARTITIONER_CLASS);
     tezRuntimeKeys.add(TEZ_RUNTIME_COMBINER_CLASS);
+    tezRuntimeKeys.add(TEZ_RUNTIME_SHUFFLE_USE_ASYNC_HTTP);
     tezRuntimeKeys.add(TEZ_RUNTIME_SHUFFLE_PARALLEL_COPIES);
     tezRuntimeKeys.add(TEZ_RUNTIME_SHUFFLE_FETCH_FAILURES_LIMIT);
     tezRuntimeKeys.add(TEZ_RUNTIME_SHUFFLE_FETCH_MAX_TASK_OUTPUT_AT_ONCE);

http://git-wip-us.apache.org/repos/asf/tez/blob/9dabf947/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 61e0151..e7c98b7 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
@@ -38,6 +38,8 @@ import java.util.concurrent.atomic.AtomicInteger;
 
 import com.google.common.annotations.VisibleForTesting;
 
+import org.apache.tez.http.BaseHttpConnection;
+import org.apache.tez.http.HttpConnectionParams;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.commons.lang.StringUtils;
@@ -59,7 +61,6 @@ import org.apache.tez.runtime.library.common.sort.impl.TezIndexRecord;
 import org.apache.tez.runtime.library.common.sort.impl.TezSpillRecord;
 import org.apache.tez.runtime.library.exceptions.FetcherReadTimeoutException;
 import org.apache.tez.runtime.library.common.shuffle.FetchedInput.Type;
-import org.apache.tez.runtime.library.common.shuffle.HttpConnection.HttpConnectionParams;
 
 import com.google.common.base.Preconditions;
 
@@ -108,7 +109,7 @@ public class Fetcher extends CallableWithNdc<FetchResult> {
   private URL url;
   private volatile DataInputStream input;
   
-  private HttpConnection httpConnection;
+  BaseHttpConnection httpConnection;
   private HttpConnectionParams httpConnectionParams;
 
   private final boolean localDiskFetchEnabled;
@@ -121,6 +122,8 @@ public class Fetcher extends CallableWithNdc<FetchResult> {
   // Initiative value is 0, which means it hasn't retried yet.
   private long retryStartTime = 0;
 
+  private final boolean asyncHttp;
+
   private final boolean isDebugEnabled = LOG.isDebugEnabled();
 
   private Fetcher(FetcherCallback fetcherCallback, HttpConnectionParams params,
@@ -132,7 +135,8 @@ public class Fetcher extends CallableWithNdc<FetchResult> {
       boolean localDiskFetchEnabled,
       boolean sharedFetchEnabled,
       String localHostname,
-      int shufflePort) {
+      int shufflePort, boolean asyncHttp) {
+    this.asyncHttp = asyncHttp;
     this.fetcherCallback = fetcherCallback;
     this.inputManager = inputManager;
     this.jobTokenSecretMgr = jobTokenSecretManager;
@@ -402,13 +406,17 @@ public class Fetcher extends CallableWithNdc<FetchResult> {
   private HostFetchResult setupConnection(List<InputAttemptIdentifier> attempts) {
     try {
       StringBuilder baseURI = ShuffleUtils.constructBaseURIForShuffleHandler(host,
-          port, partition, appId.toString(), httpConnectionParams.isSSLShuffleEnabled());
+          port, partition, appId.toString(), httpConnectionParams.isSslShuffle());
       this.url = ShuffleUtils.constructInputURL(baseURI.toString(), attempts,
-          httpConnectionParams.getKeepAlive());
+          httpConnectionParams.isKeepAlive());
 
-      httpConnection = new HttpConnection(url, httpConnectionParams, logIdentifier, jobTokenSecretMgr);
+      httpConnection = ShuffleUtils.getHttpConnection(asyncHttp, url, httpConnectionParams,
+          logIdentifier, jobTokenSecretMgr);
       httpConnection.connect();
-    } catch (IOException e) {
+    } catch (IOException | InterruptedException e) {
+      if (e instanceof InterruptedException) {
+        Thread.currentThread().interrupt();
+      }
       // ioErrs.increment(1);
       // If connect did not succeed, just mark all the maps as failed,
       // indirectly penalizing the host
@@ -449,6 +457,9 @@ public class Fetcher extends CallableWithNdc<FetchResult> {
         return new HostFetchResult(new FetchResult(host, port, partition, remaining),
             new InputAttemptIdentifier[] { firstAttempt }, false);
       }
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt(); //reset status
+      return null;
     }
     return null;
   }
@@ -903,10 +914,11 @@ public class Fetcher extends CallableWithNdc<FetchResult> {
     public FetcherBuilder(FetcherCallback fetcherCallback,
         HttpConnectionParams params, FetchedInputAllocator inputManager,
         ApplicationId appId, JobTokenSecretManager jobTokenSecretMgr, String srcNameTrimmed,
-        Configuration conf, boolean localDiskFetchEnabled, String localHostname, int shufflePort) {
+        Configuration conf, boolean localDiskFetchEnabled, String localHostname, int shufflePort,
+        boolean asyncHttp) {
       this.fetcher = new Fetcher(fetcherCallback, params, inputManager, appId,
           jobTokenSecretMgr, srcNameTrimmed, conf, null, null, null, localDiskFetchEnabled,
-          false, localHostname, shufflePort);
+          false, localHostname, shufflePort, asyncHttp);
     }
 
     public FetcherBuilder(FetcherCallback fetcherCallback,
@@ -915,10 +927,10 @@ public class Fetcher extends CallableWithNdc<FetchResult> {
         Configuration conf, RawLocalFileSystem localFs,
         LocalDirAllocator localDirAllocator, Path lockPath,
         boolean localDiskFetchEnabled, boolean sharedFetchEnabled,
-        String localHostname, int shufflePort) {
+        String localHostname, int shufflePort, boolean asyncHttp) {
       this.fetcher = new Fetcher(fetcherCallback, params, inputManager, appId,
           jobTokenSecretMgr, srcNameTrimmed, conf, localFs, localDirAllocator,
-          lockPath, localDiskFetchEnabled, sharedFetchEnabled, localHostname, shufflePort);
+          lockPath, localDiskFetchEnabled, sharedFetchEnabled, localHostname, shufflePort, asyncHttp);
     }
 
     public FetcherBuilder setHttpConnectionParameters(HttpConnectionParams httpParams) {

http://git-wip-us.apache.org/repos/asf/tez/blob/9dabf947/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/HttpConnection.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/HttpConnection.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/HttpConnection.java
deleted file mode 100644
index 7827f0a..0000000
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/HttpConnection.java
+++ /dev/null
@@ -1,428 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.runtime.library.common.shuffle;
-
-import java.io.BufferedInputStream;
-import java.io.DataInputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.net.HttpURLConnection;
-import java.net.URL;
-import java.security.GeneralSecurityException;
-import java.util.concurrent.TimeUnit;
-
-import javax.net.ssl.HttpsURLConnection;
-
-import com.google.common.annotations.VisibleForTesting;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.DataOutputBuffer;
-import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.security.ssl.SSLFactory;
-import org.apache.tez.common.security.JobTokenSecretManager;
-import org.apache.tez.runtime.library.common.security.SecureShuffleUtils;
-import org.apache.tez.runtime.library.common.shuffle.orderedgrouped.ShuffleHeader;
-
-import com.google.common.base.Stopwatch;
-
-/**
- * HttpConnection which can be used for Unordered / Ordered shuffle.
- */
-public class HttpConnection {
-
-  private static final Logger LOG = LoggerFactory.getLogger(HttpConnection.class);
-
-  /** Basic/unit connection timeout (in milliseconds) */
-  private final static int UNIT_CONNECT_TIMEOUT = 60 * 1000;
-
-  private URL url;
-  private final String logIdentifier;
-
-  //Shared by many threads
-  private static SSLFactory sslFactory;
-
-  @VisibleForTesting
-  protected volatile HttpURLConnection connection;
-  private volatile DataInputStream input;
-
-  private volatile boolean connectionSucceeed;
-  private volatile boolean cleanup;
-
-  private final JobTokenSecretManager jobTokenSecretMgr;
-  private String encHash;
-  private String msgToEncode;
-
-  private final HttpConnectionParams httpConnParams;
-  private final Stopwatch stopWatch;
-
-  /**
-   * HttpConnection
-   * 
-   * @param url
-   * @param connParams
-   * @param logIdentifier
-   * @param jobTokenSecretManager
-   * @throws IOException
-   */
-  public HttpConnection(URL url, HttpConnectionParams connParams,
-      String logIdentifier, JobTokenSecretManager jobTokenSecretManager) throws IOException {
-    this.logIdentifier = logIdentifier;
-    this.jobTokenSecretMgr = jobTokenSecretManager;
-    this.httpConnParams = connParams;
-    this.url = url;
-    this.stopWatch = new Stopwatch();
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("MapOutput URL :" + url.toString());
-    }
-  }
-
-  private void setupConnection() throws IOException {
-    connection = (HttpURLConnection) url.openConnection();
-    if (sslFactory != null && httpConnParams.sslShuffle) {
-      try {
-        ((HttpsURLConnection) connection).setSSLSocketFactory(sslFactory
-          .createSSLSocketFactory());
-        ((HttpsURLConnection) connection).setHostnameVerifier(sslFactory
-          .getHostnameVerifier());
-      } catch (GeneralSecurityException ex) {
-        throw new IOException(ex);
-      }
-    }
-    // generate hash of the url
-    msgToEncode = SecureShuffleUtils.buildMsgFrom(url);
-    encHash = SecureShuffleUtils.hashFromString(msgToEncode, jobTokenSecretMgr);
-
-    // put url hash into http header
-    connection.addRequestProperty(SecureShuffleUtils.HTTP_HEADER_URL_HASH,
-      encHash);
-    // set the read timeout
-    connection.setReadTimeout(httpConnParams.readTimeout);
-    // put shuffle version into http header
-    connection.addRequestProperty(ShuffleHeader.HTTP_HEADER_NAME,
-      ShuffleHeader.DEFAULT_HTTP_HEADER_NAME);
-    connection.addRequestProperty(ShuffleHeader.HTTP_HEADER_VERSION,
-      ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION);
-  }
-
-  /**
-   * Connect to source
-   * 
-   * @return true if connection was successful
-   *         false if connection was previously cleaned up
-   * @throws IOException upon connection failure
-   */
-  public boolean connect() throws IOException {
-    return connect(httpConnParams.connectionTimeout);
-  }
-
-  /**
-   * Connect to source with specific timeout
-   * 
-   * @param connectionTimeout
-   * @return true if connection was successful
-   *         false if connection was previously cleaned up
-   * @throws IOException upon connection failure
-   */
-  public boolean connect(int connectionTimeout) throws IOException {
-    stopWatch.reset().start();
-    if (connection == null) {
-      setupConnection();
-    }
-    int unit = 0;
-    if (connectionTimeout < 0) {
-      throw new IOException("Invalid timeout " + "[timeout = "
-          + connectionTimeout + " ms]");
-    } else if (connectionTimeout > 0) {
-      unit = Math.min(UNIT_CONNECT_TIMEOUT, connectionTimeout);
-    }
-    // set the connect timeout to the unit-connect-timeout
-    connection.setConnectTimeout(unit);
-    int connectionFailures = 0;
-    while (true) {
-      long connectStartTime = System.currentTimeMillis();
-      try {
-        connection.connect();
-        connectionSucceeed = true;
-        break;
-      } catch (IOException ioe) {
-        // Don't attempt another connect if already cleanedup.
-        connectionFailures++;
-        if (cleanup) {
-          LOG.info("Cleanup is set to true. Not attempting to"
-              + " connect again. Last exception was: ["
-              + ioe.getClass().getName() + ", " + ioe.getMessage() + "]");
-          return false;
-        }
-        // update the total remaining connect-timeout
-        connectionTimeout -= unit;
-        // throw an exception if we have waited for timeout amount of time
-        // note that the updated value if timeout is used here
-        if (connectionTimeout <= 0) {
-          throw new IOException(
-              "Failed to connect to " + url + ", #connectionFailures=" + connectionFailures, ioe);
-        }
-        long elapsed = System.currentTimeMillis() - connectStartTime;
-        if (elapsed < unit) {
-          try {
-            long sleepTime = unit - elapsed;
-            if (LOG.isDebugEnabled()) {
-              LOG.debug("Sleeping for " + sleepTime + " while establishing connection to " + url +
-                  ", since connectAttempt returned in " + elapsed + " ms");
-            }
-            Thread.sleep(sleepTime);
-          } catch (InterruptedException e) {
-            throw new IOException(
-                "Connection establishment sleep interrupted, #connectionFailures=" +
-                    connectionFailures, e);
-          }
-        }
-
-        // reset the connect timeout for the last try
-        if (connectionTimeout < unit) {
-          unit = connectionTimeout;
-          // reset the connect time out for the final connect
-          connection.setConnectTimeout(unit);
-        }
-
-      }
-    }
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Time taken to connect to " + url.toString() +
-        " " + stopWatch.elapsedTime(TimeUnit.MILLISECONDS) + " ms; connectionFailures="+ connectionFailures);
-    }
-    return true;
-  }
-
-  public void validate() throws IOException {
-    stopWatch.reset().start();
-    int rc = connection.getResponseCode();
-    if (rc != HttpURLConnection.HTTP_OK) {
-      throw new IOException("Got invalid response code " + rc + " from " + url
-          + ": " + connection.getResponseMessage());
-    }
-    // get the shuffle version
-    if (!ShuffleHeader.DEFAULT_HTTP_HEADER_NAME.equals(connection
-      .getHeaderField(ShuffleHeader.HTTP_HEADER_NAME))
-        || !ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION.equals(connection
-          .getHeaderField(ShuffleHeader.HTTP_HEADER_VERSION))) {
-      throw new IOException("Incompatible shuffle response version");
-    }
-    // get the replyHash which is HMac of the encHash we sent to the server
-    String replyHash =
-        connection
-          .getHeaderField(SecureShuffleUtils.HTTP_HEADER_REPLY_URL_HASH);
-    if (replyHash == null) {
-      throw new IOException("security validation of TT Map output failed");
-    }
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("url=" + msgToEncode + ";encHash=" + encHash + ";replyHash="
-          + replyHash);
-    }
-    // verify that replyHash is HMac of encHash
-    SecureShuffleUtils.verifyReply(replyHash, encHash, jobTokenSecretMgr);
-    //Following log statement will be used by tez-tool perf-analyzer for mapping attempt to NM host
-    LOG.info("for url=" + url +
-      " sent hash and receievd reply " + stopWatch.elapsedTime(TimeUnit.MILLISECONDS) + " ms");
-  }
-
-  /**
-   * Get the inputstream from the connection
-   * 
-   * @return DataInputStream
-   * @throws IOException
-   */
-  public DataInputStream getInputStream() throws IOException {
-    stopWatch.reset().start();
-    if (connectionSucceeed) {
-      input =
-          new DataInputStream(new BufferedInputStream(
-            connection.getInputStream(), httpConnParams.bufferSize));
-    }
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Time taken to getInputStream (connect) " + url +
-        " " + stopWatch.elapsedTime(TimeUnit.MILLISECONDS) + " ms");
-    }
-    return input;
-  }
-
-  /**
-   * Cleanup the connection.
-   * 
-   * @param disconnect
-   *          Close the connection if this is true; otherwise respect keepalive
-   * @throws IOException
-   */
-  public void cleanup(boolean disconnect) throws IOException {
-    cleanup = true;
-    stopWatch.reset().start();
-    try {
-      if (input != null) {
-        LOG.info("Closing input on " + logIdentifier);
-        input.close();
-        input = null;
-      }
-      if (httpConnParams.keepAlive && connectionSucceeed) {
-        // Refer:
-        // http://docs.oracle.com/javase/6/docs/technotes/guides/net/http-keepalive.html
-        readErrorStream(connection.getErrorStream());
-      }
-      if (connection != null && (disconnect || !httpConnParams.keepAlive)) {
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Closing connection on " + logIdentifier);
-        }
-        connection.disconnect();
-        connection = null;
-      }
-    } catch (IOException e) {
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Exception while shutting down fetcher " + logIdentifier, e);
-      } else {
-        LOG.info("Exception while shutting down fetcher " + logIdentifier
-            + ": " + e.getMessage());
-      }
-    }
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Time taken to cleanup connection to " + url +
-        " " + stopWatch.elapsedTime(TimeUnit.MILLISECONDS) + " ms");
-    }
-  }
-
-  /**
-   * Cleanup the error stream if any, for keepAlive connections
-   * 
-   * @param errorStream
-   */
-  private void readErrorStream(InputStream errorStream) {
-    if (errorStream == null) {
-      return;
-    }
-    try {
-      DataOutputBuffer errorBuffer = new DataOutputBuffer();
-      IOUtils.copyBytes(errorStream, errorBuffer, 4096);
-      IOUtils.closeStream(errorBuffer);
-      IOUtils.closeStream(errorStream);
-    } catch (IOException ioe) {
-      // ignore
-    }
-  }
-
-  public static class HttpConnectionParams {
-    private boolean keepAlive;
-    private int keepAliveMaxConnections;
-    private int connectionTimeout;
-    private int readTimeout;
-    private int bufferSize;
-    private boolean sslShuffle;
-
-    public boolean getKeepAlive() {
-      return keepAlive;
-    }
-
-    public int getKeepAliveMaxConnections() {
-      return keepAliveMaxConnections;
-    }
-
-    public int getConnectionTimeout() {
-      return connectionTimeout;
-    }
-
-    public int getReadTimeout() {
-      return readTimeout;
-    }
-
-    public void setReadTimeout(int readTimeout) {
-      this.readTimeout = readTimeout;
-    }
-
-    public int getBufferSize() {
-      return bufferSize;
-    }
-
-    public boolean isSSLShuffleEnabled() {
-      return sslShuffle;
-    }
-
-    public String toString() {
-      StringBuilder sb = new StringBuilder();
-      sb.append("keepAlive=").append(keepAlive).append(", ");
-      sb.append("keepAliveMaxConnections=").append(keepAliveMaxConnections).append(", ");
-      sb.append("connectionTimeout=").append(connectionTimeout).append(", ");
-      sb.append("readTimeout=").append(readTimeout).append(", ");
-      sb.append("bufferSize=").append(bufferSize).append(", ");
-      sb.append("sslShuffle=").append(sslShuffle);
-      return sb.toString();
-    }
-  }
-
-  public static class HttpConnectionParamsBuilder {
-    private HttpConnectionParams params;
-
-    public HttpConnectionParamsBuilder() {
-      params = new HttpConnectionParams();
-    }
-
-    public HttpConnectionParamsBuilder setKeepAlive(boolean keepAlive,
-        int keepAliveMaxConnections) {
-      params.keepAlive = keepAlive;
-      params.keepAliveMaxConnections = keepAliveMaxConnections;
-      return this;
-    }
-
-    public HttpConnectionParamsBuilder setTimeout(int connectionTimeout,
-        int readTimeout) {
-      params.connectionTimeout = connectionTimeout;
-      params.readTimeout = readTimeout;
-      return this;
-    }
-
-    public synchronized HttpConnectionParamsBuilder setSSL(boolean sslEnabled,
-        Configuration conf) {
-      synchronized (HttpConnectionParamsBuilder.class) {
-        params.sslShuffle = sslEnabled;
-        if (sslEnabled) {
-          //Create sslFactory if it is null or if it was destroyed earlier
-          if (sslFactory == null || sslFactory.getKeystoresFactory()
-              .getTrustManagers() == null) {
-            LOG.info("Initializing SSL factory in HttpConnection");
-            sslFactory = new SSLFactory(SSLFactory.Mode.CLIENT, conf);
-            try {
-              sslFactory.init();
-            } catch (Exception ex) {
-              sslFactory.destroy();
-              sslFactory = null;
-              throw new RuntimeException(ex);
-            }
-          }
-        }
-      }
-      return this;
-    }
-
-    public HttpConnectionParamsBuilder setBufferSize(int bufferSize) {
-      params.bufferSize = bufferSize;
-      return this;
-    }
-
-    public HttpConnectionParams build() {
-      return params;
-    }
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/tez/blob/9dabf947/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/ShuffleUtils.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/ShuffleUtils.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/ShuffleUtils.java
index 46489ed..8b6e847 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/ShuffleUtils.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/ShuffleUtils.java
@@ -33,13 +33,16 @@ import javax.crypto.SecretKey;
 
 import com.google.common.base.Preconditions;
 import com.google.protobuf.ByteString;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.tez.http.BaseHttpConnection;
+import org.apache.tez.http.HttpConnection;
+import org.apache.tez.http.HttpConnectionParams;
+import org.apache.tez.http.SSLFactory;
+import org.apache.tez.http.async.netty.AsyncHttpConnection;
 import org.apache.tez.runtime.api.events.DataMovementEvent;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.DataInputByteBuffer;
-import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.compress.CompressionCodec;
 import org.apache.hadoop.security.token.Token;
 import org.apache.tez.common.TezCommonUtils;
@@ -54,8 +57,6 @@ import org.apache.tez.runtime.api.events.VertexManagerEvent;
 import org.apache.tez.runtime.library.api.TezRuntimeConfiguration;
 import org.apache.tez.runtime.library.common.InputAttemptIdentifier;
 import org.apache.tez.runtime.library.common.sort.impl.IFile;
-import org.apache.tez.runtime.library.common.shuffle.HttpConnection.HttpConnectionParams;
-import org.apache.tez.runtime.library.common.shuffle.HttpConnection.HttpConnectionParamsBuilder;
 import org.apache.tez.runtime.library.common.sort.impl.TezIndexRecord;
 import org.apache.tez.runtime.library.common.sort.impl.TezSpillRecord;
 import org.apache.tez.runtime.library.shuffle.impl.ShuffleUserPayloads;
@@ -66,6 +67,9 @@ public class ShuffleUtils {
   private static final Logger LOG = LoggerFactory.getLogger(ShuffleUtils.class);
   public static final String SHUFFLE_HANDLER_SERVICE_ID = "mapreduce_shuffle";
 
+  //Shared by multiple threads
+  private static volatile SSLFactory sslFactory;
+
   static final ThreadLocal<DecimalFormat> MBPS_FORMAT =
       new ThreadLocal<DecimalFormat>() {
         @Override
@@ -213,45 +217,15 @@ public class ShuffleUtils {
     return new URL(url.toString());
   }
 
-  public static HttpConnectionParams constructHttpShuffleConnectionParams(
-      Configuration conf) {
-    HttpConnectionParamsBuilder builder = new HttpConnectionParamsBuilder();
-
-    int connectionTimeout =
-        conf.getInt(TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_CONNECT_TIMEOUT,
-          TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_STALLED_COPY_TIMEOUT_DEFAULT);
-
-    int readTimeout =
-        conf.getInt(TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_READ_TIMEOUT,
-          TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_READ_TIMEOUT_DEFAULT);
-
-    int bufferSize =
-        conf.getInt(TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_BUFFER_SIZE,
-          TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_BUFFER_SIZE_DEFAULT);
-
-    boolean keepAlive =
-        conf.getBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_KEEP_ALIVE_ENABLED,
-          TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_KEEP_ALIVE_ENABLED_DEFAULT);
-    int keepAliveMaxConnections =
-        conf.getInt(
-          TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_KEEP_ALIVE_MAX_CONNECTIONS,
-          TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_KEEP_ALIVE_MAX_CONNECTIONS_DEFAULT);
-    if (keepAlive) {
-      System.setProperty("sun.net.http.errorstream.enableBuffering", "true");
-      System.setProperty("http.maxConnections",
-        String.valueOf(keepAliveMaxConnections));
-      LOG.info("Set keepAlive max connections: " + keepAliveMaxConnections);
+  public static BaseHttpConnection getHttpConnection(boolean asyncHttp, URL url,
+      HttpConnectionParams params, String logIdentifier, JobTokenSecretManager jobTokenSecretManager)
+      throws IOException {
+    if (asyncHttp) {
+      //TODO: support other async packages? httpclient-async?
+      return new AsyncHttpConnection(url, params, logIdentifier, jobTokenSecretManager);
+    } else {
+      return new HttpConnection(url, params, logIdentifier, jobTokenSecretManager);
     }
-
-    builder.setTimeout(connectionTimeout, readTimeout)
-        .setBufferSize(bufferSize)
-        .setKeepAlive(keepAlive, keepAliveMaxConnections);
-
-    boolean sslShuffle = conf.getBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_ENABLE_SSL,
-      TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_ENABLE_SSL_DEFAULT);
-    builder.setSSL(sslShuffle, conf);
-
-    return builder.build();
   }
 
   public static String stringify(DataMovementEventPayloadProto dmProto) {
@@ -473,5 +447,62 @@ public class ShuffleUtils {
             ", EndTime=" + System.currentTimeMillis() + ", TimeTaken=" + millis + ", Rate=" +
             MBPS_FORMAT.get().format(rate) + " MB/s");
   }
+
+  /**
+   * Build {@link org.apache.tez.http.HttpConnectionParams} from configuration
+   *
+   * @param conf
+   * @return HttpConnectionParams
+   */
+  public static HttpConnectionParams getHttpConnectionParams(Configuration conf) {
+    int connectionTimeout =
+        conf.getInt(TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_CONNECT_TIMEOUT,
+            TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_STALLED_COPY_TIMEOUT_DEFAULT);
+
+    int readTimeout = conf.getInt(TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_READ_TIMEOUT,
+            TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_READ_TIMEOUT_DEFAULT);
+
+    int bufferSize = conf.getInt(TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_BUFFER_SIZE,
+            TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_BUFFER_SIZE_DEFAULT);
+
+    boolean keepAlive = conf.getBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_KEEP_ALIVE_ENABLED,
+            TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_KEEP_ALIVE_ENABLED_DEFAULT);
+
+    int keepAliveMaxConnections = conf.getInt(
+            TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_KEEP_ALIVE_MAX_CONNECTIONS,
+            TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_KEEP_ALIVE_MAX_CONNECTIONS_DEFAULT);
+
+    if (keepAlive) {
+      System.setProperty("sun.net.http.errorstream.enableBuffering", "true");
+      System.setProperty("http.maxConnections", String.valueOf(keepAliveMaxConnections));
+    }
+
+    boolean sslShuffle = conf.getBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_ENABLE_SSL,
+        TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_ENABLE_SSL_DEFAULT);
+
+    if (sslShuffle) {
+      if (sslFactory == null) {
+        synchronized (HttpConnectionParams.class) {
+          //Create sslFactory if it is null or if it was destroyed earlier
+          if (sslFactory == null || sslFactory.getKeystoresFactory().getTrustManagers() == null) {
+            sslFactory =
+                new SSLFactory(org.apache.hadoop.security.ssl.SSLFactory.Mode.CLIENT, conf);
+            try {
+              sslFactory.init();
+            } catch (Exception ex) {
+              sslFactory.destroy();
+              sslFactory = null;
+              throw new RuntimeException(ex);
+            }
+          }
+        }
+      }
+    }
+
+    HttpConnectionParams httpConnParams = new HttpConnectionParams(keepAlive,
+        keepAliveMaxConnections, connectionTimeout, readTimeout, bufferSize, sslShuffle,
+        sslFactory);
+    return httpConnParams;
+  }
 }
 

http://git-wip-us.apache.org/repos/asf/tez/blob/9dabf947/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 f354920..b7c0742 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
@@ -26,7 +26,6 @@ import java.text.DecimalFormat;
 import java.util.Arrays;
 import java.util.BitSet;
 import java.util.Collections;
-import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
@@ -46,7 +45,7 @@ import java.util.concurrent.locks.ReentrantLock;
 import javax.crypto.SecretKey;
 
 import com.google.common.annotations.VisibleForTesting;
-import com.google.common.collect.Maps;
+import org.apache.tez.http.HttpConnectionParams;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -55,7 +54,6 @@ import org.apache.hadoop.fs.LocalDirAllocator;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.RawLocalFileSystem;
 import org.apache.hadoop.io.compress.CompressionCodec;
-import org.apache.hadoop.net.NetUtils;
 import org.apache.tez.common.CallableWithNdc;
 import org.apache.tez.common.TezRuntimeFrameworkConfigs;
 import org.apache.tez.common.TezUtilsInternal;
@@ -76,7 +74,6 @@ import org.apache.tez.runtime.library.common.shuffle.FetchedInput;
 import org.apache.tez.runtime.library.common.shuffle.FetchedInputAllocator;
 import org.apache.tez.runtime.library.common.shuffle.Fetcher;
 import org.apache.tez.runtime.library.common.shuffle.FetcherCallback;
-import org.apache.tez.runtime.library.common.shuffle.HttpConnection.HttpConnectionParams;
 import org.apache.tez.runtime.library.common.shuffle.InputHost;
 import org.apache.tez.runtime.library.common.shuffle.ShuffleUtils;
 import org.apache.tez.runtime.library.common.shuffle.FetchedInput.Type;
@@ -132,6 +129,7 @@ public class ShuffleManager implements FetcherCallback {
   private final Condition wakeLoop = lock.newCondition();
   
   private final int numFetchers;
+  private final boolean asyncHttp;
   
   // Parameters required by Fetchers
   private final JobTokenSecretManager jobTokenSecretMgr;
@@ -241,8 +239,8 @@ public class ShuffleManager implements FetcherCallback {
         .getJobTokenSecretFromTokenBytes(inputContext
             .getServiceConsumerMetaData(TezConstants.TEZ_SHUFFLE_HANDLER_SERVICE_ID));
     this.jobTokenSecretMgr = new JobTokenSecretManager(shuffleSecret);
-    httpConnectionParams =
-        ShuffleUtils.constructHttpShuffleConnectionParams(conf);
+    this.asyncHttp = conf.getBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_USE_ASYNC_HTTP, false);
+    httpConnectionParams = ShuffleUtils.getHttpConnectionParams(conf);
 
     this.localFs = (RawLocalFileSystem) FileSystem.getLocal(conf).getRaw();
 
@@ -398,7 +396,7 @@ public class ShuffleManager implements FetcherCallback {
       httpConnectionParams, inputManager, inputContext.getApplicationId(),
         jobTokenSecretMgr, srcNameTrimmed, conf, localFs, localDirAllocator,
         lockDisk, localDiskFetchEnabled, sharedFetchEnabled,
-        localhostName, shufflePort);
+        localhostName, shufflePort, asyncHttp);
 
     if (codec != null) {
       fetcherBuilder.setCompressionParameters(codec);


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

Branch: refs/heads/TEZ-2003
Commit: 6a0a452bc443cb13ab93c009d4170628ea7968f1
Parents: 2fb86f3
Author: Siddharth Seth <ss...@apache.org>
Authored: Mon Feb 23 21:59:39 2015 -0800
Committer: Siddharth Seth <ss...@apache.org>
Committed: Mon Jun 1 14:09:27 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/6a0a452b/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/6a0a452b/docs/pom.xml
----------------------------------------------------------------------
diff --git a/docs/pom.xml b/docs/pom.xml
index 49168ec..2adb896 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/6a0a452b/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 4f1cc8b..4ab4063 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/6a0a452b/tez-api/pom.xml
----------------------------------------------------------------------
diff --git a/tez-api/pom.xml b/tez-api/pom.xml
index 0a5d570..46e8dd3 100644
--- a/tez-api/pom.xml
+++ b/tez-api/pom.xml
@@ -20,7 +20,7 @@
   <parent>
     <groupId>org.apache.tez</groupId>
     <artifactId>tez</artifactId>
-    <version>0.8.0-SNAPSHOT</version>
+    <version>0.8.0-TEZ-2003-SNAPSHOT</version>
   </parent>
   <artifactId>tez-api</artifactId>
 

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

http://git-wip-us.apache.org/repos/asf/tez/blob/6a0a452b/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/6a0a452b/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/6a0a452b/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/6a0a452b/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/6a0a452b/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/6a0a452b/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/6a0a452b/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/6a0a452b/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/6a0a452b/tez-runtime-library/pom.xml
----------------------------------------------------------------------
diff --git a/tez-runtime-library/pom.xml b/tez-runtime-library/pom.xml
index 4433a02..03fb020 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/6a0a452b/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/6a0a452b/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>


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

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

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

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


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

Branch: refs/heads/TEZ-2003
Commit: b5814c2284691dcee74b47665ffb9c00b7317d06
Parents: ae6d780
Author: Siddharth Seth <ss...@apache.org>
Authored: Tue Apr 7 13:13:30 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Mon Jun 1 14:09:27 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/b5814c22/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/b5814c22/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestContainerReuse.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestContainerReuse.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestContainerReuse.java
index d45346a..8b4ff07 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestContainerReuse.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestContainerReuse.java
@@ -1242,9 +1242,9 @@ public class TestContainerReuse {
     assertEquals(1, assignEvent.getRemoteTaskLocalResources().size());
 
     // Task assigned to container completed successfully. Container should be re-used.
-    taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta111, container1.getId(), TaskAttemptState.SUCCEEDED));
+    taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta111, container1.getId(), TaskAttemptState.SUCCEEDED, null, 0));
     drainableAppCallback.drain();
-    verify(taskScheduler).deallocateTask(eq(ta111), eq(true));
+    verify(taskScheduler).deallocateTask(eq(ta111), eq(true), eq((TaskAttemptEndReason)null));
     verify(taskSchedulerEventHandler).taskAllocated(eq(ta112), any(Object.class), eq(container1));
     verify(rmClient, times(0)).releaseAssignedContainer(eq(container1.getId()));
     eventHandler.verifyNoInvocations(AMContainerEventStopRequest.class);
@@ -1254,9 +1254,9 @@ public class TestContainerReuse {
 
     // Task assigned to container completed successfully.
     // Verify reuse across hosts.
-    taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta112, container1.getId(), TaskAttemptState.SUCCEEDED));
+    taskSchedulerEventHandler.handleEvent(new AMSchedulerEventTAEnded(ta112, container1.getId(), TaskAttemptState.SUCCEEDED, null, 0));
     drainableAppCallback.drain();
-    verify(taskScheduler).deallocateTask(eq(ta112), eq(true));
+    verify(taskScheduler).deallocateTask(eq(ta112), eq(true), eq((TaskAttemptEndReason)null));
     verify(rmClient, times(0)).releaseAssignedContainer(eq(container1.getId()));
     eventHandler.verifyNoInvocations(AMContainerEventStopRequest.class);
     eventHandler.reset();

http://git-wip-us.apache.org/repos/asf/tez/blob/b5814c22/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,


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

Branch: refs/heads/TEZ-2003
Commit: c61850b8d54b3160905dcae2f233cbd52c27021f
Parents: e646fc3
Author: Siddharth Seth <ss...@apache.org>
Authored: Fri May 22 11:03:11 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Mon Jun 1 14:26:41 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/c61850b8/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/c61850b8/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());
+      }
     }
   }
   


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

Branch: refs/heads/TEZ-2003
Commit: 57c44a8477a270266121eaf7013019f5ea56507a
Parents: a9c19b6
Author: Siddharth Seth <ss...@apache.org>
Authored: Mon Apr 20 13:17:31 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Mon Jun 1 14:10:27 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/57c44a84/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/57c44a84/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/57c44a84/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/57c44a84/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/57c44a84/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/57c44a84/tez-dag/src/main/java/org/apache/tez/dag/app/dag/TaskAttempt.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/TaskAttempt.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/TaskAttempt.java
index 6c85cc2..26613e9 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/TaskAttempt.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/TaskAttempt.java
@@ -122,6 +122,12 @@ public interface TaskAttempt {
    */
   long getLaunchTime();
 
+  /**
+   * Get the time at which this attempt was scheduled
+   * @return the time at which this attempt was scheduled, 0 if it hasn't been scheduled yet
+   */
+  long getScheduleTime();
+
   /** 
    * @return attempt's finish time. If attempt is not finished
    *  yet, returns 0.

http://git-wip-us.apache.org/repos/asf/tez/blob/57c44a84/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 d5c4485..3881e2b 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
@@ -696,6 +696,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/57c44a84/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 efc4b04..dcfac64 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;
@@ -675,6 +676,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 {
@@ -1033,6 +1044,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/57c44a84/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 b2eb81e..29b67f0 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
@@ -1518,7 +1518,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


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

Branch: refs/heads/TEZ-2003
Commit: c092a7ca3459e93307afbb43530c896294ea0d09
Parents: 0114142
Author: Siddharth Seth <ss...@apache.org>
Authored: Tue Apr 7 13:22:09 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Mon Jun 1 14:09:27 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/c092a7ca/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/c092a7ca/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/c092a7ca/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 0ee448b..cd39cc8 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
@@ -259,6 +259,16 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
   }
 
   @Override
+  public void taskAlive(TezTaskAttemptID taskAttemptId) {
+    taskHeartbeatHandler.pinged(taskAttemptId);
+  }
+
+  @Override
+  public void containerAlive(ContainerId containerId) {
+    pingContainerHeartbeatHandler(containerId);
+  }
+
+  @Override
   public void taskStartedRemotely(TezTaskAttemptID taskAttemptID, ContainerId containerId) {
     context.getEventHandler()
         .handle(new TaskAttemptEventStartedRemotely(taskAttemptID, containerId, null));

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


[13/50] [abbrv] tez git commit: Move TEZ-2478 into 0.7.1 release section

Posted by ss...@apache.org.
Move TEZ-2478 into 0.7.1 release section


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

Branch: refs/heads/TEZ-2003
Commit: 7bc35050530206d9c4b6281c4a486fab1a1d730c
Parents: 413c3cc
Author: Siddharth Seth <ss...@apache.org>
Authored: Fri May 29 13:53:28 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Fri May 29 13:53:28 2015 -0700

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


http://git-wip-us.apache.org/repos/asf/tez/blob/7bc35050/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 4e5f0cf..bbe4fc8 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -8,7 +8,6 @@ INCOMPATIBLE CHANGES
   TEZ-2468. Change the minimum Java version to Java 7.
 
 ALL CHANGES:
-  TEZ-2478. Move OneToOne routing to store events in Tasks.
   TEZ-2503. findbugs version isn't reported properly in test-patch report.
   TEZ-2198. Fix sorter spill counts.
   TEZ-1883. Change findbugs version to 3.x.
@@ -31,6 +30,7 @@ Release 0.7.1: Unreleased
 INCOMPATIBLE CHANGES
 
 ALL CHANGES:
+  TEZ-2478. Move OneToOne routing to store events in Tasks.
   TEZ-2482. Tez UI: Mouse events not working on IE11
   TEZ-1529. ATS and TezClient integration in secure kerberos enabled cluster.
   TEZ-2481. Tez UI: graphical view does not render properly on IE11


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

Branch: refs/heads/TEZ-2003
Commit: fe2b8a97237f9529700cd7577d54ecd961e47090
Parents: 54af7bc
Author: Siddharth Seth <ss...@apache.org>
Authored: Fri May 8 18:43:16 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Mon Jun 1 14:21:53 2015 -0700

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


http://git-wip-us.apache.org/repos/asf/tez/blob/fe2b8a97/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/fe2b8a97/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/fe2b8a97/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 d30919b..1182d54 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);
+  private final TaskHeartbeatResponse RESPONSE_SHOULD_DIE = new TaskHeartbeatResponse(true, null, 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);
-    List<TezEvent> outEvents = null;
+    TaskAttemptEventInfo eventInfo = new TaskAttemptEventInfo(0, null);
     TezTaskAttemptID taskAttemptID = request.getTaskAttemptId();
     if (taskAttemptID != null) {
       ContainerId containerIdFromMap = registeredAttempts.get(taskAttemptID);
@@ -217,12 +217,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);
         }
@@ -233,14 +238,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);
@@ -436,8 +440,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/fe2b8a97/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 825a4d2..34c8822 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
@@ -362,13 +362,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/fe2b8a97/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 be7016f..63f8039 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
@@ -44,6 +44,7 @@ import org.apache.tez.common.ContainerContext;
 import org.apache.tez.common.ContainerTask;
 import org.apache.tez.common.security.JobTokenSecretManager;
 import org.apache.tez.dag.api.TaskHeartbeatRequest;
+import org.apache.tez.dag.api.TaskHeartbeatResponse;
 import org.apache.tez.dag.api.TezException;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.tez.common.TezTaskUmbilicalProtocol;
@@ -61,14 +62,11 @@ import org.apache.tez.dag.records.TezTaskAttemptID;
 import org.apache.tez.dag.records.TezTaskID;
 import org.apache.tez.dag.records.TezVertexID;
 import org.apache.tez.runtime.api.events.DataMovementEvent;
-import org.apache.tez.runtime.api.events.InputInitializerEvent;
 import org.apache.tez.runtime.api.events.TaskAttemptCompletedEvent;
 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;
@@ -252,14 +250,13 @@ 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());
   }
 
-  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);
@@ -274,7 +271,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/fe2b8a97/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 7bd7414..5de15d7 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.List;
 
@@ -39,11 +38,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.sort.impl.TezIndexRecord;
@@ -88,7 +84,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());
@@ -104,7 +100,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());
 


[12/50] [abbrv] tez git commit: TEZ-2478. Move OneToOne routing to store events in Tasks. (Contributed by Bikas Saha and Siddharth Seth)

Posted by ss...@apache.org.
TEZ-2478. Move OneToOne routing to store events in Tasks. (Contributed by Bikas Saha and Siddharth Seth)


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

Branch: refs/heads/TEZ-2003
Commit: 413c3cc7991e810d6b49539d71ecc930e838238e
Parents: fafa804
Author: Siddharth Seth <ss...@apache.org>
Authored: Fri May 29 13:48:31 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Fri May 29 13:48:31 2015 -0700

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../apache/tez/dag/api/TezConfiguration.java    |   7 +
 .../apache/tez/dag/app/dag/impl/DAGImpl.java    |   4 +-
 .../org/apache/tez/dag/app/dag/impl/Edge.java   |  14 +-
 .../dag/app/dag/impl/OneToOneEdgeManager.java   |  52 +------
 .../dag/impl/OneToOneEdgeManagerOnDemand.java   | 137 +++++++++++++++++++
 .../apache/tez/dag/app/dag/impl/TestEdge.java   |  41 +++++-
 .../tez/dag/app/dag/impl/TestVertexImpl.java    |   2 +-
 .../tez/test/TestExceptionPropagation.java      |   7 +-
 9 files changed, 204 insertions(+), 61 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/413c3cc7/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 6fb6051..4e5f0cf 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -8,6 +8,7 @@ INCOMPATIBLE CHANGES
   TEZ-2468. Change the minimum Java version to Java 7.
 
 ALL CHANGES:
+  TEZ-2478. Move OneToOne routing to store events in Tasks.
   TEZ-2503. findbugs version isn't reported properly in test-patch report.
   TEZ-2198. Fix sorter spill counts.
   TEZ-1883. Change findbugs version to 3.x.

http://git-wip-us.apache.org/repos/asf/tez/blob/413c3cc7/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 99c7c9d..15b1333 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
@@ -322,6 +322,13 @@ public class TezConfiguration extends Configuration {
       "cancel.delegation.tokens.on.completion";
   public static final boolean TEZ_CANCEL_DELEGATION_TOKENS_ON_COMPLETION_DEFAULT = true;
 
+  @Private
+  @Unstable
+  @ConfigurationScope(Scope.DAG)
+  public static final String TEZ_AM_ONE_TO_ONE_ROUTING_USE_ON_DEMAND_ROUTING =
+      TEZ_AM_PREFIX + "one-to-one.routing.use.on-demand-routing";
+  public static final boolean TEZ_AM_ONE_TO_ONE_ROUTING_USE_ON_DEMAND_ROUTING_DEFAULT = false;
+
   /**
    * Int value. The number of threads used to listen to task heartbeat requests.
    * Expert level setting.

http://git-wip-us.apache.org/repos/asf/tez/blob/413c3cc7/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 38da302..640cd7d 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
@@ -191,7 +191,7 @@ public class DAGImpl implements org.apache.tez.dag.app.dag.DAG,
   TezCounters fullCounters = null;
   private Set<TezVertexID> reRunningVertices = new HashSet<TezVertexID>();
 
-  public final Configuration dagConf;
+  private final Configuration dagConf;
   private final DAGPlan jobPlan;
   
   Map<String, LocalResource> localResources;
@@ -1478,7 +1478,7 @@ public class DAGImpl implements org.apache.tez.dag.app.dag.DAG,
 
       // edge manager may be also set via API when using custom edge type
       dag.edges.put(edgePlan.getId(),
-          new Edge(edgeProperty, dag.getEventHandler()));
+          new Edge(edgeProperty, dag.getEventHandler(), dagConf));
     }
   }
 

http://git-wip-us.apache.org/repos/asf/tez/blob/413c3cc7/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/Edge.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/Edge.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/Edge.java
index f9cbede..ddccf8d 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/Edge.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/Edge.java
@@ -24,6 +24,8 @@ import java.util.Map;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.atomic.AtomicBoolean;
 
+import org.apache.hadoop.conf.Configuration;
+import org.apache.tez.dag.api.TezConfiguration;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.yarn.event.EventHandler;
@@ -104,7 +106,8 @@ public class Edge {
   EdgeManagerPlugin edgeManager;
   private boolean onDemandRouting = false;
   @SuppressWarnings("rawtypes")
-  private EventHandler eventHandler;
+  private final EventHandler eventHandler;
+  private final Configuration conf;
   private AtomicBoolean bufferEvents = new AtomicBoolean(false);
   private List<TezEvent> destinationEventBuffer = new ArrayList<TezEvent>();
   private List<TezEvent> sourceEventBuffer = new ArrayList<TezEvent>();
@@ -116,9 +119,10 @@ public class Edge {
       .newConcurrentMap();
 
   @SuppressWarnings("rawtypes")
-  public Edge(EdgeProperty edgeProperty, EventHandler eventHandler) {
+  public Edge(EdgeProperty edgeProperty, EventHandler eventHandler, Configuration conf) {
     this.edgeProperty = edgeProperty;
     this.eventHandler = eventHandler;
+    this.conf = conf;
     createEdgeManager();
   }
 
@@ -126,7 +130,11 @@ public class Edge {
     switch (edgeProperty.getDataMovementType()) {
       case ONE_TO_ONE:
         edgeManagerContext = new EdgeManagerPluginContextImpl(null);
-        edgeManager = new OneToOneEdgeManager(edgeManagerContext);
+        if (conf.getBoolean(TezConfiguration.TEZ_AM_ONE_TO_ONE_ROUTING_USE_ON_DEMAND_ROUTING, TezConfiguration.TEZ_AM_ONE_TO_ONE_ROUTING_USE_ON_DEMAND_ROUTING_DEFAULT)) {
+          edgeManager = new OneToOneEdgeManagerOnDemand(edgeManagerContext);
+        } else {
+          edgeManager = new OneToOneEdgeManager(edgeManagerContext);
+        }
         break;
       case BROADCAST:
         edgeManagerContext = new EdgeManagerPluginContextImpl(null);

http://git-wip-us.apache.org/repos/asf/tez/blob/413c3cc7/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/OneToOneEdgeManager.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/OneToOneEdgeManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/OneToOneEdgeManager.java
index 6053806..dd38180 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/OneToOneEdgeManager.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/OneToOneEdgeManager.java
@@ -23,23 +23,18 @@ import java.util.List;
 import java.util.Map;
 import java.util.concurrent.atomic.AtomicBoolean;
 
-import javax.annotation.Nullable;
-
+import org.apache.tez.dag.api.EdgeManagerPlugin;
 import org.apache.tez.dag.api.EdgeManagerPluginContext;
-import org.apache.tez.dag.api.EdgeManagerPluginOnDemand;
 import org.apache.tez.runtime.api.events.DataMovementEvent;
 import org.apache.tez.runtime.api.events.InputReadErrorEvent;
 
 import com.google.common.base.Preconditions;
 
-public class OneToOneEdgeManager extends EdgeManagerPluginOnDemand {
+public class OneToOneEdgeManager extends EdgeManagerPlugin {
 
-  List<Integer> destinationInputIndices = 
+  final List<Integer> destinationInputIndices =
       Collections.unmodifiableList(Collections.singletonList(0));
-  AtomicBoolean stateChecked = new AtomicBoolean(false);
- 
-  final EventRouteMetadata commonRouteMeta = 
-      EventRouteMetadata.create(1, new int[]{0}, new int[]{0});
+  final AtomicBoolean stateChecked = new AtomicBoolean(false);
 
   public OneToOneEdgeManager(EdgeManagerPluginContext context) {
     super(context);
@@ -69,37 +64,6 @@ public class OneToOneEdgeManager extends EdgeManagerPluginOnDemand {
   }
   
   @Override
-  public void prepareForRouting() throws Exception {
-    checkState();
-  }
-  
-  @Override
-  public EventRouteMetadata routeDataMovementEventToDestination(
-      int sourceTaskIndex, int sourceOutputIndex, int destinationTaskIndex)
-      throws Exception {
-    if (sourceTaskIndex == destinationTaskIndex) {
-      return commonRouteMeta;
-    }
-    return null;
-  }
-  
-  @Override
-  public @Nullable EventRouteMetadata routeCompositeDataMovementEventToDestination(
-      int sourceTaskIndex, int destinationTaskIndex)
-      throws Exception {
-    if (sourceTaskIndex == destinationTaskIndex) {
-      return commonRouteMeta;
-    }
-    return null;
-  }
-
-  @Override
-  public EventRouteMetadata routeInputSourceTaskFailedEventToDestination(
-      int sourceTaskIndex, int destinationTaskIndex) throws Exception {
-    return commonRouteMeta;
-  }
-
-  @Override
   public void routeInputSourceTaskFailedEventToDestination(int sourceTaskIndex,
       Map<Integer, List<Integer>> destinationTaskAndInputIndices) {
     destinationTaskAndInputIndices.put(sourceTaskIndex, destinationInputIndices);
@@ -112,15 +76,10 @@ public class OneToOneEdgeManager extends EdgeManagerPluginOnDemand {
   }
   
   @Override
-  public int routeInputErrorEventToSource(int destinationTaskIndex, int destinationFailedInputIndex) {
-    return destinationTaskIndex;
-  }
-
-  @Override
   public int getNumDestinationConsumerTasks(int sourceTaskIndex) {
     return 1;
   }
-  
+
   private void checkState() {
     if (stateChecked.get()) {
       return;
@@ -133,5 +92,4 @@ public class OneToOneEdgeManager extends EdgeManagerPluginOnDemand {
         + getContext().getSourceVertexName() + " tasks: " + getContext().getSourceVertexNumTasks());
     stateChecked.set(true);
   }
-
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/413c3cc7/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/OneToOneEdgeManagerOnDemand.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/OneToOneEdgeManagerOnDemand.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/OneToOneEdgeManagerOnDemand.java
new file mode 100644
index 0000000..84e7e66
--- /dev/null
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/OneToOneEdgeManagerOnDemand.java
@@ -0,0 +1,137 @@
+/**
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+*     http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.tez.dag.app.dag.impl;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import javax.annotation.Nullable;
+
+import org.apache.tez.dag.api.EdgeManagerPluginContext;
+import org.apache.tez.dag.api.EdgeManagerPluginOnDemand;
+import org.apache.tez.runtime.api.events.DataMovementEvent;
+import org.apache.tez.runtime.api.events.InputReadErrorEvent;
+
+import com.google.common.base.Preconditions;
+
+public class OneToOneEdgeManagerOnDemand extends EdgeManagerPluginOnDemand {
+
+  final List<Integer> destinationInputIndices =
+      Collections.unmodifiableList(Collections.singletonList(0));
+  final AtomicBoolean stateChecked = new AtomicBoolean(false);
+ 
+  final EventRouteMetadata commonRouteMeta = 
+      EventRouteMetadata.create(1, new int[]{0}, new int[]{0});
+
+  public OneToOneEdgeManagerOnDemand(EdgeManagerPluginContext context) {
+    super(context);
+  }
+
+  @Override
+  public void initialize() {
+    // Nothing to do.
+  }
+
+  @Override
+  public int getNumDestinationTaskPhysicalInputs(int destinationTaskIndex) {
+    return 1;
+  }
+  
+  @Override
+  public int getNumSourceTaskPhysicalOutputs(int sourceTaskIndex) {
+    return 1;
+  }
+  
+  @Override
+  public void routeDataMovementEventToDestination(DataMovementEvent event,
+      int sourceTaskIndex, int sourceOutputIndex, 
+      Map<Integer, List<Integer>> destinationTaskAndInputIndices) {
+    checkState();
+    destinationTaskAndInputIndices.put(sourceTaskIndex, destinationInputIndices);
+  }
+  
+  @Override
+  public void prepareForRouting() throws Exception {
+    checkState();
+  }
+  
+  @Override
+  public EventRouteMetadata routeDataMovementEventToDestination(
+      int sourceTaskIndex, int sourceOutputIndex, int destinationTaskIndex)
+      throws Exception {
+    if (sourceTaskIndex == destinationTaskIndex) {
+      return commonRouteMeta;
+    }
+    return null;
+  }
+  
+  @Override
+  public @Nullable EventRouteMetadata routeCompositeDataMovementEventToDestination(
+      int sourceTaskIndex, int destinationTaskIndex)
+      throws Exception {
+    if (sourceTaskIndex == destinationTaskIndex) {
+      return commonRouteMeta;
+    }
+    return null;
+  }
+
+  @Override
+  public EventRouteMetadata routeInputSourceTaskFailedEventToDestination(
+      int sourceTaskIndex, int destinationTaskIndex) throws Exception {
+    return commonRouteMeta;
+  }
+
+  @Override
+  public void routeInputSourceTaskFailedEventToDestination(int sourceTaskIndex,
+      Map<Integer, List<Integer>> destinationTaskAndInputIndices) {
+    destinationTaskAndInputIndices.put(sourceTaskIndex, destinationInputIndices);
+  }
+
+  @Override
+  public int routeInputErrorEventToSource(InputReadErrorEvent event,
+      int destinationTaskIndex, int destinationFailedInputIndex) {
+    return destinationTaskIndex;
+  }
+  
+  @Override
+  public int routeInputErrorEventToSource(int destinationTaskIndex, int destinationFailedInputIndex) {
+    return destinationTaskIndex;
+  }
+
+  @Override
+  public int getNumDestinationConsumerTasks(int sourceTaskIndex) {
+    return 1;
+  }
+  
+  private void checkState() {
+    if (stateChecked.get()) {
+      return;
+    }
+    // by the time routing is initiated all task counts must be determined and stable
+    Preconditions.checkState(getContext().getSourceVertexNumTasks() == getContext()
+        .getDestinationVertexNumTasks(), "1-1 source and destination task counts must match."
+        + " Destination: " + getContext().getDestinationVertexName() + " tasks: "
+        + getContext().getDestinationVertexNumTasks() + " Source: "
+        + getContext().getSourceVertexName() + " tasks: " + getContext().getSourceVertexNumTasks());
+    stateChecked.set(true);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/413c3cc7/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestEdge.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestEdge.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestEdge.java
index 5718b17..eb03d1e 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestEdge.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestEdge.java
@@ -53,6 +53,7 @@ import org.apache.tez.dag.api.EdgeProperty.DataSourceType;
 import org.apache.tez.dag.api.EdgeProperty.SchedulingType;
 import org.apache.tez.dag.api.InputDescriptor;
 import org.apache.tez.dag.api.OutputDescriptor;
+import org.apache.tez.dag.api.TezConfiguration;
 import org.apache.tez.dag.api.UserPayload;
 import org.apache.tez.dag.app.dag.Task;
 import org.apache.tez.dag.app.dag.Vertex;
@@ -104,6 +105,36 @@ public class TestEdge {
         .get(0).intValue());
   }
 
+  @Test (timeout = 5000)
+  public void testOneToOneEdgeManagerODR() {
+    EdgeManagerPluginContext mockContext = mock(EdgeManagerPluginContext.class);
+    when(mockContext.getSourceVertexName()).thenReturn("Source");
+    when(mockContext.getDestinationVertexName()).thenReturn("Destination");
+    when(mockContext.getSourceVertexNumTasks()).thenReturn(3);
+    OneToOneEdgeManagerOnDemand manager = new OneToOneEdgeManagerOnDemand(mockContext);
+    manager.initialize();
+    Map<Integer, List<Integer>> destinationTaskAndInputIndices = Maps.newHashMap();
+    DataMovementEvent event = DataMovementEvent.create(1, null);
+
+    // fail when source and destination are inconsistent
+    when(mockContext.getDestinationVertexNumTasks()).thenReturn(4);
+    try {
+      manager.routeDataMovementEventToDestination(event, 1, 1, destinationTaskAndInputIndices);
+      Assert.fail();
+    } catch (IllegalStateException e) {
+      Assert.assertTrue(e.getMessage().contains("1-1 source and destination task counts must match"));
+    }
+
+    // now make it consistent
+    when(mockContext.getDestinationVertexNumTasks()).thenReturn(3);
+    manager.routeDataMovementEventToDestination(event, 1, 1, destinationTaskAndInputIndices);
+    Assert.assertEquals(1, destinationTaskAndInputIndices.size());
+    Assert.assertEquals(1, destinationTaskAndInputIndices.entrySet().iterator().next().getKey()
+        .intValue());
+    Assert.assertEquals(0, destinationTaskAndInputIndices.entrySet().iterator().next().getValue()
+        .get(0).intValue());
+  }
+
   @Test(timeout = 5000)
   public void testScatterGatherManager() {
     EdgeManagerPluginContext mockContext = mock(EdgeManagerPluginContext.class);
@@ -132,7 +163,7 @@ public class TestEdge {
     EdgeProperty edgeProp = EdgeProperty.create(DataMovementType.SCATTER_GATHER,
         DataSourceType.PERSISTED, SchedulingType.SEQUENTIAL, mock(OutputDescriptor.class),
         mock(InputDescriptor.class));
-    Edge edge = new Edge(edgeProp, eventHandler);
+    Edge edge = new Edge(edgeProp, eventHandler, new TezConfiguration());
     
     TezVertexID srcVertexID = createVertexID(1);
     TezVertexID destVertexID = createVertexID(2);
@@ -246,7 +277,7 @@ public class TestEdge {
         DataSourceType.PERSISTED,
         SchedulingType.SEQUENTIAL,
         OutputDescriptor.create(""),
-        InputDescriptor.create("")), mockEventHandler);
+        InputDescriptor.create("")), mockEventHandler, new TezConfiguration());
     TezVertexID v1Id = createVertexID(1);
     TezVertexID v2Id = createVertexID(2);
     edge.setSourceVertex(mockVertex("v1", v1Id, new LinkedHashMap<TezTaskID, Task>()));
@@ -270,7 +301,7 @@ public class TestEdge {
         DataSourceType.PERSISTED,
         SchedulingType.SEQUENTIAL,
         OutputDescriptor.create(""),
-        InputDescriptor.create("")), mockEventHandler);
+        InputDescriptor.create("")), mockEventHandler, new TezConfiguration());
     TezVertexID v1Id = createVertexID(1);
     TezVertexID v2Id = createVertexID(2);
     edge.setSourceVertex(mockVertex("v1", v1Id, new LinkedHashMap<TezTaskID, Task>()));
@@ -294,7 +325,7 @@ public class TestEdge {
         DataSourceType.PERSISTED,
         SchedulingType.SEQUENTIAL,
         OutputDescriptor.create(""),
-        InputDescriptor.create("")), mockEventHandler);
+        InputDescriptor.create("")), mockEventHandler, new TezConfiguration());
     TezVertexID v1Id = createVertexID(1);
     TezVertexID v2Id = createVertexID(2);
     edge.setSourceVertex(mockVertex("v1", v1Id, new LinkedHashMap<TezTaskID, Task>()));
@@ -321,7 +352,7 @@ public class TestEdge {
         DataSourceType.PERSISTED,
         SchedulingType.SEQUENTIAL,
         OutputDescriptor.create(""),
-        InputDescriptor.create("")), mockEventHandler);
+        InputDescriptor.create("")), mockEventHandler, new TezConfiguration());
     TezVertexID v1Id = createVertexID(1);
     TezVertexID v2Id = createVertexID(2);
     edge.setSourceVertex(mockVertex("v1", v1Id, new LinkedHashMap<TezTaskID, Task>()));

http://git-wip-us.apache.org/repos/asf/tez/blob/413c3cc7/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 e569949..0176b79 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
@@ -2217,7 +2217,7 @@ public class TestVertexImpl {
     for (EdgePlan edgePlan : dagPlan.getEdgeList()) {
       EdgeProperty edgeProperty = DagTypeConverters
           .createEdgePropertyMapFromDAGPlan(edgePlan);
-      edges.put(edgePlan.getId(), new Edge(edgeProperty, dispatcher.getEventHandler()));
+      edges.put(edgePlan.getId(), new Edge(edgeProperty, dispatcher.getEventHandler(), conf));
     }
 
     parseVertexEdges();

http://git-wip-us.apache.org/repos/asf/tez/blob/413c3cc7/tez-tests/src/test/java/org/apache/tez/test/TestExceptionPropagation.java
----------------------------------------------------------------------
diff --git a/tez-tests/src/test/java/org/apache/tez/test/TestExceptionPropagation.java b/tez-tests/src/test/java/org/apache/tez/test/TestExceptionPropagation.java
index 7ba6028..49bb9f5 100644
--- a/tez-tests/src/test/java/org/apache/tez/test/TestExceptionPropagation.java
+++ b/tez-tests/src/test/java/org/apache/tez/test/TestExceptionPropagation.java
@@ -29,6 +29,7 @@ import java.util.Map;
 import java.util.Set;
 
 import org.apache.commons.lang.StringUtils;
+import org.apache.tez.dag.app.dag.impl.OneToOneEdgeManagerOnDemand;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -64,7 +65,6 @@ import org.apache.tez.dag.api.EdgeProperty.SchedulingType;
 import org.apache.tez.dag.api.client.DAGClient;
 import org.apache.tez.dag.api.client.DAGStatus;
 import org.apache.tez.dag.api.event.VertexStateUpdate;
-import org.apache.tez.dag.app.dag.impl.OneToOneEdgeManager;
 import org.apache.tez.dag.app.dag.impl.RootInputVertexManager;
 import org.apache.tez.dag.library.vertexmanager.InputReadyVertexManager;
 import org.apache.tez.runtime.api.AbstractLogicalIOProcessor;
@@ -154,6 +154,7 @@ public class TestExceptionPropagation {
     tezConf
         .setBoolean(TezConfiguration.TEZ_AM_NODE_BLACKLISTING_ENABLED, false);
     tezConf.setInt(TezConfiguration.TEZ_AM_MAX_APP_ATTEMPTS, 4);
+    tezConf.setBoolean(TezConfiguration.TEZ_AM_ONE_TO_ONE_ROUTING_USE_ON_DEMAND_ROUTING, true);
     tezConf.setInt(TezConfiguration.TEZ_AM_RESOURCE_MEMORY_MB, 500);
     tezConf.set(TezConfiguration.TEZ_AM_LAUNCH_CMD_OPTS, " -Xmx256m");
     tezConf.setBoolean(TezConfiguration.TEZ_AM_SESSION_MODE, true);
@@ -753,7 +754,7 @@ public class TestExceptionPropagation {
   }
 
   // EdgeManager for edge linking vertex1 and vertex2
-  public static class CustomEdgeManager extends OneToOneEdgeManager {
+  public static class CustomEdgeManager extends OneToOneEdgeManagerOnDemand {
 
     private ExceptionLocation exLocation;
 
@@ -822,7 +823,7 @@ public class TestExceptionPropagation {
       }
       super.prepareForRouting();
     }
-    
+
     @Override
     public EventRouteMetadata routeDataMovementEventToDestination(
         int sourceTaskIndex, int sourceOutputIndex, int destinationTaskIndex) throws Exception {


[03/50] [abbrv] tez git commit: TEZ-2450 breaks Hadoop 2.2 and 2.4 compatability (rbalamohan)

Posted by ss...@apache.org.
TEZ-2450 breaks Hadoop 2.2 and 2.4 compatability (rbalamohan)


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

Branch: refs/heads/TEZ-2003
Commit: dac59a2aa71aab5daaa6fabdda9d8f48539e1bda
Parents: 9dabf94
Author: Rajesh Balamohan <rb...@apache.org>
Authored: Wed May 27 09:22:43 2015 +0530
Committer: Rajesh Balamohan <rb...@apache.org>
Committed: Wed May 27 09:22:43 2015 +0530

----------------------------------------------------------------------
 CHANGES.txt                                                     | 1 +
 .../src/main/java/org/apache/tez/http/SSLFactory.java           | 5 +++--
 2 files changed, 4 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/dac59a2a/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 5f5dd48..484f78d 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -8,6 +8,7 @@ INCOMPATIBLE CHANGES
   TEZ-2468. Change the minimum Java version to Java 7.
 
 ALL CHANGES:
+  TEZ-2490. TEZ-2450 breaks Hadoop 2.2 and 2.4 compatability.
   TEZ-2450. support async http clients in ordered & unordered inputs.
   TEZ-2454. Change FetcherOrderedGroup to work as Callables instead of blocking threads.
   TEZ-2466. tez-history-parser breaks hadoop 2.2 compatability.

http://git-wip-us.apache.org/repos/asf/tez/blob/dac59a2a/tez-runtime-library/src/main/java/org/apache/tez/http/SSLFactory.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/http/SSLFactory.java b/tez-runtime-library/src/main/java/org/apache/tez/http/SSLFactory.java
index f23739b..9751463 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/http/SSLFactory.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/http/SSLFactory.java
@@ -37,11 +37,9 @@ import java.io.IOException;
 import java.net.HttpURLConnection;
 import java.security.GeneralSecurityException;
 
-import static org.apache.hadoop.security.ssl.SSLFactory.DEFAULT_SSL_ENABLED_PROTOCOLS;
 import static org.apache.hadoop.security.ssl.SSLFactory.DEFAULT_SSL_REQUIRE_CLIENT_CERT;
 import static org.apache.hadoop.security.ssl.SSLFactory.KEYSTORES_FACTORY_CLASS_KEY;
 import static org.apache.hadoop.security.ssl.SSLFactory.SSL_CLIENT_CONF_KEY;
-import static org.apache.hadoop.security.ssl.SSLFactory.SSL_ENABLED_PROTOCOLS;
 import static org.apache.hadoop.security.ssl.SSLFactory.SSL_HOSTNAME_VERIFIER_KEY;
 import static org.apache.hadoop.security.ssl.SSLFactory.SSL_REQUIRE_CLIENT_CERT_KEY;
 import static org.apache.hadoop.security.ssl.SSLFactory.SSL_SERVER_CONF_KEY;
@@ -60,6 +58,9 @@ import static org.apache.hadoop.security.ssl.SSLFactory.SSL_SERVER_CONF_KEY;
 @InterfaceStability.Evolving
 public class SSLFactory implements ConnectionConfigurator {
 
+  public static final String SSL_ENABLED_PROTOCOLS = "hadoop.ssl.enabled.protocols";
+  public static final String DEFAULT_SSL_ENABLED_PROTOCOLS = "TLSv1";
+
   private Configuration conf;
   private Mode mode;
   private boolean requireClientCert;


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

Branch: refs/heads/TEZ-2003
Commit: 323fa5ec6981129d1fa7e8a393a6c06524c1a9ea
Parents: 32dd525
Author: Siddharth Seth <ss...@apache.org>
Authored: Wed May 6 00:39:46 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Mon Jun 1 14:21:04 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/323fa5ec/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/323fa5ec/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/323fa5ec/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/323fa5ec/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) {


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

Branch: refs/heads/TEZ-2003
Commit: d27b897780d51aa056f4749eaa7c7d28b1ba8a8b
Parents: 4cb8f01
Author: Siddharth Seth <ss...@apache.org>
Authored: Fri Feb 20 17:23:18 2015 -0800
Committer: Siddharth Seth <ss...@apache.org>
Committed: Mon Jun 1 14:09:26 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/d27b8977/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/d27b8977/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


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

Posted by ss...@apache.org.
http://git-wip-us.apache.org/repos/asf/tez/blob/aa6c53ff/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java b/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java
new file mode 100644
index 0000000..a93c1a4
--- /dev/null
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/tests/TestExternalTezServices.java
@@ -0,0 +1,183 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.tests;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.util.Map;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.tez.client.TezClient;
+import org.apache.tez.dag.api.TezConfiguration;
+import org.apache.tez.dag.api.TezException;
+import org.apache.tez.dag.app.launcher.TezTestServiceNoOpContainerLauncher;
+import org.apache.tez.dag.app.rm.TezTestServiceTaskSchedulerService;
+import org.apache.tez.dag.app.taskcomm.TezTestServiceTaskCommunicatorImpl;
+import org.apache.tez.examples.HashJoinExample;
+import org.apache.tez.examples.JoinDataGen;
+import org.apache.tez.examples.JoinValidate;
+import org.apache.tez.service.MiniTezTestServiceCluster;
+import org.apache.tez.test.MiniTezCluster;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestExternalTezServices {
+
+  private static final Log LOG = LogFactory.getLog(TestExternalTezServices.class);
+
+  private static MiniTezCluster tezCluster;
+  private static MiniDFSCluster dfsCluster;
+  private static MiniTezTestServiceCluster tezTestServiceCluster;
+
+  private static Configuration clusterConf = new Configuration();
+  private static Configuration confForJobs;
+
+  private static FileSystem remoteFs;
+  private static FileSystem localFs;
+
+  private static TezClient sharedTezClient;
+
+  private static String TEST_ROOT_DIR = "target" + Path.SEPARATOR + TestExternalTezServices.class.getName()
+      + "-tmpDir";
+
+  @BeforeClass
+  public static void setup() throws IOException, TezException, InterruptedException {
+
+    localFs = FileSystem.getLocal(clusterConf);
+
+    try {
+      clusterConf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, TEST_ROOT_DIR);
+      dfsCluster =
+          new MiniDFSCluster.Builder(clusterConf).numDataNodes(1).format(true).racks(null).build();
+      remoteFs = dfsCluster.getFileSystem();
+      LOG.info("MiniDFSCluster started");
+    } catch (IOException io) {
+      throw new RuntimeException("problem starting mini dfs cluster", io);
+    }
+
+    tezCluster = new MiniTezCluster(TestExternalTezServices.class.getName(), 1, 1, 1);
+    Configuration conf = new Configuration();
+    conf.set("fs.defaultFS", remoteFs.getUri().toString()); // use HDFS
+    tezCluster.init(conf);
+    tezCluster.start();
+    LOG.info("MiniTezCluster started");
+
+    clusterConf.set("fs.defaultFS", remoteFs.getUri().toString()); // use HDFS
+    for (Map.Entry<String, String> entry : tezCluster.getConfig()) {
+      clusterConf.set(entry.getKey(), entry.getValue());
+    }
+    long jvmMax = Runtime.getRuntime().maxMemory();
+
+    tezTestServiceCluster = MiniTezTestServiceCluster
+        .create(TestExternalTezServices.class.getSimpleName(), 3, ((long) (jvmMax * 0.5d)), 1);
+    tezTestServiceCluster.init(clusterConf);
+    tezTestServiceCluster.start();
+    LOG.info("MiniTezTestServer started");
+
+    confForJobs = new Configuration(clusterConf);
+    for (Map.Entry<String, String> entry : tezTestServiceCluster
+        .getClusterSpecificConfiguration()) {
+      confForJobs.set(entry.getKey(), entry.getValue());
+    }
+
+    // TODO TEZ-2003 Once per vertex configuration is possible, run separate tests for push vs pull (regular threaded execution)
+
+    Path stagingDirPath = new Path("/tmp/tez-staging-dir");
+    remoteFs.mkdirs(stagingDirPath);
+    // This is currently configured to push tasks into the Service, and then use the standard RPC
+    confForJobs.set(TezConfiguration.TEZ_AM_STAGING_DIR, stagingDirPath.toString());
+    confForJobs.set(TezConfiguration.TEZ_AM_TASK_SCHEDULER_CLASS,
+        TezTestServiceTaskSchedulerService.class.getName());
+    confForJobs.set(TezConfiguration.TEZ_AM_CONTAINER_LAUNCHER_CLASS,
+        TezTestServiceNoOpContainerLauncher.class.getName());
+    confForJobs.set(TezConfiguration.TEZ_AM_TASK_COMMUNICATOR_CLASS,
+        TezTestServiceTaskCommunicatorImpl.class.getName());
+
+    TezConfiguration tezConf = new TezConfiguration(confForJobs);
+
+    sharedTezClient = TezClient.create(TestExternalTezServices.class.getSimpleName() + "_session",
+        tezConf, true);
+    sharedTezClient.start();
+    LOG.info("Shared TezSession started");
+    sharedTezClient.waitTillReady();
+    LOG.info("Shared TezSession ready for submission");
+
+  }
+
+  @AfterClass
+  public static void tearDown() throws IOException, TezException {
+    if (sharedTezClient != null) {
+      sharedTezClient.stop();
+      sharedTezClient = null;
+    }
+
+    if (tezTestServiceCluster != null) {
+      tezTestServiceCluster.stop();
+      tezTestServiceCluster = null;
+    }
+
+    if (tezCluster != null) {
+      tezCluster.stop();
+      tezCluster = null;
+    }
+    if (dfsCluster != null) {
+      dfsCluster.shutdown();
+      dfsCluster = null;
+    }
+    // TODO Add cleanup code.
+  }
+
+
+  @Test(timeout = 60000)
+  public void test1() throws Exception {
+    Path testDir = new Path("/tmp/testHashJoinExample");
+
+    remoteFs.mkdirs(testDir);
+
+    Path dataPath1 = new Path(testDir, "inPath1");
+    Path dataPath2 = new Path(testDir, "inPath2");
+    Path expectedOutputPath = new Path(testDir, "expectedOutputPath");
+    Path outPath = new Path(testDir, "outPath");
+
+    TezConfiguration tezConf = new TezConfiguration(confForJobs);
+
+    JoinDataGen dataGen = new JoinDataGen();
+    String[] dataGenArgs = new String[]{
+        dataPath1.toString(), "1048576", dataPath2.toString(), "524288",
+        expectedOutputPath.toString(), "2"};
+    assertEquals(0, dataGen.run(tezConf, dataGenArgs, sharedTezClient));
+
+    HashJoinExample joinExample = new HashJoinExample();
+    String[] args = new String[]{
+        dataPath1.toString(), dataPath2.toString(), "2", outPath.toString()};
+    assertEquals(0, joinExample.run(tezConf, args, sharedTezClient));
+
+    JoinValidate joinValidate = new JoinValidate();
+    String[] validateArgs = new String[]{
+        expectedOutputPath.toString(), outPath.toString(), "3"};
+    assertEquals(0, joinValidate.run(tezConf, validateArgs, sharedTezClient));
+
+    // Ensure this was actually submitted to the external cluster
+    assertTrue(tezTestServiceCluster.getNumSubmissions() > 0);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/aa6c53ff/tez-ext-service-tests/src/test/java/org/apache/tez/util/ProtoConverters.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/util/ProtoConverters.java b/tez-ext-service-tests/src/test/java/org/apache/tez/util/ProtoConverters.java
new file mode 100644
index 0000000..60ebc53
--- /dev/null
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/util/ProtoConverters.java
@@ -0,0 +1,172 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.util;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.tez.dag.api.DagTypeConverters;
+import org.apache.tez.dag.api.InputDescriptor;
+import org.apache.tez.dag.api.OutputDescriptor;
+import org.apache.tez.dag.api.ProcessorDescriptor;
+import org.apache.tez.dag.records.TezTaskAttemptID;
+import org.apache.tez.runtime.api.impl.GroupInputSpec;
+import org.apache.tez.runtime.api.impl.InputSpec;
+import org.apache.tez.runtime.api.impl.OutputSpec;
+import org.apache.tez.runtime.api.impl.TaskSpec;
+import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.GroupInputSpecProto;
+import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.IOSpecProto;
+import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.TaskSpecProto;
+
+public class ProtoConverters {
+
+  public static TaskSpec getTaskSpecfromProto(TaskSpecProto taskSpecProto) {
+    TezTaskAttemptID taskAttemptID =
+        TezTaskAttemptID.fromString(taskSpecProto.getTaskAttemptIdString());
+
+    ProcessorDescriptor processorDescriptor = null;
+    if (taskSpecProto.hasProcessorDescriptor()) {
+      processorDescriptor = DagTypeConverters
+          .convertProcessorDescriptorFromDAGPlan(taskSpecProto.getProcessorDescriptor());
+    }
+
+    List<InputSpec> inputSpecList = new ArrayList<InputSpec>(taskSpecProto.getInputSpecsCount());
+    if (taskSpecProto.getInputSpecsCount() > 0) {
+      for (IOSpecProto inputSpecProto : taskSpecProto.getInputSpecsList()) {
+        inputSpecList.add(getInputSpecFromProto(inputSpecProto));
+      }
+    }
+
+    List<OutputSpec> outputSpecList =
+        new ArrayList<OutputSpec>(taskSpecProto.getOutputSpecsCount());
+    if (taskSpecProto.getOutputSpecsCount() > 0) {
+      for (IOSpecProto outputSpecProto : taskSpecProto.getOutputSpecsList()) {
+        outputSpecList.add(getOutputSpecFromProto(outputSpecProto));
+      }
+    }
+
+    List<GroupInputSpec> groupInputSpecs =
+        new ArrayList<GroupInputSpec>(taskSpecProto.getGroupedInputSpecsCount());
+    if (taskSpecProto.getGroupedInputSpecsCount() > 0) {
+      for (GroupInputSpecProto groupInputSpecProto : taskSpecProto.getGroupedInputSpecsList()) {
+        groupInputSpecs.add(getGroupInputSpecFromProto(groupInputSpecProto));
+      }
+    }
+
+    TaskSpec taskSpec =
+        new TaskSpec(taskAttemptID, taskSpecProto.getDagName(), taskSpecProto.getVertexName(),
+            taskSpecProto.getVertexParallelism(), processorDescriptor, inputSpecList,
+            outputSpecList, groupInputSpecs);
+    return taskSpec;
+  }
+
+  public static TaskSpecProto convertTaskSpecToProto(TaskSpec taskSpec) {
+    TaskSpecProto.Builder builder = TaskSpecProto.newBuilder();
+    builder.setTaskAttemptIdString(taskSpec.getTaskAttemptID().toString());
+    builder.setDagName(taskSpec.getDAGName());
+    builder.setVertexName(taskSpec.getVertexName());
+    builder.setVertexParallelism(taskSpec.getVertexParallelism());
+
+    if (taskSpec.getProcessorDescriptor() != null) {
+      builder.setProcessorDescriptor(
+          DagTypeConverters.convertToDAGPlan(taskSpec.getProcessorDescriptor()));
+    }
+
+    if (taskSpec.getInputs() != null && !taskSpec.getInputs().isEmpty()) {
+      for (InputSpec inputSpec : taskSpec.getInputs()) {
+        builder.addInputSpecs(convertInputSpecToProto(inputSpec));
+      }
+    }
+
+    if (taskSpec.getOutputs() != null && !taskSpec.getOutputs().isEmpty()) {
+      for (OutputSpec outputSpec : taskSpec.getOutputs()) {
+        builder.addOutputSpecs(convertOutputSpecToProto(outputSpec));
+      }
+    }
+
+    if (taskSpec.getGroupInputs() != null && !taskSpec.getGroupInputs().isEmpty()) {
+      for (GroupInputSpec groupInputSpec : taskSpec.getGroupInputs()) {
+        builder.addGroupedInputSpecs(convertGroupInputSpecToProto(groupInputSpec));
+
+      }
+    }
+    return builder.build();
+  }
+
+
+  public static InputSpec getInputSpecFromProto(IOSpecProto inputSpecProto) {
+    InputDescriptor inputDescriptor = null;
+    if (inputSpecProto.hasIoDescriptor()) {
+      inputDescriptor =
+          DagTypeConverters.convertInputDescriptorFromDAGPlan(inputSpecProto.getIoDescriptor());
+    }
+    InputSpec inputSpec = new InputSpec(inputSpecProto.getConnectedVertexName(), inputDescriptor,
+        inputSpecProto.getPhysicalEdgeCount());
+    return inputSpec;
+  }
+
+  public static IOSpecProto convertInputSpecToProto(InputSpec inputSpec) {
+    IOSpecProto.Builder builder = IOSpecProto.newBuilder();
+    if (inputSpec.getSourceVertexName() != null) {
+      builder.setConnectedVertexName(inputSpec.getSourceVertexName());
+    }
+    if (inputSpec.getInputDescriptor() != null) {
+      builder.setIoDescriptor(DagTypeConverters.convertToDAGPlan(inputSpec.getInputDescriptor()));
+    }
+    builder.setPhysicalEdgeCount(inputSpec.getPhysicalEdgeCount());
+    return builder.build();
+  }
+
+  public static OutputSpec getOutputSpecFromProto(IOSpecProto outputSpecProto) {
+    OutputDescriptor outputDescriptor = null;
+    if (outputSpecProto.hasIoDescriptor()) {
+      outputDescriptor =
+          DagTypeConverters.convertOutputDescriptorFromDAGPlan(outputSpecProto.getIoDescriptor());
+    }
+    OutputSpec outputSpec =
+        new OutputSpec(outputSpecProto.getConnectedVertexName(), outputDescriptor,
+            outputSpecProto.getPhysicalEdgeCount());
+    return outputSpec;
+  }
+
+  public static IOSpecProto convertOutputSpecToProto(OutputSpec outputSpec) {
+    IOSpecProto.Builder builder = IOSpecProto.newBuilder();
+    if (outputSpec.getDestinationVertexName() != null) {
+      builder.setConnectedVertexName(outputSpec.getDestinationVertexName());
+    }
+    if (outputSpec.getOutputDescriptor() != null) {
+      builder.setIoDescriptor(DagTypeConverters.convertToDAGPlan(outputSpec.getOutputDescriptor()));
+    }
+    builder.setPhysicalEdgeCount(outputSpec.getPhysicalEdgeCount());
+    return builder.build();
+  }
+
+  public static GroupInputSpec getGroupInputSpecFromProto(GroupInputSpecProto groupInputSpecProto) {
+    GroupInputSpec groupSpec = new GroupInputSpec(groupInputSpecProto.getGroupName(),
+        groupInputSpecProto.getGroupVerticesList(), DagTypeConverters
+        .convertInputDescriptorFromDAGPlan(groupInputSpecProto.getMergedInputDescriptor()));
+    return groupSpec;
+  }
+
+  public static GroupInputSpecProto convertGroupInputSpecToProto(GroupInputSpec groupInputSpec) {
+    GroupInputSpecProto.Builder builder = GroupInputSpecProto.newBuilder();
+    builder.setGroupName(groupInputSpec.getGroupName());
+    builder.addAllGroupVertices(groupInputSpec.getGroupVertices());
+    builder.setMergedInputDescriptor(
+        DagTypeConverters.convertToDAGPlan(groupInputSpec.getMergedInputDescriptor()));
+    return builder.build();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/aa6c53ff/tez-ext-service-tests/src/test/proto/TezDaemonProtocol.proto
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/proto/TezDaemonProtocol.proto b/tez-ext-service-tests/src/test/proto/TezDaemonProtocol.proto
new file mode 100644
index 0000000..2f8b2e6
--- /dev/null
+++ b/tez-ext-service-tests/src/test/proto/TezDaemonProtocol.proto
@@ -0,0 +1,84 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+option java_package = "org.apache.tez.test.service.rpc";
+option java_outer_classname = "TezTestServiceProtocolProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+
+import "DAGApiRecords.proto";
+
+message IOSpecProto {
+  optional string connected_vertex_name = 1;
+  optional TezEntityDescriptorProto io_descriptor = 2;
+  optional int32 physical_edge_count = 3;
+}
+
+message GroupInputSpecProto {
+  optional string group_name = 1;
+  repeated string group_vertices = 2;
+  optional TezEntityDescriptorProto merged_input_descriptor = 3;
+}
+
+message TaskSpecProto {
+  optional string task_attempt_id_string = 1;
+  optional string dag_name = 2;
+  optional string vertex_name = 3;
+  optional TezEntityDescriptorProto processor_descriptor = 4;
+  repeated IOSpecProto input_specs = 5;
+  repeated IOSpecProto output_specs = 6;
+  repeated GroupInputSpecProto grouped_input_specs = 7;
+  optional int32 vertex_parallelism = 8;
+}
+
+
+message SubmitWorkRequestProto {
+  optional string container_id_string = 1;
+  optional string am_host = 2;
+  optional int32 am_port = 3;
+  optional string token_identifier = 4;
+  optional bytes credentials_binary = 5;
+  optional string user = 6;
+  optional string application_id_string = 7;
+  optional int32 app_attempt_number = 8;
+  optional TaskSpecProto task_spec = 9;
+}
+
+message SubmitWorkResponseProto {
+}
+
+
+
+message RunContainerRequestProto {
+  optional string container_id_string = 1;
+  optional string am_host = 2;
+  optional int32 am_port = 3;
+  optional string token_identifier = 4;
+  optional bytes credentials_binary = 5;
+  optional string user = 6;
+  optional string application_id_string = 7;
+  optional int32 app_attempt_number = 8;
+}
+
+message RunContainerResponseProto {
+}
+
+service TezTestServiceProtocol {
+  rpc runContainer(RunContainerRequestProto) returns (RunContainerResponseProto);
+  rpc submitWork(SubmitWorkRequestProto) returns (SubmitWorkResponseProto);
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/aa6c53ff/tez-ext-service-tests/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/resources/log4j.properties b/tez-ext-service-tests/src/test/resources/log4j.properties
new file mode 100644
index 0000000..531b68b
--- /dev/null
+++ b/tez-ext-service-tests/src/test/resources/log4j.properties
@@ -0,0 +1,19 @@
+#   Licensed under the Apache License, Version 2.0 (the "License");
+#   you may not use this file except in compliance with the License.
+#   You may obtain a copy of the License at
+#
+#       http://www.apache.org/licenses/LICENSE-2.0
+#
+#   Unless required by applicable law or agreed to in writing, software
+#   distributed under the License is distributed on an "AS IS" BASIS,
+#   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+#   See the License for the specific language governing permissions and
+#   limitations under the License.
+
+# log4j configuration used during build and unit tests
+
+log4j.rootLogger=info,stdout
+log4j.threshhold=ALL
+log4j.appender.stdout=org.apache.log4j.ConsoleAppender
+log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
+log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} %-5p [%t] %c{2} (%F:%M(%L)) - %m%n

http://git-wip-us.apache.org/repos/asf/tez/blob/aa6c53ff/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 83ebadb..8f19e99 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
@@ -396,7 +396,7 @@ public class TezChild {
     private final Throwable throwable;
     private final String errorMessage;
 
-    ContainerExecutionResult(ExitStatus exitStatus, @Nullable Throwable throwable,
+    public ContainerExecutionResult(ExitStatus exitStatus, @Nullable Throwable throwable,
                              @Nullable String errorMessage) {
       this.exitStatus = exitStatus;
       this.throwable = throwable;

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


[06/50] [abbrv] tez git commit: TEZ-2440. Sorter should check for indexCacheList.size() in flush() (Contributed by Mit Desai)

Posted by ss...@apache.org.
TEZ-2440. Sorter should check for indexCacheList.size() in flush() (Contributed by Mit Desai)


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

Branch: refs/heads/TEZ-2003
Commit: fe74d6b9e896dfe158392ffe91fbad944abba027
Parents: 317d45a
Author: Rajesh Balamohan <rb...@apache.org>
Authored: Thu May 28 04:44:33 2015 +0530
Committer: Rajesh Balamohan <rb...@apache.org>
Committed: Thu May 28 04:44:33 2015 +0530

----------------------------------------------------------------------
 CHANGES.txt                                                 | 1 +
 .../runtime/library/common/sort/impl/PipelinedSorter.java   | 9 +++++----
 2 files changed, 6 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/fe74d6b9/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 4779f83..f676ef7 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -8,6 +8,7 @@ INCOMPATIBLE CHANGES
   TEZ-2468. Change the minimum Java version to Java 7.
 
 ALL CHANGES:
+  TEZ-2440. Sorter should check for indexCacheList.size() in flush().
   TEZ-2490. TEZ-2450 breaks Hadoop 2.2 and 2.4 compatability.
   TEZ-2450. support async http clients in ordered & unordered inputs.
   TEZ-2454. Change FetcherOrderedGroup to work as Callables instead of blocking threads.

http://git-wip-us.apache.org/repos/asf/tez/blob/fe74d6b9/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/PipelinedSorter.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/PipelinedSorter.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/PipelinedSorter.java
index d9de921..9113fca 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/PipelinedSorter.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/PipelinedSorter.java
@@ -404,10 +404,6 @@ public class PipelinedSorter extends ExternalSorter {
     }
   }
 
-
-
-
-
   private boolean isThreadInterrupted() throws IOException {
     if (Thread.currentThread().isInterrupted()) {
       if (cleanup) {
@@ -446,6 +442,11 @@ public class PipelinedSorter extends ExternalSorter {
 
       numAdditionalSpills.increment(numSpills - 1);
 
+      if(indexCacheList.isEmpty()) {
+        LOG.warn("Index list is empty... returning");
+        return;
+      }
+
       if (!finalMergeEnabled) {
         //Generate events for all spills
         List<Event> events = Lists.newLinkedList();


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

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


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

Branch: refs/heads/TEZ-2003
Commit: ae6d78094b5f0b87f35aa52e7d2cd6c1cb9af262
Parents: 63d1e62
Author: Siddharth Seth <ss...@apache.org>
Authored: Thu Mar 26 17:57:55 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Mon Jun 1 14:09:27 2015 -0700

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


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

http://git-wip-us.apache.org/repos/asf/tez/blob/ae6d7809/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 94f6cae..0ee448b 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
@@ -67,6 +67,8 @@ import org.apache.tez.dag.records.TezTaskAttemptID;
 import org.apache.tez.dag.records.TezVertexID;
 import org.apache.tez.runtime.api.impl.TezEvent;
 import org.apache.tez.common.security.JobTokenSecretManager;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 
 @SuppressWarnings("unchecked")

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

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

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

http://git-wip-us.apache.org/repos/asf/tez/blob/ae6d7809/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 03d1456..8fc785b 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
@@ -342,7 +342,7 @@ public class MockDAGAppMaster extends DAGAppMaster {
       long startTime = System.nanoTime();
       long startCpuTime = threadMxBean.getCurrentThreadCpuTime();
       TaskHeartbeatResponse response = taListener.heartbeat(request);
-      if (response.shouldDie()) {
+      if (response.isShouldDie()) {
         cData.remove();
       } else {
         cData.nextFromEventId = response.getNextFromEventId();

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

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

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

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

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

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

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

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

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

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

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

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

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

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


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

Branch: refs/heads/TEZ-2003
Commit: f59cf567c427492b237befb83f72ad1b8f3b57e6
Parents: 57c44a8
Author: Siddharth Seth <ss...@apache.org>
Authored: Thu Apr 23 17:26:25 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Mon Jun 1 14:10:27 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/f59cf567/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/f59cf567/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/f59cf567/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 b11d697..ea554e3 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
@@ -764,7 +764,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());
@@ -778,7 +778,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/f59cf567/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 aaf9cca..03b5602 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
@@ -81,6 +81,7 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
 
   private final AppContext context;
   private final TaskCommunicator[] taskCommunicators;
+  private final TaskCommunicatorContext[] taskCommunicatorContexts;
 
   protected final TaskHeartbeatHandler taskHeartbeatHandler;
   protected final ContainerHeartbeatHandler containerHeartbeatHandler;
@@ -123,7 +124,9 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
       }
     }
     this.taskCommunicators = new TaskCommunicator[taskCommunicatorClassIdentifiers.length];
+    this.taskCommunicatorContexts = new TaskCommunicatorContext[taskCommunicatorClassIdentifiers.length];
     for (int i = 0 ; i < taskCommunicatorClassIdentifiers.length ; i++) {
+      taskCommunicatorContexts[i] = new TaskCommunicatorContextImpl(context, this, i);
       taskCommunicators[i] = createTaskCommunicator(taskCommunicatorClassIdentifiers[i], i);
     }
     // TODO TEZ-2118 Start using taskCommunicator indices properly
@@ -148,10 +151,10 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
   private TaskCommunicator createTaskCommunicator(String taskCommClassIdentifier, int taskCommIndex) {
     if (taskCommClassIdentifier.equals(TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT)) {
       LOG.info("Using Default Task Communicator");
-      return new TezTaskCommunicatorImpl(new TaskCommunicatorContextImpl(context, this, taskCommIndex));
+      return new TezTaskCommunicatorImpl(taskCommunicatorContexts[taskCommIndex]);
     } else if (taskCommClassIdentifier.equals(TezConstants.TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT)) {
       LOG.info("Using Default Local Task Communicator");
-      return new TezLocalTaskCommunicatorImpl(new TaskCommunicatorContextImpl(context, this, taskCommIndex));
+      return new TezLocalTaskCommunicatorImpl(taskCommunicatorContexts[taskCommIndex]);
     } else {
       LOG.info("Using TaskCommunicator: " + taskCommClassIdentifier);
       Class<? extends TaskCommunicator> taskCommClazz = (Class<? extends TaskCommunicator>) ReflectionUtils
@@ -159,7 +162,7 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
       try {
         Constructor<? extends TaskCommunicator> ctor = taskCommClazz.getConstructor(TaskCommunicatorContext.class);
         ctor.setAccessible(true);
-        return ctor.newInstance(new TaskCommunicatorContextImpl(context, this, taskCommIndex));
+        return ctor.newInstance(taskCommunicatorContexts[taskCommIndex]);
       } catch (NoSuchMethodException e) {
         throw new TezUncheckedException(e);
       } catch (InvocationTargetException e) {
@@ -318,6 +321,14 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
     // This becomes more relevant when task kills without container kills are allowed.
 
     // TODO TEZ-2336. Send a signal to containers indicating DAG completion.
+
+    // Inform all communicators of the dagCompletion.
+    for (int i = 0 ; i < taskCommunicators.length ; i++) {
+      ((TaskCommunicatorContextImpl)taskCommunicatorContexts[i]).dagCompleteStart(dag);
+      taskCommunicators[i].dagComplete(dag.getName());
+      ((TaskCommunicatorContextImpl)taskCommunicatorContexts[i]).dagCompleteEnd();
+    }
+
   }
 
   @Override

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

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

http://git-wip-us.apache.org/repos/asf/tez/blob/f59cf567/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 8fc785b..5cc7f38 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/f59cf567/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/f59cf567/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/f59cf567/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,


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

Branch: refs/heads/TEZ-2003
Commit: e646fc3b7d689c8e130fd0cd66f350c493a841a2
Parents: ba9d377
Author: Siddharth Seth <ss...@apache.org>
Authored: Tue May 19 13:57:58 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Mon Jun 1 14:26:41 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/e646fc3b/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/e646fc3b/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;
   }
 
 


[16/50] [abbrv] tez git commit: TEZ-2504. Tez UI: tables - show status column without scrolling, numeric 0 shown as Not available (Sreenath Somarajapuram via pramachandran)

Posted by ss...@apache.org.
TEZ-2504. Tez UI: tables - show status column without scrolling, numeric 0 shown as Not available (Sreenath Somarajapuram via pramachandran)


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

Branch: refs/heads/TEZ-2003
Commit: 7fe0a6b7122c31c4d2d274700faa6c04297ce3a7
Parents: 5a04f06
Author: Prakash Ramachandran <pr...@hortonworks.com>
Authored: Mon Jun 1 16:42:08 2015 +0530
Committer: Prakash Ramachandran <pr...@hortonworks.com>
Committed: Mon Jun 1 16:42:08 2015 +0530

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../scripts/components/basic-table/cell-view.js | 12 +++++-
 .../controllers/dag-task-attempts-controller.js | 26 ++++++-------
 .../webapp/app/scripts/controllers/dag_tasks.js | 28 +++++++-------
 .../app/scripts/controllers/dag_vertices.js     | 40 ++++++++++----------
 .../task_task_attempts_controller.js            | 26 ++++++-------
 .../vertex_task_attempts_controller.js          | 26 ++++++-------
 .../controllers/vertex_tasks_controller.js      | 28 +++++++-------
 8 files changed, 98 insertions(+), 89 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/7fe0a6b7/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index bbe4fc8..c1a7b09 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -30,6 +30,7 @@ Release 0.7.1: Unreleased
 INCOMPATIBLE CHANGES
 
 ALL CHANGES:
+  TEZ-2504. Tez UI: tables - show status column without scrolling, numeric 0 shown as Not available
   TEZ-2478. Move OneToOne routing to store events in Tasks.
   TEZ-2482. Tez UI: Mouse events not working on IE11
   TEZ-1529. ATS and TezClient integration in secure kerberos enabled cluster.

http://git-wip-us.apache.org/repos/asf/tez/blob/7fe0a6b7/tez-ui/src/main/webapp/app/scripts/components/basic-table/cell-view.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/scripts/components/basic-table/cell-view.js b/tez-ui/src/main/webapp/app/scripts/components/basic-table/cell-view.js
index 1506577..80a4c56 100644
--- a/tez-ui/src/main/webapp/app/scripts/components/basic-table/cell-view.js
+++ b/tez-ui/src/main/webapp/app/scripts/components/basic-table/cell-view.js
@@ -18,15 +18,23 @@
 
 var ObjectPromiseController = Ember.ObjectController.extend(Ember.PromiseProxyMixin);
 
+function stringifyNumbers(content) {
+  var displayText = content.displayText;
+  if(typeof displayText == 'number') {
+    content.displayText = displayText.toString();
+  }
+  return content;
+}
+
 App.BasicTableComponent.CellView = Ember.View.extend({
   templateName: 'components/basic-table/basic-cell',
 
   classNames: ['cell-content'],
 
   _normalizeContent: function (content) {
-    return content && typeof content == 'object' ? content : {
+    return stringifyNumbers(content && typeof content == 'object' ? content : {
       displayText: content
-    };
+    });
   },
 
   cellContent: function () {

http://git-wip-us.apache.org/repos/asf/tez/blob/7fe0a6b7/tez-ui/src/main/webapp/app/scripts/controllers/dag-task-attempts-controller.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/scripts/controllers/dag-task-attempts-controller.js b/tez-ui/src/main/webapp/app/scripts/controllers/dag-task-attempts-controller.js
index b26d2eb..572e226 100644
--- a/tez-ui/src/main/webapp/app/scripts/controllers/dag-task-attempts-controller.js
+++ b/tez-ui/src/main/webapp/app/scripts/controllers/dag-task-attempts-controller.js
@@ -102,6 +102,19 @@ App.DagTaskAttemptsController = App.TablePageController.extend({
         }
       },
       {
+        id: 'status',
+        headerCellName: 'Status',
+        templateName: 'components/basic-table/status-cell',
+        contentPath: 'status',
+        getCellContent: function(row) {
+          var status = App.Helpers.misc.getFixedupDisplayStatus(row.get('status'));
+          return {
+            status: status,
+            statusIcon: App.Helpers.misc.getStatusClassForEntity(status)
+          };
+        }
+      },
+      {
         id: 'vertexName',
         headerCellName: 'Vertex Name',
         contentPath: 'vertexID',
@@ -152,19 +165,6 @@ App.DagTaskAttemptsController = App.TablePageController.extend({
         },
       },
       {
-        id: 'status',
-        headerCellName: 'Status',
-        templateName: 'components/basic-table/status-cell',
-        contentPath: 'status',
-        getCellContent: function(row) {
-          var status = App.Helpers.misc.getFixedupDisplayStatus(row.get('status'));
-          return {
-            status: status,
-            statusIcon: App.Helpers.misc.getStatusClassForEntity(status)
-          };
-        }
-      },
-      {
         id: 'containerId',
         headerCellName: 'Container',
         contentPath: 'containerId'

http://git-wip-us.apache.org/repos/asf/tez/blob/7fe0a6b7/tez-ui/src/main/webapp/app/scripts/controllers/dag_tasks.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/scripts/controllers/dag_tasks.js b/tez-ui/src/main/webapp/app/scripts/controllers/dag_tasks.js
index ec4a47f..604899b 100644
--- a/tez-ui/src/main/webapp/app/scripts/controllers/dag_tasks.js
+++ b/tez-ui/src/main/webapp/app/scripts/controllers/dag_tasks.js
@@ -122,6 +122,20 @@ App.DagTasksController = App.TablePageController.extend({
         }
       },
       {
+        id: 'status',
+        headerCellName: 'Status',
+        templateName: 'components/basic-table/status-cell',
+        contentPath: 'status',
+        getCellContent: function(row) {
+          var status = row.get('status');
+          return {
+            status: status,
+            statusIcon: App.Helpers.misc.getStatusClassForEntity(status,
+              row.get('hasFailedTaskAttempts'))
+          };
+        }
+      },
+      {
         id: 'startTime',
         headerCellName: 'Start Time',
         contentPath: 'startTime',
@@ -155,20 +169,6 @@ App.DagTasksController = App.TablePageController.extend({
         },
       },
       {
-        id: 'status',
-        headerCellName: 'Status',
-        templateName: 'components/basic-table/status-cell',
-        contentPath: 'status',
-        getCellContent: function(row) {
-          var status = row.get('status');
-          return {
-            status: status,
-            statusIcon: App.Helpers.misc.getStatusClassForEntity(status,
-              row.get('hasFailedTaskAttempts'))
-          };
-        }
-      },
-      {
         id: 'actions',
         headerCellName: 'Actions',
         templateName: 'components/basic-table/task-actions-cell',

http://git-wip-us.apache.org/repos/asf/tez/blob/7fe0a6b7/tez-ui/src/main/webapp/app/scripts/controllers/dag_vertices.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/scripts/controllers/dag_vertices.js b/tez-ui/src/main/webapp/app/scripts/controllers/dag_vertices.js
index e531bdf..cdfbf21 100644
--- a/tez-ui/src/main/webapp/app/scripts/controllers/dag_vertices.js
+++ b/tez-ui/src/main/webapp/app/scripts/controllers/dag_vertices.js
@@ -112,6 +112,26 @@ App.DagVerticesController = App.TablePageController.extend({
         contentPath: 'id',
       },
       {
+        id: 'status',
+        headerCellName: 'Status',
+        templateName: 'components/basic-table/status-cell',
+        contentPath: 'status',
+        getCellContent: function(row) {
+          var status = row.get('status'),
+              content = Ember.Object.create({
+                vertex: row,
+                status: status,
+                statusIcon: App.Helpers.misc.getStatusClassForEntity(status,
+                  row.get('hasFailedTaskAttempts'))
+              });
+
+          if(status == 'RUNNING') {
+            row.addObserver('progress', content, onProgressChange);
+          }
+          return content;
+        }
+      },
+      {
         id: 'startTime',
         headerCellName: 'Start Time',
         contentPath: 'startTime',
@@ -166,26 +186,6 @@ App.DagVerticesController = App.TablePageController.extend({
         contentPath: 'processorClassName'
       },
       {
-        id: 'status',
-        headerCellName: 'Status',
-        templateName: 'components/basic-table/status-cell',
-        contentPath: 'status',
-        getCellContent: function(row) {
-          var status = row.get('status'),
-              content = Ember.Object.create({
-                vertex: row,
-                status: status,
-                statusIcon: App.Helpers.misc.getStatusClassForEntity(status,
-                  row.get('hasFailedTaskAttempts'))
-              });
-
-          if(status == 'RUNNING') {
-            row.addObserver('progress', content, onProgressChange);
-          }
-          return content;
-        }
-      },
-      {
         id: 'configurations',
         headerCellName: 'Source/Sink Configs',
         templateName: 'components/basic-table/vertex-configurations-cell',

http://git-wip-us.apache.org/repos/asf/tez/blob/7fe0a6b7/tez-ui/src/main/webapp/app/scripts/controllers/task_task_attempts_controller.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/scripts/controllers/task_task_attempts_controller.js b/tez-ui/src/main/webapp/app/scripts/controllers/task_task_attempts_controller.js
index d211479..8e491a1 100644
--- a/tez-ui/src/main/webapp/app/scripts/controllers/task_task_attempts_controller.js
+++ b/tez-ui/src/main/webapp/app/scripts/controllers/task_task_attempts_controller.js
@@ -84,6 +84,19 @@ App.TaskAttemptsController = App.TablePageController.extend(App.AutoCounterColum
         }
       },
       {
+        id: 'status',
+        headerCellName: 'Status',
+        templateName: 'components/basic-table/status-cell',
+        contentPath: 'status',
+        getCellContent: function(row) {
+          var status = App.Helpers.misc.getFixedupDisplayStatus(row.get('status'));
+          return {
+            status: status,
+            statusIcon: App.Helpers.misc.getStatusClassForEntity(status)
+          };
+        }
+      },
+      {
         id: 'startTime',
         headerCellName: 'Start Time',
         contentPath: 'startTime',
@@ -117,19 +130,6 @@ App.TaskAttemptsController = App.TablePageController.extend(App.AutoCounterColum
         },
       },
       {
-        id: 'status',
-        headerCellName: 'Status',
-        templateName: 'components/basic-table/status-cell',
-        contentPath: 'status',
-        getCellContent: function(row) {
-          var status = App.Helpers.misc.getFixedupDisplayStatus(row.get('status'));
-          return {
-            status: status,
-            statusIcon: App.Helpers.misc.getStatusClassForEntity(status)
-          };
-        }
-      },
-      {
         id: 'containerId',
         headerCellName: 'Container',
         contentPath: 'containerId'

http://git-wip-us.apache.org/repos/asf/tez/blob/7fe0a6b7/tez-ui/src/main/webapp/app/scripts/controllers/vertex_task_attempts_controller.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/scripts/controllers/vertex_task_attempts_controller.js b/tez-ui/src/main/webapp/app/scripts/controllers/vertex_task_attempts_controller.js
index b4ed89a..c4a52f5 100644
--- a/tez-ui/src/main/webapp/app/scripts/controllers/vertex_task_attempts_controller.js
+++ b/tez-ui/src/main/webapp/app/scripts/controllers/vertex_task_attempts_controller.js
@@ -102,6 +102,19 @@ App.VertexTaskAttemptsController = App.TablePageController.extend(App.AutoCounte
         }
       },
       {
+        id: 'status',
+        headerCellName: 'Status',
+        templateName: 'components/basic-table/status-cell',
+        contentPath: 'status',
+        getCellContent: function(row) {
+          var status = App.Helpers.misc.getFixedupDisplayStatus(row.get('status'));
+          return {
+            status: status,
+            statusIcon: App.Helpers.misc.getStatusClassForEntity(status)
+          };
+        }
+      },
+      {
         id: 'startTime',
         headerCellName: 'Start Time',
         contentPath: 'startTime',
@@ -135,19 +148,6 @@ App.VertexTaskAttemptsController = App.TablePageController.extend(App.AutoCounte
         },
       },
       {
-        id: 'status',
-        headerCellName: 'Status',
-        templateName: 'components/basic-table/status-cell',
-        contentPath: 'status',
-        getCellContent: function(row) {
-          var status = App.Helpers.misc.getFixedupDisplayStatus(row.get('status'));
-          return {
-            status: status,
-            statusIcon: App.Helpers.misc.getStatusClassForEntity(status)
-          };
-        }
-      },
-      {
         id: 'containerId',
         headerCellName: 'Container',
         contentPath: 'containerId'

http://git-wip-us.apache.org/repos/asf/tez/blob/7fe0a6b7/tez-ui/src/main/webapp/app/scripts/controllers/vertex_tasks_controller.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/scripts/controllers/vertex_tasks_controller.js b/tez-ui/src/main/webapp/app/scripts/controllers/vertex_tasks_controller.js
index 2cc0518..4735eac 100644
--- a/tez-ui/src/main/webapp/app/scripts/controllers/vertex_tasks_controller.js
+++ b/tez-ui/src/main/webapp/app/scripts/controllers/vertex_tasks_controller.js
@@ -105,6 +105,20 @@ App.VertexTasksController = App.TablePageController.extend(App.AutoCounterColumn
         }
       },
       {
+        id: 'status',
+        headerCellName: 'Status',
+        templateName: 'components/basic-table/status-cell',
+        contentPath: 'status',
+        getCellContent: function(row) {
+          var status = row.get('status');
+          return {
+            status: status,
+            statusIcon: App.Helpers.misc.getStatusClassForEntity(status,
+              row.get('hasFailedTaskAttempts'))
+          };
+        }
+      },
+      {
         id: 'startTime',
         headerCellName: 'Start Time',
         contentPath: 'startTime',
@@ -138,20 +152,6 @@ App.VertexTasksController = App.TablePageController.extend(App.AutoCounterColumn
         },
       },
       {
-        id: 'status',
-        headerCellName: 'Status',
-        templateName: 'components/basic-table/status-cell',
-        contentPath: 'status',
-        getCellContent: function(row) {
-          var status = row.get('status');
-          return {
-            status: status,
-            statusIcon: App.Helpers.misc.getStatusClassForEntity(status,
-              row.get('hasFailedTaskAttempts'))
-          };
-        }
-      },
-      {
         id: 'actions',
         headerCellName: 'Actions',
         templateName: 'components/basic-table/task-actions-cell',


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

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


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

Branch: refs/heads/TEZ-2003
Commit: 4cb8f01d74f0dfe6eed1860181af282d323645b9
Parents: fac7b1d
Author: Siddharth Seth <ss...@apache.org>
Authored: Fri Feb 20 16:12:52 2015 -0800
Committer: Siddharth Seth <ss...@apache.org>
Committed: Mon Jun 1 14:09:26 2015 -0700

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


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

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

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

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

http://git-wip-us.apache.org/repos/asf/tez/blob/4cb8f01d/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 49bf592..03d1456 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
@@ -514,7 +514,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/4cb8f01d/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 be97309..bf00137 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
@@ -119,7 +119,7 @@ public class TestTaskAttemptListenerImplTezDag {
     doReturn(container).when(amContainer).getContainer();
 
     taskAttemptListener = new TaskAttemptListenerImplForTest(appContext,
-        mock(TaskHeartbeatHandler.class), mock(ContainerHeartbeatHandler.class), null, null);
+        mock(TaskHeartbeatHandler.class), mock(ContainerHeartbeatHandler.class), null, null, false);
 
     taskSpec = mock(TaskSpec.class);
     doReturn(taskAttemptID).when(taskSpec).getTaskAttemptID();

http://git-wip-us.apache.org/repos/asf/tez/blob/4cb8f01d/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/4cb8f01d/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/4cb8f01d/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 8f19e99..04cdb5a 100644
--- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java
@@ -66,6 +66,7 @@ import org.apache.tez.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;
@@ -95,7 +96,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;
@@ -110,9 +110,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;
 
@@ -121,7 +122,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;
@@ -135,6 +136,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);
@@ -163,25 +166,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;
     }
   }
   
@@ -360,7 +365,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();
@@ -368,12 +373,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),
@@ -419,7 +418,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.
@@ -432,7 +432,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 {
@@ -466,7 +466,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-2090. Add tests for jobs running in external services. (sseth)

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

http://git-wip-us.apache.org/repos/asf/tez/blob/aa6c53ff/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/TezTestService.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/TezTestService.java b/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/TezTestService.java
new file mode 100644
index 0000000..012e352
--- /dev/null
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/TezTestService.java
@@ -0,0 +1,126 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.service.impl;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.Arrays;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.service.AbstractService;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.log4j.Logger;
+import org.apache.tez.service.ContainerRunner;
+import org.apache.tez.shufflehandler.ShuffleHandler;
+import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos;
+import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.RunContainerRequestProto;
+
+public class TezTestService extends AbstractService implements ContainerRunner {
+
+  private static final Logger LOG = Logger.getLogger(TezTestService.class);
+
+  private final Configuration shuffleHandlerConf;
+  private final int numExecutors;
+
+  private final TezTestServiceProtocolServerImpl server;
+  private final ContainerRunnerImpl containerRunner;
+  private final String[] localDirs;
+
+  private final AtomicInteger numSubmissions = new AtomicInteger(0);
+
+
+  private final AtomicReference<InetSocketAddress> address = new AtomicReference<InetSocketAddress>();
+
+  public TezTestService(Configuration conf, int numExecutors, long memoryAvailable, String[] localDirs) {
+    super(TezTestService.class.getSimpleName());
+    this.numExecutors = numExecutors;
+    this.localDirs = localDirs;
+
+    long memoryAvailableBytes = memoryAvailable;
+    long jvmMax = Runtime.getRuntime().maxMemory();
+
+    LOG.info(TezTestService.class.getSimpleName() + " created with the following configuration: " +
+        "numExecutors=" + numExecutors +
+        ", workDirs=" + Arrays.toString(localDirs) +
+        ", memoryAvailable=" + memoryAvailable +
+        ", jvmMaxMemory=" + jvmMax);
+
+    Preconditions.checkArgument(this.numExecutors > 0);
+    Preconditions.checkArgument(this.localDirs != null && this.localDirs.length > 0,
+        "Work dirs must be specified");
+    Preconditions.checkState(jvmMax >= memoryAvailableBytes,
+        "Invalid configuration. Xmx value too small. maxAvailable=" + jvmMax + ", configured=" +
+            memoryAvailableBytes);
+
+    this.shuffleHandlerConf = new Configuration(conf);
+    // Start Shuffle on a random port
+    this.shuffleHandlerConf.setInt(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY, 0);
+    this.shuffleHandlerConf.set(ShuffleHandler.SHUFFLE_HANDLER_LOCAL_DIRS, StringUtils.arrayToString(localDirs));
+
+    this.server = new TezTestServiceProtocolServerImpl(this, address);
+    this.containerRunner = new ContainerRunnerImpl(numExecutors, localDirs, address,
+        memoryAvailableBytes);
+  }
+
+  @Override
+  public void serviceInit(Configuration conf) {
+    server.init(conf);
+    containerRunner.init(conf);
+  }
+
+  @Override
+  public void serviceStart() throws Exception {
+    ShuffleHandler.initializeAndStart(shuffleHandlerConf);
+    containerRunner.setShufflePort(ShuffleHandler.get().getPort());
+    server.start();
+    containerRunner.start();
+  }
+
+  public void serviceStop() throws Exception {
+    containerRunner.stop();
+    server.stop();
+    ShuffleHandler.get().stop();
+  }
+
+  public InetSocketAddress getListenerAddress() {
+    return server.getBindAddress();
+  }
+
+  public int getShufflePort() {
+    return ShuffleHandler.get().getPort();
+  }
+
+
+
+  @Override
+  public void queueContainer(RunContainerRequestProto request) throws IOException {
+    numSubmissions.incrementAndGet();
+    containerRunner.queueContainer(request);
+  }
+
+  @Override
+  public void submitWork(TezTestServiceProtocolProtos.SubmitWorkRequestProto request) throws
+      IOException {
+    numSubmissions.incrementAndGet();
+    containerRunner.submitWork(request);
+  }
+
+  public int getNumSubmissions() {
+    return numSubmissions.get();
+  }
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/aa6c53ff/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/TezTestServiceProtocolClientImpl.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/TezTestServiceProtocolClientImpl.java b/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/TezTestServiceProtocolClientImpl.java
new file mode 100644
index 0000000..10d2952
--- /dev/null
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/TezTestServiceProtocolClientImpl.java
@@ -0,0 +1,82 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.service.impl;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+
+import com.google.protobuf.RpcController;
+import com.google.protobuf.ServiceException;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.ipc.ProtobufRpcEngine;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.tez.service.TezTestServiceProtocolBlockingPB;
+import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos;
+import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.RunContainerRequestProto;
+import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.RunContainerResponseProto;
+
+
+public class TezTestServiceProtocolClientImpl implements TezTestServiceProtocolBlockingPB {
+
+  private final Configuration conf;
+  private final InetSocketAddress serverAddr;
+  TezTestServiceProtocolBlockingPB proxy;
+
+
+  public TezTestServiceProtocolClientImpl(Configuration conf, String hostname, int port) {
+    this.conf = conf;
+    this.serverAddr = NetUtils.createSocketAddr(hostname, port);
+  }
+
+  @Override
+  public RunContainerResponseProto runContainer(RpcController controller,
+                                                RunContainerRequestProto request) throws
+      ServiceException {
+    try {
+      return getProxy().runContainer(null, request);
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+  @Override
+  public TezTestServiceProtocolProtos.SubmitWorkResponseProto submitWork(RpcController controller,
+                                                                         TezTestServiceProtocolProtos.SubmitWorkRequestProto request) throws
+      ServiceException {
+    try {
+      return getProxy().submitWork(null, request);
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+
+  public TezTestServiceProtocolBlockingPB getProxy() throws IOException {
+    if (proxy == null) {
+      proxy = createProxy();
+    }
+    return proxy;
+  }
+
+  public TezTestServiceProtocolBlockingPB createProxy() throws IOException {
+    TezTestServiceProtocolBlockingPB p;
+    // TODO Fix security
+    RPC.setProtocolEngine(conf, TezTestServiceProtocolBlockingPB.class, ProtobufRpcEngine.class);
+    p = (TezTestServiceProtocolBlockingPB) RPC
+        .getProxy(TezTestServiceProtocolBlockingPB.class, 0, serverAddr, conf);
+    return p;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tez/blob/aa6c53ff/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/TezTestServiceProtocolServerImpl.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/TezTestServiceProtocolServerImpl.java b/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/TezTestServiceProtocolServerImpl.java
new file mode 100644
index 0000000..d7f8444
--- /dev/null
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/TezTestServiceProtocolServerImpl.java
@@ -0,0 +1,133 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.service.impl;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.concurrent.atomic.AtomicReference;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.protobuf.BlockingService;
+import com.google.protobuf.RpcController;
+import com.google.protobuf.ServiceException;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.ipc.ProtobufRpcEngine;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.service.AbstractService;
+import org.apache.tez.service.ContainerRunner;
+import org.apache.tez.service.TezTestServiceProtocolBlockingPB;
+import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos;
+import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.RunContainerRequestProto;
+import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.RunContainerResponseProto;
+import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.SubmitWorkResponseProto;
+
+public class TezTestServiceProtocolServerImpl extends AbstractService
+    implements TezTestServiceProtocolBlockingPB {
+
+  private static final Log LOG = LogFactory.getLog(TezTestServiceProtocolServerImpl.class);
+
+  private final ContainerRunner containerRunner;
+  private RPC.Server server;
+  private final AtomicReference<InetSocketAddress> bindAddress;
+
+
+  public TezTestServiceProtocolServerImpl(ContainerRunner containerRunner,
+                                          AtomicReference<InetSocketAddress> address) {
+    super(TezTestServiceProtocolServerImpl.class.getSimpleName());
+    this.containerRunner = containerRunner;
+    this.bindAddress = address;
+  }
+
+  @Override
+  public RunContainerResponseProto runContainer(RpcController controller,
+                                                RunContainerRequestProto request) throws
+      ServiceException {
+    LOG.info("Received request: " + request);
+    try {
+      containerRunner.queueContainer(request);
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+    return RunContainerResponseProto.getDefaultInstance();
+  }
+
+  @Override
+  public SubmitWorkResponseProto submitWork(RpcController controller, TezTestServiceProtocolProtos.SubmitWorkRequestProto request) throws
+      ServiceException {
+    LOG.info("Received submitWork request: " + request);
+    try {
+      containerRunner.submitWork(request);
+    } catch (IOException e) {
+      e.printStackTrace();
+    }
+    return SubmitWorkResponseProto.getDefaultInstance();
+  }
+
+
+  @Override
+  public void serviceStart() {
+    Configuration conf = getConfig();
+
+    int numHandlers = 3;
+    InetSocketAddress addr = new InetSocketAddress(0);
+
+    try {
+      server = createServer(TezTestServiceProtocolBlockingPB.class, addr, conf, numHandlers,
+          TezTestServiceProtocolProtos.TezTestServiceProtocol.newReflectiveBlockingService(this));
+      server.start();
+    } catch (IOException e) {
+      LOG.error("Failed to run RPC Server", e);
+      throw new RuntimeException(e);
+    }
+
+    InetSocketAddress serverBindAddress = NetUtils.getConnectAddress(server);
+    this.bindAddress.set(NetUtils.createSocketAddrForHost(
+        serverBindAddress.getAddress().getCanonicalHostName(),
+        serverBindAddress.getPort()));
+    LOG.info("Instantiated TestTestServiceListener at " + bindAddress);
+  }
+
+  @Override
+  public void serviceStop() {
+    if (server != null) {
+      server.stop();
+    }
+  }
+
+  @InterfaceAudience.Private
+  @VisibleForTesting
+  InetSocketAddress getBindAddress() {
+    return this.bindAddress.get();
+  }
+
+  private RPC.Server createServer(Class<?> pbProtocol, InetSocketAddress addr, Configuration conf,
+                                  int numHandlers, BlockingService blockingService) throws
+      IOException {
+    RPC.setProtocolEngine(conf, pbProtocol, ProtobufRpcEngine.class);
+    RPC.Server server = new RPC.Builder(conf)
+        .setProtocol(pbProtocol)
+        .setInstance(blockingService)
+        .setBindAddress(addr.getHostName())
+        .setPort(0)
+        .setNumHandlers(numHandlers)
+        .build();
+    // TODO Add security.
+    return server;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/aa6c53ff/tez-ext-service-tests/src/test/java/org/apache/tez/shufflehandler/FadvisedChunkedFile.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/shufflehandler/FadvisedChunkedFile.java b/tez-ext-service-tests/src/test/java/org/apache/tez/shufflehandler/FadvisedChunkedFile.java
new file mode 100644
index 0000000..65588fe
--- /dev/null
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/shufflehandler/FadvisedChunkedFile.java
@@ -0,0 +1,78 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.shufflehandler;
+
+import java.io.FileDescriptor;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.io.ReadaheadPool;
+import org.apache.hadoop.io.ReadaheadPool.ReadaheadRequest;
+import org.apache.hadoop.io.nativeio.NativeIO;
+import org.jboss.netty.handler.stream.ChunkedFile;
+
+public class FadvisedChunkedFile extends ChunkedFile {
+
+  private static final Log LOG = LogFactory.getLog(FadvisedChunkedFile.class);
+
+  private final boolean manageOsCache;
+  private final int readaheadLength;
+  private final ReadaheadPool readaheadPool;
+  private final FileDescriptor fd;
+  private final String identifier;
+
+  private ReadaheadRequest readaheadRequest;
+
+  public FadvisedChunkedFile(RandomAccessFile file, long position, long count,
+      int chunkSize, boolean manageOsCache, int readaheadLength,
+      ReadaheadPool readaheadPool, String identifier) throws IOException {
+    super(file, position, count, chunkSize);
+    this.manageOsCache = manageOsCache;
+    this.readaheadLength = readaheadLength;
+    this.readaheadPool = readaheadPool;
+    this.fd = file.getFD();
+    this.identifier = identifier;
+  }
+
+  @Override
+  public Object nextChunk() throws Exception {
+    if (manageOsCache && readaheadPool != null) {
+      readaheadRequest = readaheadPool
+          .readaheadStream(identifier, fd, getCurrentOffset(), readaheadLength,
+              getEndOffset(), readaheadRequest);
+    }
+    return super.nextChunk();
+  }
+
+  @Override
+  public void close() throws Exception {
+    if (readaheadRequest != null) {
+      readaheadRequest.cancel();
+    }
+    if (manageOsCache && getEndOffset() - getStartOffset() > 0) {
+      try {
+        NativeIO.POSIX.getCacheManipulator().posixFadviseIfPossible(identifier,
+            fd,
+            getStartOffset(), getEndOffset() - getStartOffset(),
+            NativeIO.POSIX.POSIX_FADV_DONTNEED);
+      } catch (Throwable t) {
+        LOG.warn("Failed to manage OS cache for " + identifier, t);
+      }
+    }
+    super.close();
+  }
+}

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

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

http://git-wip-us.apache.org/repos/asf/tez/blob/aa6c53ff/tez-ext-service-tests/src/test/java/org/apache/tez/shufflehandler/ShuffleHandler.java
----------------------------------------------------------------------
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/shufflehandler/ShuffleHandler.java b/tez-ext-service-tests/src/test/java/org/apache/tez/shufflehandler/ShuffleHandler.java
new file mode 100644
index 0000000..cc82d74
--- /dev/null
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/shufflehandler/ShuffleHandler.java
@@ -0,0 +1,840 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.shufflehandler;
+
+import static org.jboss.netty.buffer.ChannelBuffers.wrappedBuffer;
+import static org.jboss.netty.handler.codec.http.HttpHeaders.Names.CONTENT_TYPE;
+import static org.jboss.netty.handler.codec.http.HttpMethod.GET;
+import static org.jboss.netty.handler.codec.http.HttpResponseStatus.BAD_REQUEST;
+import static org.jboss.netty.handler.codec.http.HttpResponseStatus.FORBIDDEN;
+import static org.jboss.netty.handler.codec.http.HttpResponseStatus.INTERNAL_SERVER_ERROR;
+import static org.jboss.netty.handler.codec.http.HttpResponseStatus.METHOD_NOT_ALLOWED;
+import static org.jboss.netty.handler.codec.http.HttpResponseStatus.NOT_FOUND;
+import static org.jboss.netty.handler.codec.http.HttpResponseStatus.OK;
+import static org.jboss.netty.handler.codec.http.HttpResponseStatus.UNAUTHORIZED;
+import static org.jboss.netty.handler.codec.http.HttpVersion.HTTP_1_1;
+
+import javax.crypto.SecretKey;
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.net.InetSocketAddress;
+import java.net.URL;
+import java.nio.ByteBuffer;
+import java.nio.channels.ClosedChannelException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.regex.Pattern;
+
+import com.google.common.base.Charsets;
+import com.google.common.base.Preconditions;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.LocalDirAllocator;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.DataInputByteBuffer;
+import org.apache.hadoop.io.DataOutputBuffer;
+import org.apache.hadoop.io.ReadaheadPool;
+import org.apache.hadoop.io.SecureIOUtils;
+import org.apache.hadoop.metrics2.annotation.Metric;
+import org.apache.hadoop.metrics2.annotation.Metrics;
+import org.apache.hadoop.metrics2.lib.MutableCounterInt;
+import org.apache.hadoop.metrics2.lib.MutableCounterLong;
+import org.apache.hadoop.metrics2.lib.MutableGaugeInt;
+import org.apache.hadoop.security.ssl.SSLFactory;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.util.Shell;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.util.ConverterUtils;
+import org.apache.tez.common.security.JobTokenIdentifier;
+import org.apache.tez.common.security.JobTokenSecretManager;
+import org.apache.tez.runtime.library.common.security.SecureShuffleUtils;
+import org.apache.tez.runtime.library.common.shuffle.orderedgrouped.ShuffleHeader;
+import org.apache.tez.runtime.library.common.sort.impl.TezIndexRecord;
+import org.jboss.netty.bootstrap.ServerBootstrap;
+import org.jboss.netty.buffer.ChannelBuffers;
+import org.jboss.netty.channel.Channel;
+import org.jboss.netty.channel.ChannelFactory;
+import org.jboss.netty.channel.ChannelFuture;
+import org.jboss.netty.channel.ChannelFutureListener;
+import org.jboss.netty.channel.ChannelHandlerContext;
+import org.jboss.netty.channel.ChannelPipeline;
+import org.jboss.netty.channel.ChannelPipelineFactory;
+import org.jboss.netty.channel.ChannelStateEvent;
+import org.jboss.netty.channel.Channels;
+import org.jboss.netty.channel.ExceptionEvent;
+import org.jboss.netty.channel.MessageEvent;
+import org.jboss.netty.channel.SimpleChannelUpstreamHandler;
+import org.jboss.netty.channel.group.ChannelGroup;
+import org.jboss.netty.channel.group.DefaultChannelGroup;
+import org.jboss.netty.channel.socket.nio.NioServerSocketChannelFactory;
+import org.jboss.netty.handler.codec.frame.TooLongFrameException;
+import org.jboss.netty.handler.codec.http.DefaultHttpResponse;
+import org.jboss.netty.handler.codec.http.HttpChunkAggregator;
+import org.jboss.netty.handler.codec.http.HttpHeaders;
+import org.jboss.netty.handler.codec.http.HttpRequest;
+import org.jboss.netty.handler.codec.http.HttpRequestDecoder;
+import org.jboss.netty.handler.codec.http.HttpResponse;
+import org.jboss.netty.handler.codec.http.HttpResponseEncoder;
+import org.jboss.netty.handler.codec.http.HttpResponseStatus;
+import org.jboss.netty.handler.codec.http.QueryStringDecoder;
+import org.jboss.netty.handler.ssl.SslHandler;
+import org.jboss.netty.handler.stream.ChunkedWriteHandler;
+import org.jboss.netty.util.CharsetUtil;
+
+public class ShuffleHandler {
+
+  private static final Log LOG = LogFactory.getLog(ShuffleHandler.class);
+
+  public static final String SHUFFLE_HANDLER_LOCAL_DIRS = "tez.shuffle.handler.local-dirs";
+
+  public static final String SHUFFLE_MANAGE_OS_CACHE = "mapreduce.shuffle.manage.os.cache";
+  public static final boolean DEFAULT_SHUFFLE_MANAGE_OS_CACHE = true;
+
+  public static final String SHUFFLE_READAHEAD_BYTES = "mapreduce.shuffle.readahead.bytes";
+  public static final int DEFAULT_SHUFFLE_READAHEAD_BYTES = 4 * 1024 * 1024;
+  
+  // pattern to identify errors related to the client closing the socket early
+  // idea borrowed from Netty SslHandler
+  private static final Pattern IGNORABLE_ERROR_MESSAGE = Pattern.compile(
+      "^.*(?:connection.*reset|connection.*closed|broken.*pipe).*$",
+      Pattern.CASE_INSENSITIVE);
+
+  private int port;
+  private final ChannelFactory selector;
+  private final ChannelGroup accepted = new DefaultChannelGroup();
+  protected HttpPipelineFactory pipelineFact;
+  private final int sslFileBufferSize;
+  private final Configuration conf;
+
+  private final ConcurrentMap<String, Boolean> registeredApps = new ConcurrentHashMap<String, Boolean>();
+
+  /**
+   * Should the shuffle use posix_fadvise calls to manage the OS cache during
+   * sendfile
+   */
+  private final boolean manageOsCache;
+  private final int readaheadLength;
+  private final int maxShuffleConnections;
+  private final int shuffleBufferSize;
+  private final boolean shuffleTransferToAllowed;
+  private final ReadaheadPool readaheadPool = ReadaheadPool.getInstance();
+
+  private Map<String,String> userRsrc;
+  private JobTokenSecretManager secretManager;
+
+  // TODO Fix this for tez.
+  public static final String MAPREDUCE_SHUFFLE_SERVICEID =
+      "mapreduce_shuffle";
+
+  public static final String SHUFFLE_PORT_CONFIG_KEY = "tez.shuffle.port";
+  public static final int DEFAULT_SHUFFLE_PORT = 15551;
+
+  // TODO Change configs to remove mapreduce references.
+  public static final String SHUFFLE_CONNECTION_KEEP_ALIVE_ENABLED =
+      "mapreduce.shuffle.connection-keep-alive.enable";
+  public static final boolean DEFAULT_SHUFFLE_CONNECTION_KEEP_ALIVE_ENABLED = false;
+
+  public static final String SHUFFLE_CONNECTION_KEEP_ALIVE_TIME_OUT =
+      "mapreduce.shuffle.connection-keep-alive.timeout";
+  public static final int DEFAULT_SHUFFLE_CONNECTION_KEEP_ALIVE_TIME_OUT = 5; //seconds
+
+  public static final String SHUFFLE_MAPOUTPUT_META_INFO_CACHE_SIZE =
+      "mapreduce.shuffle.mapoutput-info.meta.cache.size";
+  public static final int DEFAULT_SHUFFLE_MAPOUTPUT_META_INFO_CACHE_SIZE =
+      1000;
+
+  public static final String CONNECTION_CLOSE = "close";
+
+  public static final String SUFFLE_SSL_FILE_BUFFER_SIZE_KEY =
+    "mapreduce.shuffle.ssl.file.buffer.size";
+
+  public static final int DEFAULT_SUFFLE_SSL_FILE_BUFFER_SIZE = 60 * 1024;
+
+  public static final String MAX_SHUFFLE_CONNECTIONS = "mapreduce.shuffle.max.connections";
+  public static final int DEFAULT_MAX_SHUFFLE_CONNECTIONS = 0; // 0 implies no limit
+  
+  public static final String MAX_SHUFFLE_THREADS = "mapreduce.shuffle.max.threads";
+  // 0 implies Netty default of 2 * number of available processors
+  public static final int DEFAULT_MAX_SHUFFLE_THREADS = 0;
+  
+  public static final String SHUFFLE_BUFFER_SIZE = 
+      "mapreduce.shuffle.transfer.buffer.size";
+  public static final int DEFAULT_SHUFFLE_BUFFER_SIZE = 128 * 1024;
+  
+  public static final String  SHUFFLE_TRANSFERTO_ALLOWED = 
+      "mapreduce.shuffle.transferTo.allowed";
+  public static final boolean DEFAULT_SHUFFLE_TRANSFERTO_ALLOWED = true;
+  public static final boolean WINDOWS_DEFAULT_SHUFFLE_TRANSFERTO_ALLOWED = 
+      false;
+
+  final boolean connectionKeepAliveEnabled;
+  final int connectionKeepAliveTimeOut;
+  final int mapOutputMetaInfoCacheSize;
+  private static final AtomicBoolean started = new AtomicBoolean(false);
+  private static final AtomicBoolean initing = new AtomicBoolean(false);
+  private static ShuffleHandler INSTANCE;
+
+  @Metrics(about="Shuffle output metrics", context="mapred")
+  static class ShuffleMetrics implements ChannelFutureListener {
+    @Metric("Shuffle output in bytes")
+    MutableCounterLong shuffleOutputBytes;
+    @Metric("# of failed shuffle outputs")
+    MutableCounterInt shuffleOutputsFailed;
+    @Metric("# of succeeeded shuffle outputs")
+    MutableCounterInt shuffleOutputsOK;
+    @Metric("# of current shuffle connections")
+    MutableGaugeInt shuffleConnections;
+
+    @Override
+    public void operationComplete(ChannelFuture future) throws Exception {
+      if (future.isSuccess()) {
+        shuffleOutputsOK.incr();
+      } else {
+        shuffleOutputsFailed.incr();
+      }
+      shuffleConnections.decr();
+    }
+  }
+
+  public ShuffleHandler(Configuration conf) {
+    this.conf = conf;
+    manageOsCache = conf.getBoolean(SHUFFLE_MANAGE_OS_CACHE,
+        DEFAULT_SHUFFLE_MANAGE_OS_CACHE);
+
+    readaheadLength = conf.getInt(SHUFFLE_READAHEAD_BYTES,
+        DEFAULT_SHUFFLE_READAHEAD_BYTES);
+
+    maxShuffleConnections = conf.getInt(MAX_SHUFFLE_CONNECTIONS,
+        DEFAULT_MAX_SHUFFLE_CONNECTIONS);
+    int maxShuffleThreads = conf.getInt(MAX_SHUFFLE_THREADS,
+        DEFAULT_MAX_SHUFFLE_THREADS);
+    if (maxShuffleThreads == 0) {
+      maxShuffleThreads = 2 * Runtime.getRuntime().availableProcessors();
+    }
+
+    shuffleBufferSize = conf.getInt(SHUFFLE_BUFFER_SIZE,
+        DEFAULT_SHUFFLE_BUFFER_SIZE);
+
+    shuffleTransferToAllowed = conf.getBoolean(SHUFFLE_TRANSFERTO_ALLOWED,
+        (Shell.WINDOWS)?WINDOWS_DEFAULT_SHUFFLE_TRANSFERTO_ALLOWED:
+            DEFAULT_SHUFFLE_TRANSFERTO_ALLOWED);
+
+    ThreadFactory bossFactory = new ThreadFactoryBuilder()
+        .setNameFormat("ShuffleHandler Netty Boss #%d")
+        .build();
+    ThreadFactory workerFactory = new ThreadFactoryBuilder()
+        .setNameFormat("ShuffleHandler Netty Worker #%d")
+        .build();
+
+    selector = new NioServerSocketChannelFactory(
+        Executors.newCachedThreadPool(bossFactory),
+        Executors.newCachedThreadPool(workerFactory),
+        maxShuffleThreads);
+
+    sslFileBufferSize = conf.getInt(SUFFLE_SSL_FILE_BUFFER_SIZE_KEY,
+        DEFAULT_SUFFLE_SSL_FILE_BUFFER_SIZE);
+    connectionKeepAliveEnabled =
+        conf.getBoolean(SHUFFLE_CONNECTION_KEEP_ALIVE_ENABLED,
+            DEFAULT_SHUFFLE_CONNECTION_KEEP_ALIVE_ENABLED);
+    connectionKeepAliveTimeOut =
+        Math.max(1, conf.getInt(SHUFFLE_CONNECTION_KEEP_ALIVE_TIME_OUT,
+            DEFAULT_SHUFFLE_CONNECTION_KEEP_ALIVE_TIME_OUT));
+    mapOutputMetaInfoCacheSize =
+        Math.max(1, conf.getInt(SHUFFLE_MAPOUTPUT_META_INFO_CACHE_SIZE,
+            DEFAULT_SHUFFLE_MAPOUTPUT_META_INFO_CACHE_SIZE));
+
+    userRsrc = new ConcurrentHashMap<String,String>();
+    secretManager = new JobTokenSecretManager();
+  }
+
+
+  public void start() throws Exception {
+    ServerBootstrap bootstrap = new ServerBootstrap(selector);
+    try {
+      pipelineFact = new HttpPipelineFactory(conf);
+    } catch (Exception ex) {
+      throw new RuntimeException(ex);
+    }
+    bootstrap.setPipelineFactory(pipelineFact);
+    port = conf.getInt(SHUFFLE_PORT_CONFIG_KEY, DEFAULT_SHUFFLE_PORT);
+    Channel ch = bootstrap.bind(new InetSocketAddress(port));
+    accepted.add(ch);
+    port = ((InetSocketAddress)ch.getLocalAddress()).getPort();
+    conf.set(SHUFFLE_PORT_CONFIG_KEY, Integer.toString(port));
+    pipelineFact.SHUFFLE.setPort(port);
+    LOG.info("TezShuffleHandler" + " listening on port " + port);
+  }
+
+  public static void initializeAndStart(Configuration conf) throws Exception {
+    if (!initing.getAndSet(true)) {
+      INSTANCE = new ShuffleHandler(conf);
+      INSTANCE.start();
+      started.set(true);
+    }
+  }
+
+  public static ShuffleHandler get() {
+    Preconditions.checkState(started.get(), "ShuffleHandler must be started before invoking started");
+    return INSTANCE;
+  }
+
+  /**
+   * Serialize the shuffle port into a ByteBuffer for use later on.
+   * @param port the port to be sent to the ApplciationMaster
+   * @return the serialized form of the port.
+   */
+  public static ByteBuffer serializeMetaData(int port) throws IOException {
+    //TODO these bytes should be versioned
+    DataOutputBuffer port_dob = new DataOutputBuffer();
+    port_dob.writeInt(port);
+    return ByteBuffer.wrap(port_dob.getData(), 0, port_dob.getLength());
+  }
+
+  /**
+   * A helper function to deserialize the metadata returned by ShuffleHandler.
+   * @param meta the metadata returned by the ShuffleHandler
+   * @return the port the Shuffle Handler is listening on to serve shuffle data.
+   */
+  public static int deserializeMetaData(ByteBuffer meta) throws IOException {
+    //TODO this should be returning a class not just an int
+    DataInputByteBuffer in = new DataInputByteBuffer();
+    in.reset(meta);
+    int port = in.readInt();
+    return port;
+  }
+
+  /**
+   * A helper function to serialize the JobTokenIdentifier to be sent to the
+   * ShuffleHandler as ServiceData.
+   * @param jobToken the job token to be used for authentication of
+   * shuffle data requests.
+   * @return the serialized version of the jobToken.
+   */
+  public static ByteBuffer serializeServiceData(Token<JobTokenIdentifier> jobToken) throws IOException {
+    //TODO these bytes should be versioned
+    DataOutputBuffer jobToken_dob = new DataOutputBuffer();
+    jobToken.write(jobToken_dob);
+    return ByteBuffer.wrap(jobToken_dob.getData(), 0, jobToken_dob.getLength());
+  }
+
+  static Token<JobTokenIdentifier> deserializeServiceData(ByteBuffer secret) throws IOException {
+    DataInputByteBuffer in = new DataInputByteBuffer();
+    in.reset(secret);
+    Token<JobTokenIdentifier> jt = new Token<JobTokenIdentifier>();
+    jt.readFields(in);
+    return jt;
+  }
+
+  public int getPort() {
+    return port;
+  }
+
+  public void registerApplication(String applicationIdString, Token<JobTokenIdentifier> appToken,
+                                  String user) {
+    Boolean registered = registeredApps.putIfAbsent(applicationIdString, Boolean.valueOf(true));
+    if (registered == null) {
+      recordJobShuffleInfo(applicationIdString, user, appToken);
+    }
+  }
+
+  public void unregisterApplication(String applicationIdString) {
+    removeJobShuffleInfo(applicationIdString);
+  }
+
+
+  public void stop() throws Exception {
+    accepted.close().awaitUninterruptibly(10, TimeUnit.SECONDS);
+    if (selector != null) {
+      ServerBootstrap bootstrap = new ServerBootstrap(selector);
+      bootstrap.releaseExternalResources();
+    }
+    if (pipelineFact != null) {
+      pipelineFact.destroy();
+    }
+  }
+
+  protected Shuffle getShuffle(Configuration conf) {
+    return new Shuffle(conf);
+  }
+
+
+  private void addJobToken(String appIdString, String user,
+      Token<JobTokenIdentifier> jobToken) {
+    String jobIdString = appIdString.replace("application", "job");
+    userRsrc.put(jobIdString, user);
+    secretManager.addTokenForJob(jobIdString, jobToken);
+    LOG.info("Added token for " + jobIdString);
+  }
+
+  private void recordJobShuffleInfo(String appIdString, String user,
+      Token<JobTokenIdentifier> jobToken) {
+    addJobToken(appIdString, user, jobToken);
+  }
+
+  private void removeJobShuffleInfo(String appIdString) {
+    secretManager.removeTokenForJob(appIdString);
+    userRsrc.remove(appIdString);
+  }
+
+  class HttpPipelineFactory implements ChannelPipelineFactory {
+
+    final Shuffle SHUFFLE;
+    private SSLFactory sslFactory;
+
+    public HttpPipelineFactory(Configuration conf) throws Exception {
+      SHUFFLE = getShuffle(conf);
+      // TODO Setup SSL Shuffle
+//      if (conf.getBoolean(MRConfig.SHUFFLE_SSL_ENABLED_KEY,
+//                          MRConfig.SHUFFLE_SSL_ENABLED_DEFAULT)) {
+//        LOG.info("Encrypted shuffle is enabled.");
+//        sslFactory = new SSLFactory(SSLFactory.Mode.SERVER, conf);
+//        sslFactory.init();
+//      }
+    }
+
+    public void destroy() {
+      if (sslFactory != null) {
+        sslFactory.destroy();
+      }
+    }
+
+    @Override
+    public ChannelPipeline getPipeline() throws Exception {
+      ChannelPipeline pipeline = Channels.pipeline();
+      if (sslFactory != null) {
+        pipeline.addLast("ssl", new SslHandler(sslFactory.createSSLEngine()));
+      }
+      pipeline.addLast("decoder", new HttpRequestDecoder());
+      pipeline.addLast("aggregator", new HttpChunkAggregator(1 << 16));
+      pipeline.addLast("encoder", new HttpResponseEncoder());
+      pipeline.addLast("chunking", new ChunkedWriteHandler());
+      pipeline.addLast("shuffle", SHUFFLE);
+      return pipeline;
+      // TODO factor security manager into pipeline
+      // TODO factor out encode/decode to permit binary shuffle
+      // TODO factor out decode of index to permit alt. models
+    }
+
+  }
+
+  class Shuffle extends SimpleChannelUpstreamHandler {
+
+    private final Configuration conf;
+    private final IndexCache indexCache;
+    private final LocalDirAllocator lDirAlloc =
+      new LocalDirAllocator(SHUFFLE_HANDLER_LOCAL_DIRS);
+    private int port;
+
+    public Shuffle(Configuration conf) {
+      this.conf = conf;
+      indexCache = new IndexCache(conf);
+      this.port = conf.getInt(SHUFFLE_PORT_CONFIG_KEY, DEFAULT_SHUFFLE_PORT);
+    }
+    
+    public void setPort(int port) {
+      this.port = port;
+    }
+
+    private List<String> splitMaps(List<String> mapq) {
+      if (null == mapq) {
+        return null;
+      }
+      final List<String> ret = new ArrayList<String>();
+      for (String s : mapq) {
+        Collections.addAll(ret, s.split(","));
+      }
+      return ret;
+    }
+
+    @Override
+    public void channelOpen(ChannelHandlerContext ctx, ChannelStateEvent evt) 
+        throws Exception {
+      if ((maxShuffleConnections > 0) && (accepted.size() >= maxShuffleConnections)) {
+        LOG.info(String.format("Current number of shuffle connections (%d) is " + 
+            "greater than or equal to the max allowed shuffle connections (%d)", 
+            accepted.size(), maxShuffleConnections));
+        evt.getChannel().close();
+        return;
+      }
+      accepted.add(evt.getChannel());
+      super.channelOpen(ctx, evt);
+     
+    }
+
+    @Override
+    public void messageReceived(ChannelHandlerContext ctx, MessageEvent evt)
+        throws Exception {
+      HttpRequest request = (HttpRequest) evt.getMessage();
+      if (request.getMethod() != GET) {
+          sendError(ctx, METHOD_NOT_ALLOWED);
+          return;
+      }
+      // Check whether the shuffle version is compatible
+      if (!ShuffleHeader.DEFAULT_HTTP_HEADER_NAME.equals(
+          request.getHeader(ShuffleHeader.HTTP_HEADER_NAME))
+          || !ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION.equals(
+              request.getHeader(ShuffleHeader.HTTP_HEADER_VERSION))) {
+        sendError(ctx, "Incompatible shuffle request version", BAD_REQUEST);
+      }
+      final Map<String,List<String>> q =
+        new QueryStringDecoder(request.getUri()).getParameters();
+      final List<String> keepAliveList = q.get("keepAlive");
+      boolean keepAliveParam = false;
+      if (keepAliveList != null && keepAliveList.size() == 1) {
+        keepAliveParam = Boolean.valueOf(keepAliveList.get(0));
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("KeepAliveParam : " + keepAliveList
+            + " : " + keepAliveParam);
+        }
+      }
+      final List<String> mapIds = splitMaps(q.get("map"));
+      final List<String> reduceQ = q.get("reduce");
+      final List<String> jobQ = q.get("job");
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("RECV: " + request.getUri() +
+            "\n  mapId: " + mapIds +
+            "\n  reduceId: " + reduceQ +
+            "\n  jobId: " + jobQ +
+            "\n  keepAlive: " + keepAliveParam);
+      }
+
+      if (mapIds == null || reduceQ == null || jobQ == null) {
+        sendError(ctx, "Required param job, map and reduce", BAD_REQUEST);
+        return;
+      }
+      if (reduceQ.size() != 1 || jobQ.size() != 1) {
+        sendError(ctx, "Too many job/reduce parameters", BAD_REQUEST);
+        return;
+      }
+      int reduceId;
+      String jobId;
+      try {
+        reduceId = Integer.parseInt(reduceQ.get(0));
+        jobId = jobQ.get(0);
+      } catch (NumberFormatException e) {
+        sendError(ctx, "Bad reduce parameter", BAD_REQUEST);
+        return;
+      } catch (IllegalArgumentException e) {
+        sendError(ctx, "Bad job parameter", BAD_REQUEST);
+        return;
+      }
+      final String reqUri = request.getUri();
+      if (null == reqUri) {
+        // TODO? add upstream?
+        sendError(ctx, FORBIDDEN);
+        return;
+      }
+      HttpResponse response = new DefaultHttpResponse(HTTP_1_1, OK);
+      try {
+        verifyRequest(jobId, ctx, request, response,
+            new URL("http", "", this.port, reqUri));
+      } catch (IOException e) {
+        LOG.warn("Shuffle failure ", e);
+        sendError(ctx, e.getMessage(), UNAUTHORIZED);
+        return;
+      }
+
+      Map<String, MapOutputInfo> mapOutputInfoMap =
+          new HashMap<String, MapOutputInfo>();
+      Channel ch = evt.getChannel();
+      String user = userRsrc.get(jobId);
+
+      // $x/$user/appcache/$appId/output/$mapId
+      // TODO: Once Shuffle is out of NM, this can use MR APIs to convert
+      // between App and Job
+      String outputBasePathStr = getBaseLocation(jobId, user);
+
+      try {
+        populateHeaders(mapIds, outputBasePathStr, user, reduceId, request,
+          response, keepAliveParam, mapOutputInfoMap);
+      } catch(IOException e) {
+        ch.write(response);
+        LOG.error("Shuffle error in populating headers :", e);
+        String errorMessage = getErrorMessage(e);
+        sendError(ctx,errorMessage , INTERNAL_SERVER_ERROR);
+        return;
+      }
+      ch.write(response);
+      // TODO refactor the following into the pipeline
+      ChannelFuture lastMap = null;
+      for (String mapId : mapIds) {
+        try {
+          MapOutputInfo info = mapOutputInfoMap.get(mapId);
+          if (info == null) {
+            info = getMapOutputInfo(outputBasePathStr, mapId, reduceId, user);
+          }
+          lastMap =
+              sendMapOutput(ctx, ch, user, mapId,
+                reduceId, info);
+          if (null == lastMap) {
+            sendError(ctx, NOT_FOUND);
+            return;
+          }
+        } catch (IOException e) {
+          LOG.error("Shuffle error :", e);
+          String errorMessage = getErrorMessage(e);
+          sendError(ctx,errorMessage , INTERNAL_SERVER_ERROR);
+          return;
+        }
+      }
+      lastMap.addListener(ChannelFutureListener.CLOSE);
+    }
+
+    private String getErrorMessage(Throwable t) {
+      StringBuffer sb = new StringBuffer(t.getMessage());
+      while (t.getCause() != null) {
+        sb.append(t.getCause().getMessage());
+        t = t.getCause();
+      }
+      return sb.toString();
+    }
+
+    private final String USERCACHE_CONSTANT = "usercache";
+    private final String APPCACHE_CONSTANT = "appcache";
+
+    private String getBaseLocation(String jobIdString, String user) {
+      String parts[] = jobIdString.split("_");
+      Preconditions.checkArgument(parts.length == 3, "Invalid jobId. Expecting 3 parts");
+      final ApplicationId appID =
+          ApplicationId.newInstance(Long.parseLong(parts[1]), Integer.parseInt(parts[2]));
+      final String baseStr =
+          USERCACHE_CONSTANT + "/" + user + "/"
+              + APPCACHE_CONSTANT + "/"
+              + ConverterUtils.toString(appID) + "/output" + "/";
+      return baseStr;
+    }
+
+    protected MapOutputInfo getMapOutputInfo(String base, String mapId,
+        int reduce, String user) throws IOException {
+      // Index file
+      Path indexFileName =
+          lDirAlloc.getLocalPathToRead(base + "/file.out.index", conf);
+      TezIndexRecord info =
+          indexCache.getIndexInformation(mapId, reduce, indexFileName, user);
+
+      Path mapOutputFileName =
+          lDirAlloc.getLocalPathToRead(base + "/file.out", conf);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug(base + " : " + mapOutputFileName + " : " + indexFileName);
+      }
+      MapOutputInfo outputInfo = new MapOutputInfo(mapOutputFileName, info);
+      return outputInfo;
+    }
+
+    protected void populateHeaders(List<String> mapIds, String outputBaseStr,
+        String user, int reduce, HttpRequest request, HttpResponse response,
+        boolean keepAliveParam, Map<String, MapOutputInfo> mapOutputInfoMap)
+        throws IOException {
+
+      long contentLength = 0;
+      for (String mapId : mapIds) {
+        String base = outputBaseStr + mapId;
+        MapOutputInfo outputInfo = getMapOutputInfo(base, mapId, reduce, user);
+        if (mapOutputInfoMap.size() < mapOutputMetaInfoCacheSize) {
+          mapOutputInfoMap.put(mapId, outputInfo);
+        }
+        // Index file
+        Path indexFileName =
+            lDirAlloc.getLocalPathToRead(base + "/file.out.index", conf);
+        TezIndexRecord info =
+            indexCache.getIndexInformation(mapId, reduce, indexFileName, user);
+        ShuffleHeader header =
+            new ShuffleHeader(mapId, info.getPartLength(), info.getRawLength(), reduce);
+        DataOutputBuffer dob = new DataOutputBuffer();
+        header.write(dob);
+
+        contentLength += info.getPartLength();
+        contentLength += dob.getLength();
+      }
+
+      // Now set the response headers.
+      setResponseHeaders(response, keepAliveParam, contentLength);
+    }
+
+    protected void setResponseHeaders(HttpResponse response,
+        boolean keepAliveParam, long contentLength) {
+      if (!connectionKeepAliveEnabled && !keepAliveParam) {
+        LOG.info("Setting connection close header...");
+        response.setHeader(HttpHeaders.Names.CONNECTION, CONNECTION_CLOSE);
+      } else {
+        response.setHeader(HttpHeaders.Names.CONTENT_LENGTH,
+          String.valueOf(contentLength));
+        response.setHeader(HttpHeaders.Names.CONNECTION, HttpHeaders.Values.KEEP_ALIVE);
+        response.setHeader(HttpHeaders.Values.KEEP_ALIVE, "timeout="
+            + connectionKeepAliveTimeOut);
+        LOG.info("Content Length in shuffle : " + contentLength);
+      }
+    }
+
+    class MapOutputInfo {
+      final Path mapOutputFileName;
+      final TezIndexRecord indexRecord;
+
+      MapOutputInfo(Path mapOutputFileName, TezIndexRecord indexRecord) {
+        this.mapOutputFileName = mapOutputFileName;
+        this.indexRecord = indexRecord;
+      }
+    }
+
+    protected void verifyRequest(String appid, ChannelHandlerContext ctx,
+        HttpRequest request, HttpResponse response, URL requestUri)
+        throws IOException {
+      SecretKey tokenSecret = secretManager.retrieveTokenSecret(appid);
+      if (null == tokenSecret) {
+        LOG.info("Request for unknown token " + appid);
+        throw new IOException("could not find jobid");
+      }
+      // string to encrypt
+      String enc_str = SecureShuffleUtils.buildMsgFrom(requestUri);
+      // hash from the fetcher
+      String urlHashStr =
+        request.getHeader(SecureShuffleUtils.HTTP_HEADER_URL_HASH);
+      if (urlHashStr == null) {
+        LOG.info("Missing header hash for " + appid);
+        throw new IOException("fetcher cannot be authenticated");
+      }
+      if (LOG.isDebugEnabled()) {
+        int len = urlHashStr.length();
+        LOG.debug("verifying request. enc_str=" + enc_str + "; hash=..." +
+            urlHashStr.substring(len-len/2, len-1));
+      }
+      // verify - throws exception
+      SecureShuffleUtils.verifyReply(urlHashStr, enc_str, tokenSecret);
+      // verification passed - encode the reply
+      String reply =
+        SecureShuffleUtils.generateHash(urlHashStr.getBytes(Charsets.UTF_8), 
+            tokenSecret);
+      response.setHeader(SecureShuffleUtils.HTTP_HEADER_REPLY_URL_HASH, reply);
+      // Put shuffle version into http header
+      response.setHeader(ShuffleHeader.HTTP_HEADER_NAME,
+          ShuffleHeader.DEFAULT_HTTP_HEADER_NAME);
+      response.setHeader(ShuffleHeader.HTTP_HEADER_VERSION,
+          ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION);
+      if (LOG.isDebugEnabled()) {
+        int len = reply.length();
+        LOG.debug("Fetcher request verfied. enc_str=" + enc_str + ";reply=" +
+            reply.substring(len-len/2, len-1));
+      }
+    }
+
+    protected ChannelFuture sendMapOutput(ChannelHandlerContext ctx, Channel ch,
+        String user, String mapId, int reduce, MapOutputInfo mapOutputInfo)
+        throws IOException {
+      final TezIndexRecord info = mapOutputInfo.indexRecord;
+      final ShuffleHeader header =
+        new ShuffleHeader(mapId, info.getPartLength(), info.getRawLength(), reduce);
+      final DataOutputBuffer dob = new DataOutputBuffer();
+      header.write(dob);
+      ch.write(wrappedBuffer(dob.getData(), 0, dob.getLength()));
+      final File spillfile =
+          new File(mapOutputInfo.mapOutputFileName.toString());
+      RandomAccessFile spill;
+      try {
+        spill = SecureIOUtils.openForRandomRead(spillfile, "r", user, null);
+      } catch (FileNotFoundException e) {
+        LOG.info(spillfile + " not found");
+        return null;
+      }
+      ChannelFuture writeFuture;
+      if (ch.getPipeline().get(SslHandler.class) == null) {
+        final FadvisedFileRegion partition = new FadvisedFileRegion(spill,
+            info.getStartOffset(), info.getPartLength(), manageOsCache, readaheadLength,
+            readaheadPool, spillfile.getAbsolutePath(), 
+            shuffleBufferSize, shuffleTransferToAllowed);
+        writeFuture = ch.write(partition);
+        writeFuture.addListener(new ChannelFutureListener() {
+            // TODO error handling; distinguish IO/connection failures,
+            //      attribute to appropriate spill output
+          @Override
+          public void operationComplete(ChannelFuture future) {
+            if (future.isSuccess()) {
+              partition.transferSuccessful();
+            }
+            partition.releaseExternalResources();
+          }
+        });
+      } else {
+        // HTTPS cannot be done with zero copy.
+        final FadvisedChunkedFile chunk = new FadvisedChunkedFile(spill,
+            info.getStartOffset(), info.getPartLength(), sslFileBufferSize,
+            manageOsCache, readaheadLength, readaheadPool,
+            spillfile.getAbsolutePath());
+        writeFuture = ch.write(chunk);
+      }
+      return writeFuture;
+    }
+
+    protected void sendError(ChannelHandlerContext ctx,
+        HttpResponseStatus status) {
+      sendError(ctx, "", status);
+    }
+
+    protected void sendError(ChannelHandlerContext ctx, String message,
+        HttpResponseStatus status) {
+      HttpResponse response = new DefaultHttpResponse(HTTP_1_1, status);
+      response.setHeader(CONTENT_TYPE, "text/plain; charset=UTF-8");
+      // Put shuffle version into http header
+      response.setHeader(ShuffleHeader.HTTP_HEADER_NAME,
+          ShuffleHeader.DEFAULT_HTTP_HEADER_NAME);
+      response.setHeader(ShuffleHeader.HTTP_HEADER_VERSION,
+          ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION);
+      response.setContent(
+        ChannelBuffers.copiedBuffer(message, CharsetUtil.UTF_8));
+
+      // Close the connection as soon as the error message is sent.
+      ctx.getChannel().write(response).addListener(ChannelFutureListener.CLOSE);
+    }
+
+    @Override
+    public void exceptionCaught(ChannelHandlerContext ctx, ExceptionEvent e)
+        throws Exception {
+      Channel ch = e.getChannel();
+      Throwable cause = e.getCause();
+      if (cause instanceof TooLongFrameException) {
+        sendError(ctx, BAD_REQUEST);
+        return;
+      } else if (cause instanceof IOException) {
+        if (cause instanceof ClosedChannelException) {
+          LOG.debug("Ignoring closed channel error", cause);
+          return;
+        }
+        String message = String.valueOf(cause.getMessage());
+        if (IGNORABLE_ERROR_MESSAGE.matcher(message).matches()) {
+          LOG.debug("Ignoring client socket close", cause);
+          return;
+        }
+      }
+
+      LOG.error("Shuffle error: ", cause);
+      if (ch.isConnected()) {
+        LOG.error("Shuffle error " + e);
+        sendError(ctx, INTERNAL_SERVER_ERROR);
+      }
+    }
+  }
+}


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

Branch: refs/heads/TEZ-2003
Commit: 0114142d0301e636475567ef1507451297e9bfed
Parents: b5814c2
Author: Siddharth Seth <ss...@apache.org>
Authored: Tue Apr 7 13:21:35 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Mon Jun 1 14:09:27 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/0114142d/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/0114142d/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/0114142d/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/0114142d/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 04cdb5a..7fbc0f7 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
@@ -68,6 +68,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;
 
@@ -113,7 +114,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/0114142d/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 {


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

Branch: refs/heads/TEZ-2003
Commit: 32dd525043aa89ca3f8d9e6fbfab4b5b85cb5482
Parents: c4fa9f5
Author: Siddharth Seth <ss...@apache.org>
Authored: Wed Apr 29 08:20:05 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Mon Jun 1 14:13:37 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/32dd5250/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/32dd5250/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 e7c98b7..075d9a2 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/Fetcher.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/Fetcher.java
@@ -87,6 +87,7 @@ public class Fetcher extends CallableWithNdc<FetchResult> {
   private final FetcherCallback fetcherCallback;
   private final FetchedInputAllocator inputManager;
   private final ApplicationId appId;
+  private final int dagIdentifier;
   
   private final String logIdentifier;
 
@@ -127,7 +128,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,
@@ -141,6 +142,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;
@@ -406,7 +408,7 @@ public class Fetcher extends CallableWithNdc<FetchResult> {
   private HostFetchResult setupConnection(List<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());
 
@@ -913,22 +915,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/32dd5250/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 8b6e847..977c23d 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
@@ -178,19 +178,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/32dd5250/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 b7c0742..6f89a09 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
@@ -393,7 +393,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/32dd5250/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/32dd5250/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 34c2ca7..7bd7414 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
@@ -70,7 +70,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());
@@ -120,7 +120,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());
@@ -154,7 +154,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());
 


[07/50] [abbrv] tez git commit: TEZ-2483. Tez should close task if processor fail (zjffdu)

Posted by ss...@apache.org.
TEZ-2483. Tez should close task if processor fail (zjffdu)


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

Branch: refs/heads/TEZ-2003
Commit: 02aafb55e9b1adc3384adaccf44cba712a753fac
Parents: fe74d6b
Author: Jeff Zhang <zj...@apache.org>
Authored: Thu May 28 09:45:20 2015 +0800
Committer: Jeff Zhang <zj...@apache.org>
Committed: Thu May 28 09:45:20 2015 +0800

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../runtime/LogicalIOProcessorRuntimeTask.java  | 78 +++++++++++++++++++-
 2 files changed, 75 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/02aafb55/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index f676ef7..c9318d2 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -220,6 +220,7 @@ Release 0.6.2: Unreleased
 INCOMPATIBLE CHANGES
 
 ALL CHANGES:
+  TEZ-2483. Tez should close task if processor fail
 
 Release 0.6.1: 2015-05-18
 

http://git-wip-us.apache.org/repos/asf/tez/blob/02aafb55/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 24f62a0..84e5e0d 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
@@ -26,6 +26,7 @@ import java.util.ArrayList;
 import java.util.Calendar;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.Iterator;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
@@ -114,6 +115,10 @@ public class LogicalIOProcessorRuntimeTask extends RuntimeTask {
   final List<GroupInputSpec> groupInputSpecs;
   ConcurrentHashMap<String, MergedLogicalInput> groupInputsMap;
 
+  final ConcurrentHashMap<String, LogicalInput> initializedInputs;
+  final ConcurrentHashMap<String, LogicalOutput> initializedOutputs;
+
+  private boolean processorClosed = false;
   final ProcessorDescriptor processorDescriptor;
   AbstractLogicalIOProcessor processor;
   ProcessorContext processorContext;
@@ -163,6 +168,9 @@ 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>();
 
@@ -344,11 +352,13 @@ public class LogicalIOProcessorRuntimeTask extends RuntimeTask {
       this.state.set(State.CLOSED);
 
       // Close the Processor.
+      processorClosed = true;
       processor.close();
 
       // Close the Inputs.
       for (InputSpec inputSpec : inputSpecs) {
         String srcVertexName = inputSpec.getSourceVertexName();
+        initializedInputs.remove(srcVertexName);
         List<Event> closeInputEvents = ((InputFrameworkInterface)inputsMap.get(srcVertexName)).close();
         sendTaskGeneratedEvents(closeInputEvents,
             EventProducerConsumerType.INPUT, taskSpec.getVertexName(),
@@ -358,6 +368,7 @@ public class LogicalIOProcessorRuntimeTask extends RuntimeTask {
       // Close the Outputs.
       for (OutputSpec outputSpec : outputSpecs) {
         String destVertexName = outputSpec.getDestinationVertexName();
+        initializedOutputs.remove(destVertexName);
         List<Event> closeOutputEvents = ((LogicalOutputFrameworkInterface)outputsMap.get(destVertexName)).close();
         sendTaskGeneratedEvents(closeOutputEvents,
             EventProducerConsumerType.OUTPUT, taskSpec.getVertexName(),
@@ -407,6 +418,7 @@ public class LogicalIOProcessorRuntimeTask extends RuntimeTask {
       sendTaskGeneratedEvents(events, EventProducerConsumerType.INPUT,
           inputContext.getTaskVertexName(), inputContext.getSourceVertexName(),
           taskSpec.getTaskAttemptID());
+      initializedInputs.put(edgeName, input);
       LOG.info("Initialized Input with src edge: " + edgeName);
       return null;
     }
@@ -455,6 +467,7 @@ public class LogicalIOProcessorRuntimeTask extends RuntimeTask {
       sendTaskGeneratedEvents(events, EventProducerConsumerType.OUTPUT,
           outputContext.getTaskVertexName(),
           outputContext.getDestinationVertexName(), taskSpec.getTaskAttemptID());
+      initializedOutputs.put(edgeName, output);
       LOG.info("Initialized Output with dest edge: " + edgeName);
       return null;
     }
@@ -748,6 +761,65 @@ public class LogicalIOProcessorRuntimeTask extends RuntimeTask {
       }
       eventRouterThread = null;
     }
+
+    // Close the unclosed IPO
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Processor closed={}", processorClosed);
+      LOG.debug("Num of inputs to be closed={}", initializedInputs.size());
+      LOG.debug("Num of outputs to be closed={}", initializedOutputs.size());
+    }
+    // Close processor
+    if (!processorClosed && processor != null) {
+      try {
+        processorClosed = true;
+        processor.close();
+        LOG.info("Closed processor for vertex={}, index={}",
+            processor
+                .getContext().getTaskVertexName(),
+            processor.getContext().getTaskVertexIndex());
+      } catch (Throwable e) {
+        LOG.warn(
+            "Ignoring Exception when closing processor(cleanup). Exception class={}, message={}",
+                e.getClass().getName(), e.getMessage());
+      }
+    }
+
+    // Close the remaining inited Inputs.
+    Iterator<Map.Entry<String, LogicalInput>> inputIterator = initializedInputs.entrySet().iterator();
+    while (inputIterator.hasNext()) {
+      Map.Entry<String, LogicalInput> entry = inputIterator.next();
+      String srcVertexName = entry.getKey();
+      inputIterator.remove();
+      try {
+        ((InputFrameworkInterface)entry.getValue()).close();
+      } 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);
+      }
+    }
+
+    // Close the remaining inited Outputs.
+    Iterator<Map.Entry<String, LogicalOutput>> outputIterator = initializedOutputs.entrySet().iterator();
+    while (outputIterator.hasNext()) {
+      Map.Entry<String, LogicalOutput> entry = outputIterator.next();
+      String destVertexName = entry.getKey();
+      outputIterator.remove();
+      try {
+        ((OutputFrameworkInterface) entry.getValue()).close();
+      } 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);
+      }
+    }
+
     try {
       closeContexts();
       // Cleanup references which may be held by misbehaved tasks.
@@ -764,10 +836,8 @@ public class LogicalIOProcessorRuntimeTask extends RuntimeTask {
     inputsMap.clear();
     outputsMap.clear();
 
-
-
-    inputsMap.clear();
-    outputsMap.clear();
+    initializedInputs.clear();
+    initializedOutputs.clear();
 
     inputContextMap.clear();
     outputContextMap.clear();


[04/50] [abbrv] tez git commit: TEZ-1529. ATS and TezClient integration in secure kerberos enabled cluster. (pramachandran)

Posted by ss...@apache.org.
TEZ-1529. ATS and TezClient integration in secure kerberos enabled cluster. (pramachandran)


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

Branch: refs/heads/TEZ-2003
Commit: 8710df0d1264a453218220ed48e5d2b5d2923da1
Parents: dac59a2
Author: Prakash Ramachandran <pr...@hortonworks.com>
Authored: Wed May 27 17:59:17 2015 +0530
Committer: Prakash Ramachandran <pr...@hortonworks.com>
Committed: Wed May 27 17:59:17 2015 +0530

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../org/apache/tez/common/ReflectionUtils.java  |  19 +
 .../tez/dag/api/client/DAGClientImpl.java       |   9 +-
 .../dag/api/client/DAGClientTimelineImpl.java   |  55 ++-
 .../dag/api/client/TimelineReaderFactory.java   | 387 +++++++++++++++++++
 .../tez/dag/api/client/TestATSHttpClient.java   |   6 +-
 .../api/client/TestTimelineReaderFactory.java   |  91 +++++
 7 files changed, 529 insertions(+), 39 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/8710df0d/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 484f78d..513285f 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -26,6 +26,7 @@ Release 0.7.1: Unreleased
 INCOMPATIBLE CHANGES
 
 ALL CHANGES:
+  TEZ-1529. ATS and TezClient integration in secure kerberos enabled cluster.
   TEZ-2481. Tez UI: graphical view does not render properly on IE11
   TEZ-2474. The old taskNum is logged incorrectly when parallelism is changed
   TEZ-2460. Temporary solution for issue due to YARN-2560

http://git-wip-us.apache.org/repos/asf/tez/blob/8710df0d/tez-api/src/main/java/org/apache/tez/common/ReflectionUtils.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/common/ReflectionUtils.java b/tez-api/src/main/java/org/apache/tez/common/ReflectionUtils.java
index 0fc529b..f1eb0ae 100644
--- a/tez-api/src/main/java/org/apache/tez/common/ReflectionUtils.java
+++ b/tez-api/src/main/java/org/apache/tez/common/ReflectionUtils.java
@@ -100,6 +100,25 @@ public class ReflectionUtils {
   }
 
   @Private
+  @SuppressWarnings("unchecked")
+  public static <T> T invokeMethod(Object target, Method method, Object... args) {
+    try {
+      return (T) method.invoke(target, args);
+    } catch (Exception e) {
+      throw new TezUncheckedException(e);
+    }
+  }
+
+  @Private
+  public static Method getMethod(Class<?> targetClazz, String methodName, Class<?>... parameterTypes) {
+    try {
+      return targetClazz.getMethod(methodName, parameterTypes);
+    } catch (NoSuchMethodException e) {
+      throw new TezUncheckedException(e);
+    }
+  }
+
+  @Private
   public static synchronized void addResourcesToClasspath(List<URL> urls) {
     ClassLoader classLoader = new URLClassLoader(urls.toArray(new URL[urls.size()]), Thread
         .currentThread().getContextClassLoader());

http://git-wip-us.apache.org/repos/asf/tez/blob/8710df0d/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClientImpl.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClientImpl.java b/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClientImpl.java
index baacdb9..66fc986 100644
--- a/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClientImpl.java
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClientImpl.java
@@ -33,7 +33,6 @@ import com.google.common.base.Preconditions;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
-import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
@@ -95,10 +94,7 @@ public class DAGClientImpl extends DAGClient {
             conf.getBoolean(TezConfiguration.TEZ_AM_HISTORY_LOGGING_ENABLED,
                  TezConfiguration.TEZ_AM_HISTORY_LOGGING_ENABLED_DEFAULT);
 
-    if (UserGroupInformation.isSecurityEnabled()){
-      //TODO: enable ATS integration in kerberos secured cluster - see TEZ-1529
-      isATSEnabled = false;
-    }
+    isATSEnabled = DAGClientTimelineImpl.isSupported();
 
     realClient = new DAGClientRPCImpl(appId, dagId, conf, this.frameworkClient);
     statusPollInterval = conf.getLong(
@@ -583,7 +579,8 @@ public class DAGClientImpl extends DAGClient {
 
   private void switchToTimelineClient() throws IOException, TezException {
     realClient.close();
-    realClient = new DAGClientTimelineImpl(appId, dagId, conf, frameworkClient);
+    realClient = new DAGClientTimelineImpl(appId, dagId, conf, frameworkClient,
+        (int) (2 * PRINT_STATUS_INTERVAL_MILLIS));
     if (LOG.isDebugEnabled()) {
       LOG.debug("dag completed switching to DAGClientTimelineImpl");
     }

http://git-wip-us.apache.org/repos/asf/tez/blob/8710df0d/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClientTimelineImpl.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClientTimelineImpl.java b/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClientTimelineImpl.java
index cc000df..9a0949b 100644
--- a/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClientTimelineImpl.java
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClientTimelineImpl.java
@@ -23,9 +23,6 @@ import javax.ws.rs.core.MediaType;
 import java.io.IOException;
 import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
-import java.net.HttpURLConnection;
-import java.net.URL;
-import java.net.URLEncoder;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
@@ -40,16 +37,10 @@ import com.sun.jersey.api.client.ClientHandlerException;
 import com.sun.jersey.api.client.ClientResponse;
 import com.sun.jersey.api.client.UniformInterfaceException;
 import com.sun.jersey.api.client.WebResource;
-import com.sun.jersey.api.client.config.ClientConfig;
-import com.sun.jersey.api.client.config.DefaultClientConfig;
-import com.sun.jersey.client.urlconnection.HttpURLConnectionFactory;
-import com.sun.jersey.client.urlconnection.URLConnectionClientHandler;
-import com.sun.jersey.json.impl.provider.entity.JSONRootElementProvider;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
 import org.apache.hadoop.yarn.exceptions.YarnException;
@@ -78,10 +69,10 @@ public class DAGClientTimelineImpl extends DAGClient {
   private static final String FILTER_BY_FIELDS = "primaryfilters,otherinfo";
   private static final String HTTPS_SCHEME = "https://";
   private static final String HTTP_SCHEME = "http://";
-  private static Client httpClient = null;
+  private Client httpClient = null;
+  private final TimelineReaderFactory.TimelineReaderStrategy timelineReaderStrategy;
   private final ApplicationId appId;
   private final String dagId;
-  private final TezConfiguration conf;
   private final FrameworkClient frameworkClient;
 
   private Map<String, VertexTaskStats> vertexTaskStatsCache = null;
@@ -90,16 +81,21 @@ public class DAGClientTimelineImpl extends DAGClient {
   protected String baseUri;
 
   public DAGClientTimelineImpl(ApplicationId appId, String dagId, TezConfiguration conf,
-                               FrameworkClient frameworkClient)
+                               FrameworkClient frameworkClient, int connTimeout)
       throws TezException {
+
+    if (!TimelineReaderFactory.isTimelineClientSupported()) {
+      throw new TezException("Reading from secure timeline is supported only for hadoop 2.6 and above.");
+    }
+
     this.appId = appId;
     this.dagId = dagId;
-    this.conf = conf;
     this.frameworkClient = frameworkClient;
 
     String scheme;
     String webAppAddress;
-    if (webappHttpsOnly(conf)) {
+    boolean useHttps = webappHttpsOnly(conf);
+    if (useHttps) {
       scheme = HTTPS_SCHEME;
       webAppAddress = conf.get(ATSConstants.TIMELINE_SERVICE_WEBAPP_HTTPS_ADDRESS_CONF_NAME);
     } else {
@@ -111,8 +107,14 @@ public class DAGClientTimelineImpl extends DAGClient {
     }
 
     baseUri = Joiner.on("").join(scheme, webAppAddress, ATSConstants.RESOURCE_URI_BASE);
+
+    timelineReaderStrategy =
+        TimelineReaderFactory.getTimelineReaderStrategy(conf, useHttps, connTimeout);
   }
 
+  public static boolean isSupported() {
+    return TimelineReaderFactory.isTimelineClientSupported();
+  }
 
   @Override
   public String getExecutionContext() {
@@ -407,13 +409,15 @@ public class DAGClientTimelineImpl extends DAGClient {
   @VisibleForTesting
   protected JSONObject getJsonRootEntity(String url) throws TezException {
     try {
-      WebResource wr = getHttpClient().resource(url);
+      WebResource wr = getCachedHttpClient().resource(url);
       ClientResponse response = wr.accept(MediaType.APPLICATION_JSON_TYPE)
           .type(MediaType.APPLICATION_JSON_TYPE)
           .get(ClientResponse.class);
 
-      if (response.getClientResponseStatus() != ClientResponse.Status.OK) {
-        throw new TezException("Failed to get response from YARN Timeline: url: " + url);
+      final ClientResponse.Status clientResponseStatus = response.getClientResponseStatus();
+      if (clientResponseStatus != ClientResponse.Status.OK) {
+        throw new TezException("Failed to get response from YARN Timeline:" +
+            " errorCode:" + clientResponseStatus + ", url:" + url);
       }
 
       return response.getEntity(JSONObject.class);
@@ -423,6 +427,8 @@ public class DAGClientTimelineImpl extends DAGClient {
       throw new TezException("Error accessing content from YARN Timeline - unexpected response", e);
     } catch (IllegalArgumentException e) {
       throw new TezException("Error accessing content from YARN Timeline - invalid url", e);
+    } catch (IOException e) {
+      throw new TezException("Error failed to get http client", e);
     }
   }
 
@@ -460,11 +466,9 @@ public class DAGClientTimelineImpl extends DAGClient {
     }
   }
 
-  protected Client getHttpClient() {
+  protected Client getCachedHttpClient() throws IOException {
     if (httpClient == null) {
-      ClientConfig config = new DefaultClientConfig(JSONRootElementProvider.App.class);
-      HttpURLConnectionFactory urlFactory = new PseudoAuthenticatedURLConnectionFactory();
-      httpClient = new Client(new URLConnectionClientHandler(urlFactory), config);
+      httpClient = timelineReaderStrategy.getHttpClient();
     }
     return httpClient;
   }
@@ -498,15 +502,6 @@ public class DAGClientTimelineImpl extends DAGClient {
       }});
 
 
-  static class PseudoAuthenticatedURLConnectionFactory implements HttpURLConnectionFactory {
-    @Override
-    public HttpURLConnection getHttpURLConnection(URL url) throws IOException {
-      String tokenString = (url.getQuery() == null ? "?" : "&") + "user.name=" +
-          URLEncoder.encode(UserGroupInformation.getCurrentUser().getShortUserName(), "UTF8");
-      return (HttpURLConnection) (new URL(url.toString() + tokenString)).openConnection();
-    }
-  }
-
   @Override
   public DAGStatus getDAGStatus(@Nullable Set<StatusGetOpts> statusOptions,
       long timeout) throws IOException, TezException {

http://git-wip-us.apache.org/repos/asf/tez/blob/8710df0d/tez-api/src/main/java/org/apache/tez/dag/api/client/TimelineReaderFactory.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/client/TimelineReaderFactory.java b/tez-api/src/main/java/org/apache/tez/dag/api/client/TimelineReaderFactory.java
new file mode 100644
index 0000000..f544198
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/client/TimelineReaderFactory.java
@@ -0,0 +1,387 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.dag.api.client;
+
+import static org.apache.hadoop.security.ssl.SSLFactory.Mode.CLIENT;
+
+import javax.net.ssl.HostnameVerifier;
+import javax.net.ssl.HttpsURLConnection;
+import javax.net.ssl.SSLSocketFactory;
+import java.io.IOException;
+import java.lang.reflect.Method;
+import java.net.HttpURLConnection;
+import java.net.URL;
+import java.net.URLEncoder;
+import java.security.GeneralSecurityException;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.sun.jersey.api.client.Client;
+import com.sun.jersey.api.client.config.ClientConfig;
+import com.sun.jersey.api.client.config.DefaultClientConfig;
+import com.sun.jersey.client.urlconnection.HttpURLConnectionFactory;
+import com.sun.jersey.client.urlconnection.URLConnectionClientHandler;
+import com.sun.jersey.json.impl.provider.entity.JSONRootElementProvider;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.authentication.client.AuthenticatedURL;
+import org.apache.hadoop.security.authentication.client.Authenticator;
+import org.apache.hadoop.security.authentication.client.ConnectionConfigurator;
+import org.apache.hadoop.security.ssl.SSLFactory;
+import org.apache.tez.common.ReflectionUtils;
+import org.apache.tez.dag.api.TezException;
+import org.apache.tez.dag.api.TezUncheckedException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/*
+ *  TimelineReaderFactory getTimelineReaderStrategy returns a Strategy class, which is used to
+ *  create a httpclient, configured for the appropriate runtime.
+ *
+ *  on hadoop 2.6+ the factory returns TimelineReaderTokenAuthenticatedStrategy, which supports
+ *  kerberos based auth (secure cluster) or psuedo auth (un-secure cluster).
+ *
+ *  on hadoop 2.4 where the token delegation auth is not supported, TimelineReaderPseudoAuthenticatedStrategy
+ *  is used which supports only unsecure timeline.
+ *
+ */
+@InterfaceAudience.Private
+public class TimelineReaderFactory {
+
+  private static final Logger LOG = LoggerFactory.getLogger(TimelineReaderFactory.class);
+
+  private static final String KERBEROS_DELEGATION_TOKEN_AUTHENTICATOR_CLAZZ_NAME =
+      "org.apache.hadoop.security.token.delegation.web.KerberosDelegationTokenAuthenticator";
+  private static final String PSEUDO_DELEGATION_TOKEN_AUTHENTICATOR_CLAZZ_NAME =
+      "org.apache.hadoop.security.token.delegation.web.PseudoDelegationTokenAuthenticator";
+  private static final String DELEGATION_TOKEN_AUTHENTICATED_URL_CLAZZ_NAME =
+      "org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticatedURL";
+  private static final String DELEGATION_TOKEN_AUTHENTICATOR_CLAZZ_NAME =
+      "org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticator";
+  private static final String DELEGATION_TOKEN_AUTHENTICATED_URL_TOKEN_CLASS_NAME =
+      "org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticatedURL$Token";
+
+  private static Class<?> delegationTokenAuthenticatorClazz = null;
+  private static Method delegationTokenAuthenticateURLOpenConnectionMethod = null;
+
+  public static TimelineReaderStrategy getTimelineReaderStrategy(Configuration conf,
+                                                                 boolean useHttps,
+                                                                 int connTimeout) throws TezException {
+
+    TimelineReaderStrategy timelineReaderStrategy;
+
+    if (!isTimelineClientSupported()) {
+      throw new TezException("Reading from timeline is not supported." +
+          " token delegation support: " + tokenDelegationSupported() +
+          ", is secure timeline: " + UserGroupInformation.isSecurityEnabled());
+    }
+
+    timelineReaderStrategy = getTimelineReaderStrategy(tokenDelegationSupported(), conf, useHttps,
+        connTimeout);
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Using " + timelineReaderStrategy.getClass().getName() + " to read timeline data");
+    }
+
+    return timelineReaderStrategy;
+  }
+
+  private static TimelineReaderStrategy getTimelineReaderStrategy(boolean isTokenDelegationSupported,
+                                                                    Configuration conf,
+                                                                    boolean useHttps,
+                                                                    int connTimeout) {
+    TimelineReaderStrategy timelineReaderStrategy;
+
+    if (isTokenDelegationSupported) {
+      timelineReaderStrategy =
+          new TimelineReaderTokenAuthenticatedStrategy(conf, useHttps, connTimeout);
+    } else {
+      timelineReaderStrategy =
+          new TimelineReaderPseudoAuthenticatedStrategy(conf, useHttps, connTimeout);
+    }
+
+    return timelineReaderStrategy;
+  }
+
+  /**
+   * Check if timeline client can be supported.
+   *
+   * @return boolean value indicating if timeline client to read data is supported.
+   */
+  public static boolean isTimelineClientSupported() {
+    // support to read data from timeline is based on the version of hadoop.
+    // reads are supported for non-secure cluster from hadoop 2.4 and up.
+    // reads are supported for secure cluster only from hadoop 2.6. check the presence of the classes
+    // required upfront if security is enabled.
+    return !UserGroupInformation.isSecurityEnabled() || tokenDelegationSupported();
+  }
+
+  public interface TimelineReaderStrategy {
+    Client getHttpClient() throws IOException;
+  }
+
+  /*
+   * auth strategy for secured and unsecured environment with delegation token (hadoop 2.6 and above)
+   */
+  private static class TimelineReaderTokenAuthenticatedStrategy implements TimelineReaderStrategy {
+    private final Configuration conf;
+    private final boolean useHttps;
+    private final int connTimeout;
+
+    public TimelineReaderTokenAuthenticatedStrategy(final Configuration conf,
+                                                    final boolean useHttps,
+                                                    final int connTimeout) {
+
+      this.conf = conf;
+      this.useHttps = useHttps;
+      this.connTimeout = connTimeout;
+    }
+
+    @Override
+    public Client getHttpClient() throws IOException {
+      Authenticator authenticator;
+      UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
+      UserGroupInformation realUgi = ugi.getRealUser();
+      UserGroupInformation authUgi;
+      String doAsUser;
+      ClientConfig clientConfig = new DefaultClientConfig(JSONRootElementProvider.App.class);
+      ConnectionConfigurator connectionConfigurator = getNewConnectionConf(conf, useHttps,
+          connTimeout);
+
+      try {
+        authenticator = getTokenAuthenticator();
+        authenticator.setConnectionConfigurator(connectionConfigurator);
+      } catch (TezUncheckedException e) {
+        throw new IOException("Failed to get authenticator", e);
+      }
+
+      if (realUgi != null) {
+        authUgi = realUgi;
+        doAsUser = ugi.getShortUserName();
+      } else {
+        authUgi = ugi;
+        doAsUser = null;
+      }
+
+      HttpURLConnectionFactory connectionFactory =
+          new TokenAuthenticatedURLConnectionFactory(connectionConfigurator, authenticator,
+              authUgi, doAsUser);
+      return new Client(new URLConnectionClientHandler(connectionFactory), clientConfig);
+    }
+
+    private static Authenticator getTokenAuthenticator() {
+      String authenticatorClazzName;
+
+      if (UserGroupInformation.isSecurityEnabled()) {
+        authenticatorClazzName = KERBEROS_DELEGATION_TOKEN_AUTHENTICATOR_CLAZZ_NAME;
+      } else {
+        authenticatorClazzName = PSEUDO_DELEGATION_TOKEN_AUTHENTICATOR_CLAZZ_NAME;
+      }
+
+      return ReflectionUtils.createClazzInstance(authenticatorClazzName);
+    }
+
+    private static class TokenAuthenticatedURLConnectionFactory implements HttpURLConnectionFactory {
+
+      private final Authenticator authenticator;
+      private final ConnectionConfigurator connConfigurator;
+      private final UserGroupInformation authUgi;
+      private final String doAsUser;
+      private final AuthenticatedURL.Token token;
+
+      public TokenAuthenticatedURLConnectionFactory(ConnectionConfigurator connConfigurator,
+                                                    Authenticator authenticator,
+                                                    UserGroupInformation authUgi,
+                                                    String doAsUser) {
+        this.connConfigurator = connConfigurator;
+        this.authenticator = authenticator;
+        this.authUgi = authUgi;
+        this.doAsUser = doAsUser;
+        this.token = ReflectionUtils.createClazzInstance(
+            DELEGATION_TOKEN_AUTHENTICATED_URL_TOKEN_CLASS_NAME, null, null);
+      }
+
+      @Override
+      public HttpURLConnection getHttpURLConnection(URL url) throws IOException {
+        try {
+          AuthenticatedURL authenticatedURL= ReflectionUtils.createClazzInstance(
+              DELEGATION_TOKEN_AUTHENTICATED_URL_CLAZZ_NAME, new Class[] {
+              delegationTokenAuthenticatorClazz,
+              ConnectionConfigurator.class
+          }, new Object[] {
+              authenticator,
+              connConfigurator
+          });
+          return ReflectionUtils.invokeMethod(authenticatedURL,
+              delegationTokenAuthenticateURLOpenConnectionMethod, url, token, doAsUser);
+        } catch (Exception e) {
+          throw new IOException(e);
+        }
+      }
+    }
+  }
+
+  /*
+   * Pseudo auth strategy for env where delegation token is not supported (hadoop 2.4)
+   */
+  @VisibleForTesting
+  protected static class TimelineReaderPseudoAuthenticatedStrategy implements TimelineReaderStrategy {
+
+    private final ConnectionConfigurator connectionConf;
+
+    public TimelineReaderPseudoAuthenticatedStrategy(final Configuration conf,
+                                                     final boolean useHttps,
+                                                     final int connTimeout) {
+      connectionConf = getNewConnectionConf(conf, useHttps, connTimeout);
+    }
+
+    @Override
+    public Client getHttpClient() {
+      ClientConfig config = new DefaultClientConfig(JSONRootElementProvider.App.class);
+      HttpURLConnectionFactory urlFactory = new PseudoAuthenticatedURLConnectionFactory(connectionConf);
+      Client httpClient = new Client(new URLConnectionClientHandler(urlFactory), config);
+      return httpClient;
+    }
+
+    @VisibleForTesting
+    protected static class PseudoAuthenticatedURLConnectionFactory implements HttpURLConnectionFactory {
+      private final ConnectionConfigurator connectionConf;
+
+      public PseudoAuthenticatedURLConnectionFactory(ConnectionConfigurator connectionConf) {
+        this.connectionConf = connectionConf;
+      }
+
+      @Override
+      public HttpURLConnection getHttpURLConnection(URL url) throws IOException {
+        String tokenString = (url.getQuery() == null ? "?" : "&") + "user.name=" +
+            URLEncoder.encode(UserGroupInformation.getCurrentUser().getShortUserName(), "UTF8");
+
+        HttpURLConnection httpURLConnection =
+            (HttpURLConnection) (new URL(url.toString() + tokenString)).openConnection();
+        this.connectionConf.configure(httpURLConnection);
+
+        return httpURLConnection;
+      }
+    }
+  }
+
+  private static ConnectionConfigurator getNewConnectionConf(final Configuration conf,
+                                                             final boolean useHttps,
+                                                             final int connTimeout) {
+    ConnectionConfigurator connectionConf = null;
+    if (useHttps) {
+      try {
+        connectionConf = getNewSSLConnectionConf(conf, connTimeout);
+      } catch (IOException e) {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Cannot load customized ssl related configuration."
+              + " Falling back to system-generic settings.", e);
+        }
+      }
+    }
+
+    if (connectionConf == null) {
+      connectionConf = new ConnectionConfigurator() {
+        @Override
+        public HttpURLConnection configure(HttpURLConnection httpURLConnection) throws IOException {
+          setTimeouts(httpURLConnection, connTimeout);
+          return httpURLConnection;
+        }
+      };
+    }
+
+    return connectionConf;
+  }
+
+  private static ConnectionConfigurator getNewSSLConnectionConf(final Configuration conf,
+                                                                final int connTimeout)
+      throws IOException {
+    final SSLFactory sslFactory;
+    final SSLSocketFactory sslSocketFactory;
+    final HostnameVerifier hostnameVerifier;
+
+    sslFactory = new SSLFactory(CLIENT, conf);
+    try {
+      sslFactory.init();
+      sslSocketFactory = sslFactory.createSSLSocketFactory();
+    } catch (GeneralSecurityException e) {
+      sslFactory.destroy();
+      throw new IOException("Failed to initialize ssl factory");
+    }
+    hostnameVerifier = sslFactory.getHostnameVerifier();
+
+    return new ConnectionConfigurator() {
+      @Override
+      public HttpURLConnection configure(HttpURLConnection httpURLConnection) throws IOException {
+        if (!(httpURLConnection instanceof HttpsURLConnection)) {
+          throw new IOException("Expected https connection");
+        }
+        HttpsURLConnection httpsURLConnection = (HttpsURLConnection) httpURLConnection;
+        httpsURLConnection.setSSLSocketFactory(sslSocketFactory);
+        httpsURLConnection.setHostnameVerifier(hostnameVerifier);
+        setTimeouts(httpsURLConnection, connTimeout);
+
+        return httpsURLConnection;
+      }
+    };
+  }
+
+  private static void setTimeouts(HttpURLConnection httpURLConnection, int connTimeout) {
+    httpURLConnection.setConnectTimeout(connTimeout);
+    httpURLConnection.setReadTimeout(connTimeout);
+  }
+
+  private static boolean isTokenDelegationSupportChecksDone = false;
+  private static boolean isTokenDelegationClassesPresent = false;
+
+  // Check if all the classes required for doing token authentication are present. These classes
+  // are present only from hadoop 2.6 onwards.
+  private static synchronized boolean tokenDelegationSupported() {
+
+    if (!isTokenDelegationSupportChecksDone) {
+
+      isTokenDelegationSupportChecksDone = true;
+
+      try {
+        ReflectionUtils.getClazz(KERBEROS_DELEGATION_TOKEN_AUTHENTICATOR_CLAZZ_NAME);
+        ReflectionUtils.getClazz(PSEUDO_DELEGATION_TOKEN_AUTHENTICATOR_CLAZZ_NAME);
+
+        delegationTokenAuthenticatorClazz =
+            ReflectionUtils.getClazz(DELEGATION_TOKEN_AUTHENTICATOR_CLAZZ_NAME);
+
+        Class<?> delegationTokenAuthenticatedURLClazz =
+            ReflectionUtils.getClazz(DELEGATION_TOKEN_AUTHENTICATED_URL_CLAZZ_NAME);
+
+        Class<?> delegationTokenAuthenticatedURLTokenClazz =
+            ReflectionUtils.getClazz(DELEGATION_TOKEN_AUTHENTICATED_URL_TOKEN_CLASS_NAME);
+
+        delegationTokenAuthenticateURLOpenConnectionMethod =
+            ReflectionUtils.getMethod(delegationTokenAuthenticatedURLClazz, "openConnection",
+                URL.class, delegationTokenAuthenticatedURLTokenClazz, String.class);
+
+        isTokenDelegationClassesPresent = true;
+
+      } catch (TezUncheckedException e) {
+        LOG.info("Could not find class required for token delegation, will fallback to pseudo auth");
+      }
+    }
+
+    return isTokenDelegationClassesPresent;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/8710df0d/tez-api/src/test/java/org/apache/tez/dag/api/client/TestATSHttpClient.java
----------------------------------------------------------------------
diff --git a/tez-api/src/test/java/org/apache/tez/dag/api/client/TestATSHttpClient.java b/tez-api/src/test/java/org/apache/tez/dag/api/client/TestATSHttpClient.java
index a72b799..ef1b0a5 100644
--- a/tez-api/src/test/java/org/apache/tez/dag/api/client/TestATSHttpClient.java
+++ b/tez-api/src/test/java/org/apache/tez/dag/api/client/TestATSHttpClient.java
@@ -56,7 +56,7 @@ public class TestATSHttpClient {
   public void testGetDagStatusThrowsExceptionOnEmptyJson() throws TezException {
     ApplicationId mockAppId = mock(ApplicationId.class);
     DAGClientTimelineImpl httpClient = new DAGClientTimelineImpl(mockAppId, "EXAMPLE_DAG_ID",
-        new TezConfiguration(), null);
+        new TezConfiguration(), null, 0);
     DAGClientTimelineImpl spyClient = spy(httpClient);
     spyClient.baseUri = "http://yarn.ats.webapp/ws/v1/timeline";
     final String expectedDagUrl = "http://yarn.ats.webapp/ws/v1/timeline/TEZ_DAG_ID/EXAMPLE_DAG_ID" +
@@ -80,7 +80,7 @@ public class TestATSHttpClient {
   public void testGetDagStatusSimple() throws TezException, JSONException, IOException {
     DAGClientTimelineImpl
         httpClient = new DAGClientTimelineImpl(mock(ApplicationId.class),"EXAMPLE_DAG_ID",
-        new TezConfiguration(), null);
+        new TezConfiguration(), null, 0);
     DAGClientTimelineImpl spyClient = spy(httpClient);
     spyClient.baseUri = "http://yarn.ats.webapp/ws/v1/timeline";
     final String expectedDagUrl = "http://yarn.ats.webapp/ws/v1/timeline/TEZ_DAG_ID/EXAMPLE_DAG_ID" +
@@ -140,7 +140,7 @@ public class TestATSHttpClient {
   public void testGetVertexStatusSimple() throws JSONException, TezException, IOException {
     DAGClientTimelineImpl
         httpClient = new DAGClientTimelineImpl(mock(ApplicationId.class), "EXAMPLE_DAG_ID",
-        new TezConfiguration(), null);
+        new TezConfiguration(), null, 0);
     DAGClientTimelineImpl spyClient = spy(httpClient);
     spyClient.baseUri = "http://yarn.ats.webapp/ws/v1/timeline";
     final String expectedVertexUrl = "http://yarn.ats.webapp/ws/v1/timeline/TEZ_VERTEX_ID" +

http://git-wip-us.apache.org/repos/asf/tez/blob/8710df0d/tez-api/src/test/java/org/apache/tez/dag/api/client/TestTimelineReaderFactory.java
----------------------------------------------------------------------
diff --git a/tez-api/src/test/java/org/apache/tez/dag/api/client/TestTimelineReaderFactory.java b/tez-api/src/test/java/org/apache/tez/dag/api/client/TestTimelineReaderFactory.java
new file mode 100644
index 0000000..4aff0ca
--- /dev/null
+++ b/tez-api/src/test/java/org/apache/tez/dag/api/client/TestTimelineReaderFactory.java
@@ -0,0 +1,91 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.dag.api.client;
+
+import static org.mockito.Mockito.mock;
+
+import java.net.HttpURLConnection;
+import java.net.URL;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.authentication.client.ConnectionConfigurator;
+import org.apache.tez.dag.api.TezException;
+import org.apache.tez.dag.api.client.TimelineReaderFactory.TimelineReaderPseudoAuthenticatedStrategy;
+import org.junit.Assert;
+import org.junit.Assume;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestTimelineReaderFactory {
+
+  @Before
+  public void setup() {
+    // Disable tests if hadoop version is less than 2.4.0
+    // as Timeline is not supported in 2.2.x or 2.3.x
+    String hadoopVersion = System.getProperty("tez.hadoop.version");
+    Assume.assumeFalse(hadoopVersion.startsWith("2.2.") || hadoopVersion.startsWith("2.3."));
+  }
+
+  // ensure on hadoop 2.4 TimelinePseudoAuthenticatedStrategy is used.
+  @Test(timeout = 5000)
+  public void testShouldUsePseudoAuthStrategyForHadoop24() throws TezException {
+    String hadoopVersion = System.getProperty("tez.hadoop.version");
+    Assume.assumeTrue(hadoopVersion.startsWith("2.4.") || hadoopVersion.startsWith("2.5."));
+
+    String returnedClassName =
+        TimelineReaderFactory.getTimelineReaderStrategy(mock(Configuration.class), false, 0)
+            .getClass()
+            .getCanonicalName();
+    Assert.assertEquals("should use pseudo auth on hadoop2.4",
+        "org.apache.tez.dag.api.client.TimelineReaderFactory.TimelineReaderPseudoAuthenticatedStrategy",
+        returnedClassName);
+  }
+
+  // ensure on hadoop 2.6+ TimelineReaderTokenAuthenticatedStrategy is used.
+  @Test(timeout = 5000)
+  public void testShouldUseTokenDelegationAuthStrategyForHadoop26() throws TezException {
+    String hadoopVersion = System.getProperty("tez.hadoop.version");
+    Assume.assumeFalse(hadoopVersion.startsWith("2.2.") ||
+        hadoopVersion.startsWith("2.3.") ||
+            hadoopVersion.startsWith("2.4.") ||
+            hadoopVersion.startsWith("2.5."));
+
+    String returnedClassName =
+        TimelineReaderFactory.getTimelineReaderStrategy(mock(Configuration.class), false, 0)
+            .getClass()
+            .getCanonicalName();
+    Assert.assertEquals("should use pseudo auth on hadoop2.4",
+        "org.apache.tez.dag.api.client.TimelineReaderFactory.TimelineReaderTokenAuthenticatedStrategy",
+        returnedClassName);
+  }
+
+  @Test(timeout = 5000)
+  public void testPseudoAuthenticatorConnectionUrlShouldHaveUserName() throws Exception {
+    ConnectionConfigurator connConf = mock(ConnectionConfigurator.class);
+    TimelineReaderPseudoAuthenticatedStrategy.PseudoAuthenticatedURLConnectionFactory
+        connectionFactory = new TimelineReaderPseudoAuthenticatedStrategy
+          .PseudoAuthenticatedURLConnectionFactory(connConf);
+    String inputUrl = "http://host:8080/path";
+    String expectedUrl = inputUrl + "?user.name=" + UserGroupInformation.getCurrentUser().getShortUserName();
+    HttpURLConnection httpURLConnection = connectionFactory.getHttpURLConnection(new URL(inputUrl));
+    Assert.assertEquals(expectedUrl, httpURLConnection.getURL().toString());
+  }
+
+}


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

Branch: refs/heads/TEZ-2003
Commit: df757f1b62a8edfabad455bdc88899479996c3f9
Parents: c61850b
Author: Siddharth Seth <ss...@apache.org>
Authored: Thu May 28 02:01:04 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Mon Jun 1 14:26:41 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/df757f1b/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/df757f1b/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/df757f1b/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/df757f1b/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/df757f1b/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/df757f1b/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/df757f1b/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 1182d54..1d9bdc1 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.dag.app.dag.event.TaskAttemptEventType;
@@ -356,7 +357,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);
     }
@@ -364,7 +365,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
@@ -405,7 +406,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");
@@ -419,7 +420,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/df757f1b/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 34c8822..ee6ce6e 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;
@@ -183,7 +185,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) {
@@ -229,7 +231,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/df757f1b/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/df757f1b/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 39df2e8..e9e0f04 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;
@@ -624,7 +627,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();
     }
   }
@@ -654,7 +657,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()
@@ -680,7 +683,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);
@@ -732,7 +735,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();
     }
   }
@@ -749,7 +752,7 @@ public class AMContainerImpl implements AMContainer {
                 container.getState(), TaskAttemptTerminationCause.FRAMEWORK_ERROR);
       }
       container.logStopped(ContainerExitStatus.ABORTED);
-      container.unregisterFromTAListener();
+      container.unregisterFromTAListener(ContainerEndReason.FRAMEWORK_ERROR);
       container.sendStopRequestToNM();
     }
   }
@@ -821,7 +824,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);
     }
   }
@@ -832,7 +835,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;
     }
   }
@@ -849,7 +852,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);
@@ -859,7 +862,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);
     }
   }
@@ -880,7 +883,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);
     }
   }
 
@@ -889,7 +892,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 " +
@@ -1015,7 +1018,7 @@ public class AMContainerImpl implements AMContainer {
     LOG.warn(errorMessage);
     this.logStopped(ContainerExitStatus.INVALID);
     this.sendStopRequestToNM();
-    this.unregisterFromTAListener();
+    this.unregisterFromTAListener(ContainerEndReason.FRAMEWORK_ERROR);
     this.unregisterFromContainerListener();
   }
 
@@ -1073,8 +1076,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) {
@@ -1085,8 +1088,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/df757f1b/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 63f8039..6043ece 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
@@ -43,6 +43,8 @@ import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.tez.common.ContainerContext;
 import org.apache.tez.common.ContainerTask;
 import org.apache.tez.common.security.JobTokenSecretManager;
+import org.apache.tez.dag.api.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;
@@ -154,12 +156,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());
 
@@ -173,7 +175,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/df757f1b/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 bdd0f61..b8b4774 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;
@@ -132,14 +134,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());
@@ -181,13 +183,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());
@@ -232,7 +234,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);
@@ -247,14 +249,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());
@@ -287,7 +289,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());
@@ -324,7 +326,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());
@@ -347,7 +349,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);
@@ -385,7 +387,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);
@@ -421,7 +423,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);
@@ -455,7 +457,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);
@@ -489,7 +491,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,
@@ -539,7 +541,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,
@@ -569,7 +571,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,
@@ -600,7 +602,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,
@@ -631,7 +633,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);
 
@@ -660,7 +662,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);
 
@@ -695,7 +697,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);
 
@@ -732,7 +734,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);
 
@@ -769,7 +772,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/df757f1b/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/df757f1b/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());
+        }
       }
     }
   }


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

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


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

Branch: refs/heads/TEZ-2003
Commit: 3b36e0db11bb1f12e1059cb88330525493ba3dfe
Parents: 8d82845
Author: Siddharth Seth <ss...@apache.org>
Authored: Thu Feb 12 11:25:45 2015 -0800
Committer: Siddharth Seth <ss...@apache.org>
Committed: Mon Jun 1 14:09:25 2015 -0700

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


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

http://git-wip-us.apache.org/repos/asf/tez/blob/3b36e0db/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicator.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicator.java b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicator.java
new file mode 100644
index 0000000..97f9c16
--- /dev/null
+++ b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicator.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.dag.api;
+
+import java.net.InetSocketAddress;
+import java.util.Map;
+
+import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.service.AbstractService;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.LocalResource;
+import org.apache.tez.dag.records.TezTaskAttemptID;
+import org.apache.tez.runtime.api.impl.TaskSpec;
+
+// TODO TEZ-2003 Move this into the tez-api module
+public abstract class TaskCommunicator extends AbstractService {
+  public TaskCommunicator(String name) {
+    super(name);
+  }
+
+  // TODO TEZ-2003 Ideally, don't expose YARN containerId; instead expose a Tez specific construct.
+  // TODO When talking to an external service, this plugin implementer may need access to a host:port
+  public abstract void registerRunningContainer(ContainerId containerId, String hostname, int port);
+
+  // TODO TEZ-2003 Ideally, don't expose YARN containerId; instead expose a Tez specific construct.
+  public abstract void registerContainerEnd(ContainerId containerId);
+
+  // TODO TEZ-2003 TaskSpec breakup into a clean interface
+  // TODO TEZ-2003 Add support for priority
+  public abstract void registerRunningTaskAttempt(ContainerId containerId, TaskSpec taskSpec,
+                                                  Map<String, LocalResource> additionalResources,
+                                                  Credentials credentials,
+                                                  boolean credentialsChanged);
+
+  // TODO TEZ-2003 Remove reference to TaskAttemptID
+  public abstract void unregisterRunningTaskAttempt(TezTaskAttemptID taskAttemptID);
+
+  // TODO TEZ-2003 This doesn't necessarily belong here. A server may not start within the AM.
+  public abstract InetSocketAddress getAddress();
+
+  // TODO Eventually. Add methods here to support preemption of tasks.
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/3b36e0db/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorContext.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorContext.java b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorContext.java
new file mode 100644
index 0000000..9b2d889
--- /dev/null
+++ b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskCommunicatorContext.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.dag.api;
+
+import java.io.IOException;
+
+import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.tez.dag.records.TezTaskAttemptID;
+
+
+// Do not make calls into this from within a held lock.
+
+// TODO TEZ-2003 Move this into the tez-api module
+public interface TaskCommunicatorContext {
+
+  // TODO TEZ-2003 Add signalling back into this to indicate errors - e.g. Container unregsitered, task no longer running, etc.
+
+  // TODO TEZ-2003 Maybe add book-keeping as a helper library, instead of each impl tracking container to task etc.
+
+  ApplicationAttemptId getApplicationAttemptId();
+  Credentials getCredentials();
+
+  // TODO TEZ-2003 Move to vertex, taskIndex, version
+  boolean canCommit(TezTaskAttemptID taskAttemptId) throws IOException;
+
+  TaskHeartbeatResponse heartbeat(TaskHeartbeatRequest request) throws IOException, TezException;
+
+  boolean isKnownContainer(ContainerId containerId);
+
+  // TODO TEZ-2003 Move to vertex, taskIndex, version
+  void taskStartedRemotely(TezTaskAttemptID taskAttemptID, ContainerId containerId);
+
+  // TODO Eventually Add methods to report availability stats to the scheduler.
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/3b36e0db/tez-dag/src/main/java/org/apache/tez/dag/api/TaskHeartbeatRequest.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/api/TaskHeartbeatRequest.java b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskHeartbeatRequest.java
new file mode 100644
index 0000000..f6bc8f0
--- /dev/null
+++ b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskHeartbeatRequest.java
@@ -0,0 +1,63 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.dag.api;
+
+import java.util.List;
+
+import org.apache.tez.dag.records.TezTaskAttemptID;
+import org.apache.tez.runtime.api.impl.TezEvent;
+
+// TODO TEZ-2003 Move this into the tez-api module
+public class TaskHeartbeatRequest {
+
+  // TODO TEZ-2003 Ideally containerIdentifier should not be part of the request.
+  // Replace with a task lookup - vertex name + task index
+  private final String containerIdentifier;
+  // TODO TEZ-2003 Get rid of the task attemptId reference if possible
+  private final TezTaskAttemptID taskAttemptId;
+  private final List<TezEvent> events;
+  private final int startIndex;
+  private final int maxEvents;
+
+
+  public TaskHeartbeatRequest(String containerIdentifier, TezTaskAttemptID taskAttemptId, List<TezEvent> events, int startIndex,
+                              int maxEvents) {
+    this.containerIdentifier = containerIdentifier;
+    this.taskAttemptId = taskAttemptId;
+    this.events = events;
+    this.startIndex = startIndex;
+    this.maxEvents = maxEvents;
+  }
+
+  public String getContainerIdentifier() {
+    return containerIdentifier;
+  }
+
+  public TezTaskAttemptID getTaskAttemptId() {
+    return taskAttemptId;
+  }
+
+  public List<TezEvent> getEvents() {
+    return events;
+  }
+
+  public int getStartIndex() {
+    return startIndex;
+  }
+
+  public int getMaxEvents() {
+    return maxEvents;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/3b36e0db/tez-dag/src/main/java/org/apache/tez/dag/api/TaskHeartbeatResponse.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/api/TaskHeartbeatResponse.java b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskHeartbeatResponse.java
new file mode 100644
index 0000000..c82a743
--- /dev/null
+++ b/tez-dag/src/main/java/org/apache/tez/dag/api/TaskHeartbeatResponse.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.dag.api;
+
+import java.util.List;
+
+import org.apache.tez.runtime.api.impl.TezEvent;
+
+// TODO TEZ-2003 Move this into the tez-api module
+public class TaskHeartbeatResponse {
+
+  private final boolean shouldDie;
+  private List<TezEvent> events;
+
+  public TaskHeartbeatResponse(boolean shouldDie, List<TezEvent> events) {
+    this.shouldDie = shouldDie;
+    this.events = events;
+  }
+
+  public boolean isShouldDie() {
+    return shouldDie;
+  }
+
+  public List<TezEvent> getEvents() {
+    return events;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/3b36e0db/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 c005447..1f14887 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/AppContext.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/AppContext.java
@@ -24,6 +24,7 @@ import java.util.Set;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
@@ -108,4 +109,6 @@ public interface AppContext {
   String[] getLocalDirs();
 
   String getAMUser();
+
+  Credentials getAppCredentials();
 }

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

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

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

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

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

http://git-wip-us.apache.org/repos/asf/tez/blob/3b36e0db/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerEventAssignTA.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerEventAssignTA.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerEventAssignTA.java
index 682cd02..0398882 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerEventAssignTA.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerEventAssignTA.java
@@ -27,6 +27,8 @@ import org.apache.tez.runtime.api.impl.TaskSpec;
 
 public class AMContainerEventAssignTA extends AMContainerEvent {
 
+  // TODO TEZ-2003. Add the task priority to this event.
+
   private final TezTaskAttemptID attemptId;
   // TODO Maybe have tht TAL pull the remoteTask from the TaskAttempt itself ?
   private final TaskSpec remoteTaskSpec;

http://git-wip-us.apache.org/repos/asf/tez/blob/3b36e0db/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerImpl.java
index 330f2b7..1acec9c 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerImpl.java
@@ -35,6 +35,7 @@ import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.LocalResource;
+import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.event.Event;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.state.InvalidStateTransitonException;

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

http://git-wip-us.apache.org/repos/asf/tez/blob/3b36e0db/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 ac816f4..f41b51b 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag.java
@@ -1,16 +1,16 @@
 /*
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
+* Licensed under the Apache License, Version 2.0 (the "License");
+* you may not use this file except in compliance with the License.
+* You may obtain a copy of the License at
+*
+*     http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
 
 package org.apache.tez.dag.app;
 
@@ -18,6 +18,7 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
+import static org.mockito.Matchers.any;
 import static org.mockito.Mockito.doReturn;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.times;
@@ -34,6 +35,7 @@ import java.util.Map;
 import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.event.Event;
 import org.apache.hadoop.yarn.event.EventHandler;
@@ -41,6 +43,12 @@ import org.apache.tez.common.ContainerContext;
 import org.apache.tez.common.ContainerTask;
 import org.apache.tez.common.security.JobTokenSecretManager;
 import org.apache.tez.dag.api.TezException;
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.tez.common.ContainerContext;
+import org.apache.tez.common.ContainerTask;
+import org.apache.tez.common.TezTaskUmbilicalProtocol;
+import org.apache.tez.dag.api.TaskCommunicatorContext;
 import org.apache.tez.dag.app.dag.DAG;
 import org.apache.tez.dag.app.dag.Vertex;
 import org.apache.tez.dag.app.dag.event.TaskAttemptEventType;
@@ -101,9 +109,18 @@ public class TestTaskAttemptListenerImplTezDag {
     doReturn(dag).when(appContext).getCurrentDAG();
     doReturn(appAcls).when(appContext).getApplicationACLs();
     doReturn(amContainerMap).when(appContext).getAllContainers();
-
-    taskAttemptListener = new TaskAttemptListenerImplForTest(appContext,
-        mock(TaskHeartbeatHandler.class), mock(ContainerHeartbeatHandler.class), null);
+    NodeId nodeId = NodeId.newInstance("localhost", 0);
+    AMContainer amContainer = mock(AMContainer.class);
+    Container container = mock(Container.class);
+    doReturn(nodeId).when(container).getNodeId();
+    doReturn(amContainer).when(amContainerMap).get(any(ContainerId.class));
+    doReturn(container).when(amContainer).getContainer();
+
+    taskAttemptListener =
+        new TaskAttemptListenerImpTezDag(appContext, mock(TaskHeartbeatHandler.class),
+            mock(ContainerHeartbeatHandler.class), null);
+    TezTaskCommunicatorImpl taskCommunicator = (TezTaskCommunicatorImpl)taskAttemptListener.getTaskCommunicator();
+    TezTaskUmbilicalProtocol tezUmbilical = taskCommunicator.getUmbilical();
 
     taskSpec = mock(TaskSpec.class);
     doReturn(taskAttemptID).when(taskSpec).getTaskAttemptID();
@@ -115,32 +132,30 @@ public class TestTaskAttemptListenerImplTezDag {
   public void testGetTask() throws IOException {
 
     ContainerId containerId1 = createContainerId(appId, 1);
-    doReturn(mock(AMContainer.class)).when(amContainerMap).get(containerId1);
     ContainerContext containerContext1 = new ContainerContext(containerId1.toString());
-    containerTask = taskAttemptListener.getTask(containerContext1);
+    containerTask = tezUmbilical.getTask(containerContext1);
     assertTrue(containerTask.shouldDie());
 
     ContainerId containerId2 = createContainerId(appId, 2);
-    doReturn(mock(AMContainer.class)).when(amContainerMap).get(containerId2);
     ContainerContext containerContext2 = new ContainerContext(containerId2.toString());
     taskAttemptListener.registerRunningContainer(containerId2);
-    containerTask = taskAttemptListener.getTask(containerContext2);
+    containerTask = tezUmbilical.getTask(containerContext2);
     assertNull(containerTask);
 
     // Valid task registered
     taskAttemptListener.registerTaskAttempt(amContainerTask, containerId2);
-    containerTask = taskAttemptListener.getTask(containerContext2);
+    containerTask = tezUmbilical.getTask(containerContext2);
     assertFalse(containerTask.shouldDie());
     assertEquals(taskSpec, containerTask.getTaskSpec());
 
     // Task unregistered. Should respond to heartbeats
-    taskAttemptListener.unregisterTaskAttempt(taskAttemptID);
-    containerTask = taskAttemptListener.getTask(containerContext2);
+    taskAttemptListener.unregisterTaskAttempt(taskAttemptId);
+    containerTask = tezUmbilical.getTask(containerContext2);
     assertNull(containerTask);
 
     // Container unregistered. Should send a shouldDie = true
     taskAttemptListener.unregisterRunningContainer(containerId2);
-    containerTask = taskAttemptListener.getTask(containerContext2);
+    containerTask = tezUmbilical.getTask(containerContext2);
     assertTrue(containerTask.shouldDie());
 
     ContainerId containerId3 = createContainerId(appId, 3);
@@ -154,27 +169,30 @@ public class TestTaskAttemptListenerImplTezDag {
     AMContainerTask amContainerTask2 = new AMContainerTask(taskSpec, null, null, false, 0);
     taskAttemptListener.registerTaskAttempt(amContainerTask2, containerId3);
     taskAttemptListener.unregisterRunningContainer(containerId3);
-    containerTask = taskAttemptListener.getTask(containerContext3);
+    containerTask = tezUmbilical.getTask(containerContext3);
     assertTrue(containerTask.shouldDie());
   }
 
   @Test(timeout = 5000)
   public void testGetTaskMultiplePulls() throws IOException {
+    TezTaskCommunicatorImpl taskCommunicator = (TezTaskCommunicatorImpl)taskAttemptListener.getTaskCommunicator();
+    TezTaskUmbilicalProtocol tezUmbilical = taskCommunicator.getUmbilical();
+
     ContainerId containerId1 = createContainerId(appId, 1);
     doReturn(mock(AMContainer.class)).when(amContainerMap).get(containerId1);
     ContainerContext containerContext1 = new ContainerContext(containerId1.toString());
     taskAttemptListener.registerRunningContainer(containerId1);
-    containerTask = taskAttemptListener.getTask(containerContext1);
+    containerTask = tezUmbilical.getTask(containerContext1);
     assertNull(containerTask);
 
     // Register task
     taskAttemptListener.registerTaskAttempt(amContainerTask, containerId1);
-    containerTask = taskAttemptListener.getTask(containerContext1);
+    containerTask = tezUmbilical.getTask(containerContext1);
     assertFalse(containerTask.shouldDie());
     assertEquals(taskSpec, containerTask.getTaskSpec());
 
     // Try pulling again - simulates re-use pull
-    containerTask = taskAttemptListener.getTask(containerContext1);
+    containerTask = tezUmbilical.getTask(containerContext1);
     assertNull(containerTask);
   }
 
@@ -266,13 +284,11 @@ public class TestTaskAttemptListenerImplTezDag {
     return ContainerId.newInstance(appAttemptId, containerIdx);
   }
 
-  private static class TaskAttemptListenerImplForTest extends TaskAttemptListenerImpTezDag {
+  private static class TezTaskCommunicatorImplForTest extends TezTaskCommunicatorImpl {
 
-    public TaskAttemptListenerImplForTest(AppContext context,
-                                          TaskHeartbeatHandler thh,
-                                          ContainerHeartbeatHandler chh,
-                                          JobTokenSecretManager jobTokenSecretManager) {
-      super(context, thh, chh, jobTokenSecretManager);
+    public TezTaskCommunicatorImplForTest(
+        TaskCommunicatorContext taskCommunicatorContext) {
+      super(taskCommunicatorContext);
     }
 
     @Override


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

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


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

Branch: refs/heads/TEZ-2003
Commit: 2fb86f3204a6f6c19019605315b9d52a36180737
Parents: d27b897
Author: Siddharth Seth <ss...@apache.org>
Authored: Mon Feb 23 20:53:24 2015 -0800
Committer: Siddharth Seth <ss...@apache.org>
Committed: Mon Jun 1 14:09:26 2015 -0700

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


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

http://git-wip-us.apache.org/repos/asf/tez/blob/2fb86f32/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 c834cf9..b11d697 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
@@ -1517,7 +1517,7 @@ public class DAGAppMaster extends AbstractService {
 
     @Override
     public Integer getContainerLauncherIdentifier(String name) {
-      return taskCommunicators.get(name);
+      return containerLaunchers.get(name);
     }
 
     @Override

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

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

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


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

Branch: refs/heads/TEZ-2003
Commit: fac7b1d7edaa698fa5cc2ee57a6a3ae1e88138d7
Parents: caab578
Author: Siddharth Seth <ss...@apache.org>
Authored: Fri Feb 20 11:59:03 2015 -0800
Committer: Siddharth Seth <ss...@apache.org>
Committed: Mon Jun 1 14:09:26 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   |  39 +++--
 .../dag/app/rm/container/AMContainerMap.java    |   4 +-
 .../apache/tez/dag/app/rm/node/AMNodeImpl.java  |   6 +-
 .../apache/tez/dag/app/MockDAGAppMaster.java    |   2 +-
 .../app/TestTaskAttemptListenerImplTezDag.java  |  31 ++--
 .../tez/dag/app/dag/impl/TestTaskAttempt.java   |  69 ++++----
 .../tez/dag/app/dag/impl/TestVertexImpl.java    |   8 +-
 .../tez/dag/app/rm/TestContainerReuse.java      |  34 ++--
 .../tez/dag/app/rm/TestLocalTaskScheduler.java  |   2 +-
 .../app/rm/TestLocalTaskSchedulerService.java   |  18 ++-
 .../app/rm/TestTaskSchedulerEventHandler.java   |  11 +-
 .../dag/app/rm/TestTaskSchedulerHelpers.java    |   2 +-
 .../dag/app/rm/container/TestAMContainer.java   | 108 +++++++------
 .../app/rm/container/TestAMContainerMap.java    |   6 +-
 .../org/apache/tez/examples/JoinValidate.java   |  30 +++-
 .../TezTestServiceContainerLauncher.java        |   5 +-
 .../rm/TezTestServiceTaskSchedulerService.java  | 100 ++----------
 .../tez/examples/JoinValidateConfigured.java    |  53 ++++++
 .../tez/tests/TestExternalTezServices.java      | 160 ++++++++++++++-----
 39 files changed, 638 insertions(+), 356 deletions(-)
----------------------------------------------------------------------


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

http://git-wip-us.apache.org/repos/asf/tez/blob/fac7b1d7/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 4ab1201..47b76cb 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
@@ -1221,7 +1221,7 @@ public class TaskAttemptImpl implements TaskAttempt,
       // Inform the scheduler
       if (sendSchedulerEvent()) {
         ta.sendEvent(new AMSchedulerEventTAEnded(ta, ta.containerId, helper
-            .getTaskAttemptState()));
+            .getTaskAttemptState(), ta.getVertex().getTaskSchedulerIdentifier()));
       }
     }
   }
@@ -1303,7 +1303,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/fac7b1d7/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/fac7b1d7/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 a677dea..5343c9e 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
@@ -4405,7 +4405,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/fac7b1d7/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexManager.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexManager.java
index 945d9ba..1300fc0 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexManager.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexManager.java
@@ -286,7 +286,7 @@ public class VertexManager {
     @Override
     public synchronized Resource getTotalAvailableResource() {
       checkAndThrowIfDone();
-      return appContext.getTaskScheduler().getTotalResources();
+      return appContext.getTaskScheduler().getTotalResources(managedVertex.getTaskSchedulerIdentifier());
     }
 
     @Override

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

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

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

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

http://git-wip-us.apache.org/repos/asf/tez/blob/fac7b1d7/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/fac7b1d7/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/fac7b1d7/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 6683d84..49bf592 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/fac7b1d7/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 62c5b21..be97309 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
@@ -110,9 +110,16 @@ public class TestTaskAttemptListenerImplTezDag {
     doReturn(dag).when(appContext).getCurrentDAG();
     doReturn(appAcls).when(appContext).getApplicationACLs();
     doReturn(amContainerMap).when(appContext).getAllContainers();
+    NodeId nodeId = NodeId.newInstance("localhost", 0);
+
+    AMContainer amContainer = mock(AMContainer.class);
+    Container container = mock(Container.class);
+    doReturn(nodeId).when(container).getNodeId();
+    doReturn(amContainer).when(amContainerMap).get(any(ContainerId.class));
+    doReturn(container).when(amContainer).getContainer();
 
     taskAttemptListener = new TaskAttemptListenerImplForTest(appContext,
-        mock(TaskHeartbeatHandler.class), mock(ContainerHeartbeatHandler.class), null);
+        mock(TaskHeartbeatHandler.class), mock(ContainerHeartbeatHandler.class), null, null);
 
     taskSpec = mock(TaskSpec.class);
     doReturn(taskAttemptID).when(taskSpec).getTaskAttemptID();
@@ -123,7 +130,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);
@@ -133,55 +140,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/fac7b1d7/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 86251cc..5046b84 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/fac7b1d7/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 0176b79..2776ca0 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
@@ -2179,7 +2179,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();
 
@@ -3071,7 +3071,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));
@@ -3106,7 +3106,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));
@@ -3142,7 +3142,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));


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

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


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

Branch: refs/heads/TEZ-2003
Commit: 63d1e6278af1cb1b8593a3845adb313be8d8d858
Parents: 2b663b1
Author: Siddharth Seth <ss...@apache.org>
Authored: Tue Mar 10 01:25:39 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Mon Jun 1 14:09:27 2015 -0700

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


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

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

http://git-wip-us.apache.org/repos/asf/tez/blob/63d1e627/tez-common/src/main/java/org/apache/tez/dag/api/TaskAttemptEndReason.java
----------------------------------------------------------------------
diff --git a/tez-common/src/main/java/org/apache/tez/dag/api/TaskAttemptEndReason.java b/tez-common/src/main/java/org/apache/tez/dag/api/TaskAttemptEndReason.java
new file mode 100644
index 0000000..96a4768
--- /dev/null
+++ b/tez-common/src/main/java/org/apache/tez/dag/api/TaskAttemptEndReason.java
@@ -0,0 +1,24 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.dag.api;
+
+// TODO TEZ-2003 Expose as a public API
+public enum TaskAttemptEndReason {
+  COMMUNICATION_ERROR,
+  SERVICE_BUSY,
+  INTERRUPTED_BY_SYSTEM,
+  INTERRUPTED_BY_USER,
+  OTHER
+}

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

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

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

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

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

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

http://git-wip-us.apache.org/repos/asf/tez/blob/63d1e627/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventType.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventType.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventType.java
index b7aca36..6d20368 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventType.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventType.java
@@ -26,14 +26,15 @@ public enum TaskAttemptEventType {
 //Producer:Task, Speculator
   TA_SCHEDULE,
 
-//Producer: TaskAttemptListener
+//Producer: TaskAttemptListener | Vertex after routing events
   TA_STARTED_REMOTELY,
   TA_STATUS_UPDATE,
   TA_DIAGNOSTICS_UPDATE, // REMOVE THIS - UNUSED
   TA_DONE,
   TA_FAILED,
+  TA_KILLED, // Generated by TaskCommunicators
   TA_TIMED_OUT,
-  
+
 //Producer: Client, Scheduler, On speculation.
   TA_KILL_REQUEST,
 

http://git-wip-us.apache.org/repos/asf/tez/blob/63d1e627/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 47b76cb..efc4b04 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java
@@ -51,6 +51,7 @@ import org.apache.hadoop.yarn.state.StateMachineFactory;
 import org.apache.hadoop.yarn.util.Clock;
 import org.apache.hadoop.yarn.util.RackResolver;
 import org.apache.hadoop.yarn.util.Records;
+import org.apache.tez.common.TezUtilsInternal;
 import org.apache.tez.common.counters.DAGCounter;
 import org.apache.tez.common.counters.TezCounters;
 import org.apache.tez.dag.api.TezConfiguration;
@@ -184,6 +185,11 @@ public class TaskAttemptImpl implements TaskAttempt,
 
   private final StateMachine<TaskAttemptStateInternal, TaskAttemptEventType, TaskAttemptEvent> stateMachine;
 
+  // TODO TEZ-2003 We may need some additional state management for STATUS_UPDATES, FAILED, KILLED coming in before
+  // TASK_STARTED_REMOTELY. In case of a PUSH it's more intuitive to send TASK_STARTED_REMOTELY after communicating
+  // with the listening service and getting a response, which in turn can trigger STATUS_UPDATES / FAILED / KILLED
+
+  // TA_KILLED handled the same as TA_KILL_REQUEST. Just a different name indicating a request / already killed.
   private static StateMachineFactory
   <TaskAttemptImpl, TaskAttemptStateInternal, TaskAttemptEventType, TaskAttemptEvent>
   stateMachineFactory
@@ -224,6 +230,10 @@ public class TaskAttemptImpl implements TaskAttempt,
           new TerminatedBeforeRunningTransition(KILLED_HELPER))
       .addTransition(TaskAttemptStateInternal.START_WAIT,
           TaskAttemptStateInternal.KILL_IN_PROGRESS,
+          TaskAttemptEventType.TA_KILLED,
+          new TerminatedBeforeRunningTransition(KILLED_HELPER))
+      .addTransition(TaskAttemptStateInternal.START_WAIT,
+          TaskAttemptStateInternal.KILL_IN_PROGRESS,
           TaskAttemptEventType.TA_NODE_FAILED,
           new NodeFailedBeforeRunningTransition())
       .addTransition(TaskAttemptStateInternal.START_WAIT,
@@ -264,6 +274,10 @@ public class TaskAttemptImpl implements TaskAttempt,
           new TerminatedWhileRunningTransition(KILLED_HELPER))
       .addTransition(TaskAttemptStateInternal.RUNNING,
           TaskAttemptStateInternal.KILL_IN_PROGRESS,
+          TaskAttemptEventType.TA_KILLED,
+          new TerminatedWhileRunningTransition(KILLED_HELPER))
+      .addTransition(TaskAttemptStateInternal.RUNNING,
+          TaskAttemptStateInternal.KILL_IN_PROGRESS,
           TaskAttemptEventType.TA_NODE_FAILED,
           new TerminatedWhileRunningTransition(KILLED_HELPER))
       .addTransition(TaskAttemptStateInternal.RUNNING,
@@ -302,6 +316,7 @@ public class TaskAttemptImpl implements TaskAttempt,
               TaskAttemptEventType.TA_DONE, TaskAttemptEventType.TA_FAILED,
               TaskAttemptEventType.TA_TIMED_OUT,
               TaskAttemptEventType.TA_KILL_REQUEST,
+              TaskAttemptEventType.TA_KILLED,
               TaskAttemptEventType.TA_NODE_FAILED,
               TaskAttemptEventType.TA_CONTAINER_TERMINATING,
               TaskAttemptEventType.TA_OUTPUT_FAILED))
@@ -323,6 +338,7 @@ public class TaskAttemptImpl implements TaskAttempt,
               TaskAttemptEventType.TA_DONE, TaskAttemptEventType.TA_FAILED,
               TaskAttemptEventType.TA_TIMED_OUT,
               TaskAttemptEventType.TA_KILL_REQUEST,
+              TaskAttemptEventType.TA_KILLED,
               TaskAttemptEventType.TA_NODE_FAILED,
               TaskAttemptEventType.TA_CONTAINER_TERMINATING,
               TaskAttemptEventType.TA_OUTPUT_FAILED))
@@ -341,6 +357,7 @@ public class TaskAttemptImpl implements TaskAttempt,
               TaskAttemptEventType.TA_DONE, TaskAttemptEventType.TA_FAILED,
               TaskAttemptEventType.TA_TIMED_OUT,
               TaskAttemptEventType.TA_KILL_REQUEST,
+              TaskAttemptEventType.TA_KILLED,
               TaskAttemptEventType.TA_NODE_FAILED,
               TaskAttemptEventType.TA_CONTAINER_TERMINATING,
               TaskAttemptEventType.TA_CONTAINER_TERMINATED,
@@ -360,6 +377,7 @@ public class TaskAttemptImpl implements TaskAttempt,
               TaskAttemptEventType.TA_DONE, TaskAttemptEventType.TA_FAILED,
               TaskAttemptEventType.TA_TIMED_OUT,
               TaskAttemptEventType.TA_KILL_REQUEST,
+              TaskAttemptEventType.TA_KILLED,
               TaskAttemptEventType.TA_NODE_FAILED,
               TaskAttemptEventType.TA_CONTAINER_TERMINATING,
               TaskAttemptEventType.TA_CONTAINER_TERMINATED,
@@ -382,6 +400,12 @@ public class TaskAttemptImpl implements TaskAttempt,
           TaskAttemptStateInternal.SUCCEEDED,
           EnumSet.of(TaskAttemptStateInternal.KILLED,
               TaskAttemptStateInternal.SUCCEEDED),
+          TaskAttemptEventType.TA_KILLED,
+          new TerminatedAfterSuccessTransition())
+      .addTransition(
+          TaskAttemptStateInternal.SUCCEEDED,
+          EnumSet.of(TaskAttemptStateInternal.KILLED,
+              TaskAttemptStateInternal.SUCCEEDED),
           TaskAttemptEventType.TA_NODE_FAILED,
           new TerminatedAfterSuccessTransition())
       .addTransition(
@@ -433,7 +457,6 @@ public class TaskAttemptImpl implements TaskAttempt,
     this.leafVertex = leafVertex;
   }
 
-
   @Override
   public TezTaskAttemptID getID() {
     return attemptId;
@@ -1033,6 +1056,7 @@ public class TaskAttemptImpl implements TaskAttempt,
 
       // Compute node/rack location request even if re-scheduled.
       Set<String> racks = new HashSet<String>();
+      // TODO Post TEZ-2003. Allow for a policy in the VMPlugin to define localicty for different attempts.
       TaskLocationHint locationHint = ta.getTaskLocationHint();
       if (locationHint != null) {
         if (locationHint.getRacks() != null) {
@@ -1107,6 +1131,8 @@ public class TaskAttemptImpl implements TaskAttempt,
 
     @Override
     public void transition(TaskAttemptImpl ta, TaskAttemptEvent event) {
+      // This transition should not be invoked directly, if a scheduler event has already been sent out.
+      // Sub-classes should be used if a scheduler request has been sent.
       ta.setFinishTime();
 
       if (event instanceof DiagnosableEvent) {
@@ -1221,7 +1247,8 @@ public class TaskAttemptImpl implements TaskAttempt,
       // Inform the scheduler
       if (sendSchedulerEvent()) {
         ta.sendEvent(new AMSchedulerEventTAEnded(ta, ta.containerId, helper
-            .getTaskAttemptState(), ta.getVertex().getTaskSchedulerIdentifier()));
+            .getTaskAttemptState(), TezUtilsInternal.toTaskAttemptEndReason(ta.terminationCause),
+            ta.getVertex().getTaskSchedulerIdentifier()));
       }
     }
   }
@@ -1303,7 +1330,7 @@ public class TaskAttemptImpl implements TaskAttempt,
 
       // Inform the Scheduler.
       ta.sendEvent(new AMSchedulerEventTAEnded(ta, ta.containerId,
-          TaskAttemptState.SUCCEEDED, ta.getVertex().getTaskSchedulerIdentifier()));
+          TaskAttemptState.SUCCEEDED, null, ta.getVertex().getTaskSchedulerIdentifier()));
 
       // Inform the task.
       ta.sendEvent(new TaskEventTAUpdate(ta.attemptId,

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

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

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

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

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

http://git-wip-us.apache.org/repos/asf/tez/blob/63d1e627/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 bf00137..15b2a04 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
@@ -76,6 +76,7 @@ import org.junit.Test;
 import org.mockito.ArgumentCaptor;
 
 @SuppressWarnings("unchecked")
+// TODO TEZ-2003 Rename to TestTezTaskCommunicator
 public class TestTaskAttemptListenerImplTezDag {
   private ApplicationId appId;
   private AppContext appContext;

http://git-wip-us.apache.org/repos/asf/tez/blob/63d1e627/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag2.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag2.java b/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag2.java
new file mode 100644
index 0000000..934543f
--- /dev/null
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskAttemptListenerImplTezDag2.java
@@ -0,0 +1,126 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.dag.app;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.event.Event;
+import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.tez.dag.api.TaskAttemptEndReason;
+import org.apache.tez.dag.app.dag.DAG;
+import org.apache.tez.dag.app.dag.event.TaskAttemptEventAttemptFailed;
+import org.apache.tez.dag.app.dag.event.TaskAttemptEventAttemptKilled;
+import org.apache.tez.dag.app.rm.container.AMContainer;
+import org.apache.tez.dag.app.rm.container.AMContainerMap;
+import org.apache.tez.dag.app.rm.container.AMContainerTask;
+import org.apache.tez.dag.records.TaskAttemptTerminationCause;
+import org.apache.tez.dag.records.TezTaskAttemptID;
+import org.apache.tez.runtime.api.impl.TaskSpec;
+import org.junit.Test;
+import org.mockito.ArgumentCaptor;
+
+// TODO TEZ-2003. Rename to TestTaskAttemptListener | whatever TaskAttemptListener is renamed to.
+public class TestTaskAttemptListenerImplTezDag2 {
+
+  @Test(timeout = 5000)
+  public void testTaskAttemptFailedKilled() {
+    ApplicationId appId = ApplicationId.newInstance(1000, 1);
+    ApplicationAttemptId appAttemptId = ApplicationAttemptId.newInstance(appId, 1);
+    Credentials credentials = new Credentials();
+    AppContext appContext = mock(AppContext.class);
+    EventHandler eventHandler = mock(EventHandler.class);
+    DAG dag = mock(DAG.class);
+    AMContainerMap amContainerMap = mock(AMContainerMap.class);
+    Map<ApplicationAccessType, String> appAcls = new HashMap<ApplicationAccessType, String>();
+    doReturn(eventHandler).when(appContext).getEventHandler();
+    doReturn(dag).when(appContext).getCurrentDAG();
+    doReturn(appAttemptId).when(appContext).getApplicationAttemptId();
+    doReturn(credentials).when(appContext).getAppCredentials();
+    doReturn(appAcls).when(appContext).getApplicationACLs();
+    doReturn(amContainerMap).when(appContext).getAllContainers();
+    NodeId nodeId = NodeId.newInstance("localhost", 0);
+    AMContainer amContainer = mock(AMContainer.class);
+    Container container = mock(Container.class);
+    doReturn(nodeId).when(container).getNodeId();
+    doReturn(amContainer).when(amContainerMap).get(any(ContainerId.class));
+    doReturn(container).when(amContainer).getContainer();
+
+    TaskAttemptListenerImpTezDag taskAttemptListener =
+        new TaskAttemptListenerImpTezDag(appContext, mock(TaskHeartbeatHandler.class),
+            mock(ContainerHeartbeatHandler.class), null, null, false);
+
+    TaskSpec taskSpec1 = mock(TaskSpec.class);
+    TezTaskAttemptID taskAttemptId1 = mock(TezTaskAttemptID.class);
+    doReturn(taskAttemptId1).when(taskSpec1).getTaskAttemptID();
+    AMContainerTask amContainerTask1 = new AMContainerTask(taskSpec1, null, null, false, 10);
+
+    TaskSpec taskSpec2 = mock(TaskSpec.class);
+    TezTaskAttemptID taskAttemptId2 = mock(TezTaskAttemptID.class);
+    doReturn(taskAttemptId2).when(taskSpec2).getTaskAttemptID();
+    AMContainerTask amContainerTask2 = new AMContainerTask(taskSpec2, null, null, false, 10);
+
+    ContainerId containerId1 = createContainerId(appId, 1);
+    taskAttemptListener.registerRunningContainer(containerId1, 0);
+    taskAttemptListener.registerTaskAttempt(amContainerTask1, containerId1, 0);
+    ContainerId containerId2 = createContainerId(appId, 2);
+    taskAttemptListener.registerRunningContainer(containerId2, 0);
+    taskAttemptListener.registerTaskAttempt(amContainerTask2, containerId2, 0);
+
+
+    taskAttemptListener
+        .taskFailed(taskAttemptId1, TaskAttemptEndReason.COMMUNICATION_ERROR, "Diagnostics1");
+    taskAttemptListener
+        .taskKilled(taskAttemptId2, TaskAttemptEndReason.SERVICE_BUSY, "Diagnostics2");
+
+    ArgumentCaptor<Event> argumentCaptor = ArgumentCaptor.forClass(Event.class);
+    verify(eventHandler, times(2)).handle(argumentCaptor.capture());
+    assertTrue(argumentCaptor.getAllValues().get(0) instanceof TaskAttemptEventAttemptFailed);
+    assertTrue(argumentCaptor.getAllValues().get(1) instanceof TaskAttemptEventAttemptKilled);
+    TaskAttemptEventAttemptFailed failedEvent =
+        (TaskAttemptEventAttemptFailed) argumentCaptor.getAllValues().get(0);
+    TaskAttemptEventAttemptKilled killedEvent =
+        (TaskAttemptEventAttemptKilled) argumentCaptor.getAllValues().get(1);
+
+    assertEquals("Diagnostics1", failedEvent.getDiagnosticInfo());
+    assertEquals(TaskAttemptTerminationCause.COMMUNICATION_ERROR,
+        failedEvent.getTerminationCause());
+
+    assertEquals("Diagnostics2", killedEvent.getDiagnosticInfo());
+    assertEquals(TaskAttemptTerminationCause.SERVICE_BUSY, killedEvent.getTerminationCause());
+    // TODO TEZ-2003. Verify unregistration from the registered list
+  }
+
+  private ContainerId createContainerId(ApplicationId applicationId, int containerIdx) {
+    ApplicationAttemptId appAttemptId = ApplicationAttemptId.newInstance(applicationId, 1);
+    ContainerId containerId = ContainerId.newInstance(appAttemptId, containerIdx);
+    return containerId;
+  }
+}

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

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

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

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

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

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

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

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

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

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

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


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

Branch: refs/heads/TEZ-2003
Commit: c4b800d0aebc82e4a87be2cf28b45ce95e50064d
Parents: fe2b8a9
Author: Siddharth Seth <ss...@apache.org>
Authored: Mon May 11 16:42:45 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Mon Jun 1 14:21:53 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/c4b800d0/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/c4b800d0/tez-tools/pom.xml
----------------------------------------------------------------------
diff --git a/tez-tools/pom.xml b/tez-tools/pom.xml
index bf8fdf8..53cfe8b 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/c4b800d0/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>
 


[15/50] [abbrv] tez git commit: SITE UPDATE: Add Jeff Zhang to PMC list (zjffdu)

Posted by ss...@apache.org.
SITE UPDATE: Add Jeff Zhang to PMC list (zjffdu)


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

Branch: refs/heads/TEZ-2003
Commit: 5a04f06028ea4d3814bab6b044113d6eec559b9a
Parents: 87f26a2
Author: Jeff Zhang <zj...@apache.org>
Authored: Mon Jun 1 14:23:31 2015 +0800
Committer: Jeff Zhang <zj...@apache.org>
Committed: Mon Jun 1 14:23:31 2015 +0800

----------------------------------------------------------------------
 docs/pom.xml                        | 2 +-
 docs/src/site/resources/pmc/tez.rdf | 5 +++++
 2 files changed, 6 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/5a04f060/docs/pom.xml
----------------------------------------------------------------------
diff --git a/docs/pom.xml b/docs/pom.xml
index d732f44..49168ec 100644
--- a/docs/pom.xml
+++ b/docs/pom.xml
@@ -379,7 +379,7 @@
         <email>zjffdu@apache.org</email>
         <timezone>+8</timezone>
         <roles>
-          <role>Committer</role>
+          <role>PMC</role>
         </roles>
       </developer>
       <developer>

http://git-wip-us.apache.org/repos/asf/tez/blob/5a04f060/docs/src/site/resources/pmc/tez.rdf
----------------------------------------------------------------------
diff --git a/docs/src/site/resources/pmc/tez.rdf b/docs/src/site/resources/pmc/tez.rdf
index 86faad0..1aa8d07 100644
--- a/docs/src/site/resources/pmc/tez.rdf
+++ b/docs/src/site/resources/pmc/tez.rdf
@@ -93,6 +93,11 @@
     </asfext:member>
     <asfext:member>
       <foaf:Person>
+        <foaf:name>Jeff Zhang</foaf:name>
+      </foaf:Person>
+    </asfext:member>
+    <asfext:member>
+      <foaf:Person>
         <foaf:name>Jitendra Pandey</foaf:name>
       </foaf:Person>
     </asfext:member>


[11/50] [abbrv] tez git commit: TEZ-2503. findbugs version isn't reported properly in test-patch report. (sseth)

Posted by ss...@apache.org.
TEZ-2503. findbugs version isn't reported properly in test-patch report. (sseth)


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

Branch: refs/heads/TEZ-2003
Commit: fafa804c953aa528c839be7a14d0ccbdd35c75f9
Parents: 42b7756
Author: Siddharth Seth <ss...@apache.org>
Authored: Fri May 29 11:42:23 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Fri May 29 11:42:23 2015 -0700

----------------------------------------------------------------------
 CHANGES.txt               | 1 +
 build-tools/test-patch.sh | 2 +-
 2 files changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/fafa804c/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 7f240c1..6fb6051 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -8,6 +8,7 @@ INCOMPATIBLE CHANGES
   TEZ-2468. Change the minimum Java version to Java 7.
 
 ALL CHANGES:
+  TEZ-2503. findbugs version isn't reported properly in test-patch report.
   TEZ-2198. Fix sorter spill counts.
   TEZ-1883. Change findbugs version to 3.x.
   TEZ-2440. Sorter should check for indexCacheList.size() in flush().

http://git-wip-us.apache.org/repos/asf/tez/blob/fafa804c/build-tools/test-patch.sh
----------------------------------------------------------------------
diff --git a/build-tools/test-patch.sh b/build-tools/test-patch.sh
index 0ac2bac..d41c54b 100755
--- a/build-tools/test-patch.sh
+++ b/build-tools/test-patch.sh
@@ -525,7 +525,7 @@ buildAndInstall () {
 ###############################################################################
 ### Check there are no changes in the number of Findbugs warnings
 checkFindbugsWarnings () {
-  findbugs_version=$(${AWK} 'match($0, /findbugs-maven-plugin:[^:]*:findbugs/) { print substr($0, RSTART + 22, RLENGTH - 31); exit }' "${PATCH_DIR}/patchFindBugsOutput${module_suffix}.txt")
+  findbugs_version=$(${AWK} 'match($0, /findbugs-maven-plugin:[^:]*:findbugs/) { print substr($0, RSTART + 22, RLENGTH - 31); exit }' "${PATCH_DIR}/patchFindBugsOutput.txt")
   echo ""
   echo ""
   echo "======================================================================"


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

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


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

Branch: refs/heads/TEZ-2003
Commit: 8d828457aefa742d217b90c21509bbdc9bde1129
Parents: 7fe0a6b
Author: Siddharth Seth <ss...@apache.org>
Authored: Fri Jan 30 16:02:32 2015 -0800
Committer: Siddharth Seth <ss...@apache.org>
Committed: Mon Jun 1 14:04:56 2015 -0700

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


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

http://git-wip-us.apache.org/repos/asf/tez/blob/8d828457/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 15b1333..828405e 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
@@ -1175,6 +1175,12 @@ public class TezConfiguration extends Configuration {
       + "tez-ui.webservice.enable";
   public static final boolean TEZ_AM_WEBSERVICE_ENABLE_DEFAULT = true;
 
+  @ConfigurationScope(Scope.VERTEX)
+  public static final String TEZ_AM_CONTAINER_LAUNCHER_CLASS = TEZ_AM_PREFIX + "container-launcher.class";
+  @ConfigurationScope(Scope.VERTEX)
+  public static final String TEZ_AM_TASK_SCHEDULER_CLASS = TEZ_AM_PREFIX + "task-scheduler.class";
+
+
   // TODO only validate property here, value can also be validated if necessary
   public static void validateProperty(String property, Scope usedScope) {
     Scope validScope = PropertyScope.get(property);

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

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

http://git-wip-us.apache.org/repos/asf/tez/blob/8d828457/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 36297a9..83ebadb 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
@@ -362,7 +362,8 @@ public class TezChild {
       }
       if (!isLocal) {
         RPC.stopProxy(umbilical);
-        LogManager.shutdown();
+        // TODO Temporary change. Revert. Ideally, move this over to the main method in TezChild if possible.
+//        LogManager.shutdown();
       }
     }
   }


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

Branch: refs/heads/TEZ-2003
Commit: c4fa9f508b7334c23a9194ec373735ab8dde39fd
Parents: f59cf56
Author: Siddharth Seth <ss...@apache.org>
Authored: Tue Apr 28 13:41:12 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Mon Jun 1 14:10:28 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/c4fa9f50/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/c4fa9f50/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 03b5602..d30919b 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
@@ -17,28 +17,22 @@
 
 package org.apache.tez.dag.app;
 
-import javax.annotation.Nullable;
 import java.io.IOException;
 import java.lang.reflect.Constructor;
 import java.lang.reflect.InvocationTargetException;
-import java.net.InetSocketAddress;
-import java.net.URISyntaxException;
-import java.net.UnknownHostException;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
-import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.commons.collections4.ListUtils;
 import org.apache.tez.dag.app.dag.event.TaskAttemptEvent;
 import org.apache.tez.dag.app.dag.event.TaskAttemptEventStatusUpdate;
 import org.apache.tez.dag.app.dag.event.TaskAttemptEventType;
 import org.apache.tez.runtime.api.events.TaskStatusUpdateEvent;
 import org.apache.tez.runtime.api.impl.EventType;
-import com.google.common.base.Preconditions;
-import org.apache.tez.dag.api.event.VertexState;
 import org.apache.tez.dag.api.event.VertexStateUpdate;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -62,7 +56,6 @@ import org.apache.tez.dag.app.dag.Task;
 import org.apache.tez.dag.app.dag.event.TaskAttemptEventAttemptFailed;
 import org.apache.tez.dag.app.dag.event.TaskAttemptEventAttemptKilled;
 import org.apache.tez.dag.app.dag.event.TaskAttemptEventStartedRemotely;
-import org.apache.tez.dag.app.dag.event.TaskAttemptEventType;
 import org.apache.tez.dag.app.dag.event.VertexEventRouteEvent;
 import org.apache.tez.dag.app.rm.container.AMContainerTask;
 import org.apache.tez.dag.records.TezTaskAttemptID;
@@ -151,7 +144,7 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
   private TaskCommunicator createTaskCommunicator(String taskCommClassIdentifier, int taskCommIndex) {
     if (taskCommClassIdentifier.equals(TezConstants.TEZ_AM_SERVICE_PLUGINS_NAME_DEFAULT)) {
       LOG.info("Using Default Task Communicator");
-      return new TezTaskCommunicatorImpl(taskCommunicatorContexts[taskCommIndex]);
+      return createTezTaskCommunicator(taskCommunicatorContexts[taskCommIndex]);
     } else if (taskCommClassIdentifier.equals(TezConstants.TEZ_AM_SERVICE_PLUGINS_LOCAL_MODE_NAME_DEFAULT)) {
       LOG.info("Using Default Local Task Communicator");
       return new TezLocalTaskCommunicatorImpl(taskCommunicatorContexts[taskCommIndex]);
@@ -174,6 +167,12 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
       }
     }
   }
+
+  @VisibleForTesting
+  protected TezTaskCommunicatorImpl createTezTaskCommunicator(TaskCommunicatorContext context) {
+    return new TezTaskCommunicatorImpl(context);
+  }
+
   public TaskHeartbeatResponse heartbeat(TaskHeartbeatRequest request)
       throws IOException, TezException {
     ContainerId containerId = ConverterUtils.toContainerId(request

http://git-wip-us.apache.org/repos/asf/tez/blob/c4fa9f50/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 15b2a04..be7016f 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
@@ -43,11 +43,9 @@ import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.tez.common.ContainerContext;
 import org.apache.tez.common.ContainerTask;
 import org.apache.tez.common.security.JobTokenSecretManager;
+import org.apache.tez.dag.api.TaskHeartbeatRequest;
 import org.apache.tez.dag.api.TezException;
 import org.apache.hadoop.yarn.api.records.NodeId;
-import org.apache.hadoop.yarn.event.EventHandler;
-import org.apache.tez.common.ContainerContext;
-import org.apache.tez.common.ContainerTask;
 import org.apache.tez.common.TezTaskUmbilicalProtocol;
 import org.apache.tez.dag.api.TaskCommunicatorContext;
 import org.apache.tez.dag.app.dag.DAG;
@@ -79,7 +77,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;
@@ -95,11 +95,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>();
@@ -111,6 +113,8 @@ public class TestTaskAttemptListenerImplTezDag {
     doReturn(dag).when(appContext).getCurrentDAG();
     doReturn(appAcls).when(appContext).getApplicationACLs();
     doReturn(amContainerMap).when(appContext).getAllContainers();
+    doReturn(appAttemptId).when(appContext).getApplicationAttemptId();
+    doReturn(credentials).when(appContext).getAppCredentials();
     NodeId nodeId = NodeId.newInstance("localhost", 0);
 
     AMContainer amContainer = mock(AMContainer.class);
@@ -152,7 +156,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);
 
@@ -182,7 +186,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);
@@ -259,7 +263,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);
@@ -267,13 +270,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();
@@ -287,6 +290,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(


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

Branch: refs/heads/TEZ-2003
Commit: 1ed50e1e5655c7b849667bd594ab3751861cedab
Parents: c4b800d
Author: Siddharth Seth <ss...@apache.org>
Authored: Mon May 11 23:34:43 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Mon Jun 1 14:24:30 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/1ed50e1e/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/1ed50e1e/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/1ed50e1e/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/1ed50e1e/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/1ed50e1e/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 7fbc0f7..f3f86a9 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
@@ -247,27 +247,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 hearbeats. Shutting down");
             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/1ed50e1e/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/1ed50e1e/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


[14/50] [abbrv] tez git commit: TEZ-2503 addendum. Move findbugs_version determination to after running findbugs. (sseth)

Posted by ss...@apache.org.
TEZ-2503 addendum. Move findbugs_version determination to after running
findbugs. (sseth)


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

Branch: refs/heads/TEZ-2003
Commit: 87f26a2ed10a097d20f69c5ab0b6e33a0877125e
Parents: 7bc3505
Author: Siddharth Seth <ss...@apache.org>
Authored: Fri May 29 14:05:16 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Fri May 29 14:05:16 2015 -0700

----------------------------------------------------------------------
 build-tools/test-patch.sh | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/87f26a2e/build-tools/test-patch.sh
----------------------------------------------------------------------
diff --git a/build-tools/test-patch.sh b/build-tools/test-patch.sh
index d41c54b..d1d5a24 100755
--- a/build-tools/test-patch.sh
+++ b/build-tools/test-patch.sh
@@ -525,7 +525,6 @@ buildAndInstall () {
 ###############################################################################
 ### Check there are no changes in the number of Findbugs warnings
 checkFindbugsWarnings () {
-  findbugs_version=$(${AWK} 'match($0, /findbugs-maven-plugin:[^:]*:findbugs/) { print substr($0, RSTART + 22, RLENGTH - 31); exit }' "${PATCH_DIR}/patchFindBugsOutput.txt")
   echo ""
   echo ""
   echo "======================================================================"
@@ -541,6 +540,7 @@ checkFindbugsWarnings () {
   echo "$MVN clean test findbugs:findbugs -DskipTests < /dev/null > $PATCH_DIR/patchFindBugsOutput.txt 2>&1"
   $MVN clean test findbugs:findbugs -DskipTests < /dev/null > $PATCH_DIR/patchFindBugsOutput.txt 2>&1
   rc=$?
+  findbugs_version=$(${AWK} 'match($0, /findbugs-maven-plugin:[^:]*:findbugs/) { print substr($0, RSTART + 22, RLENGTH - 31); exit }' "${PATCH_DIR}/patchFindBugsOutput.txt")
 
   if [ $rc != 0 ] ; then
     JIRA_COMMENT="$JIRA_COMMENT


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

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

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

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

http://git-wip-us.apache.org/repos/asf/tez/blob/fac7b1d7/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/fac7b1d7/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/fac7b1d7/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/fac7b1d7/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));
   }
 }


[10/50] [abbrv] tez git commit: TEZ-2198. Fix sorter spill counts (rbalamohan)

Posted by ss...@apache.org.
TEZ-2198. Fix sorter spill counts (rbalamohan)


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

Branch: refs/heads/TEZ-2003
Commit: 42b7756ebea108f81dbf633c847e2f70cd099e0f
Parents: b905665
Author: Rajesh Balamohan <rb...@apache.org>
Authored: Fri May 29 05:16:46 2015 +0530
Committer: Rajesh Balamohan <rb...@apache.org>
Committed: Fri May 29 05:16:46 2015 +0530

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../apache/tez/common/counters/TaskCounter.java | 11 ++-
 .../common/sort/impl/ExternalSorter.java        | 21 +++++-
 .../common/sort/impl/PipelinedSorter.java       | 54 +++++++++-----
 .../common/sort/impl/dflt/DefaultSorter.java    | 76 ++++++++++----------
 .../common/sort/impl/TestPipelinedSorter.java   | 58 ++++++++++++++-
 .../sort/impl/dflt/TestDefaultSorter.java       | 43 +++++++++--
 .../apache/tez/test/TestPipelinedShuffle.java   |  2 +-
 8 files changed, 197 insertions(+), 69 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/42b7756e/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index ece14a3..7f240c1 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -8,6 +8,7 @@ INCOMPATIBLE CHANGES
   TEZ-2468. Change the minimum Java version to Java 7.
 
 ALL CHANGES:
+  TEZ-2198. Fix sorter spill counts.
   TEZ-1883. Change findbugs version to 3.x.
   TEZ-2440. Sorter should check for indexCacheList.size() in flush().
   TEZ-2490. TEZ-2450 breaks Hadoop 2.2 and 2.4 compatability.

http://git-wip-us.apache.org/repos/asf/tez/blob/42b7756e/tez-api/src/main/java/org/apache/tez/common/counters/TaskCounter.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/common/counters/TaskCounter.java b/tez-api/src/main/java/org/apache/tez/common/counters/TaskCounter.java
index 128b067..7dcdf8a 100644
--- a/tez-api/src/main/java/org/apache/tez/common/counters/TaskCounter.java
+++ b/tez-api/src/main/java/org/apache/tez/common/counters/TaskCounter.java
@@ -131,10 +131,19 @@ public enum TaskCounter {
   ADDITIONAL_SPILLS_BYTES_READ,
   
   /**
-   * Actual number of unnecessary spills. (lac of adequate memory)
+   * Spills that were generated & read by the same task (unnecessary spills due to lac of
+   * adequate memory).
+   *
    * Used by OnFileSortedOutput
    */
   ADDITIONAL_SPILL_COUNT,
+
+  /**
+   * Number of spill files being offered via shuffle-handler.
+   * e.g Without pipelined shuffle, this would be 1. With pipelined shuffle, this could be many
+   * as final merge is avoided.
+   */
+  SHUFFLE_CHUNK_COUNT,
   
   INPUT_GROUPS, // Not used at the moment. Will eventually replace REDUCE_INPUT_GROUPS
 

http://git-wip-us.apache.org/repos/asf/tez/blob/42b7756e/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/ExternalSorter.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/ExternalSorter.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/ExternalSorter.java
index 40d22fe..27fe37a 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/ExternalSorter.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/ExternalSorter.java
@@ -24,6 +24,7 @@ import java.io.InputStream;
 import java.util.Iterator;
 import java.util.Map;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.Maps;
 import org.apache.tez.runtime.library.api.IOInterruptedException;
 import org.slf4j.Logger;
@@ -118,6 +119,9 @@ public abstract class ExternalSorter {
 
   protected final boolean cleanup;
 
+  protected final boolean finalMergeEnabled;
+  protected final boolean sendEmptyPartitionDetails;
+
   // Counters
   // MR compatilbity layer needs to rename counters back to what MR requries.
 
@@ -141,9 +145,10 @@ public abstract class ExternalSorter {
   protected final TezCounter additionalSpillBytesWritten;
   
   protected final TezCounter additionalSpillBytesRead;
-  // Number of additional spills. (This will be 0 if there's no additional
-  // spills)
+  // Number of spills written & consumed by the same task to generate the final file
   protected final TezCounter numAdditionalSpills;
+  // Number of files offered via shuffle-handler to consumers.
+  protected final TezCounter numShuffleChunks;
 
   public ExternalSorter(OutputContext outputContext, Configuration conf, int numOutputs,
       long initialMemoryAvailable) throws IOException {
@@ -187,6 +192,7 @@ public abstract class ExternalSorter {
     additionalSpillBytesWritten = outputContext.getCounters().findCounter(TaskCounter.ADDITIONAL_SPILLS_BYTES_WRITTEN);
     additionalSpillBytesRead = outputContext.getCounters().findCounter(TaskCounter.ADDITIONAL_SPILLS_BYTES_READ);
     numAdditionalSpills = outputContext.getCounters().findCounter(TaskCounter.ADDITIONAL_SPILL_COUNT);
+    numShuffleChunks = outputContext.getCounters().findCounter(TaskCounter.SHUFFLE_CHUNK_COUNT);
 
     // compression
     if (ConfigUtils.shouldCompressIntermediateOutput(this.conf)) {
@@ -235,6 +241,17 @@ public abstract class ExternalSorter {
     this.conf.setInt(TezRuntimeFrameworkConfigs.TEZ_RUNTIME_NUM_EXPECTED_PARTITIONS, this.partitions);
     this.partitioner = TezRuntimeUtils.instantiatePartitioner(this.conf);
     this.combiner = TezRuntimeUtils.instantiateCombiner(this.conf, outputContext);
+    this.finalMergeEnabled = conf.getBoolean(
+        TezRuntimeConfiguration.TEZ_RUNTIME_ENABLE_FINAL_MERGE_IN_OUTPUT,
+        TezRuntimeConfiguration.TEZ_RUNTIME_ENABLE_FINAL_MERGE_IN_OUTPUT_DEFAULT);
+    this.sendEmptyPartitionDetails = conf.getBoolean(
+        TezRuntimeConfiguration.TEZ_RUNTIME_EMPTY_PARTITION_INFO_VIA_EVENTS_ENABLED,
+        TezRuntimeConfiguration.TEZ_RUNTIME_EMPTY_PARTITION_INFO_VIA_EVENTS_ENABLED_DEFAULT);
+  }
+
+  @VisibleForTesting
+  public boolean isFinalMergeEnabled() {
+    return finalMergeEnabled;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/tez/blob/42b7756e/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/PipelinedSorter.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/PipelinedSorter.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/PipelinedSorter.java
index 9113fca..6e4d72e 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/PipelinedSorter.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/PipelinedSorter.java
@@ -111,8 +111,6 @@ public class PipelinedSorter extends ExternalSorter {
   private int indexCacheMemoryLimit;
 
   private final boolean pipelinedShuffle;
-  private final boolean finalMergeEnabled;
-  private final boolean sendEmptyPartitionDetails;
 
   // TODO Set additional countesr - total bytes written, spills etc.
 
@@ -127,20 +125,11 @@ public class PipelinedSorter extends ExternalSorter {
     
     partitionBits = bitcount(partitions)+1;
 
-    finalMergeEnabled = conf.getBoolean(
-        TezRuntimeConfiguration.TEZ_RUNTIME_ENABLE_FINAL_MERGE_IN_OUTPUT,
-        TezRuntimeConfiguration.TEZ_RUNTIME_ENABLE_FINAL_MERGE_IN_OUTPUT_DEFAULT);
-
     boolean confPipelinedShuffle = this.conf.getBoolean(TezRuntimeConfiguration
         .TEZ_RUNTIME_PIPELINED_SHUFFLE_ENABLED, TezRuntimeConfiguration
         .TEZ_RUNTIME_PIPELINED_SHUFFLE_ENABLED_DEFAULT);
 
-    sendEmptyPartitionDetails = conf.getBoolean(
-        TezRuntimeConfiguration.TEZ_RUNTIME_EMPTY_PARTITION_INFO_VIA_EVENTS_ENABLED,
-        TezRuntimeConfiguration.TEZ_RUNTIME_EMPTY_PARTITION_INFO_VIA_EVENTS_ENABLED_DEFAULT);
-
-
-    pipelinedShuffle = !finalMergeEnabled && confPipelinedShuffle;
+    pipelinedShuffle = !isFinalMergeEnabled() && confPipelinedShuffle;
 
     //sanity checks
     final long sortmb = this.availableMemoryMb;
@@ -157,7 +146,7 @@ public class PipelinedSorter extends ExternalSorter {
     int numberOfBlocks = Math.max(1, (int) Math.ceil(1.0 * usage / blockSize));
     LOG.info("Number of Blocks : " + numberOfBlocks
         + ", maxMemUsage=" + maxMemUsage + ", BLOCK_SIZE=" + blockSize + ", finalMergeEnabled="
-        + finalMergeEnabled + ", pipelinedShuffle=" + pipelinedShuffle + ", "
+        + isFinalMergeEnabled() + ", pipelinedShuffle=" + pipelinedShuffle + ", "
         + "sendEmptyPartitionDetails=" + sendEmptyPartitionDetails);
     long totalCapacityWithoutMeta = 0;
     for (int i = 0; i < numberOfBlocks; i++) {
@@ -235,7 +224,7 @@ public class PipelinedSorter extends ExternalSorter {
       if (pipelinedShuffle) {
         List<Event> events = Lists.newLinkedList();
         String pathComponent = (outputContext.getUniqueIdentifier() + "_" + (numSpills-1));
-        ShuffleUtils.generateEventOnSpill(events, finalMergeEnabled, false, outputContext,
+        ShuffleUtils.generateEventOnSpill(events, isFinalMergeEnabled(), false, outputContext,
             (numSpills - 1), indexCacheList.get(numSpills - 1), partitions, sendEmptyPartitionDetails,
             pathComponent);
         outputContext.sendEvents(events);
@@ -343,6 +332,21 @@ public class PipelinedSorter extends ExternalSorter {
     mapOutputByteCounter.increment(valend - keystart);
   }
 
+  private void adjustSpillCounters(long rawLength, long compLength) {
+    if (!isFinalMergeEnabled()) {
+      outputBytesWithOverheadCounter.increment(rawLength);
+    } else {
+      if (numSpills > 0) {
+        additionalSpillBytesWritten.increment(compLength);
+        // Reset the value will be set during the final merge.
+        outputBytesWithOverheadCounter.setValue(0);
+      } else {
+        // Set this up for the first write only. Subsequent ones will be handled in the final merge.
+        outputBytesWithOverheadCounter.increment(rawLength);
+      }
+    }
+  }
+
   public void spill() throws IOException {
     // create spill file
     final long size = capacity +
@@ -381,6 +385,7 @@ public class PipelinedSorter extends ExternalSorter {
         }
         //close
         writer.close();
+        adjustSpillCounters(writer.getRawLength(), writer.getCompressedLength());
 
         // record offsets
         final TezIndexRecord rec = 
@@ -399,6 +404,11 @@ public class PipelinedSorter extends ExternalSorter {
       //TODO: honor cache limits
       indexCacheList.add(spillRec);
       ++numSpills;
+      if (!isFinalMergeEnabled()) {
+        fileOutputByteCounter.increment(rfs.getFileStatus(filename).getLen());
+        //No final merge. Set the number of files offered via shuffle-handler
+        numShuffleChunks.setValue(numSpills);
+      }
     } finally {
       out.close();
     }
@@ -440,14 +450,13 @@ public class PipelinedSorter extends ExternalSorter {
       //safe to clean up
       bufferList.clear();
 
-      numAdditionalSpills.increment(numSpills - 1);
 
       if(indexCacheList.isEmpty()) {
         LOG.warn("Index list is empty... returning");
         return;
       }
 
-      if (!finalMergeEnabled) {
+      if (!isFinalMergeEnabled()) {
         //Generate events for all spills
         List<Event> events = Lists.newLinkedList();
 
@@ -459,16 +468,17 @@ public class PipelinedSorter extends ExternalSorter {
           boolean isLastEvent = (i == numSpills - 1);
 
           String pathComponent = (outputContext.getUniqueIdentifier() + "_" + i);
-          ShuffleUtils.generateEventOnSpill(events, finalMergeEnabled, isLastEvent,
+          ShuffleUtils.generateEventOnSpill(events, isFinalMergeEnabled(), isLastEvent,
               outputContext, i, indexCacheList.get(i), partitions,
               sendEmptyPartitionDetails, pathComponent);
           LOG.info("Adding spill event for spill (final update=" + isLastEvent + "), spillId=" + i);
         }
         outputContext.sendEvents(events);
-        //No need to generate final merge
         return;
       }
 
+      numAdditionalSpills.increment(numSpills - 1);
+
       //In case final merge is required, the following code path is executed.
       if (numSpills == 1) {
         // someday be able to pass this directly to shuffle
@@ -485,6 +495,8 @@ public class PipelinedSorter extends ExternalSorter {
               + "finalIndexFile=" + finalIndexFile + ", filename=" + filename + ", indexFilename=" +
               indexFilename);
         }
+        numShuffleChunks.setValue(numSpills);
+        fileOutputByteCounter.increment(rfs.getFileStatus(finalOutputFile).getLen());
         return;
       }
 
@@ -531,7 +543,7 @@ public class PipelinedSorter extends ExternalSorter {
             new Path(uniqueIdentifier),
             (RawComparator) ConfigUtils.getIntermediateOutputKeyComparator(conf),
             nullProgressable, sortSegments, true,
-            null, spilledRecordsCounter, null,
+            null, spilledRecordsCounter, additionalSpillBytesRead,
             null); // Not using any Progress in TezMerger. Should just work.
 
         //write merged output to disk
@@ -548,6 +560,7 @@ public class PipelinedSorter extends ExternalSorter {
 
         //close
         writer.close();
+        outputBytesWithOverheadCounter.increment(writer.getRawLength());
 
         // record offsets
         final TezIndexRecord rec =
@@ -558,6 +571,9 @@ public class PipelinedSorter extends ExternalSorter {
         spillRec.putIndex(rec, parts);
       }
 
+      numShuffleChunks.setValue(1); //final merge has happened.
+      fileOutputByteCounter.increment(rfs.getFileStatus(finalOutputFile).getLen());
+
       spillRec.writeToFile(finalIndexFile, conf);
       finalOut.close();
       for (int i = 0; i < numSpills; i++) {

http://git-wip-us.apache.org/repos/asf/tez/blob/42b7756e/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/dflt/DefaultSorter.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/dflt/DefaultSorter.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/dflt/DefaultSorter.java
index afe07f0..ebf40f3 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/dflt/DefaultSorter.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/dflt/DefaultSorter.java
@@ -120,8 +120,6 @@ public class DefaultSorter extends ExternalSorter implements IndexedSortable {
   private long totalKeys = 0;
   private long sameKey = 0;
 
-  private final boolean finalMergeEnabled;
-  private final boolean sendEmptyPartitionDetails;
 
   public DefaultSorter(OutputContext outputContext, Configuration conf, int numOutputs,
       long initialMemoryAvailable) throws IOException {
@@ -139,10 +137,6 @@ public class DefaultSorter extends ExternalSorter implements IndexedSortable {
     indexCacheMemoryLimit = this.conf.getInt(TezRuntimeConfiguration.TEZ_RUNTIME_INDEX_CACHE_MEMORY_LIMIT_BYTES,
                                        TezRuntimeConfiguration.TEZ_RUNTIME_INDEX_CACHE_MEMORY_LIMIT_BYTES_DEFAULT);
 
-    finalMergeEnabled = conf.getBoolean(
-        TezRuntimeConfiguration.TEZ_RUNTIME_ENABLE_FINAL_MERGE_IN_OUTPUT,
-        TezRuntimeConfiguration.TEZ_RUNTIME_ENABLE_FINAL_MERGE_IN_OUTPUT_DEFAULT);
-
     boolean confPipelinedShuffle = this.conf.getBoolean(TezRuntimeConfiguration
         .TEZ_RUNTIME_PIPELINED_SHUFFLE_ENABLED, TezRuntimeConfiguration
         .TEZ_RUNTIME_PIPELINED_SHUFFLE_ENABLED_DEFAULT);
@@ -152,10 +146,6 @@ public class DefaultSorter extends ExternalSorter implements IndexedSortable {
           + "with DefaultSorter. It is supported only with PipelinedSorter.");
     }
 
-    sendEmptyPartitionDetails = conf.getBoolean(
-        TezRuntimeConfiguration.TEZ_RUNTIME_EMPTY_PARTITION_INFO_VIA_EVENTS_ENABLED,
-        TezRuntimeConfiguration.TEZ_RUNTIME_EMPTY_PARTITION_INFO_VIA_EVENTS_ENABLED_DEFAULT);
-
     // buffers and accounting
     int maxMemUsage = sortmb << 20;
     maxMemUsage -= maxMemUsage % METASIZE;
@@ -175,7 +165,7 @@ public class DefaultSorter extends ExternalSorter implements IndexedSortable {
       LOG.info(TezRuntimeConfiguration.TEZ_RUNTIME_IO_SORT_MB + ": " + sortmb);
       LOG.info("soft limit at " + softLimit);
       LOG.info("bufstart = " + bufstart + "; bufvoid = " + bufvoid);
-      LOG.info("kvstart = " + kvstart + "; length = " + maxRec + "; finalMergeEnabled = " + finalMergeEnabled);
+      LOG.info("kvstart = " + kvstart + "; length = " + maxRec + "; finalMergeEnabled = " + isFinalMergeEnabled());
     }
 
     // k/v serialization
@@ -716,7 +706,7 @@ public class DefaultSorter extends ExternalSorter implements IndexedSortable {
       cleanup();
       Thread.currentThread().interrupt();
     }
-    if (finalMergeEnabled) {
+    if (isFinalMergeEnabled()) {
       fileOutputByteCounter.increment(rfs.getFileStatus(finalOutputFile).getLen());
     }
   }
@@ -818,6 +808,22 @@ public class DefaultSorter extends ExternalSorter implements IndexedSortable {
     spill(mstart, mend);
   }
 
+  private void adjustSpillCounters(long rawLen, long compLength) {
+    if (!isFinalMergeEnabled()) {
+      outputBytesWithOverheadCounter.increment(rawLen);
+    } else {
+      if (numSpills > 0) {
+        additionalSpillBytesWritten.increment(compLength);
+        numAdditionalSpills.increment(1);
+        // Reset the value will be set during the final merge.
+        outputBytesWithOverheadCounter.setValue(0);
+      } else {
+        // Set this up for the first write only. Subsequent ones will be handled in the final merge.
+        outputBytesWithOverheadCounter.increment(rawLen);
+      }
+    }
+  }
+
   protected void spill(int mstart, int mend)
       throws IOException, InterruptedException {
 
@@ -879,15 +885,7 @@ public class DefaultSorter extends ExternalSorter implements IndexedSortable {
 
           // close the writer
           writer.close();
-          if (numSpills > 0) {
-            additionalSpillBytesWritten.increment(writer.getCompressedLength());
-            numAdditionalSpills.increment(1);
-            // Reset the value will be set during the final merge.
-            outputBytesWithOverheadCounter.setValue(0);
-          } else {
-            // Set this up for the first write only. Subsequent ones will be handled in the final merge.
-            outputBytesWithOverheadCounter.increment(writer.getRawLength());
-          }
+          adjustSpillCounters(writer.getRawLength(), writer.getCompressedLength());
           // record offsets
           final TezIndexRecord rec =
               new TezIndexRecord(
@@ -916,6 +914,9 @@ public class DefaultSorter extends ExternalSorter implements IndexedSortable {
       }
       LOG.info("Finished spill " + numSpills);
       ++numSpills;
+      if (!isFinalMergeEnabled()) {
+        numShuffleChunks.setValue(numSpills);
+      }
     } finally {
       if (out != null) out.close();
     }
@@ -956,14 +957,7 @@ public class DefaultSorter extends ExternalSorter implements IndexedSortable {
           }
           writer.close();
 
-          if (numSpills > 0) {
-            additionalSpillBytesWritten.increment(writer.getCompressedLength());
-            numAdditionalSpills.increment(1);
-            outputBytesWithOverheadCounter.setValue(0);
-          } else {
-            // Set this up for the first write only. Subsequent ones will be handled in the final merge.
-            outputBytesWithOverheadCounter.increment(writer.getRawLength());
-          }
+          adjustSpillCounters(writer.getRawLength(), writer.getCompressedLength());
 
           // record offsets
           TezIndexRecord rec =
@@ -992,6 +986,9 @@ public class DefaultSorter extends ExternalSorter implements IndexedSortable {
           spillRec.size() * MAP_OUTPUT_INDEX_RECORD_LENGTH;
       }
       ++numSpills;
+      if (!isFinalMergeEnabled()) {
+        numShuffleChunks.setValue(numSpills);
+      }
     } finally {
       if (out != null) out.close();
     }
@@ -1085,13 +1082,13 @@ public class DefaultSorter extends ExternalSorter implements IndexedSortable {
 
   private void maybeSendEventForSpill(List<Event> events, boolean isLastEvent,
       TezSpillRecord spillRecord, int index, boolean sendEvent) throws IOException {
-    if (finalMergeEnabled) {
+    if (isFinalMergeEnabled()) {
       return;
     }
     Preconditions.checkArgument(spillRecord != null, "Spill record can not be null");
 
     String pathComponent = (outputContext.getUniqueIdentifier() + "_" + index);
-    ShuffleUtils.generateEventOnSpill(events, finalMergeEnabled, isLastEvent,
+    ShuffleUtils.generateEventOnSpill(events, isFinalMergeEnabled(), isLastEvent,
         outputContext, index, spillRecord, partitions, sendEmptyPartitionDetails, pathComponent);
 
     LOG.info("Adding spill event for spill (final update=" + isLastEvent + "), spillId=" + index);
@@ -1102,7 +1099,7 @@ public class DefaultSorter extends ExternalSorter implements IndexedSortable {
   }
 
   private void maybeAddEventsForSpills() throws IOException {
-    if (finalMergeEnabled) {
+    if (isFinalMergeEnabled()) {
       return;
     }
     List<Event> events = Lists.newLinkedList();
@@ -1124,7 +1121,6 @@ public class DefaultSorter extends ExternalSorter implements IndexedSortable {
       maybeSendEventForSpill(events, (i == numSpills - 1), spillRecord, i, false);
       fileOutputByteCounter.increment(rfs.getFileStatus(spillFilePaths.get(i)).getLen());
     }
-
     outputContext.sendEvents(events);
   }
 
@@ -1140,7 +1136,7 @@ public class DefaultSorter extends ExternalSorter implements IndexedSortable {
       finalOutFileSize += rfs.getFileStatus(filename[i]).getLen();
     }
     if (numSpills == 1) { //the spill is the final output
-      if (finalMergeEnabled) {
+      if (isFinalMergeEnabled()) {
         finalOutputFile = mapOutputFile.getOutputFileForWriteInVolume(filename[0]);
         finalIndexFile = mapOutputFile.getOutputIndexFileForWriteInVolume(filename[0]);
         sameVolRename(filename[0], finalOutputFile);
@@ -1160,6 +1156,7 @@ public class DefaultSorter extends ExternalSorter implements IndexedSortable {
         fileOutputByteCounter.increment(rfs.getFileStatus(spillFilePaths.get(0)).getLen());
         //No need to populate finalIndexFile, finalOutputFile etc when finalMerge is disabled
       }
+      numShuffleChunks.setValue(numSpills);
       return;
     }
 
@@ -1170,7 +1167,7 @@ public class DefaultSorter extends ExternalSorter implements IndexedSortable {
     }
 
     //Check if it is needed to do final merge. Or else, exit early.
-    if (numSpills > 0 && !finalMergeEnabled) {
+    if (numSpills > 0 && !isFinalMergeEnabled()) {
       maybeAddEventsForSpills();
       //No need to do final merge.
       return;
@@ -1181,7 +1178,7 @@ public class DefaultSorter extends ExternalSorter implements IndexedSortable {
     finalOutFileSize += partitions * APPROX_HEADER_LENGTH;
     finalIndexFileSize = partitions * MAP_OUTPUT_INDEX_RECORD_LENGTH;
 
-    if (finalMergeEnabled) {
+    if (isFinalMergeEnabled()) {
       finalOutputFile = mapOutputFile.getOutputFileForWrite(finalOutFileSize);
       finalIndexFile = mapOutputFile.getOutputIndexFileForWrite(finalIndexFileSize);
     } else if (numSpills == 0) {
@@ -1219,12 +1216,13 @@ public class DefaultSorter extends ExternalSorter implements IndexedSortable {
       } finally {
         finalOut.close();
       }
-
-      if (!finalMergeEnabled) {
+      ++numSpills;
+      if (!isFinalMergeEnabled()) {
         List<Event> events = Lists.newLinkedList();
         maybeSendEventForSpill(events, true, sr, 0, true);
         fileOutputByteCounter.increment(rfs.getFileStatus(finalOutputFile).getLen());
       }
+      numShuffleChunks.setValue(numSpills);
       return;
     }
     else {
@@ -1277,6 +1275,7 @@ public class DefaultSorter extends ExternalSorter implements IndexedSortable {
           runCombineProcessor(kvIter, writer);
         }
         writer.close();
+        outputBytesWithOverheadCounter.increment(writer.getRawLength());
 
         // record offsets
         final TezIndexRecord rec =
@@ -1286,6 +1285,7 @@ public class DefaultSorter extends ExternalSorter implements IndexedSortable {
                 writer.getCompressedLength());
         spillRec.putIndex(rec, parts);
       }
+      numShuffleChunks.setValue(1); //final merge has happened
       spillRec.writeToFile(finalIndexFile, conf);
       finalOut.close();
       for(int i = 0; i < numSpills; i++) {

http://git-wip-us.apache.org/repos/asf/tez/blob/42b7756e/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestPipelinedSorter.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestPipelinedSorter.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestPipelinedSorter.java
index 5de96c9..8bf91ce 100644
--- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestPipelinedSorter.java
+++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestPipelinedSorter.java
@@ -13,6 +13,8 @@ import org.apache.hadoop.io.serializer.Deserializer;
 import org.apache.hadoop.io.serializer.SerializationFactory;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.tez.common.TezRuntimeFrameworkConfigs;
+import org.apache.tez.common.counters.TaskCounter;
+import org.apache.tez.common.counters.TezCounter;
 import org.apache.tez.common.counters.TezCounters;
 import org.apache.tez.runtime.api.Event;
 import org.apache.tez.runtime.api.ExecutionContext;
@@ -60,7 +62,7 @@ import static org.mockito.internal.verification.VerificationModeFactory.times;
  * limitations under the License.
  */
 public class TestPipelinedSorter {
-  private static final Configuration conf = new Configuration();
+  private static Configuration conf = new Configuration();
   private static FileSystem localFs = null;
   private static Path workDir = null;
   private OutputContext outputContext;
@@ -114,6 +116,7 @@ public class TestPipelinedSorter {
   public void reset() throws IOException {
     cleanup();
     localFs.mkdirs(workDir);
+    conf = new Configuration();
   }
 
   @Test
@@ -124,6 +127,17 @@ public class TestPipelinedSorter {
     conf.setInt(TezRuntimeConfiguration.TEZ_RUNTIME_IO_SORT_MB, 5);
     basicTest(1, 100000, 100, (10 * 1024l * 1024l), 3 << 20);
 
+  }
+
+  @Test
+  public void testWithEmptyData() throws IOException {
+    conf.setInt(TezRuntimeConfiguration.TEZ_RUNTIME_IO_SORT_MB, 5);
+    //# partition, # of keys, size per key, InitialMem, blockSize
+    basicTest(1, 0, 0, (10 * 1024l * 1024l), 3 << 20);
+  }
+
+  @Test
+  public void basicTestWithSmallBlockSize() throws IOException {
     try {
       //3 MB key & 3 MB value, whereas block size is just 3 MB
       basicTest(1, 5, (3 << 20), (10 * 1024l * 1024l), 3 << 20);
@@ -133,11 +147,13 @@ public class TestPipelinedSorter {
           ioe.getMessage().contains("Record too large for in-memory buffer."
               + " Exceeded buffer overflow limit"));
     }
+  }
 
+  @Test
+  public void testWithLargeKeyValue() throws IOException {
     //15 MB key & 15 MB value, 48 MB sort buffer.  block size is 48MB (or 1 block)
     //meta would be 16 MB
     basicTest(1, 5, (15 << 20), (48 * 1024l * 1024l), 48 << 20);
-
   }
 
   @Test
@@ -154,7 +170,7 @@ public class TestPipelinedSorter {
     conf.setInt(TezRuntimeConfiguration.TEZ_RUNTIME_IO_SORT_MB, 5);
     conf.setBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_ENABLE_FINAL_MERGE_IN_OUTPUT, false);
     PipelinedSorter sorter = new PipelinedSorter(this.outputContext, conf, numOutputs,
-        initialAvailableMem, 1<<20);
+        initialAvailableMem, 1 << 20);
 
     //Write 100 keys each of size 10
     writeData(sorter, 10000, 100);
@@ -172,6 +188,7 @@ public class TestPipelinedSorter {
 
     writeData(sorter, numKeys, keySize);
 
+    verifyCounters(sorter, outputContext);
     Path outputFile = sorter.finalOutputFile;
     FileSystem fs = outputFile.getFileSystem(conf);
 
@@ -181,6 +198,41 @@ public class TestPipelinedSorter {
     reader.close();
   }
 
+  private void verifyCounters(PipelinedSorter sorter, OutputContext context) {
+    TezCounter numShuffleChunks = context.getCounters().findCounter(TaskCounter.SHUFFLE_CHUNK_COUNT);
+    TezCounter additionalSpills =
+        context.getCounters().findCounter(TaskCounter.ADDITIONAL_SPILL_COUNT);
+    TezCounter additionalSpillBytesWritten =
+        context.getCounters().findCounter(TaskCounter.ADDITIONAL_SPILLS_BYTES_WRITTEN);
+    TezCounter additionalSpillBytesRead =
+        context.getCounters().findCounter(TaskCounter.ADDITIONAL_SPILLS_BYTES_READ);
+
+    if (sorter.isFinalMergeEnabled()) {
+      assertTrue(additionalSpills.getValue() == (sorter.getNumSpills() - 1));
+      //Number of files served by shuffle-handler
+      assertTrue(1 == numShuffleChunks.getValue());
+      if (sorter.getNumSpills() > 1) {
+        assertTrue(additionalSpillBytesRead.getValue() > 0);
+        assertTrue(additionalSpillBytesWritten.getValue() > 0);
+      }
+    } else {
+      assertTrue(0 == additionalSpills.getValue());
+      //Number of files served by shuffle-handler
+      assertTrue(sorter.getNumSpills() == numShuffleChunks.getValue());
+      assertTrue(additionalSpillBytesRead.getValue() == 0);
+      assertTrue(additionalSpillBytesWritten.getValue() == 0);
+    }
+
+    TezCounter finalOutputBytes =
+        context.getCounters().findCounter(TaskCounter.OUTPUT_BYTES_PHYSICAL);
+    assertTrue(finalOutputBytes.getValue() > 0);
+
+    TezCounter outputBytesWithOverheadCounter = context.getCounters().findCounter
+        (TaskCounter.OUTPUT_BYTES_WITH_OVERHEAD);
+    assertTrue(outputBytesWithOverheadCounter.getValue() > 0);
+  }
+
+
   @Test
   //Its not possible to allocate > 2 GB in test environment.  Carry out basic checks here.
   public void memTest() throws IOException {

http://git-wip-us.apache.org/repos/asf/tez/blob/42b7756e/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/dflt/TestDefaultSorter.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/dflt/TestDefaultSorter.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/dflt/TestDefaultSorter.java
index 70dce13..072eafc 100644
--- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/dflt/TestDefaultSorter.java
+++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/dflt/TestDefaultSorter.java
@@ -42,6 +42,7 @@ import org.apache.hadoop.io.Text;
 import org.apache.tez.common.TezRuntimeFrameworkConfigs;
 import org.apache.tez.common.TezUtils;
 import org.apache.tez.common.counters.TaskCounter;
+import org.apache.tez.common.counters.TezCounter;
 import org.apache.tez.common.counters.TezCounters;
 import org.apache.tez.dag.api.UserPayload;
 import org.apache.tez.runtime.api.Event;
@@ -171,6 +172,7 @@ public class TestDefaultSorter {
     try {
       writeData(sorter, 1000, 1000);
       assertTrue(sorter.getNumSpills() > 2);
+      verifyCounters(sorter, context);
     } catch(IOException ioe) {
       fail(ioe.getMessage());
     }
@@ -191,6 +193,7 @@ public class TestDefaultSorter {
       sorter.flush();
       sorter.close();
       assertTrue(sorter.getFinalOutputFile().getParent().getName().equalsIgnoreCase(UniqueID));
+      verifyCounters(sorter, context);
     } catch(Exception e) {
       fail();
     }
@@ -213,13 +216,13 @@ public class TestDefaultSorter {
       sorter.close();
       assertTrue(sorter.getFinalOutputFile().getParent().getName().equalsIgnoreCase(UniqueID +
           "_0"));
-      assertTrue(context.getCounters().findCounter(TaskCounter.OUTPUT_BYTES_PHYSICAL).getValue() > 0);
+      verifyCounters(sorter, context);
     } catch(Exception e) {
       fail();
     }
   }
 
-  @Test(timeout = 30000)
+  @Test(timeout = 60000)
   @SuppressWarnings("unchecked")
   public void testWithSingleSpillWithFinalMergeDisabled() throws IOException {
     OutputContext context = createTezOutputContext();
@@ -245,10 +248,10 @@ public class TestDefaultSorter {
       }
     }
 
-    assertTrue(context.getCounters().findCounter(TaskCounter.OUTPUT_BYTES_PHYSICAL).getValue() > 0);
+    verifyCounters(sorter, context);
   }
 
-  @Test(timeout = 30000)
+  @Test(timeout = 60000)
   @SuppressWarnings("unchecked")
   public void testWithMultipleSpillsWithFinalMergeDisabled() throws IOException {
     OutputContext context = createTezOutputContext();
@@ -277,7 +280,37 @@ public class TestDefaultSorter {
       }
     }
     assertTrue(spillIndex == spillCount);
-    assertTrue(context.getCounters().findCounter(TaskCounter.OUTPUT_BYTES_PHYSICAL).getValue() > 0);
+    verifyCounters(sorter, context);
+  }
+
+  private void verifyCounters(DefaultSorter sorter, OutputContext context) {
+    TezCounter numShuffleChunks = context.getCounters().findCounter(TaskCounter.SHUFFLE_CHUNK_COUNT);
+    TezCounter additionalSpills = context.getCounters().findCounter(TaskCounter.ADDITIONAL_SPILL_COUNT);
+    TezCounter additionalSpillBytesWritten = context.getCounters().findCounter(TaskCounter.ADDITIONAL_SPILLS_BYTES_WRITTEN);
+    TezCounter additionalSpillBytesRead = context.getCounters().findCounter(TaskCounter.ADDITIONAL_SPILLS_BYTES_READ);
+
+    if (sorter.isFinalMergeEnabled()) {
+      assertTrue(additionalSpills.getValue() == (sorter.getNumSpills() - 1));
+      //Number of files served by shuffle-handler
+      assertTrue(1 == numShuffleChunks.getValue());
+      if (sorter.getNumSpills() > 1) {
+        assertTrue(additionalSpillBytesRead.getValue() > 0);
+        assertTrue(additionalSpillBytesWritten.getValue() > 0);
+      }
+    } else {
+      assertTrue(0 == additionalSpills.getValue());
+      //Number of files served by shuffle-handler
+      assertTrue(sorter.getNumSpills() == numShuffleChunks.getValue());
+      assertTrue(additionalSpillBytesRead.getValue() == 0);
+      assertTrue(additionalSpillBytesWritten.getValue() == 0);
+    }
+
+    TezCounter finalOutputBytes = context.getCounters().findCounter(TaskCounter.OUTPUT_BYTES_PHYSICAL);
+    assertTrue(finalOutputBytes.getValue() > 0);
+
+    TezCounter outputBytesWithOverheadCounter = context.getCounters().findCounter
+        (TaskCounter.OUTPUT_BYTES_WITH_OVERHEAD);
+    assertTrue(outputBytesWithOverheadCounter.getValue() > 0);
   }
 
   private void writeData(ExternalSorter sorter, int numKeys, int keyLen) throws IOException {

http://git-wip-us.apache.org/repos/asf/tez/blob/42b7756e/tez-tests/src/test/java/org/apache/tez/test/TestPipelinedShuffle.java
----------------------------------------------------------------------
diff --git a/tez-tests/src/test/java/org/apache/tez/test/TestPipelinedShuffle.java b/tez-tests/src/test/java/org/apache/tez/test/TestPipelinedShuffle.java
index 25c149d..52342a2 100644
--- a/tez-tests/src/test/java/org/apache/tez/test/TestPipelinedShuffle.java
+++ b/tez-tests/src/test/java/org/apache/tez/test/TestPipelinedShuffle.java
@@ -256,7 +256,7 @@ public class TestPipelinedShuffle {
       TezCounters counters = dagStatus.getDAGCounters();
 
       //Ensure that atleast 10 spills were there in this job.
-      assertTrue(counters.findCounter(TaskCounter.ADDITIONAL_SPILL_COUNT).getValue() > 10);
+      assertTrue(counters.findCounter(TaskCounter.SHUFFLE_CHUNK_COUNT).getValue() > 10);
 
       if (dagStatus.getState() != DAGStatus.State.SUCCEEDED) {
         System.out.println("DAG diagnostics: " + dagStatus.getDiagnostics());


[09/50] [abbrv] tez git commit: TEZ-1883. Change findbugs version to 3.x. (sseth)

Posted by ss...@apache.org.
TEZ-1883. Change findbugs version to 3.x. (sseth)


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

Branch: refs/heads/TEZ-2003
Commit: b9056657a101b33c96195655079b1eb03ebb3c95
Parents: 17b6aba
Author: Siddharth Seth <ss...@apache.org>
Authored: Thu May 28 14:23:43 2015 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Thu May 28 14:23:43 2015 -0700

----------------------------------------------------------------------
 CHANGES.txt                                                   | 1 +
 build-tools/test-patch.sh                                     | 2 +-
 pom.xml                                                       | 4 ++--
 .../src/main/java/org/apache/tez/dag/records/TezDAGID.java    | 2 +-
 tez-dag/findbugs-exclude.xml                                  | 7 +++++++
 .../src/main/java/org/apache/tez/dag/app/DAGAppMaster.java    | 2 +-
 tez-dag/src/main/java/org/apache/tez/dag/utils/Graph.java     | 7 ++++---
 .../apache/tez/dag/utils/Simple2LevelVersionComparator.java   | 2 +-
 .../apache/hadoop/mapred/split/TezMapredSplitsGrouper.java    | 3 +--
 .../hadoop/mapreduce/split/TezMapReduceSplitsGrouper.java     | 2 +-
 .../java/org/apache/tez/mapreduce/hadoop/IDConverter.java     | 2 +-
 tez-runtime-library/findbugs-exclude.xml                      | 6 ++++++
 .../apache/tez/runtime/library/processor/SleepProcessor.java  | 2 +-
 .../tez/mapreduce/examples/BroadcastAndOneToOneExample.java   | 2 +-
 .../java/org/apache/tez/mapreduce/examples/MRRSleepJob.java   | 2 +-
 15 files changed, 30 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/b9056657/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index ef9a839..ece14a3 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -8,6 +8,7 @@ INCOMPATIBLE CHANGES
   TEZ-2468. Change the minimum Java version to Java 7.
 
 ALL CHANGES:
+  TEZ-1883. Change findbugs version to 3.x.
   TEZ-2440. Sorter should check for indexCacheList.size() in flush().
   TEZ-2490. TEZ-2450 breaks Hadoop 2.2 and 2.4 compatability.
   TEZ-2450. support async http clients in ordered & unordered inputs.

http://git-wip-us.apache.org/repos/asf/tez/blob/b9056657/build-tools/test-patch.sh
----------------------------------------------------------------------
diff --git a/build-tools/test-patch.sh b/build-tools/test-patch.sh
index 7218b87..0ac2bac 100755
--- a/build-tools/test-patch.sh
+++ b/build-tools/test-patch.sh
@@ -525,7 +525,7 @@ buildAndInstall () {
 ###############################################################################
 ### Check there are no changes in the number of Findbugs warnings
 checkFindbugsWarnings () {
-  findbugs_version=`${FINDBUGS_HOME}/bin/findbugs -version`
+  findbugs_version=$(${AWK} 'match($0, /findbugs-maven-plugin:[^:]*:findbugs/) { print substr($0, RSTART + 22, RLENGTH - 31); exit }' "${PATCH_DIR}/patchFindBugsOutput${module_suffix}.txt")
   echo ""
   echo ""
   echo "======================================================================"

http://git-wip-us.apache.org/repos/asf/tez/blob/b9056657/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 2922cab..cafba97 100644
--- a/pom.xml
+++ b/pom.xml
@@ -629,7 +629,7 @@
       <dependency>
         <groupId>com.google.code.findbugs</groupId>
         <artifactId>jsr305</artifactId>
-        <version>2.0.3</version>
+        <version>3.0.0</version>
       </dependency>
       <dependency>
         <groupId>com.sun.jersey</groupId>
@@ -781,7 +781,7 @@
         <plugin>
           <groupId>org.codehaus.mojo</groupId>
           <artifactId>findbugs-maven-plugin</artifactId>
-          <version>2.5.2</version>
+          <version>3.0.1</version>
         </plugin>
         <plugin>
           <groupId>org.apache.maven.plugins</groupId>

http://git-wip-us.apache.org/repos/asf/tez/blob/b9056657/tez-common/src/main/java/org/apache/tez/dag/records/TezDAGID.java
----------------------------------------------------------------------
diff --git a/tez-common/src/main/java/org/apache/tez/dag/records/TezDAGID.java b/tez-common/src/main/java/org/apache/tez/dag/records/TezDAGID.java
index 0fe1d44..27cc751 100644
--- a/tez-common/src/main/java/org/apache/tez/dag/records/TezDAGID.java
+++ b/tez-common/src/main/java/org/apache/tez/dag/records/TezDAGID.java
@@ -99,7 +99,7 @@ public class TezDAGID extends TezID {
 
   
   private TezDAGID(String yarnRMIdentifier, int appId, int id) {
-    this(ApplicationId.newInstance(Long.valueOf(yarnRMIdentifier),
+    this(ApplicationId.newInstance(Long.parseLong(yarnRMIdentifier),
         appId), id);
   }
 

http://git-wip-us.apache.org/repos/asf/tez/blob/b9056657/tez-dag/findbugs-exclude.xml
----------------------------------------------------------------------
diff --git a/tez-dag/findbugs-exclude.xml b/tez-dag/findbugs-exclude.xml
index 57c0aca..c89a9d2 100644
--- a/tez-dag/findbugs-exclude.xml
+++ b/tez-dag/findbugs-exclude.xml
@@ -142,6 +142,13 @@
       <Field name="taskSchedulerEventHandler"/>
       <Field name="versionMismatch"/>
       <Field name="versionMismatchDiagnostics"/>
+      <Field name="containers"/>
+      <Field name="currentRecoveryDataDir"/>
+      <Field name="execService"/>
+      <Field name="historyEventHandler"/>
+      <Field name="nodes"/>
+      <Field name="recoveryEnabled"/>
+      <Field name="isLocal"/>
     </Or>
     <Bug pattern="IS2_INCONSISTENT_SYNC"/>
   </Match>

http://git-wip-us.apache.org/repos/asf/tez/blob/b9056657/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 0fadcfa..3805b6c 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
@@ -2000,7 +2000,7 @@ public class DAGAppMaster extends AbstractService {
       String maxAppAttemptsEnv = System.getenv(
           ApplicationConstants.MAX_APP_ATTEMPTS_ENV);
       if (maxAppAttemptsEnv != null) {
-        maxAppAttempts = Integer.valueOf(maxAppAttemptsEnv);
+        maxAppAttempts = Integer.parseInt(maxAppAttemptsEnv);
       }
 
       validateInputParam(appSubmitTimeStr,

http://git-wip-us.apache.org/repos/asf/tez/blob/b9056657/tez-dag/src/main/java/org/apache/tez/dag/utils/Graph.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/utils/Graph.java b/tez-dag/src/main/java/org/apache/tez/dag/utils/Graph.java
index cc9033d..6de9c59 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/utils/Graph.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/utils/Graph.java
@@ -230,9 +230,10 @@ public class Graph {
   }
 
   public void save(String filePath) throws IOException {
-    FileOutputStream fout = new FileOutputStream(filePath);
-    fout.write(generateGraphViz().getBytes("UTF-8"));
-    fout.close();
+    try (FileOutputStream fout = new FileOutputStream(filePath)){
+      fout.write(generateGraphViz().getBytes("UTF-8"));
+      fout.close();
+    }
   }
 
   public static List<Edge> combineEdges(List<Edge> edges) {

http://git-wip-us.apache.org/repos/asf/tez/blob/b9056657/tez-dag/src/main/java/org/apache/tez/dag/utils/Simple2LevelVersionComparator.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/utils/Simple2LevelVersionComparator.java b/tez-dag/src/main/java/org/apache/tez/dag/utils/Simple2LevelVersionComparator.java
index ef0ad29..b54187c 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/utils/Simple2LevelVersionComparator.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/utils/Simple2LevelVersionComparator.java
@@ -53,7 +53,7 @@ public class Simple2LevelVersionComparator {
           continue;
         }
         try {
-          int i = Integer.valueOf(token);
+          int i = Integer.parseInt(token);
           if (index == 0) {
             majorVersion = i;
           } else if (index == 1) {

http://git-wip-us.apache.org/repos/asf/tez/blob/b9056657/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/split/TezMapredSplitsGrouper.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/split/TezMapredSplitsGrouper.java b/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/split/TezMapredSplitsGrouper.java
index 29b5e1e..2194551 100644
--- a/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/split/TezMapredSplitsGrouper.java
+++ b/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/split/TezMapredSplitsGrouper.java
@@ -33,7 +33,6 @@ import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceStability.Evolving;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.mapred.InputSplit;
-import org.apache.hadoop.mapred.split.SplitSizeEstimator;
 import org.apache.hadoop.mapreduce.split.TezMapReduceSplitsGrouper;
 import org.apache.hadoop.yarn.util.RackResolver;
 import org.apache.tez.dag.api.TezUncheckedException;
@@ -109,6 +108,7 @@ public class TezMapredSplitsGrouper {
       InputSplit[] originalSplits, int desiredNumSplits,
       String wrappedInputFormatName, SplitSizeEstimator estimator) throws IOException {
     LOG.info("Grouping splits in Tez");
+    Preconditions.checkArgument(originalSplits != null, "Splits must be specified");
 
     int configNumSplits = conf.getInt(TezMapReduceSplitsGrouper.TEZ_GROUPING_SPLIT_COUNT, 0);
     if (configNumSplits > 0) {
@@ -122,7 +122,6 @@ public class TezMapredSplitsGrouper {
     }
 
     if (! (configNumSplits > 0 || 
-          originalSplits == null || 
           originalSplits.length == 0) ) {
       // numSplits has not been overridden by config
       // numSplits has been set at runtime

http://git-wip-us.apache.org/repos/asf/tez/blob/b9056657/tez-mapreduce/src/main/java/org/apache/hadoop/mapreduce/split/TezMapReduceSplitsGrouper.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/hadoop/mapreduce/split/TezMapReduceSplitsGrouper.java b/tez-mapreduce/src/main/java/org/apache/hadoop/mapreduce/split/TezMapReduceSplitsGrouper.java
index 88b9845..4be3931 100644
--- a/tez-mapreduce/src/main/java/org/apache/hadoop/mapreduce/split/TezMapReduceSplitsGrouper.java
+++ b/tez-mapreduce/src/main/java/org/apache/hadoop/mapreduce/split/TezMapReduceSplitsGrouper.java
@@ -169,6 +169,7 @@ public class TezMapReduceSplitsGrouper {
       List<InputSplit> originalSplits, int desiredNumSplits,
       String wrappedInputFormatName, SplitSizeEstimator estimator) throws IOException, InterruptedException {
     LOG.info("Grouping splits in Tez");
+    Preconditions.checkArgument(originalSplits != null, "Splits must be specified");
 
     int configNumSplits = conf.getInt(TEZ_GROUPING_SPLIT_COUNT, 0);
     if (configNumSplits > 0) {
@@ -182,7 +183,6 @@ public class TezMapReduceSplitsGrouper {
     }
 
     if (! (configNumSplits > 0 || 
-          originalSplits == null || 
           originalSplits.size() == 0)) {
       // numSplits has not been overridden by config
       // numSplits has been set at runtime

http://git-wip-us.apache.org/repos/asf/tez/blob/b9056657/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/IDConverter.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/IDConverter.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/IDConverter.java
index a528098..0f1b56d 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/IDConverter.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/IDConverter.java
@@ -60,7 +60,7 @@ public class IDConverter {
   public static TezDAGID fromMRJobId(
       org.apache.hadoop.mapreduce.JobID jobId) {
     return TezDAGID.getInstance(ApplicationId.newInstance(
-        Long.valueOf(jobId.getJtIdentifier()), jobId.getId()), 1);
+        Long.parseLong(jobId.getJtIdentifier()), jobId.getId()), 1);
   }
 
   // FIXME hack alert converting objects with hard coded id

http://git-wip-us.apache.org/repos/asf/tez/blob/b9056657/tez-runtime-library/findbugs-exclude.xml
----------------------------------------------------------------------
diff --git a/tez-runtime-library/findbugs-exclude.xml b/tez-runtime-library/findbugs-exclude.xml
index 919e1e3..9866ba4 100644
--- a/tez-runtime-library/findbugs-exclude.xml
+++ b/tez-runtime-library/findbugs-exclude.xml
@@ -122,4 +122,10 @@
     <Bug pattern="NN_NAKED_NOTIFY"/>
   </Match>
 
+  <Match>
+    <Class name="org.apache.tez.runtime.library.common.shuffle.ShuffleUtils"/>
+    <Method name="getHttpConnectionParams" params="org.apache.hadoop.conf.Configuration" return="org.apache.tez.http.HttpConnectionParams"/>
+    <Bug pattern="DC_PARTIALLY_CONSTRUCTED"/>
+  </Match>
+
 </FindBugsFilter>

http://git-wip-us.apache.org/repos/asf/tez/blob/b9056657/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/processor/SleepProcessor.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/processor/SleepProcessor.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/processor/SleepProcessor.java
index 1122bbe..91dcb6d 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/processor/SleepProcessor.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/processor/SleepProcessor.java
@@ -120,7 +120,7 @@ public class SleepProcessor extends AbstractLogicalIOProcessor {
     }
 
     public void fromUserPayload(UserPayload userPayload) throws CharacterCodingException {
-      timeToSleepMS = Integer.valueOf(charSet.newDecoder().decode(userPayload.getPayload()).toString()).intValue();
+      timeToSleepMS = Integer.parseInt(charSet.newDecoder().decode(userPayload.getPayload()).toString());
     }
 
     public int getTimeToSleepMS() {

http://git-wip-us.apache.org/repos/asf/tez/blob/b9056657/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/BroadcastAndOneToOneExample.java
----------------------------------------------------------------------
diff --git a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/BroadcastAndOneToOneExample.java b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/BroadcastAndOneToOneExample.java
index 96dff26..9dd9b59 100644
--- a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/BroadcastAndOneToOneExample.java
+++ b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/BroadcastAndOneToOneExample.java
@@ -114,7 +114,7 @@ public class BroadcastAndOneToOneExample extends Configured implements Tool {
       if (doLocalityCheck) {
         ObjectRegistry objectRegistry = getContext().getObjectRegistry();
         String index = (String) objectRegistry.get(String.valueOf(getContext().getTaskIndex()));
-        if (index == null || Integer.valueOf(index).intValue() != getContext().getTaskIndex()) {
+        if (index == null || Integer.parseInt(index) != getContext().getTaskIndex()) {
           String msg = "Did not find expected local producer "
               + getContext().getTaskIndex() + " in the same JVM";
           System.out.println(msg);

http://git-wip-us.apache.org/repos/asf/tez/blob/b9056657/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/MRRSleepJob.java
----------------------------------------------------------------------
diff --git a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/MRRSleepJob.java b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/MRRSleepJob.java
index 9342668..0a6d9a1 100644
--- a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/MRRSleepJob.java
+++ b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/MRRSleepJob.java
@@ -235,7 +235,7 @@ public class MRRSleepJob extends Configured implements Tool {
         }
         if (!found) {
           for (String taskId : taskIds) {
-            if (Integer.valueOf(taskId).intValue() ==
+            if (Integer.parseInt(taskId) ==
                 taId.getTaskID().getId()) {
               found = true;
               break;