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;