From 79048f9e44cc41154a3d741090053a0b0e6f21b0 Mon Sep 17 00:00:00 2001 From: Laszlo Bodor Date: Tue, 1 Sep 2020 12:42:10 +0200 Subject: [PATCH 001/137] TEZ-4228: TezClassLoader should be used in TezChild and for Configuration objects --- .../org/apache/tez/common/TezClassLoader.java | 18 ++++++++++++++++++ .../java/org/apache/tez/common/TezUtils.java | 3 +++ .../org/apache/tez/runtime/task/TezChild.java | 4 +++- 3 files changed, 24 insertions(+), 1 deletion(-) diff --git a/tez-api/src/main/java/org/apache/tez/common/TezClassLoader.java b/tez-api/src/main/java/org/apache/tez/common/TezClassLoader.java index 923d2177db..53c9e51fc6 100644 --- a/tez-api/src/main/java/org/apache/tez/common/TezClassLoader.java +++ b/tez-api/src/main/java/org/apache/tez/common/TezClassLoader.java @@ -18,6 +18,10 @@ import java.security.AccessController; import java.security.PrivilegedAction; +import org.apache.hadoop.conf.Configuration; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + /** * ClassLoader to allow addition of new paths to classpath in the runtime. * @@ -30,6 +34,7 @@ */ public class TezClassLoader extends URLClassLoader { private static final TezClassLoader INSTANCE; + private static final Logger LOG = LoggerFactory.getLogger(TezClassLoader.class); static { INSTANCE = AccessController.doPrivileged(new PrivilegedAction() { @@ -41,6 +46,11 @@ public TezClassLoader run() { private TezClassLoader() { super(new URL[] {}, TezClassLoader.class.getClassLoader()); + + LOG.info( + "Created TezClassLoader with parent classloader: {}, thread: {}, system classloader: {}", + TezClassLoader.class.getClassLoader(), Thread.currentThread().getId(), + ClassLoader.getSystemClassLoader()); } public void addURL(URL url) { @@ -52,6 +62,14 @@ public static TezClassLoader getInstance() { } public static void setupTezClassLoader() { + LOG.debug( + "Setting up TezClassLoader: thread: {}, current thread classloader: {} system classloader: {}", + Thread.currentThread().getId(), Thread.currentThread().getContextClassLoader(), + ClassLoader.getSystemClassLoader()); Thread.currentThread().setContextClassLoader(INSTANCE); } + + public static void setupForConfiguration(Configuration configuration) { + configuration.setClassLoader(INSTANCE); + } } diff --git a/tez-api/src/main/java/org/apache/tez/common/TezUtils.java b/tez-api/src/main/java/org/apache/tez/common/TezUtils.java index 23811aa7f1..3f0241834a 100644 --- a/tez-api/src/main/java/org/apache/tez/common/TezUtils.java +++ b/tez-api/src/main/java/org/apache/tez/common/TezUtils.java @@ -117,6 +117,7 @@ public static Configuration createConfFromByteString(ByteString byteString) thro DAGProtos.ConfigurationProto confProto = DAGProtos.ConfigurationProto.parseFrom(in); Configuration conf = new Configuration(false); readConfFromPB(confProto, conf); + TezClassLoader.setupForConfiguration(conf); return conf; } } @@ -130,6 +131,7 @@ public static Configuration createConfFromBaseConfAndPayload(TaskContext context try(SnappyInputStream uncompressIs = new SnappyInputStream(byteString.newInput())) { DAGProtos.ConfigurationProto confProto = DAGProtos.ConfigurationProto.parseFrom(uncompressIs); readConfFromPB(confProto, configuration); + TezClassLoader.setupForConfiguration(configuration); return configuration; } } @@ -139,6 +141,7 @@ public static void addToConfFromByteString(Configuration configuration, ByteStri try(SnappyInputStream uncompressIs = new SnappyInputStream(byteString.newInput())) { DAGProtos.ConfigurationProto confProto = DAGProtos.ConfigurationProto.parseFrom(uncompressIs); readConfFromPB(confProto, configuration); + TezClassLoader.setupForConfiguration(configuration); } } 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 6dd146a8fb..86ceb12d7d 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 @@ -77,6 +77,8 @@ import com.google.common.base.Function; import org.apache.tez.common.Preconditions; +import org.apache.tez.common.TezClassLoader; + import com.google.common.collect.HashMultimap; import com.google.common.collect.Maps; import com.google.common.collect.Multimap; @@ -480,7 +482,7 @@ public static TezChild newTezChild(Configuration conf, String host, int port, St } public static void main(String[] args) throws IOException, InterruptedException, TezException { - + TezClassLoader.setupTezClassLoader(); final Configuration defaultConf = new Configuration(); Thread.setDefaultUncaughtExceptionHandler(new YarnUncaughtExceptionHandler()); From 1fba8f71a0f496e1fcb67a223fb32b389bdfb2e0 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?L=C3=A1szl=C3=B3=20Bodor?= Date: Tue, 8 Sep 2020 18:43:23 +0200 Subject: [PATCH 002/137] =?UTF-8?q?TEZ-4230:=20LocalContainerLauncher=20ca?= =?UTF-8?q?n=20kill=20task=20future=20too=20early,=20causing=20app=20hang?= =?UTF-8?q?=20(L=C3=A1szl=C3=B3=20Bodor=20reviewed=20by=20Jonathan=20Turne?= =?UTF-8?q?r=20Eagles)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Laszlo Bodor --- .../tez/dag/app/launcher/LocalContainerLauncher.java | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) 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 a63ce3fdfe..ae308098a7 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 @@ -299,10 +299,11 @@ private void stop(ContainerStopRequest event) { if (future == null) { LOG.info("Ignoring stop request for containerId: " + event.getContainerId()); } else { - LOG.info( - "Stopping containerId: {}", - event.getContainerId()); - future.cancel(true); + LOG.info("Stopping containerId: {}, isDone: {}", event.getContainerId(), + future.isDone()); + future.cancel(false); + LOG.debug("Stopped containerId: {}, isCancelled: {}", event.getContainerId(), + future.isCancelled()); } // Send this event to maintain regular control flow. This isn't of much use though. getContext().containerStopRequested(event.getContainerId()); From 629714afd692770d64aed73f6e48b7b46838a592 Mon Sep 17 00:00:00 2001 From: Laszlo Bodor Date: Mon, 28 Sep 2020 08:28:53 +0200 Subject: [PATCH 003/137] TEZ-4233: Map task should be blamed earlier for local fetch failures (Laszlo Bdoor reviewed by Rajesh Balamohan) --- .../api/events/InputReadErrorEvent.java | 40 +++- .../shuffle/api/ShuffleHandlerError.java | 27 +++ .../common/shuffle/api/package-info.java | 22 +++ tez-api/src/main/proto/Events.proto | 2 + .../tez/dag/app/dag/impl/TaskAttemptImpl.java | 9 +- .../tez/dag/app/dag/impl/TestTaskAttempt.java | 49 ++++- .../tez/auxservices/ShuffleHandler.java | 52 ++++-- .../tez/auxservices/TestShuffleHandler.java | 102 +++++++++- .../apache/tez/runtime/api/impl/TezEvent.java | 9 +- .../library/common/shuffle/Fetcher.java | 63 ++++--- .../common/shuffle/FetcherCallback.java | 3 +- .../shuffle/InputAttemptFetchFailure.java | 115 ++++++++++++ .../common/shuffle/impl/ShuffleManager.java | 22 ++- .../orderedgrouped/FetcherOrderedGrouped.java | 74 +++++--- .../shuffle/orderedgrouped/ShuffleHeader.java | 7 + .../orderedgrouped/ShuffleScheduler.java | 45 ++--- .../library/common/shuffle/TestFetcher.java | 54 +++++- .../shuffle/impl/TestShuffleManager.java | 15 +- .../shuffle/orderedgrouped/TestFetcher.java | 66 +++++-- .../orderedgrouped/TestShuffleScheduler.java | 174 +++++++++--------- .../library/testutils/RuntimeTestUtils.java | 44 +++++ 21 files changed, 766 insertions(+), 228 deletions(-) create mode 100644 tez-api/src/main/java/org/apache/tez/runtime/library/common/shuffle/api/ShuffleHandlerError.java create mode 100644 tez-api/src/main/java/org/apache/tez/runtime/library/common/shuffle/api/package-info.java create mode 100644 tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/InputAttemptFetchFailure.java create mode 100644 tez-runtime-library/src/test/java/org/apache/tez/runtime/library/testutils/RuntimeTestUtils.java diff --git a/tez-api/src/main/java/org/apache/tez/runtime/api/events/InputReadErrorEvent.java b/tez-api/src/main/java/org/apache/tez/runtime/api/events/InputReadErrorEvent.java index cabc39fc8e..8ef50ebac4 100644 --- a/tez-api/src/main/java/org/apache/tez/runtime/api/events/InputReadErrorEvent.java +++ b/tez-api/src/main/java/org/apache/tez/runtime/api/events/InputReadErrorEvent.java @@ -51,26 +51,44 @@ public final class InputReadErrorEvent extends Event { */ private final int numFailures; - private InputReadErrorEvent(final String diagnostics, final int index, - final int version, final int numFailures) { + /** + * Whether this input read error is caused while fetching local file. + */ + private final boolean isLocalFetch; + + /** + * Whether this input read error is caused because the fetcher detected a fatal, unrecoverable, + * local file read issue from the shuffle handler. + */ + private final boolean isDiskErrorAtSource; + + private InputReadErrorEvent(final String diagnostics, final int index, final int version, + final int numFailures, boolean isLocalFetch, boolean isDiskErrorAtSource) { super(); this.diagnostics = diagnostics; this.index = index; this.version = version; this.numFailures = numFailures; + this.isLocalFetch = isLocalFetch; + this.isDiskErrorAtSource = isDiskErrorAtSource; } - public static InputReadErrorEvent create(String diagnostics, int index, - int version) { - return create(diagnostics, index, version, 1); + public static InputReadErrorEvent create(String diagnostics, int index, int version, + boolean isLocalFetch, boolean isDiskErrorAtSource) { + return create(diagnostics, index, version, 1, isLocalFetch, isDiskErrorAtSource); + } + + public static InputReadErrorEvent create(String diagnostics, int index, int version) { + return create(diagnostics, index, version, 1, false, false); } /** * Create an InputReadErrorEvent. */ public static InputReadErrorEvent create(final String diagnostics, final int index, - final int version, final int numFailures) { - return new InputReadErrorEvent(diagnostics, index, version, numFailures); + final int version, final int numFailures, boolean isLocalFetch, boolean isDiskErrorAtSource) { + return new InputReadErrorEvent(diagnostics, index, version, numFailures, isLocalFetch, + isDiskErrorAtSource); } public String getDiagnostics() { @@ -92,6 +110,14 @@ public int getNumFailures() { return numFailures; } + public boolean isLocalFetch() { + return isLocalFetch; + } + + public boolean isDiskErrorAtSource() { + return isDiskErrorAtSource; + } + @Override public int hashCode() { return Objects.hash(index, version); diff --git a/tez-api/src/main/java/org/apache/tez/runtime/library/common/shuffle/api/ShuffleHandlerError.java b/tez-api/src/main/java/org/apache/tez/runtime/library/common/shuffle/api/ShuffleHandlerError.java new file mode 100644 index 0000000000..09137de673 --- /dev/null +++ b/tez-api/src/main/java/org/apache/tez/runtime/library/common/shuffle/api/ShuffleHandlerError.java @@ -0,0 +1,27 @@ +/** +* Licensed to the Apache Software Foundation (ASF) under one +* or more contributor license agreements. See the NOTICE file +* distributed with this work for additional information +* regarding copyright ownership. The ASF licenses this file +* to you under the Apache License, Version 2.0 (the +* "License"); you may not use this file except in compliance +* with the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, software +* distributed under the License is distributed on an "AS IS" BASIS, +* WITHOUT WARRANTIES OR CONDITIONS OF 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.api; + +/** + * ShuffleHandlerError enum encapsulates possible error messages that can be propagated from + * ShuffleHandler to fetchers. Depending on the message, fetchers can make better decisions, or give + * AM a hint in order to let it make better decisions in case of shuffle issues. + */ +public enum ShuffleHandlerError { + DISK_ERROR_EXCEPTION +} diff --git a/tez-api/src/main/java/org/apache/tez/runtime/library/common/shuffle/api/package-info.java b/tez-api/src/main/java/org/apache/tez/runtime/library/common/shuffle/api/package-info.java new file mode 100644 index 0000000000..9ad8e61d50 --- /dev/null +++ b/tez-api/src/main/java/org/apache/tez/runtime/library/common/shuffle/api/package-info.java @@ -0,0 +1,22 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +@Private +package org.apache.tez.runtime.library.common.shuffle.api; + +import org.apache.hadoop.classification.InterfaceAudience.Private; \ No newline at end of file diff --git a/tez-api/src/main/proto/Events.proto b/tez-api/src/main/proto/Events.proto index 71235004da..e041c33f60 100644 --- a/tez-api/src/main/proto/Events.proto +++ b/tez-api/src/main/proto/Events.proto @@ -39,6 +39,8 @@ message InputReadErrorEventProto { optional int32 index = 1; optional string diagnostics = 2; optional int32 version = 3; + optional bool is_local_fetch = 4; + optional bool is_disk_error_at_source = 5; } message InputFailedEventProto { 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 37e818e271..9a5e73de5a 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 @@ -1825,6 +1825,7 @@ public TaskAttemptStateInternal transition(TaskAttemptImpl attempt, + " at inputIndex " + failedInputIndexOnDestTa); long time = attempt.clock.getTime(); Long firstErrReportTime = attempt.uniquefailedOutputReports.get(failedDestTaId); + if (firstErrReportTime == null) { attempt.uniquefailedOutputReports.put(failedDestTaId, time); firstErrReportTime = time; @@ -1851,7 +1852,8 @@ public TaskAttemptStateInternal transition(TaskAttemptImpl attempt, // If needed we can launch a background task without failing this task // to generate a copy of the output just in case. // If needed we can consider only running consumer tasks - if (!crossTimeDeadline && withinFailureFractionLimits && withinOutputFailureLimits) { + if (!crossTimeDeadline && withinFailureFractionLimits && withinOutputFailureLimits + && !(readErrorEvent.isLocalFetch() || readErrorEvent.isDiskErrorAtSource())) { return attempt.getInternalState(); } String message = attempt.getID() + " being failed for too many output errors. " @@ -1862,7 +1864,10 @@ public TaskAttemptStateInternal transition(TaskAttemptImpl attempt, + ", MAX_ALLOWED_OUTPUT_FAILURES=" + maxAllowedOutputFailures + ", MAX_ALLOWED_TIME_FOR_TASK_READ_ERROR_SEC=" + maxAllowedTimeForTaskReadErrorSec - + ", readErrorTimespan=" + readErrorTimespanSec; + + ", readErrorTimespan=" + readErrorTimespanSec + + ", isLocalFetch=" + readErrorEvent.isLocalFetch() + + ", isDiskErrorAtSource=" + readErrorEvent.isDiskErrorAtSource(); + LOG.info(message); attempt.addDiagnosticInfo(message); // send input failed event 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 41cce3b60e..6862bec2ee 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 @@ -19,7 +19,6 @@ package org.apache.tez.dag.app.dag.impl; import org.apache.tez.dag.app.MockClock; -import org.apache.tez.dag.app.rm.AMSchedulerEventTAStateUpdated; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.fail; @@ -83,6 +82,7 @@ import org.apache.tez.dag.app.TaskCommunicatorWrapper; import org.apache.tez.dag.app.TaskHeartbeatHandler; import org.apache.tez.dag.app.dag.TaskAttemptStateInternal; +import org.apache.tez.dag.app.dag.DAG; import org.apache.tez.dag.app.dag.Task; import org.apache.tez.dag.app.dag.Vertex; import org.apache.tez.dag.app.dag.event.DAGEvent; @@ -127,6 +127,7 @@ import org.junit.BeforeClass; import org.junit.Test; import org.mockito.ArgumentCaptor; +import org.mockito.Mockito; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -2154,6 +2155,52 @@ taListener, taskConf, new SystemClock(), Assert.assertEquals(1, taImpl.taskAttemptFinishedEventLogged); } + @Test + public void testMapTaskIsBlamedImmediatelyOnLocalFetchFailure() throws ServicePluginException { + // local fetch failure or disk read error at source -> turn source attempt to FAIL_IN_PROGRESS + testMapTaskFailingForFetchFailureType(true, true, TaskAttemptStateInternal.FAIL_IN_PROGRESS); + testMapTaskFailingForFetchFailureType(true, false, TaskAttemptStateInternal.FAIL_IN_PROGRESS); + testMapTaskFailingForFetchFailureType(false, true, TaskAttemptStateInternal.FAIL_IN_PROGRESS); + + // remote fetch failure -> won't change current state + testMapTaskFailingForFetchFailureType(false, false, TaskAttemptStateInternal.NEW); + } + + private void testMapTaskFailingForFetchFailureType(boolean isLocalFetch, + boolean isDiskErrorAtSource, TaskAttemptStateInternal expectedState) { + EventHandler eventHandler = mock(EventHandler.class); + TezTaskID taskID = + TezTaskID.getInstance(TezVertexID.getInstance(TezDAGID.getInstance("1", 1, 1), 1), 1); + TaskAttemptImpl sourceAttempt = new MockTaskAttemptImpl(taskID, 1, eventHandler, null, + new Configuration(), SystemClock.getInstance(), mock(TaskHeartbeatHandler.class), appCtx, + false, null, null, false); + + // the original read error event, sent by reducer task + InputReadErrorEvent inputReadErrorEvent = + InputReadErrorEvent.create("", 0, 1, 1, isLocalFetch, isDiskErrorAtSource); + TezTaskAttemptID destTaskAttemptId = mock(TezTaskAttemptID.class); + when(destTaskAttemptId.getTaskID()).thenReturn(mock(TezTaskID.class)); + when(destTaskAttemptId.getTaskID().getVertexID()).thenReturn(mock(TezVertexID.class)); + when(appCtx.getCurrentDAG()).thenReturn(mock(DAG.class)); + when(appCtx.getCurrentDAG().getVertex(Mockito.any(TezVertexID.class))) + .thenReturn(mock(Vertex.class)); + when(appCtx.getCurrentDAG().getVertex(Mockito.any(TezVertexID.class)).getRunningTasks()) + .thenReturn(100); + + EventMetaData mockMeta = mock(EventMetaData.class); + when(mockMeta.getTaskAttemptID()).thenReturn(destTaskAttemptId); + TezEvent tezEvent = new TezEvent(inputReadErrorEvent, mockMeta); + + // the event is propagated to map task's event handler + TaskAttemptEventOutputFailed outputFailedEvent = + new TaskAttemptEventOutputFailed(sourceAttempt.getID(), tezEvent, 11); + + Assert.assertEquals(TaskAttemptStateInternal.NEW, sourceAttempt.getInternalState()); + TaskAttemptStateInternal resultState = new TaskAttemptImpl.OutputReportedFailedTransition() + .transition(sourceAttempt, outputFailedEvent); + Assert.assertEquals(expectedState, resultState); + } + private Event verifyEventType(List events, Class eventClass, int expectedOccurences) { int count = 0; diff --git a/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java b/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java index f294edced9..b67883dfcf 100644 --- a/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java +++ b/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java @@ -71,6 +71,7 @@ import org.apache.tez.common.security.JobTokenSecretManager; import org.apache.tez.runtime.library.common.Constants; import org.apache.tez.runtime.library.common.security.SecureShuffleUtils; +import org.apache.tez.runtime.library.common.shuffle.api.ShuffleHandlerError; import org.apache.tez.runtime.library.common.shuffle.orderedgrouped.ShuffleHeader; import org.apache.hadoop.metrics2.MetricsSystem; import org.apache.hadoop.metrics2.annotation.Metric; @@ -84,6 +85,7 @@ import org.apache.hadoop.security.token.Token; import org.apache.tez.runtime.library.common.sort.impl.TezIndexRecord; import org.apache.hadoop.util.Shell; +import org.apache.hadoop.util.DiskChecker.DiskErrorException; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.proto.YarnServerCommonProtos.VersionProto; @@ -101,6 +103,7 @@ import org.iq80.leveldb.Logger; import org.iq80.leveldb.Options; import org.jboss.netty.bootstrap.ServerBootstrap; +import org.jboss.netty.buffer.ChannelBuffer; import org.jboss.netty.buffer.ChannelBuffers; import org.jboss.netty.channel.Channel; import org.jboss.netty.channel.ChannelFactory; @@ -608,6 +611,10 @@ protected Shuffle getShuffle(Configuration conf) { return new Shuffle(conf); } + protected JobTokenSecretManager getSecretManager() { + return secretManager; + } + private void recoverState(Configuration conf) throws IOException { Path recoveryRoot = getRecoveryPath(); if (recoveryRoot != null) { @@ -727,7 +734,7 @@ private void checkVersion() throws IOException { private void addJobToken(JobID jobId, String user, Token jobToken) { userRsrc.put(jobId.toString(), user); - secretManager.addTokenForJob(jobId.toString(), jobToken); + getSecretManager().addTokenForJob(jobId.toString(), jobToken); LOG.info("Added token for " + jobId.toString()); } @@ -772,7 +779,7 @@ private void recordJobShuffleInfo(JobID jobId, String user, private void removeJobShuffleInfo(JobID jobId) throws IOException { String jobIdStr = jobId.toString(); - secretManager.removeTokenForJob(jobIdStr); + getSecretManager().removeTokenForJob(jobIdStr); userRsrc.remove(jobIdStr); if (stateDb != null) { try { @@ -1080,11 +1087,19 @@ public void messageReceived(ChannelHandlerContext ctx, MessageEvent evt) try { populateHeaders(mapIds, jobId, dagId, user, reduceRange, response, keepAliveParam, mapOutputInfoMap); - } catch(IOException e) { + } catch (DiskErrorException e) { // fatal error: fetcher should be aware of that + LOG.error("Shuffle error in populating headers (fatal: DiskErrorException):", e); + String errorMessage = getErrorMessage(e); + // custom message, might be noticed by fetchers + // it should reuse the current response object, as headers have been already set for it + sendFakeShuffleHeaderWithError(ctx, + ShuffleHandlerError.DISK_ERROR_EXCEPTION + ": " + errorMessage, response); + return; + } catch (IOException e) { ch.write(response); LOG.error("Shuffle error in populating headers :", e); String errorMessage = getErrorMessage(e); - sendError(ctx,errorMessage , INTERNAL_SERVER_ERROR); + sendError(ctx, errorMessage, INTERNAL_SERVER_ERROR); return; } ch.write(response); @@ -1337,7 +1352,7 @@ public void finish() { protected void verifyRequest(String appid, ChannelHandlerContext ctx, HttpRequest request, HttpResponse response, URL requestUri) throws IOException { - SecretKey tokenSecret = secretManager.retrieveTokenSecret(appid); + SecretKey tokenSecret = getSecretManager().retrieveTokenSecret(appid); if (null == tokenSecret) { LOG.info("Request for unknown token " + appid); throw new IOException("could not find jobid"); @@ -1444,22 +1459,37 @@ public void operationComplete(ChannelFuture future) { return writeFuture; } - protected void sendError(ChannelHandlerContext ctx, - HttpResponseStatus status) { + protected void sendError(ChannelHandlerContext ctx, HttpResponseStatus status) { sendError(ctx, "", status); } - protected void sendError(ChannelHandlerContext ctx, String message, - HttpResponseStatus status) { + protected void sendError(ChannelHandlerContext ctx, String message, HttpResponseStatus status) { HttpResponse response = new DefaultHttpResponse(HTTP_1_1, status); + sendError(ctx, message, response); + } + + protected void sendError(ChannelHandlerContext ctx, String message, HttpResponse response) { + sendError(ctx, ChannelBuffers.copiedBuffer(message, CharsetUtil.UTF_8), response); + } + + private void sendFakeShuffleHeaderWithError(ChannelHandlerContext ctx, String message, + HttpResponse response) throws IOException { + ShuffleHeader header = new ShuffleHeader(message, -1, -1, -1); + DataOutputBuffer out = new DataOutputBuffer(); + header.write(out); + + sendError(ctx, wrappedBuffer(out.getData(), 0, out.getLength()), response); + } + + protected void sendError(ChannelHandlerContext ctx, ChannelBuffer content, + HttpResponse response) { response.headers().set(CONTENT_TYPE, "text/plain; charset=UTF-8"); // Put shuffle version into http header response.headers().set(ShuffleHeader.HTTP_HEADER_NAME, ShuffleHeader.DEFAULT_HTTP_HEADER_NAME); response.headers().set(ShuffleHeader.HTTP_HEADER_VERSION, ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION); - response.setContent( - ChannelBuffers.copiedBuffer(message, CharsetUtil.UTF_8)); + response.setContent(content); // Close the connection as soon as the error message is sent. ctx.getChannel().write(response).addListener(ChannelFutureListener.CLOSE); diff --git a/tez-plugins/tez-aux-services/src/test/java/org/apache/tez/auxservices/TestShuffleHandler.java b/tez-plugins/tez-aux-services/src/test/java/org/apache/tez/auxservices/TestShuffleHandler.java index a7f444680b..5ca4ed8e26 100644 --- a/tez-plugins/tez-aux-services/src/test/java/org/apache/tez/auxservices/TestShuffleHandler.java +++ b/tez-plugins/tez-aux-services/src/test/java/org/apache/tez/auxservices/TestShuffleHandler.java @@ -20,6 +20,7 @@ //import static org.apache.hadoop.test.MetricsAsserts.assertCounter; //import static org.apache.hadoop.test.MetricsAsserts.assertGauge; //import static org.apache.hadoop.test.MetricsAsserts.getMetrics; +import org.apache.hadoop.util.DiskChecker.DiskErrorException; import static org.junit.Assert.assertTrue; import static org.jboss.netty.buffer.ChannelBuffers.wrappedBuffer; import static org.jboss.netty.handler.codec.http.HttpResponseStatus.OK; @@ -41,15 +42,12 @@ import java.net.URL; import java.nio.ByteBuffer; import java.util.ArrayList; -import java.util.Arrays; import java.util.List; import java.util.Map; -import java.util.zip.CheckedOutputStream; import java.util.zip.Checksum; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.CommonConfigurationKeysPublic; -import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.fs.Path; @@ -58,11 +56,14 @@ import org.apache.hadoop.io.WritableUtils; import org.apache.hadoop.io.nativeio.NativeIO; import org.apache.hadoop.mapred.JobID; -import org.apache.hadoop.mapred.MapTask; import org.apache.hadoop.mapreduce.TypeConverter; import org.apache.tez.runtime.library.common.security.SecureShuffleUtils; import org.apache.tez.common.security.JobTokenIdentifier; import org.apache.tez.common.security.JobTokenSecretManager; +import org.apache.tez.http.BaseHttpConnection; +import org.apache.tez.http.HttpConnectionParams; +import org.apache.tez.runtime.library.common.shuffle.ShuffleUtils; +import org.apache.tez.runtime.library.common.shuffle.api.ShuffleHandlerError; import org.apache.tez.runtime.library.common.shuffle.orderedgrouped.ShuffleHeader; import org.apache.hadoop.metrics2.MetricsSystem; import org.apache.hadoop.metrics2.impl.MetricsSystemImpl; @@ -172,6 +173,52 @@ protected boolean isSocketKeepAlive() { } } + class MockShuffleHandlerWithFatalDiskError extends org.apache.tez.auxservices.ShuffleHandler { + public static final String MESSAGE = + "Could not find application_1234/240/output/attempt_1234_0/file.out.index"; + + private JobTokenSecretManager secretManager = + new JobTokenSecretManager(JobTokenSecretManager.createSecretKey(getSecret().getBytes())); + + protected JobTokenSecretManager getSecretManager(){ + return secretManager; + } + + @Override + protected Shuffle getShuffle(final Configuration conf) { + return new Shuffle(conf) { + @Override + protected void verifyRequest(String appid, ChannelHandlerContext ctx, HttpRequest request, + HttpResponse response, URL requestUri) throws IOException { + super.verifyRequest(appid, ctx, request, response, requestUri); + } + + @Override + protected MapOutputInfo getMapOutputInfo(String dagId, String mapId, Range reduceRange, + String jobId, String user) { + return null; + } + + @Override + protected void populateHeaders(List mapIds, String jobId, String dagId, String user, + Range reduceRange, HttpResponse response, boolean keepAliveParam, + Map infoMap) throws IOException { + throw new DiskErrorException(MESSAGE); + } + + @Override + protected ChannelFuture sendMapOutput(ChannelHandlerContext ctx, Channel ch, String user, + String mapId, Range reduceRange, MapOutputInfo info) throws IOException { + return null; + } + }; + } + + public String getSecret() { + return "secret"; + } + } + /** * Test the validation of ShuffleHandler's meta-data's serialization and * de-serialization. @@ -1322,6 +1369,53 @@ public void testSendMapCount() throws Exception { sh.close(); } + @Test + public void testShuffleHandlerSendsDiskError() throws Exception { + Configuration conf = new Configuration(); + conf.setInt(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY, 0); + + DataInputStream input = null; + MockShuffleHandlerWithFatalDiskError shuffleHandler = + new MockShuffleHandlerWithFatalDiskError(); + try { + shuffleHandler.init(conf); + shuffleHandler.start(); + + String shuffleBaseURL = "http://127.0.0.1:" + + shuffleHandler.getConfig().get(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY); + URL url = new URL( + shuffleBaseURL + "/mapOutput?job=job_12345_1&dag=1&reduce=1&map=attempt_12345_1_m_1_0"); + shuffleHandler.secretManager.addTokenForJob("job_12345_1", + new Token<>("id".getBytes(), shuffleHandler.getSecret().getBytes(), null, null)); + + HttpConnectionParams httpConnectionParams = ShuffleUtils.getHttpConnectionParams(conf); + BaseHttpConnection httpConnection = ShuffleUtils.getHttpConnection(true, url, + httpConnectionParams, "testFetcher", shuffleHandler.secretManager); + + boolean connectSucceeded = httpConnection.connect(); + Assert.assertTrue(connectSucceeded); + + input = httpConnection.getInputStream(); + httpConnection.validate(); + + ShuffleHeader header = new ShuffleHeader(); + header.readFields(input); + + // message is encoded in the shuffle header, and can be checked by fetchers + Assert.assertEquals( + ShuffleHandlerError.DISK_ERROR_EXCEPTION + ": " + MockShuffleHandlerWithFatalDiskError.MESSAGE, + header.getMapId()); + Assert.assertEquals(-1, header.getCompressedLength()); + Assert.assertEquals(-1, header.getUncompressedLength()); + Assert.assertEquals(-1, header.getPartition()); + } finally { + if (input != null) { + input.close(); + } + shuffleHandler.close(); + } + } + public ChannelFuture createMockChannelFuture(Channel mockCh, final List listenerList) { final ChannelFuture mockFuture = mock(ChannelFuture.class); diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezEvent.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezEvent.java index e7af4a1ebe..ebea9a4f3f 100644 --- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezEvent.java +++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezEvent.java @@ -191,6 +191,8 @@ private void serializeEvent(DataOutput out) throws IOException { .setIndex(ideEvt.getIndex()) .setDiagnostics(ideEvt.getDiagnostics()) .setVersion(ideEvt.getVersion()) + .setIsLocalFetch(ideEvt.isLocalFetch()) + .setIsDiskErrorAtSource(ideEvt.isDiskErrorAtSource()) .build(); break; case TASK_ATTEMPT_FAILED_EVENT: @@ -294,10 +296,9 @@ private void deserializeEvent(DataInput in) throws IOException { event = ProtoConverters.convertVertexManagerEventFromProto(vmProto); break; case INPUT_READ_ERROR_EVENT: - InputReadErrorEventProto ideProto = - InputReadErrorEventProto.parseFrom(input); - event = InputReadErrorEvent.create(ideProto.getDiagnostics(), - ideProto.getIndex(), ideProto.getVersion()); + InputReadErrorEventProto ideProto = InputReadErrorEventProto.parseFrom(input); + event = InputReadErrorEvent.create(ideProto.getDiagnostics(), ideProto.getIndex(), + ideProto.getVersion(), ideProto.getIsLocalFetch(), ideProto.getIsDiskErrorAtSource()); break; case TASK_ATTEMPT_FAILED_EVENT: TaskAttemptFailedEventProto tfProto = 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 18f66cc8ef..f295c06b8e 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 @@ -66,7 +66,7 @@ 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.api.ShuffleHandlerError; import org.apache.tez.common.Preconditions; /** @@ -277,7 +277,8 @@ public FetchResult callInternal() throws Exception { HostFetchResult hostFetchResult; - if (localDiskFetchEnabled && host.equals(localHostname) && port == shufflePort) { + boolean isLocalFetch = localDiskFetchEnabled && host.equals(localHostname) && port == shufflePort; + if (isLocalFetch) { hostFetchResult = setupLocalDiskFetch(); } else if (multiplex) { hostFetchResult = doSharedFetch(); @@ -288,7 +289,7 @@ public FetchResult callInternal() throws Exception { if (hostFetchResult.failedInputs != null && hostFetchResult.failedInputs.length > 0) { if (!isShutDown.get()) { LOG.warn("copyInputs failed for tasks " + Arrays.toString(hostFetchResult.failedInputs)); - for (InputAttemptIdentifier left : hostFetchResult.failedInputs) { + for (InputAttemptFetchFailure left : hostFetchResult.failedInputs) { fetcherCallback.fetchFailed(host, left, hostFetchResult.connectFailed); } } else { @@ -504,7 +505,7 @@ private HostFetchResult setupConnection(Collection attem // ioErrs.increment(1); // If connect did not succeed, just mark all the maps as failed, // indirectly penalizing the host - InputAttemptIdentifier[] failedFetches = null; + InputAttemptFetchFailure[] failedFetches = null; if (isShutDown.get()) { if (isDebugEnabled) { LOG.debug( @@ -512,8 +513,7 @@ private HostFetchResult setupConnection(Collection attem e.getClass().getName() + ", Message: " + e.getMessage()); } } else { - failedFetches = srcAttemptsRemaining.values(). - toArray(new InputAttemptIdentifier[srcAttemptsRemaining.values().size()]); + failedFetches = InputAttemptFetchFailure.fromAttempts(srcAttemptsRemaining.values()); } return new HostFetchResult(new FetchResult(host, port, partition, partitionCount, srcAttemptsRemaining.values()), failedFetches, true); } @@ -547,7 +547,7 @@ private HostFetchResult setupConnection(Collection attem "Fetch Failure while connecting from %s to: %s:%d, attempt: %s Informing ShuffleManager: ", localHostname, host, port, firstAttempt), e); return new HostFetchResult(new FetchResult(host, port, partition, partitionCount, srcAttemptsRemaining.values()), - new InputAttemptIdentifier[] { firstAttempt }, true); + new InputAttemptFetchFailure[] { new InputAttemptFetchFailure(firstAttempt) }, true); } } catch (InterruptedException e) { Thread.currentThread().interrupt(); //reset status @@ -584,7 +584,7 @@ protected HostFetchResult doHttpFetch(CachingCallBack callback) { // On any error, faildTasks is not null and we exit // after putting back the remaining maps to the // yet_to_be_fetched list and marking the failed tasks. - InputAttemptIdentifier[] failedInputs = null; + InputAttemptFetchFailure[] failedInputs = null; while (!srcAttemptsRemaining.isEmpty() && failedInputs == null) { InputAttemptIdentifier inputAttemptIdentifier = srcAttemptsRemaining.entrySet().iterator().next().getValue(); @@ -711,7 +711,7 @@ public void freeResources(FetchedInput fetchedInput) { } } - InputAttemptIdentifier[] failedFetches = null; + InputAttemptFetchFailure[] failedFetches = null; if (failMissing && srcAttemptsRemaining.size() > 0) { if (isShutDown.get()) { if (isDebugEnabled) { @@ -720,8 +720,8 @@ public void freeResources(FetchedInput fetchedInput) { " remaining inputs"); } } else { - failedFetches = srcAttemptsRemaining.values(). - toArray(new InputAttemptIdentifier[srcAttemptsRemaining.values().size()]); + failedFetches = + InputAttemptFetchFailure.fromAttemptsLocalFetchFailure(srcAttemptsRemaining.values()); } } else { // nothing needs to be done to requeue remaining entries @@ -770,10 +770,10 @@ public Map getPathToAttemptMap() { static class HostFetchResult { private final FetchResult fetchResult; - private final InputAttemptIdentifier[] failedInputs; + private final InputAttemptFetchFailure[] failedInputs; private final boolean connectFailed; - public HostFetchResult(FetchResult fetchResult, InputAttemptIdentifier[] failedInputs, + public HostFetchResult(FetchResult fetchResult, InputAttemptFetchFailure[] failedInputs, boolean connectFailed) { this.fetchResult = fetchResult; this.failedInputs = failedInputs; @@ -831,8 +831,11 @@ public String toString() { return "id: " + srcAttemptId + ", decompressed length: " + decompressedLength + ", compressed length: " + compressedLength + ", reduce: " + forReduce; } } - private InputAttemptIdentifier[] fetchInputs(DataInputStream input, - CachingCallBack callback, InputAttemptIdentifier inputAttemptIdentifier) throws FetcherReadTimeoutException { + + @VisibleForTesting + InputAttemptFetchFailure[] fetchInputs(DataInputStream input, CachingCallBack callback, + InputAttemptIdentifier inputAttemptIdentifier) + throws FetcherReadTimeoutException { FetchedInput fetchedInput = null; InputAttemptIdentifier srcAttemptId = null; long decompressedLength = 0; @@ -856,9 +859,19 @@ private InputAttemptIdentifier[] fetchInputs(DataInputStream input, header.readFields(input); pathComponent = header.getMapId(); if (!pathComponent.startsWith(InputAttemptIdentifier.PATH_PREFIX)) { - throw new IllegalArgumentException("Invalid map id: " + header.getMapId() + ", expected to start with " + - InputAttemptIdentifier.PATH_PREFIX + ", partition: " + header.getPartition() - + " while fetching " + inputAttemptIdentifier); + if (pathComponent.startsWith(ShuffleHandlerError.DISK_ERROR_EXCEPTION.toString())) { + LOG.warn("Invalid map id: " + header.getMapId() + ", expected to start with " + + InputAttemptIdentifier.PATH_PREFIX + ", partition: " + header.getPartition() + + " while fetching " + inputAttemptIdentifier); + // this should be treated as local fetch failure while reporting later + return new InputAttemptFetchFailure[] { + InputAttemptFetchFailure.fromDiskErrorAtSource(inputAttemptIdentifier) }; + } else { + throw new IllegalArgumentException( + "Invalid map id: " + header.getMapId() + ", expected to start with " + + InputAttemptIdentifier.PATH_PREFIX + ", partition: " + header.getPartition() + + " while fetching " + inputAttemptIdentifier); + } } srcAttemptId = pathToAttemptMap.get(new PathPartition(pathComponent, header.getPartition())); @@ -883,7 +896,7 @@ private InputAttemptIdentifier[] fetchInputs(DataInputStream input, if (!isShutDown.get()) { LOG.warn("Invalid src id ", e); // Don't know which one was bad, so consider all of them as bad - return srcAttemptsRemaining.values().toArray(new InputAttemptIdentifier[srcAttemptsRemaining.size()]); + return InputAttemptFetchFailure.fromAttempts(srcAttemptsRemaining.values()); } else { if (isDebugEnabled) { LOG.debug("Already shutdown. Ignoring badId error with message: " + e.getMessage()); @@ -902,7 +915,8 @@ private InputAttemptIdentifier[] fetchInputs(DataInputStream input, srcAttemptId = getNextRemainingAttempt(); } assert (srcAttemptId != null); - return new InputAttemptIdentifier[]{srcAttemptId}; + return new InputAttemptFetchFailure[] { + InputAttemptFetchFailure.fromAttempt(srcAttemptId) }; } else { if (isDebugEnabled) { LOG.debug("Already shutdown. Ignoring verification failure."); @@ -1004,10 +1018,10 @@ private InputAttemptIdentifier[] fetchInputs(DataInputStream input, // Cleanup the fetchedInput before returning. cleanupFetchedInput(fetchedInput); if (srcAttemptId == null) { - return srcAttemptsRemaining.values() - .toArray(new InputAttemptIdentifier[srcAttemptsRemaining.size()]); + return InputAttemptFetchFailure.fromAttempts(srcAttemptsRemaining.values()); } else { - return new InputAttemptIdentifier[] { srcAttemptId }; + return new InputAttemptFetchFailure[] { + new InputAttemptFetchFailure(srcAttemptId) }; } } LOG.warn("Failed to shuffle output of " + srcAttemptId + " from " + host + " (to " @@ -1016,7 +1030,8 @@ private InputAttemptIdentifier[] fetchInputs(DataInputStream input, // Cleanup the fetchedInput cleanupFetchedInput(fetchedInput); // metrics.failedFetch(); - return new InputAttemptIdentifier[] { srcAttemptId }; + return new InputAttemptFetchFailure[] { + new InputAttemptFetchFailure(srcAttemptId) }; } return null; } diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/FetcherCallback.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/FetcherCallback.java index 34bd272909..b751fb9ce0 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/FetcherCallback.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/FetcherCallback.java @@ -28,6 +28,7 @@ public void fetchSucceeded(String host, InputAttemptIdentifier srcAttemptIdentif FetchedInput fetchedInput, long fetchedBytes, long decompressedLength, long copyDuration) throws IOException; - public void fetchFailed(String host, InputAttemptIdentifier srcAttemptIdentifier, boolean connectFailed); + public void fetchFailed(String host, InputAttemptFetchFailure srcAttemptFetchFailure, + boolean connectFailed); } diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/InputAttemptFetchFailure.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/InputAttemptFetchFailure.java new file mode 100644 index 0000000000..d94db35c2f --- /dev/null +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/InputAttemptFetchFailure.java @@ -0,0 +1,115 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.util.Arrays; +import java.util.Collection; + +import org.apache.tez.runtime.library.common.CompositeInputAttemptIdentifier; +import org.apache.tez.runtime.library.common.InputAttemptIdentifier; + +/** + * InputAttemptFetchFailure is supposed to wrap an InputAttemptIdentifier with any kind of failure + * information during fetch. It can be useful for propagating as a single object instead of multiple + * parameters (local fetch error, remote fetch error, connect failed, read failed, etc.). + */ +public class InputAttemptFetchFailure { + + private final InputAttemptIdentifier inputAttemptIdentifier; + private final boolean isLocalFetch; + private final boolean isDiskErrorAtSource; + + public InputAttemptFetchFailure(InputAttemptIdentifier inputAttemptIdentifier) { + this(inputAttemptIdentifier, false, false); + } + + public InputAttemptFetchFailure(InputAttemptIdentifier inputAttemptIdentifier, + boolean isLocalFetch, boolean isDiskErrorAtSource) { + this.inputAttemptIdentifier = inputAttemptIdentifier; + this.isLocalFetch = isLocalFetch; + this.isDiskErrorAtSource = isDiskErrorAtSource; + } + + public InputAttemptIdentifier getInputAttemptIdentifier() { + return inputAttemptIdentifier; + } + + public boolean isLocalFetch() { + return isLocalFetch; + } + + public boolean isDiskErrorAtSource() { + return isDiskErrorAtSource; + } + + public static InputAttemptFetchFailure fromAttempt(InputAttemptIdentifier attempt) { + return new InputAttemptFetchFailure(attempt, false, false); + } + + public static InputAttemptFetchFailure fromLocalFetchFailure(InputAttemptIdentifier attempt) { + return new InputAttemptFetchFailure(attempt, true, false); + } + + public static InputAttemptFetchFailure fromDiskErrorAtSource(InputAttemptIdentifier attempt) { + return new InputAttemptFetchFailure(attempt, false, true); + } + + public static InputAttemptFetchFailure[] fromAttempts(Collection values) { + return values.stream().map(identifier -> new InputAttemptFetchFailure(identifier, false, false)) + .toArray(InputAttemptFetchFailure[]::new); + } + + public static InputAttemptFetchFailure[] fromAttempts(InputAttemptIdentifier[] values) { + return Arrays.asList(values).stream() + .map(identifier -> new InputAttemptFetchFailure(identifier, false, false)) + .toArray(InputAttemptFetchFailure[]::new); + } + + public static InputAttemptFetchFailure[] fromAttemptsLocalFetchFailure( + Collection values) { + return values.stream().map(identifier -> new InputAttemptFetchFailure(identifier, true, false)) + .toArray(InputAttemptFetchFailure[]::new); + } + + public static InputAttemptFetchFailure fromCompositeAttemptLocalFetchFailure( + CompositeInputAttemptIdentifier compositeInputAttemptIdentifier) { + return new InputAttemptFetchFailure(compositeInputAttemptIdentifier, true, false); + } + + @Override + public boolean equals(Object obj) { + if (obj == null || (obj.getClass() != this.getClass())) { + return false; + } + return inputAttemptIdentifier.equals(((InputAttemptFetchFailure) obj).inputAttemptIdentifier) + && isLocalFetch == ((InputAttemptFetchFailure) obj).isLocalFetch + && isDiskErrorAtSource == ((InputAttemptFetchFailure) obj).isDiskErrorAtSource; + } + + @Override + public int hashCode() { + return 31 * inputAttemptIdentifier.hashCode() + 31 * (isLocalFetch ? 0 : 1) + + 31 * (isDiskErrorAtSource ? 0 : 1); + } + + @Override + public String toString() { + return String.format("%s, isLocalFetch: %s, isDiskErrorAtSource: %s", + inputAttemptIdentifier.toString(), isLocalFetch, isDiskErrorAtSource); + } +} 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 742fc18632..2b83ad8920 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 @@ -84,6 +84,7 @@ import org.apache.tez.runtime.library.common.shuffle.Fetcher.FetcherBuilder; import org.apache.tez.runtime.library.common.shuffle.FetcherCallback; import org.apache.tez.runtime.library.common.shuffle.HostPort; +import org.apache.tez.runtime.library.common.shuffle.InputAttemptFetchFailure; import org.apache.tez.runtime.library.common.shuffle.InputHost; import org.apache.tez.runtime.library.common.shuffle.InputHost.PartitionToInputs; import org.apache.tez.runtime.library.common.shuffle.ShuffleUtils; @@ -391,9 +392,9 @@ protected Void callInternal() throws Exception { List failedEventsToSend = Lists.newArrayListWithCapacity( failedEvents.size()); for (InputReadErrorEvent key : failedEvents.keySet()) { - failedEventsToSend.add(InputReadErrorEvent - .create(key.getDiagnostics(), key.getIndex(), - key.getVersion(), failedEvents.get(key))); + failedEventsToSend.add(InputReadErrorEvent.create(key.getDiagnostics(), + key.getIndex(), key.getVersion(), failedEvents.get(key), key.isLocalFetch(), + key.isDiskErrorAtSource())); } inputContext.sendEvents(failedEventsToSend); failedEvents.clear(); @@ -939,12 +940,15 @@ private void reportFatalError(Throwable exception, String message) { @Override public void fetchFailed(String host, - InputAttemptIdentifier srcAttemptIdentifier, boolean connectFailed) { + InputAttemptFetchFailure inputAttemptFetchFailure, boolean connectFailed) { // TODO NEWTEZ. Implement logic to report fetch failures after a threshold. // For now, reporting immediately. - LOG.info(srcNameTrimmed + ": " + "Fetch failed for src: " + srcAttemptIdentifier - + "InputIdentifier: " + srcAttemptIdentifier + ", connectFailed: " - + connectFailed); + InputAttemptIdentifier srcAttemptIdentifier = inputAttemptFetchFailure.getInputAttemptIdentifier(); + LOG.info( + "{}: Fetch failed for src: {} InputIdentifier: {}, connectFailed: {}, " + + "local fetch: {}, remote fetch failure reported as local failure: {})", + srcNameTrimmed, srcAttemptIdentifier, srcAttemptIdentifier, connectFailed, + inputAttemptFetchFailure.isLocalFetch(), inputAttemptFetchFailure.isDiskErrorAtSource()); failedShufflesCounter.increment(1); inputContext.notifyProgress(); if (srcAttemptIdentifier == null) { @@ -957,7 +961,9 @@ public void fetchFailed(String host, srcAttemptIdentifier.getInputIdentifier(), srcAttemptIdentifier.getAttemptNumber()), srcAttemptIdentifier.getInputIdentifier(), - srcAttemptIdentifier.getAttemptNumber()); + srcAttemptIdentifier.getAttemptNumber(), + inputAttemptFetchFailure.isLocalFetch(), + inputAttemptFetchFailure.isDiskErrorAtSource()); if (maxTimeToWaitForReportMillis > 0) { try { reportLock.lock(); 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 e732ab0063..327232710b 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 @@ -50,7 +50,9 @@ 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.InputAttemptFetchFailure; import org.apache.tez.runtime.library.common.shuffle.ShuffleUtils; +import org.apache.tez.runtime.library.common.shuffle.api.ShuffleHandlerError; import com.google.common.annotations.VisibleForTesting; @@ -272,7 +274,8 @@ protected void copyFromHost(MapHost host) throws IOException { // On any error, faildTasks is not null and we exit // after putting back the remaining maps to the // yet_to_be_fetched list and marking the failed tasks. - InputAttemptIdentifier[] failedTasks = null; + InputAttemptFetchFailure[] failedTasks = null; + while (!remaining.isEmpty() && failedTasks == null) { InputAttemptIdentifier inputAttemptIdentifier = remaining.entrySet().iterator().next().getValue(); @@ -300,25 +303,14 @@ protected void copyFromHost(MapHost host) throws IOException { } return; } - failedTasks = new InputAttemptIdentifier[] {getNextRemainingAttempt()}; + failedTasks = new InputAttemptFetchFailure[] { + new InputAttemptFetchFailure(getNextRemainingAttempt()) }; break; } } } - if (failedTasks != null && failedTasks.length > 0) { - if (stopped) { - if (LOG.isDebugEnabled()) { - LOG.debug("Ignoring copyMapOutput failures for tasks: " + Arrays.toString(failedTasks) + - " since Fetcher has been stopped"); - } - } else { - LOG.warn("copyMapOutput failed for tasks " + Arrays.toString(failedTasks)); - for (InputAttemptIdentifier left : failedTasks) { - scheduler.copyFailed(left, host, true, false, false); - } - } - } + invokeCopyFailedForFailedTasks(host, failedTasks); cleanupCurrentConnection(false); @@ -332,6 +324,23 @@ protected void copyFromHost(MapHost host) throws IOException { } } + private void invokeCopyFailedForFailedTasks(MapHost host, + InputAttemptFetchFailure[] failedTasks) { + if (failedTasks != null && failedTasks.length > 0) { + if (stopped) { + if (LOG.isDebugEnabled()) { + LOG.debug("Ignoring copyMapOutput failures for tasks: " + Arrays.toString(failedTasks) + + " since Fetcher has been stopped"); + } + } else { + LOG.warn("copyMapOutput failed for tasks " + Arrays.toString(failedTasks)); + for (InputAttemptFetchFailure left : failedTasks) { + scheduler.copyFailed(left, host, true, false); + } + } + } + } + @VisibleForTesting boolean setupConnection(MapHost host, Collection attempts) throws IOException { @@ -380,7 +389,8 @@ boolean setupConnection(MapHost host, Collection attempt for (InputAttemptIdentifier left : remaining.values()) { // Need to be handling temporary glitches .. // Report read error to the AM to trigger source failure heuristics - scheduler.copyFailed(left, host, connectSucceeded, !connectSucceeded, false); + scheduler.copyFailed(InputAttemptFetchFailure.fromAttempt(left), host, connectSucceeded, + !connectSucceeded); } return false; } @@ -404,7 +414,8 @@ protected void putBackRemainingMapOutputs(MapHost host) { } } - private static InputAttemptIdentifier[] EMPTY_ATTEMPT_ID_ARRAY = new InputAttemptIdentifier[0]; + private static final InputAttemptFetchFailure[] EMPTY_ATTEMPT_ID_ARRAY = + new InputAttemptFetchFailure[0]; private static class MapOutputStat { final InputAttemptIdentifier srcAttemptId; @@ -425,8 +436,8 @@ public String toString() { } } - protected InputAttemptIdentifier[] copyMapOutput(MapHost host, - DataInputStream input, InputAttemptIdentifier inputAttemptIdentifier) throws FetcherReadTimeoutException { + protected InputAttemptFetchFailure[] copyMapOutput(MapHost host, DataInputStream input, + InputAttemptIdentifier inputAttemptIdentifier) throws FetcherReadTimeoutException { MapOutput mapOutput = null; InputAttemptIdentifier srcAttemptId = null; long decompressedLength = 0; @@ -452,7 +463,13 @@ protected InputAttemptIdentifier[] copyMapOutput(MapHost host, badIdErrs.increment(1); LOG.warn("Invalid map id: " + header.mapId + ", expected to start with " + InputAttemptIdentifier.PATH_PREFIX + ", partition: " + header.forReduce); - return new InputAttemptIdentifier[]{getNextRemainingAttempt()}; + if (header.mapId.startsWith(ShuffleHandlerError.DISK_ERROR_EXCEPTION.toString())) { + //this should be treated as local fetch failure while reporting later + return new InputAttemptFetchFailure[] { + InputAttemptFetchFailure.fromDiskErrorAtSource(getNextRemainingAttempt()) }; + } + return new InputAttemptFetchFailure[] { + InputAttemptFetchFailure.fromAttempt(getNextRemainingAttempt()) }; } else { if (LOG.isDebugEnabled()) { LOG.debug("Already shutdown. Ignoring invalid map id error"); @@ -477,7 +494,8 @@ protected InputAttemptIdentifier[] copyMapOutput(MapHost host, LOG.warn("Invalid map id ", e); // Don't know which one was bad, so consider this one bad and dont read // the remaining because we dont know where to start reading from. YARN-1773 - return new InputAttemptIdentifier[]{getNextRemainingAttempt()}; + return new InputAttemptFetchFailure[] { + new InputAttemptFetchFailure(getNextRemainingAttempt()) }; } else { if (LOG.isDebugEnabled()) { LOG.debug("Already shutdown. Ignoring invalid map id error. Exception: " + @@ -497,7 +515,8 @@ protected InputAttemptIdentifier[] copyMapOutput(MapHost host, LOG.warn("Was expecting " + srcAttemptId + " but got null"); } assert (srcAttemptId != null); - return new InputAttemptIdentifier[]{srcAttemptId}; + return new InputAttemptFetchFailure[] { + new InputAttemptFetchFailure(getNextRemainingAttempt()) }; } else { if (LOG.isDebugEnabled()) { LOG.debug("Already stopped. Ignoring verification failure."); @@ -595,9 +614,10 @@ protected InputAttemptIdentifier[] copyMapOutput(MapHost host, srcAttemptId + " decomp: " + decompressedLength + ", " + compressedLength, ioe); if (srcAttemptId == null) { - return remaining.values().toArray(new InputAttemptIdentifier[remaining.values().size()]); + return InputAttemptFetchFailure.fromAttempts(remaining.values()); } else { - return new InputAttemptIdentifier[]{srcAttemptId}; + return new InputAttemptFetchFailure[] { + new InputAttemptFetchFailure(srcAttemptId) }; } } LOG.warn("Failed to shuffle output of " + srcAttemptId + @@ -605,7 +625,8 @@ protected InputAttemptIdentifier[] copyMapOutput(MapHost host, // Inform the shuffle-scheduler mapOutput.abort(); - return new InputAttemptIdentifier[] {srcAttemptId}; + return new InputAttemptFetchFailure[] { + new InputAttemptFetchFailure(srcAttemptId) }; } return null; } @@ -734,7 +755,8 @@ protected void setupLocalDiskFetch(MapHost host) throws InterruptedException { if (!stopped) { hasFailures = true; ioErrs.increment(1); - scheduler.copyFailed(srcAttemptId, host, true, false, true); + scheduler.copyFailed(InputAttemptFetchFailure.fromLocalFetchFailure(srcAttemptId), + host, true, false); LOG.warn("Failed to read local disk output of " + srcAttemptId + " from " + host.getHostIdentifier(), e); } else { diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleHeader.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleHeader.java index 9f883dbfdc..f074e897e8 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleHeader.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleHeader.java @@ -103,4 +103,11 @@ public void write(DataOutput out) throws IOException { WritableUtils.writeVLong(out, uncompressedLength); WritableUtils.writeVInt(out, forReduce); } + + @Override + public String toString() { + return String.format( + "ShuffleHeader [mapId=%s, uncompressedLength=%d, compressedLength=%d, forReduce=%d]", mapId, + uncompressedLength, compressedLength, forReduce); + } } 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 0954a76ec9..e7f63ab7ad 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 @@ -80,6 +80,7 @@ import org.apache.tez.runtime.library.common.shuffle.ShuffleUtils; import org.apache.tez.runtime.library.common.shuffle.ShuffleUtils.FetchStatsLogger; import org.apache.tez.runtime.library.common.shuffle.HostPort; +import org.apache.tez.runtime.library.common.shuffle.InputAttemptFetchFailure; import org.apache.tez.runtime.library.common.shuffle.orderedgrouped.MapHost.HostPortPartition; import org.apache.tez.runtime.library.common.shuffle.orderedgrouped.MapOutput.Type; @@ -755,16 +756,13 @@ private void logProgress() { } } - public synchronized void copyFailed(InputAttemptIdentifier srcAttempt, - MapHost host, - boolean readError, - boolean connectError, - boolean isLocalFetch) { + public synchronized void copyFailed(InputAttemptFetchFailure fetchFailure, MapHost host, + boolean readError, boolean connectError) { failedShuffleCounter.increment(1); inputContext.notifyProgress(); - int failures = incrementAndGetFailureAttempt(srcAttempt); + int failures = incrementAndGetFailureAttempt(fetchFailure.getInputAttemptIdentifier()); - if (!isLocalFetch) { + if (!fetchFailure.isLocalFetch()) { /** * Track the number of failures that has happened since last completion. * This gets reset on a successful copy. @@ -789,11 +787,11 @@ public synchronized void copyFailed(InputAttemptIdentifier srcAttempt, if (shouldInformAM) { //Inform AM. In case producer needs to be restarted, it is handled at AM. - informAM(srcAttempt); + informAM(fetchFailure); } //Restart consumer in case shuffle is not healthy - if (!isShuffleHealthy(srcAttempt)) { + if (!isShuffleHealthy(fetchFailure.getInputAttemptIdentifier())) { return; } @@ -868,21 +866,24 @@ public void reportLocalError(IOException ioe) { } // Notify AM - private void informAM(InputAttemptIdentifier srcAttempt) { + private void informAM(InputAttemptFetchFailure fetchFailure) { + InputAttemptIdentifier srcAttempt = fetchFailure.getInputAttemptIdentifier(); LOG.info( - srcNameTrimmed + ": " + "Reporting fetch failure for InputIdentifier: " - + srcAttempt + " taskAttemptIdentifier: " + TezRuntimeUtils - .getTaskAttemptIdentifier(inputContext.getSourceVertexName(), - srcAttempt.getInputIdentifier(), - srcAttempt.getAttemptNumber()) + " to AM."); + "{}: Reporting fetch failure for InputIdentifier: {} taskAttemptIdentifier: {}, " + + "local fetch: {}, remote fetch failure reported as local failure: {}) to AM.", + srcNameTrimmed, srcAttempt, + TezRuntimeUtils.getTaskAttemptIdentifier(inputContext.getSourceVertexName(), + srcAttempt.getInputIdentifier(), srcAttempt.getAttemptNumber()), + fetchFailure.isLocalFetch(), fetchFailure.isDiskErrorAtSource()); List failedEvents = Lists.newArrayListWithCapacity(1); - failedEvents.add(InputReadErrorEvent.create( - "Fetch failure for " + TezRuntimeUtils - .getTaskAttemptIdentifier(inputContext.getSourceVertexName(), - srcAttempt.getInputIdentifier(), - srcAttempt.getAttemptNumber()) + " to jobtracker.", - srcAttempt.getInputIdentifier(), - srcAttempt.getAttemptNumber())); + failedEvents.add( + InputReadErrorEvent.create( + "Fetch failure for " + + TezRuntimeUtils.getTaskAttemptIdentifier(inputContext.getSourceVertexName(), + srcAttempt.getInputIdentifier(), srcAttempt.getAttemptNumber()) + + " to jobtracker.", + srcAttempt.getInputIdentifier(), srcAttempt.getAttemptNumber(), + fetchFailure.isLocalFetch(), fetchFailure.isDiskErrorAtSource())); inputContext.sendEvents(failedEvents); } 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 db9c7afad0..05d4eb4145 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 @@ -32,6 +32,7 @@ import static org.mockito.Mockito.spy; import static org.mockito.Mockito.verify; +import java.io.DataInputStream; import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; @@ -46,10 +47,16 @@ import org.apache.hadoop.fs.LocalFileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.tez.common.counters.TezCounters; import org.apache.tez.dag.api.TezConfiguration; +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.shuffle.api.ShuffleHandlerError; +import org.apache.tez.runtime.library.common.shuffle.impl.ShuffleManager; +import org.apache.tez.runtime.library.common.shuffle.orderedgrouped.ShuffleHeader; import org.apache.tez.runtime.library.common.sort.impl.TezIndexRecord; +import org.apache.tez.runtime.library.testutils.RuntimeTestUtils; import org.junit.Assert; import org.junit.Test; import org.mockito.ArgumentCaptor; @@ -79,7 +86,8 @@ public void testLocalFetchModeSetting() throws Exception { Fetcher fetcher = spy(builder.build()); FetchResult fr = new FetchResult(HOST, PORT, 0, 1, Arrays.asList(srcAttempts)); - Fetcher.HostFetchResult hfr = new Fetcher.HostFetchResult(fr, srcAttempts, false); + Fetcher.HostFetchResult hfr = + new Fetcher.HostFetchResult(fr, InputAttemptFetchFailure.fromAttempts(srcAttempts), false); doReturn(hfr).when(fetcher).setupLocalDiskFetch(); doReturn(null).when(fetcher).doHttpFetch(); doNothing().when(fetcher).shutdown(); @@ -151,7 +159,7 @@ public void testSetupLocalDiskFetch() throws Exception { }; final int FIRST_FAILED_ATTEMPT_IDX = 2; final int SECOND_FAILED_ATTEMPT_IDX = 4; - final int[] sucessfulAttempts = {0, 1, 3}; + final int[] successfulAttempts = {0, 1, 3}; TezConfiguration conf = new TezConfiguration(); conf.set(TezRuntimeConfiguration.TEZ_RUNTIME_OPTIMIZE_LOCAL_FETCH, "true"); @@ -206,18 +214,24 @@ public TezIndexRecord answer(InvocationOnMock invocation) throws Throwable { doNothing().when(fetcher).shutdown(); doNothing().when(callback).fetchSucceeded(anyString(), any(InputAttemptIdentifier.class), any(FetchedInput.class), anyLong(), anyLong(), anyLong()); - doNothing().when(callback).fetchFailed(anyString(), any(InputAttemptIdentifier.class), eq(false)); + doNothing().when(callback).fetchFailed(anyString(), any(InputAttemptFetchFailure.class), eq(false)); FetchResult fetchResult = fetcher.call(); verify(fetcher).setupLocalDiskFetch(); - // expect 3 sucesses and 2 failures - for (int i : sucessfulAttempts) { + // expect 3 successes and 2 failures + for (int i : successfulAttempts) { verifyFetchSucceeded(callback, srcAttempts[i], conf); } - verify(callback).fetchFailed(eq(HOST), eq(srcAttempts[FIRST_FAILED_ATTEMPT_IDX]), eq(false)); - verify(callback).fetchFailed(eq(HOST), eq(srcAttempts[SECOND_FAILED_ATTEMPT_IDX]), eq(false)); + verify(callback).fetchFailed(eq(HOST), + eq(InputAttemptFetchFailure + .fromCompositeAttemptLocalFetchFailure(srcAttempts[FIRST_FAILED_ATTEMPT_IDX])), + eq(false)); + verify(callback).fetchFailed(eq(HOST), + eq(InputAttemptFetchFailure + .fromCompositeAttemptLocalFetchFailure(srcAttempts[SECOND_FAILED_ATTEMPT_IDX])), + eq(false)); Assert.assertEquals("fetchResult host", fetchResult.getHost(), HOST); Assert.assertEquals("fetchResult partition", fetchResult.getPartition(), partition); @@ -304,4 +318,30 @@ public void testInputAttemptIdentifierMap() { Assert.assertTrue(expectedSrcAttempts[count++].toString().compareTo(key) == 0); } } + + @Test + public void testShuffleHandlerDiskErrorUnordered() + throws Exception { + Configuration conf = new Configuration(); + + InputContext inputContext = mock(InputContext.class); + doReturn(new TezCounters()).when(inputContext).getCounters(); + doReturn("vertex").when(inputContext).getSourceVertexName(); + + Fetcher.FetcherBuilder builder = new Fetcher.FetcherBuilder(mock(ShuffleManager.class), null, + null, ApplicationId.newInstance(0, 1), 1, null, "fetcherTest", conf, true, HOST, PORT, + false, true, false); + builder.assignWork(HOST, PORT, 0, 1, Arrays.asList(new InputAttemptIdentifier(0, 0))); + + Fetcher fetcher = builder.build(); + ShuffleHeader header = + new ShuffleHeader(ShuffleHandlerError.DISK_ERROR_EXCEPTION.toString(), -1, -1, -1); + DataInputStream input = RuntimeTestUtils.shuffleHeaderToDataInput(header); + + InputAttemptFetchFailure[] failures = + fetcher.fetchInputs(input, null, new InputAttemptIdentifier(0, 0)); + Assert.assertEquals(1, failures.length); + Assert.assertTrue(failures[0].isDiskErrorAtSource()); + Assert.assertFalse(failures[0].isLocalFetch()); + } } diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/impl/TestShuffleManager.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/impl/TestShuffleManager.java index 94f7f5a487..041fd03854 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/impl/TestShuffleManager.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/impl/TestShuffleManager.java @@ -22,7 +22,6 @@ import static org.junit.Assert.assertTrue; import static org.mockito.Matchers.any; import static org.mockito.Matchers.anyInt; -import static org.mockito.Matchers.anyList; import static org.mockito.Matchers.anyString; import static org.mockito.Mockito.atLeast; import static org.mockito.Mockito.doAnswer; @@ -37,7 +36,6 @@ import java.io.InputStream; import java.io.OutputStream; import java.nio.ByteBuffer; -import java.util.ArrayList; import java.util.LinkedList; import java.util.List; import java.util.concurrent.ExecutorService; @@ -55,7 +53,6 @@ import org.apache.tez.common.security.JobTokenIdentifier; import org.apache.tez.common.security.JobTokenSecretManager; import org.apache.tez.dag.api.TezConfiguration; -import org.apache.tez.dag.api.TezConstants; import org.apache.tez.runtime.api.Event; import org.apache.tez.runtime.api.ExecutionContext; import org.apache.tez.runtime.api.InputContext; @@ -66,9 +63,9 @@ 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.InputAttemptFetchFailure; import org.apache.tez.runtime.library.common.shuffle.FetchResult; import org.apache.tez.runtime.library.common.shuffle.InputHost; -import org.apache.tez.runtime.library.common.shuffle.ShuffleUtils; import org.apache.tez.runtime.library.shuffle.impl.ShuffleUserPayloads.DataMovementEventPayloadProto; import org.junit.After; import org.junit.Assert; @@ -234,12 +231,12 @@ public void run() { } } }); - InputAttemptIdentifier inputAttemptIdentifier - = new InputAttemptIdentifier(1, 1); + InputAttemptFetchFailure inputAttemptFetchFailure = + new InputAttemptFetchFailure(new InputAttemptIdentifier(1, 1)); schedulerGetHostThread.start(); Thread.sleep(1000); - shuffleManager.fetchFailed("host1", inputAttemptIdentifier, false); + shuffleManager.fetchFailed("host1", inputAttemptFetchFailure, false); Thread.sleep(1000); ArgumentCaptor captor = ArgumentCaptor.forClass(List.class); @@ -254,8 +251,8 @@ public void run() { Assert.assertEquals("Number of failures was: " + inputEvent.getNumFailures(), inputEvent.getNumFailures(), 1); - shuffleManager.fetchFailed("host1", inputAttemptIdentifier, false); - shuffleManager.fetchFailed("host1", inputAttemptIdentifier, false); + shuffleManager.fetchFailed("host1", inputAttemptFetchFailure, false); + shuffleManager.fetchFailed("host1", inputAttemptFetchFailure, false); Thread.sleep(1000); verify(inputContext, times(1)).sendEvents(any()); 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 5f7fe4ba7e..028fbce96a 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 @@ -42,6 +42,7 @@ import java.net.URL; import java.util.Arrays; import java.util.Collection; +import java.util.HashMap; import java.util.Iterator; import java.util.LinkedHashMap; import java.util.LinkedList; @@ -72,7 +73,10 @@ 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.testutils.RuntimeTestUtils; +import org.apache.tez.runtime.library.common.shuffle.InputAttemptFetchFailure; import org.apache.tez.runtime.library.common.shuffle.ShuffleUtils; +import org.apache.tez.runtime.library.common.shuffle.api.ShuffleHandlerError; import org.junit.Assert; import org.junit.Test; import org.mockito.ArgumentCaptor; @@ -241,7 +245,7 @@ public void testSetupLocalDiskFetch() throws Exception { ); final int FIRST_FAILED_ATTEMPT_IDX = 2; final int SECOND_FAILED_ATTEMPT_IDX = 4; - final int[] sucessfulAttemptsIndexes = { 0, 1, 3 }; + final int[] successfulAttemptsIndexes = { 0, 1, 3 }; doReturn(srcAttempts).when(scheduler).getMapsForHost(host); @@ -311,13 +315,17 @@ public TezIndexRecord answer(InvocationOnMock invocation) throws Throwable { spyFetcher.setupLocalDiskFetch(host); // should have exactly 3 success and 1 failure. - for (int i : sucessfulAttemptsIndexes) { + for (int i : successfulAttemptsIndexes) { for (int j = 0; j < host.getPartitionCount(); j++) { verifyCopySucceeded(scheduler, host, srcAttempts, i, j); } } - verify(scheduler).copyFailed(srcAttempts.get(FIRST_FAILED_ATTEMPT_IDX).expand(0), host, true, false, true); - verify(scheduler).copyFailed(srcAttempts.get(SECOND_FAILED_ATTEMPT_IDX).expand(0), host, true, false, true); + verify(scheduler).copyFailed( + eq(InputAttemptFetchFailure.fromLocalFetchFailure(srcAttempts.get(FIRST_FAILED_ATTEMPT_IDX).expand(0))), + eq(host), eq(true), eq(false)); + verify(scheduler).copyFailed( + eq(InputAttemptFetchFailure.fromLocalFetchFailure(srcAttempts.get(SECOND_FAILED_ATTEMPT_IDX).expand(0))), + eq(host), eq(true), eq(false)); verify(spyFetcher).putBackRemainingMapOutputs(host); verify(scheduler).putBackKnownMapOutput(host, srcAttempts.get(FIRST_FAILED_ATTEMPT_IDX)); @@ -426,7 +434,7 @@ public void testSetupLocalDiskFetchAutoReduce() throws Exception { ); final int FIRST_FAILED_ATTEMPT_IDX = 2; final int SECOND_FAILED_ATTEMPT_IDX = 4; - final int[] sucessfulAttemptsIndexes = { 0, 1, 3 }; + final int[] successfulAttemptsIndexes = { 0, 1, 3 }; doReturn(srcAttempts).when(scheduler).getMapsForHost(host); final ConcurrentMap pathToIdentifierMap @@ -503,15 +511,23 @@ public TezIndexRecord answer(InvocationOnMock invocation) throws Throwable { spyFetcher.setupLocalDiskFetch(host); // should have exactly 3 success and 1 failure. - for (int i : sucessfulAttemptsIndexes) { + for (int i : successfulAttemptsIndexes) { for (int j = 0; j < host.getPartitionCount(); j++) { verifyCopySucceeded(scheduler, host, srcAttempts, i, j); } } - verify(scheduler).copyFailed(srcAttempts.get(FIRST_FAILED_ATTEMPT_IDX).expand(0), host, true, false, true); - verify(scheduler).copyFailed(srcAttempts.get(FIRST_FAILED_ATTEMPT_IDX).expand(1), host, true, false, true); - verify(scheduler).copyFailed(srcAttempts.get(SECOND_FAILED_ATTEMPT_IDX).expand(0), host, true, false, true); - verify(scheduler).copyFailed(srcAttempts.get(SECOND_FAILED_ATTEMPT_IDX).expand(1), host, true, false, true); + verify(scheduler).copyFailed( + eq(InputAttemptFetchFailure.fromLocalFetchFailure(srcAttempts.get(FIRST_FAILED_ATTEMPT_IDX).expand(0))), + eq(host), eq(true), eq(false)); + verify(scheduler).copyFailed( + eq(InputAttemptFetchFailure.fromLocalFetchFailure(srcAttempts.get(FIRST_FAILED_ATTEMPT_IDX).expand(1))), + eq(host), eq(true), eq(false)); + verify(scheduler).copyFailed(eq( + InputAttemptFetchFailure.fromLocalFetchFailure(srcAttempts.get(SECOND_FAILED_ATTEMPT_IDX).expand(0))), + eq(host), eq(true), eq(false)); + verify(scheduler).copyFailed(eq( + InputAttemptFetchFailure.fromLocalFetchFailure(srcAttempts.get(SECOND_FAILED_ATTEMPT_IDX).expand(1))), + eq(host), eq(true), eq(false)); verify(spyFetcher).putBackRemainingMapOutputs(host); verify(scheduler).putBackKnownMapOutput(host, srcAttempts.get(FIRST_FAILED_ATTEMPT_IDX)); @@ -630,8 +646,8 @@ public MapOutput answer(InvocationOnMock invocation) throws Throwable { //setup connection should be called twice (1 for connect and another for retry) verify(fetcher, times(2)).setupConnection(any(MapHost.class), any(Collection.class)); //since copyMapOutput consistently fails, it should call copyFailed once - verify(scheduler, times(1)).copyFailed(any(InputAttemptIdentifier.class), any(MapHost.class), - anyBoolean(), anyBoolean(), anyBoolean()); + verify(scheduler, times(1)).copyFailed(any(InputAttemptFetchFailure.class), any(MapHost.class), + anyBoolean(), anyBoolean()); verify(fetcher, times(1)).putBackRemainingMapOutputs(any(MapHost.class)); verify(scheduler, times(3)).putBackKnownMapOutput(any(MapHost.class), @@ -750,6 +766,32 @@ public void testInputAttemptIdentifierMap() { } } + @Test + public void testShuffleHandlerDiskErrorOrdered() + throws Exception { + MapHost mapHost = new MapHost(HOST, PORT, 0, 1); + InputAttemptIdentifier inputAttemptIdentifier = new InputAttemptIdentifier(0, 0, "attempt"); + + FetcherOrderedGrouped fetcher = new FetcherOrderedGrouped(null, null, null, null, null, false, + 0, null, new TezConfiguration(), null, false, HOST, PORT, "src vertex", mapHost, + ioErrsCounter, wrongLengthErrsCounter, badIdErrsCounter, wrongMapErrsCounter, + connectionErrsCounter, wrongReduceErrsCounter, APP_ID, DAG_ID, false, false, true, false); + fetcher.remaining = new HashMap(); + + ShuffleHeader header = + new ShuffleHeader(ShuffleHandlerError.DISK_ERROR_EXCEPTION.toString(), -1, -1, -1); + DataInputStream input = RuntimeTestUtils.shuffleHeaderToDataInput(header); + + // copyMapOutput is used for remote fetch, this time it returns a fetch failure, which is fatal + // and should be treated as a local fetch failure + InputAttemptFetchFailure[] failures = + fetcher.copyMapOutput(mapHost, input, inputAttemptIdentifier); + + Assert.assertEquals(1, failures.length); + Assert.assertTrue(failures[0].isDiskErrorAtSource()); + Assert.assertFalse(failures[0].isLocalFetch()); + } + private RawLocalFileSystem getRawFs(Configuration conf) { try { return (RawLocalFileSystem) FileSystem.getLocal(conf).getRaw(); diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestShuffleScheduler.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestShuffleScheduler.java index fabfa270d7..b89ffb0ce9 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestShuffleScheduler.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestShuffleScheduler.java @@ -55,6 +55,7 @@ import org.apache.tez.runtime.library.api.TezRuntimeConfiguration; import org.apache.tez.runtime.library.common.CompositeInputAttemptIdentifier; import org.apache.tez.runtime.library.common.InputAttemptIdentifier; +import org.apache.tez.runtime.library.common.shuffle.InputAttemptFetchFailure; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -248,8 +249,8 @@ public void _testReducerHealth_1(Configuration conf) throws IOException { for (int i = 100; i < 199; i++) { InputAttemptIdentifier inputAttemptIdentifier = new InputAttemptIdentifier(i, 0, "attempt_"); - scheduler.copyFailed(inputAttemptIdentifier, new MapHost("host" + (i % totalProducerNodes), - 10000, i, 1), false, true, false); + scheduler.copyFailed(InputAttemptFetchFailure.fromAttempt(inputAttemptIdentifier), + new MapHost("host" + (i % totalProducerNodes), 10000, i, 1), false, true); } @@ -257,9 +258,8 @@ public void _testReducerHealth_1(Configuration conf) throws IOException { new InputAttemptIdentifier(200, 0, "attempt_"); //Should fail here and report exception as reducer is not healthy - scheduler.copyFailed(inputAttemptIdentifier, new MapHost("host" + (200 % - totalProducerNodes), - 10000, 200, 1), false, true, false); + scheduler.copyFailed(InputAttemptFetchFailure.fromAttempt(inputAttemptIdentifier), + new MapHost("host" + (200 % totalProducerNodes), 10000, 200, 1), false, true); int minFailurePerHost = conf.getInt( TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_MIN_FAILURES_PER_HOST, @@ -330,8 +330,8 @@ public void testReducerHealth_2() throws IOException, InterruptedException { for (int i = 190; i < 200; i++) { InputAttemptIdentifier inputAttemptIdentifier = new InputAttemptIdentifier(i, 0, "attempt_"); - scheduler.copyFailed(inputAttemptIdentifier, new MapHost("host" + (i % totalProducerNodes), - 10000, i, 1), false, true, false); + scheduler.copyFailed(InputAttemptFetchFailure.fromAttempt(inputAttemptIdentifier), + new MapHost("host" + (i % totalProducerNodes), 10000, i, 1), false, true); } //Shuffle has not stalled. so no issues. @@ -342,9 +342,8 @@ public void testReducerHealth_2() throws IOException, InterruptedException { InputAttemptIdentifier inputAttemptIdentifier = new InputAttemptIdentifier(190, 0, "attempt_"); - scheduler.copyFailed(inputAttemptIdentifier, new MapHost("host" + - (190 % totalProducerNodes), - 10000, 190, 1), false, true, false); + scheduler.copyFailed(InputAttemptFetchFailure.fromAttempt(inputAttemptIdentifier), + new MapHost("host" + (190 % totalProducerNodes), 10000, 190, 1), false, true); //Even when it is stalled, need (320 - 300 = 20) * 3 = 60 failures verify(scheduler.reporter, times(0)).reportException(any(Throwable.class)); @@ -355,16 +354,17 @@ public void testReducerHealth_2() throws IOException, InterruptedException { for (int i = 190; i < 200; i++) { inputAttemptIdentifier = new InputAttemptIdentifier(i, 0, "attempt_"); - scheduler.copyFailed(inputAttemptIdentifier, new MapHost("host" + (i % totalProducerNodes), - 10000, i, 1), false, true, false); - scheduler.copyFailed(inputAttemptIdentifier, new MapHost("host" + (i % totalProducerNodes), - 10000, i, 1), false, true, false); - scheduler.copyFailed(inputAttemptIdentifier, new MapHost("host" + (i % totalProducerNodes), - 10000, i, 1), false, true, false); - scheduler.copyFailed(inputAttemptIdentifier, new MapHost("host" + (i % totalProducerNodes), - 10000, i, 1), false, true, false); - scheduler.copyFailed(inputAttemptIdentifier, new MapHost("host" + (i % totalProducerNodes), - 10000, i, 1), false, true, false); + InputAttemptFetchFailure failure = InputAttemptFetchFailure.fromAttempt(inputAttemptIdentifier); + scheduler.copyFailed(failure, new MapHost("host" + (i % totalProducerNodes), + 10000, i, 1), false, true); + scheduler.copyFailed(failure, new MapHost("host" + (i % totalProducerNodes), + 10000, i, 1), false, true); + scheduler.copyFailed(failure, new MapHost("host" + (i % totalProducerNodes), + 10000, i, 1), false, true); + scheduler.copyFailed(failure, new MapHost("host" + (i % totalProducerNodes), + 10000, i, 1), false, true); + scheduler.copyFailed(failure, new MapHost("host" + (i % totalProducerNodes), + 10000, i, 1), false, true); } assertEquals(61, scheduler.failedShufflesSinceLastCompletion); @@ -376,12 +376,14 @@ public void testReducerHealth_2() throws IOException, InterruptedException { for (int i = 110; i < 120; i++) { inputAttemptIdentifier = new InputAttemptIdentifier(i, 0, "attempt_"); - scheduler.copyFailed(inputAttemptIdentifier, new MapHost("host" + (i % totalProducerNodes), - 10000, i, 1), false, true, false); - scheduler.copyFailed(inputAttemptIdentifier, new MapHost("host" + (i % totalProducerNodes), - 10000, i, 1), false, true, false); - scheduler.copyFailed(inputAttemptIdentifier, new MapHost("host" + (i % totalProducerNodes), - 10000, i, 1), false, true, false); + InputAttemptFetchFailure failure = + InputAttemptFetchFailure.fromAttempt(inputAttemptIdentifier); + scheduler.copyFailed(failure, new MapHost("host" + (i % totalProducerNodes), 10000, i, 1), + false, true); + scheduler.copyFailed(failure, new MapHost("host" + (i % totalProducerNodes), 10000, i, 1), + false, true); + scheduler.copyFailed(failure, new MapHost("host" + (i % totalProducerNodes), 10000, i, 1), + false, true); } // Should fail now due to fetcherHealthy. (stall has already happened and @@ -432,8 +434,8 @@ public void testReducerHealth_3() throws IOException { //1 fails (last fetch) InputAttemptIdentifier inputAttemptIdentifier = new InputAttemptIdentifier(319, 0, "attempt_"); - scheduler.copyFailed(inputAttemptIdentifier, new MapHost("host" + (319 % totalProducerNodes), - 10000, 319, 1), false, true, false); + scheduler.copyFailed(InputAttemptFetchFailure.fromAttempt(inputAttemptIdentifier), + new MapHost("host" + (319 % totalProducerNodes), 10000, 319, 1), false, true); //stall the shuffle scheduler.lastProgressTime = System.currentTimeMillis() - 1000000; @@ -441,15 +443,13 @@ public void testReducerHealth_3() throws IOException { assertEquals(scheduler.remainingMaps.get(), 1); //Retry for 3 more times - scheduler.copyFailed(inputAttemptIdentifier, new MapHost("host" + (319 % - totalProducerNodes), - 10000, 319, 1), false, true, false); - scheduler.copyFailed(inputAttemptIdentifier, new MapHost("host" + (319 % - totalProducerNodes), - 10000, 310, 1), false, true, false); - scheduler.copyFailed(inputAttemptIdentifier, new MapHost("host" + (319 % - totalProducerNodes), - 10000, 310, 1), false, true, false); + InputAttemptFetchFailure failure = InputAttemptFetchFailure.fromAttempt(inputAttemptIdentifier); + scheduler.copyFailed(failure, new MapHost("host" + (319 % totalProducerNodes), 10000, 319, 1), + false, true); + scheduler.copyFailed(failure, new MapHost("host" + (319 % totalProducerNodes), 10000, 310, 1), + false, true); + scheduler.copyFailed(failure, new MapHost("host" + (319 % totalProducerNodes), 10000, 310, 1), + false, true); // failedShufflesSinceLastCompletion has crossed the limits. Throw error verify(shuffle, times(0)).reportException(any(Throwable.class)); @@ -487,15 +487,15 @@ public void testReducerHealth_4() throws IOException { for (int i = 0; i < 64; i++) { InputAttemptIdentifier inputAttemptIdentifier = new InputAttemptIdentifier(i, 0, "attempt_"); + InputAttemptFetchFailure failure = + InputAttemptFetchFailure.fromAttempt(inputAttemptIdentifier); - scheduler.copyFailed(inputAttemptIdentifier, new MapHost("host" + (i % - totalProducerNodes), 10000, i, 1), false, true, false); - - scheduler.copyFailed(inputAttemptIdentifier, new MapHost("host" + (i % - totalProducerNodes), 10000, i, 1), false, true, false); - - scheduler.copyFailed(inputAttemptIdentifier, new MapHost("host" + (i % - totalProducerNodes), 10000, i, 1), false, true, false); + scheduler.copyFailed(failure, new MapHost("host" + (i % totalProducerNodes), 10000, i, 1), + false, true); + scheduler.copyFailed(failure, new MapHost("host" + (i % totalProducerNodes), 10000, i, 1), + false, true); + scheduler.copyFailed(failure, new MapHost("host" + (i % totalProducerNodes), 10000, i, 1), + false, true); MapOutput mapOutput = MapOutput .createMemoryMapOutput(inputAttemptIdentifier, mock(FetchedInputAllocatorOrderedGrouped.class), @@ -518,8 +518,8 @@ public void testReducerHealth_4() throws IOException { //1 fails (last fetch) InputAttemptIdentifier inputAttemptIdentifier = new InputAttemptIdentifier(319, 0, "attempt_"); - scheduler.copyFailed(inputAttemptIdentifier, new MapHost("host" + (319 % totalProducerNodes), - 10000, 319, 1), false, true, false); + scheduler.copyFailed(new InputAttemptFetchFailure(inputAttemptIdentifier), + new MapHost("host" + (319 % totalProducerNodes), 10000, 319, 1), false, true); //stall the shuffle (but within limits) scheduler.lastProgressTime = System.currentTimeMillis() - 100000; @@ -527,15 +527,13 @@ public void testReducerHealth_4() throws IOException { assertEquals(scheduler.remainingMaps.get(), 1); //Retry for 3 more times - scheduler.copyFailed(inputAttemptIdentifier, new MapHost("host" + (319 % - totalProducerNodes), - 10000, 319, 1), false, true, false); - scheduler.copyFailed(inputAttemptIdentifier, new MapHost("host" + (319 % - totalProducerNodes), - 10000, 319, 1), false, true, false); - scheduler.copyFailed(inputAttemptIdentifier, new MapHost("host" + (319 % - totalProducerNodes), - 10000, 319, 1), false, true, false); + InputAttemptFetchFailure failure = InputAttemptFetchFailure.fromAttempt(inputAttemptIdentifier); + scheduler.copyFailed(failure, new MapHost("host" + (319 % totalProducerNodes), 10000, 319, 1), + false, true); + scheduler.copyFailed(failure, new MapHost("host" + (319 % totalProducerNodes), 10000, 319, 1), + false, true); + scheduler.copyFailed(failure, new MapHost("host" + (319 % totalProducerNodes), 10000, 319, 1), + false, true); // failedShufflesSinceLastCompletion has crossed the limits. 20% of other nodes had failures as // well. However, it has failed only in one host. So this should proceed @@ -544,9 +542,8 @@ public void testReducerHealth_4() throws IOException { //stall the shuffle (but within limits) scheduler.lastProgressTime = System.currentTimeMillis() - 300000; - scheduler.copyFailed(inputAttemptIdentifier, new MapHost("host" + (319 % - totalProducerNodes), - 10000, 319, 1), false, true, false); + scheduler.copyFailed(InputAttemptFetchFailure.fromAttempt(inputAttemptIdentifier), + new MapHost("host" + (319 % totalProducerNodes), 10000, 319, 1), false, true); verify(shuffle, times(1)).reportException(any(Throwable.class)); } @@ -592,8 +589,9 @@ public void testReducerHealth_5() throws IOException { //1 fails (last fetch) InputAttemptIdentifier inputAttemptIdentifier = new InputAttemptIdentifier(318, 0, "attempt_"); - scheduler.copyFailed(inputAttemptIdentifier, new MapHost("host" + (318 % totalProducerNodes), - 10000, 318, 1), false, true, false); + InputAttemptFetchFailure failure = new InputAttemptFetchFailure(inputAttemptIdentifier); + scheduler.copyFailed(failure, new MapHost("host" + (318 % totalProducerNodes), + 10000, 318, 1), false, true); //stall the shuffle scheduler.lastProgressTime = System.currentTimeMillis() - 1000000; @@ -601,15 +599,12 @@ public void testReducerHealth_5() throws IOException { assertEquals(scheduler.remainingMaps.get(), 1); //Retry for 3 more times - scheduler.copyFailed(inputAttemptIdentifier, new MapHost("host" + (318 % - totalProducerNodes), - 10000, 318, 1), false, true, false); - scheduler.copyFailed(inputAttemptIdentifier, new MapHost("host" + (318 % - totalProducerNodes), - 10000, 318, 1), false, true, false); - scheduler.copyFailed(inputAttemptIdentifier, new MapHost("host" + (318 % - totalProducerNodes), - 10000, 318, 1), false, true, false); + scheduler.copyFailed(failure, new MapHost("host" + (318 % totalProducerNodes), 10000, 318, 1), + false, true); + scheduler.copyFailed(failure, new MapHost("host" + (318 % totalProducerNodes), 10000, 318, 1), + false, true); + scheduler.copyFailed(failure, new MapHost("host" + (318 % totalProducerNodes), 10000, 318, 1), + false, true); //Shuffle has not received the events completely. So do not bail out yet. verify(shuffle, times(0)).reportException(any(Throwable.class)); @@ -672,8 +667,8 @@ public void _testReducerHealth_6(Configuration conf) throws IOException { for (int i = 10; i < 15; i++) { InputAttemptIdentifier inputAttemptIdentifier = new InputAttemptIdentifier(i, 0, "attempt_"); - scheduler.copyFailed(inputAttemptIdentifier, new MapHost("host" + (i % totalProducerNodes), - 10000, i, 1), false, true, false); + scheduler.copyFailed(InputAttemptFetchFailure.fromAttempt(inputAttemptIdentifier), + new MapHost("host" + (i % totalProducerNodes), 10000, i, 1), false, true); } assertTrue(scheduler.failureCounts.size() >= 5); @@ -686,10 +681,10 @@ public void _testReducerHealth_6(Configuration conf) throws IOException { for (int i = 10; i < 15; i++) { InputAttemptIdentifier inputAttemptIdentifier = new InputAttemptIdentifier(i, 0, "attempt_"); - scheduler.copyFailed(inputAttemptIdentifier, new MapHost("host" + (i % totalProducerNodes), - 10000, i, 1), false, true, false); - scheduler.copyFailed(inputAttemptIdentifier, new MapHost("host" + (i % totalProducerNodes), - 10000, i, 1), false, true, false); + scheduler.copyFailed(InputAttemptFetchFailure.fromAttempt(inputAttemptIdentifier), + new MapHost("host" + (i % totalProducerNodes), 10000, i, 1), false, true); + scheduler.copyFailed(InputAttemptFetchFailure.fromAttempt(inputAttemptIdentifier), + new MapHost("host" + (i % totalProducerNodes), 10000, i, 1), false, true); } boolean checkFailedFetchSinceLastCompletion = conf.getBoolean @@ -749,18 +744,15 @@ public void testReducerHealth_7() throws IOException { for (int i = 100; i < 199; i++) { InputAttemptIdentifier inputAttemptIdentifier = new InputAttemptIdentifier(i, 0, "attempt_"); - scheduler.copyFailed(inputAttemptIdentifier, - new MapHost("host" + (i % totalProducerNodes), 10000, i, 1), - false, true, false); - scheduler.copyFailed(inputAttemptIdentifier, - new MapHost("host" + (i % totalProducerNodes), 10000, i, 1), - false, true, false); - scheduler.copyFailed(inputAttemptIdentifier, - new MapHost("host" + (i % totalProducerNodes), 10000, i, 1), - false, true, false); - scheduler.copyFailed(inputAttemptIdentifier, - new MapHost("host" + (i % totalProducerNodes), 10000, i, 1), - false, true, false); + InputAttemptFetchFailure failure = InputAttemptFetchFailure.fromAttempt(inputAttemptIdentifier); + scheduler.copyFailed(failure, new MapHost("host" + (i % totalProducerNodes), 10000, i, 1), + false, true); + scheduler.copyFailed(failure, new MapHost("host" + (i % totalProducerNodes), 10000, i, 1), + false, true); + scheduler.copyFailed(failure, new MapHost("host" + (i % totalProducerNodes), 10000, i, 1), + false, true); + scheduler.copyFailed(failure, new MapHost("host" + (i % totalProducerNodes), 10000, i, 1), + false, true); } verify(shuffle, atLeast(1)).reportException(any(Throwable.class)); @@ -799,7 +791,8 @@ public void testPenalty() throws IOException, InterruptedException { MapHost mapHost = scheduler.pendingHosts.iterator().next(); //Fails to pull from host0. host0 should be added to penalties - scheduler.copyFailed(inputAttemptIdentifier, mapHost, false, true, false); + scheduler.copyFailed(InputAttemptFetchFailure.fromAttempt(inputAttemptIdentifier), mapHost, + false, true); //Should not get host, as it is added to penalty loop MapHost host = scheduler.getHost(); @@ -993,7 +986,8 @@ public Void call() throws Exception { } for (int i = 0; i < 10; i++) { - scheduler.copyFailed(identifiers[0], mapHosts[0], false, false, false); + scheduler.copyFailed(InputAttemptFetchFailure.fromAttempt(identifiers[0]), mapHosts[0], false, + false); } ShuffleScheduler.Penalty[] penaltyArray = new ShuffleScheduler.Penalty[scheduler.getPenalties().size()]; scheduler.getPenalties().toArray(penaltyArray); diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/testutils/RuntimeTestUtils.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/testutils/RuntimeTestUtils.java new file mode 100644 index 0000000000..0885178ee5 --- /dev/null +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/testutils/RuntimeTestUtils.java @@ -0,0 +1,44 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.tez.runtime.library.testutils; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.io.InputStream; + +import org.apache.tez.runtime.library.common.shuffle.orderedgrouped.ShuffleHeader; + +public final class RuntimeTestUtils { + + private RuntimeTestUtils() { + } + + public static DataInputStream shuffleHeaderToDataInput(ShuffleHeader header) throws IOException { + ByteArrayOutputStream byteOutput = new ByteArrayOutputStream(1000); + DataOutputStream output = new DataOutputStream(byteOutput); + header.write(output); + + InputStream inputStream = new ByteArrayInputStream(byteOutput.toByteArray()); + DataInputStream input = new DataInputStream(inputStream); + + return input; + } +} From d0f498781e9675e88138db49fd8e9e140cc07f0c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?L=C3=A1szl=C3=B3=20Bodor?= Date: Tue, 6 Oct 2020 08:56:57 +0200 Subject: [PATCH 004/137] =?UTF-8?q?TEZ-4234:=20Compressor=20can=20cause=20?= =?UTF-8?q?IllegalArgumentException=20in=20Buffer.limit=20where=20limit=20?= =?UTF-8?q?exceeds=20capacity=20(L=C3=A1szl=C3=B3=20Bodor=20reviewed=20by?= =?UTF-8?q?=20Rajesh=20Balamohan,=20Jonathan=20Turner=20Eagles)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Laszlo Bodor --- .../hadoop/TestConfigTranslationMRToTez.java | 1 - .../runtime/library/common/ConfigUtils.java | 23 ---- .../library/common/TezRuntimeUtils.java | 18 ++- .../shuffle/orderedgrouped/Shuffle.java | 15 +-- .../common/sort/impl/ExternalSorter.java | 29 +--- .../library/common/sort/impl/IFile.java | 25 +--- .../BaseUnorderedPartitionedKVWriter.java | 17 +-- .../library/input/UnorderedKVInput.java | 14 +- .../tez/runtime/library/utils/CodecUtils.java | 127 ++++++++++++++++++ .../common/shuffle/TestShuffleUtils.java | 8 +- .../library/common/sort/impl/TestIFile.java | 59 +++++++- 11 files changed, 217 insertions(+), 119 deletions(-) create mode 100644 tez-runtime-library/src/main/java/org/apache/tez/runtime/library/utils/CodecUtils.java diff --git a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/hadoop/TestConfigTranslationMRToTez.java b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/hadoop/TestConfigTranslationMRToTez.java index deab64feef..df68c8dff8 100644 --- a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/hadoop/TestConfigTranslationMRToTez.java +++ b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/hadoop/TestConfigTranslationMRToTez.java @@ -70,6 +70,5 @@ public void testMRToTezKeyTranslation() { assertEquals(LongWritable.class.getName(), ConfigUtils .getIntermediateInputValueClass(confVertex1).getName()); assertTrue(ConfigUtils.shouldCompressIntermediateOutput(confVertex1)); - assertTrue(ConfigUtils.isIntermediateInputCompressed(confVertex1)); } } diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/ConfigUtils.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/ConfigUtils.java index 76d3dff868..f83fdc99d7 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/ConfigUtils.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/ConfigUtils.java @@ -56,24 +56,6 @@ public static Class getIntermediateOutputCompressorC } return codecClass; } - - public static Class getIntermediateInputCompressorClass( - Configuration conf, Class defaultValue) { - Class codecClass = defaultValue; - String name = conf - .get(TezRuntimeConfiguration.TEZ_RUNTIME_COMPRESS_CODEC); - if (name != null) { - try { - codecClass = conf.getClassByName(name).asSubclass( - CompressionCodec.class); - } catch (ClassNotFoundException e) { - throw new IllegalArgumentException("Compression codec " + name - + " was not found.", e); - } - } - return codecClass; - } - // TODO Move defaults over to a constants file. @@ -82,11 +64,6 @@ public static boolean shouldCompressIntermediateOutput(Configuration conf) { TezRuntimeConfiguration.TEZ_RUNTIME_COMPRESS, false); } - public static boolean isIntermediateInputCompressed(Configuration conf) { - return conf.getBoolean( - TezRuntimeConfiguration.TEZ_RUNTIME_COMPRESS, false); - } - public static Class getIntermediateOutputValueClass(Configuration conf) { Class retv = (Class) conf.getClass( TezRuntimeConfiguration.TEZ_RUNTIME_VALUE_CLASS, null, diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/TezRuntimeUtils.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/TezRuntimeUtils.java index 8be8fa20d1..daeafbc6fe 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/TezRuntimeUtils.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/TezRuntimeUtils.java @@ -265,16 +265,22 @@ public static int deserializeShuffleProviderMetaData(ByteBuffer meta) } public static String getBufferSizeProperty(CompressionCodec codec) { - switch (codec.getClass().getSimpleName().toString()) { - case "DefaultCodec": + return getBufferSizeProperty(codec.getClass().getName()); + } + + public static String getBufferSizeProperty(String className) { + switch (className) { + case "org.apache.hadoop.io.compress.DefaultCodec": return "io.file.buffer.size"; - case "SnappyCodec": + case "org.apache.hadoop.io.compress.SnappyCodec": return CommonConfigurationKeys.IO_COMPRESSION_CODEC_SNAPPY_BUFFERSIZE_KEY; - case "ZStandardCodec": + case "org.apache.hadoop.io.compress.ZStandardCodec": return CommonConfigurationKeys.IO_COMPRESSION_CODEC_ZSTD_BUFFER_SIZE_KEY; - case "LzoCodec": + case "org.apache.hadoop.io.compress.LzoCodec": + return CommonConfigurationKeys.IO_COMPRESSION_CODEC_LZO_BUFFERSIZE_KEY; + case "com.hadoop.compression.lzo.LzoCodec": return CommonConfigurationKeys.IO_COMPRESSION_CODEC_LZO_BUFFERSIZE_KEY; - case "Lz4Codec": + case "org.apache.hadoop.io.compress.Lz4Codec": return CommonConfigurationKeys.IO_COMPRESSION_CODEC_LZ4_BUFFERSIZE_KEY; default: return null; diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/Shuffle.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/Shuffle.java index 38f079a20b..db5ef734de 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/Shuffle.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/Shuffle.java @@ -39,8 +39,6 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.LocalDirAllocator; import org.apache.hadoop.io.compress.CompressionCodec; -import org.apache.hadoop.io.compress.DefaultCodec; -import org.apache.hadoop.util.ReflectionUtils; import org.apache.tez.common.CallableWithNdc; import org.apache.tez.common.GuavaShim; import org.apache.tez.common.TezRuntimeFrameworkConfigs; @@ -51,12 +49,11 @@ import org.apache.tez.runtime.api.Event; import org.apache.tez.runtime.api.InputContext; import org.apache.tez.runtime.library.api.TezRuntimeConfiguration; -import org.apache.tez.runtime.library.common.ConfigUtils; import org.apache.tez.runtime.library.common.TezRuntimeUtils; import org.apache.tez.runtime.library.common.combine.Combiner; import org.apache.tez.runtime.library.common.sort.impl.TezRawKeyValueIterator; import org.apache.tez.runtime.library.exceptions.InputAlreadyClosedException; - +import org.apache.tez.runtime.library.utils.CodecUtils; import org.apache.tez.common.Preconditions; import com.google.common.util.concurrent.FutureCallback; import com.google.common.util.concurrent.Futures; @@ -114,16 +111,8 @@ public Shuffle(InputContext inputContext, Configuration conf, int numInputs, this.srcNameTrimmed = TezUtilsInternal.cleanVertexName(inputContext.getSourceVertexName()); + this.codec = CodecUtils.getCodec(conf); - if (ConfigUtils.isIntermediateInputCompressed(conf)) { - Class codecClass = - ConfigUtils.getIntermediateInputCompressorClass(conf, DefaultCodec.class); - codec = ReflectionUtils.newInstance(codecClass, conf); - // Work around needed for HADOOP-12191. Avoids the native initialization synchronization race - codec.getDecompressorType(); - } else { - codec = null; - } this.ifileReadAhead = conf.getBoolean( TezRuntimeConfiguration.TEZ_RUNTIME_IFILE_READAHEAD, TezRuntimeConfiguration.TEZ_RUNTIME_IFILE_READAHEAD_DEFAULT); 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 194e899fea..3ff74f72bb 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 @@ -42,8 +42,6 @@ import org.apache.hadoop.fs.RawLocalFileSystem; import org.apache.hadoop.io.RawComparator; import org.apache.hadoop.io.compress.CompressionCodec; -import org.apache.hadoop.io.compress.Compressor; -import org.apache.hadoop.io.compress.DefaultCodec; import org.apache.hadoop.io.serializer.Serializer; import org.apache.hadoop.util.IndexedSorter; import org.apache.hadoop.util.Progressable; @@ -63,7 +61,7 @@ import org.apache.tez.runtime.library.common.shuffle.orderedgrouped.ShuffleHeader; import org.apache.tez.runtime.library.common.sort.impl.IFile.Writer; import org.apache.tez.runtime.library.common.task.local.output.TezTaskOutput; - +import org.apache.tez.runtime.library.utils.CodecUtils; import org.apache.tez.common.Preconditions; @SuppressWarnings({"rawtypes"}) @@ -224,30 +222,7 @@ public ExternalSorter(OutputContext outputContext, Configuration conf, int numOu numShuffleChunks = outputContext.getCounters().findCounter(TaskCounter.SHUFFLE_CHUNK_COUNT); // compression - if (ConfigUtils.shouldCompressIntermediateOutput(this.conf)) { - Class codecClass = - ConfigUtils.getIntermediateOutputCompressorClass(this.conf, DefaultCodec.class); - codec = ReflectionUtils.newInstance(codecClass, this.conf); - - if (codec != null) { - Class compressorType = null; - Throwable cause = null; - try { - compressorType = codec.getCompressorType(); - } catch (RuntimeException e) { - cause = e; - } - if (compressorType == null) { - String errMsg = - String.format("Unable to get CompressorType for codec (%s). This is most" + - " likely due to missing native libraries for the codec.", - conf.get(TezRuntimeConfiguration.TEZ_RUNTIME_COMPRESS_CODEC)); - throw new IOException(errMsg, cause); - } - } - } else { - codec = null; - } + this.codec = CodecUtils.getCodec(conf); this.ifileReadAhead = this.conf.getBoolean( TezRuntimeConfiguration.TEZ_RUNTIME_IFILE_READAHEAD, diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFile.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFile.java index 6aa44e206a..1b2aefff41 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFile.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFile.java @@ -30,20 +30,18 @@ import com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.io.BoundedByteArrayOutputStream; -import org.apache.tez.runtime.library.common.TezRuntimeUtils; import org.apache.tez.runtime.library.common.task.local.output.TezTaskOutput; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; -import org.apache.hadoop.conf.Configurable; -import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.DataInputBuffer; import org.apache.hadoop.io.DataOutputBuffer; import org.apache.tez.runtime.library.utils.BufferUtils; +import org.apache.tez.runtime.library.utils.CodecUtils; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.WritableUtils; import org.apache.hadoop.io.compress.CodecPool; @@ -823,7 +821,8 @@ public static void readToMemory(byte[] buffer, InputStream in, int compressedLen decompressor = CodecPool.getDecompressor(codec); if (decompressor != null) { decompressor.reset(); - in = getDecompressedInputStreamWithBufferSize(codec, checksumIn, decompressor, compressedLength); + in = CodecUtils.getDecompressedInputStreamWithBufferSize(codec, checksumIn, decompressor, + compressedLength); } else { LOG.warn("Could not obtain decompressor from CodecPool"); in = checksumIn; @@ -859,24 +858,6 @@ public static void readToMemory(byte[] buffer, InputStream in, int compressedLen } } - private static InputStream getDecompressedInputStreamWithBufferSize(CompressionCodec codec, - IFileInputStream checksumIn, Decompressor decompressor, int compressedLength) - throws IOException { - String bufferSizeProp = TezRuntimeUtils.getBufferSizeProperty(codec); - - if (bufferSizeProp != null) { - Configurable configurableCodec = (Configurable) codec; - Configuration conf = configurableCodec.getConf(); - - int bufSize = Math.min(compressedLength, DEFAULT_BUFFER_SIZE); - LOG.trace("buffer size was set according to min(compressedLength, {}): {}={}", - DEFAULT_BUFFER_SIZE, bufferSizeProp, bufSize); - conf.setInt(bufferSizeProp, bufSize); - } - - return codec.createInputStream(checksumIn, decompressor); - } - /** * Read entire IFile content to disk. * diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/writers/BaseUnorderedPartitionedKVWriter.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/writers/BaseUnorderedPartitionedKVWriter.java index ecc9e03a5d..adea49fe80 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/writers/BaseUnorderedPartitionedKVWriter.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/writers/BaseUnorderedPartitionedKVWriter.java @@ -29,10 +29,8 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.RawLocalFileSystem; import org.apache.hadoop.io.compress.CompressionCodec; -import org.apache.hadoop.io.compress.DefaultCodec; import org.apache.hadoop.io.serializer.SerializationFactory; import org.apache.hadoop.io.serializer.Serializer; -import org.apache.hadoop.util.ReflectionUtils; import org.apache.tez.common.counters.TaskCounter; import org.apache.tez.common.counters.TezCounter; import org.apache.tez.runtime.api.Event; @@ -43,6 +41,7 @@ import org.apache.tez.runtime.library.common.ConfigUtils; import org.apache.tez.runtime.library.common.TezRuntimeUtils; import org.apache.tez.runtime.library.common.task.local.output.TezTaskOutput; +import org.apache.tez.runtime.library.utils.CodecUtils; @SuppressWarnings("rawtypes") public abstract class BaseUnorderedPartitionedKVWriter extends KeyValuesWriter { @@ -141,16 +140,14 @@ public BaseUnorderedPartitionedKVWriter(OutputContext outputContext, Configurati additionalSpillBytesReadCounter = outputContext.getCounters().findCounter(TaskCounter.ADDITIONAL_SPILLS_BYTES_READ); numAdditionalSpillsCounter = outputContext.getCounters().findCounter(TaskCounter.ADDITIONAL_SPILL_COUNT); dataViaEventSize = outputContext.getCounters().findCounter(TaskCounter.DATA_BYTES_VIA_EVENT); - + // compression - if (ConfigUtils.shouldCompressIntermediateOutput(this.conf)) { - Class codecClass = - ConfigUtils.getIntermediateOutputCompressorClass(this.conf, DefaultCodec.class); - codec = ReflectionUtils.newInstance(codecClass, this.conf); - } else { - codec = null; + try { + this.codec = CodecUtils.getCodec(conf); + } catch (IOException e) { + throw new RuntimeException(e); } - + this.ifileReadAhead = this.conf.getBoolean( TezRuntimeConfiguration.TEZ_RUNTIME_IFILE_READAHEAD, TezRuntimeConfiguration.TEZ_RUNTIME_IFILE_READAHEAD_DEFAULT); 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 1db786995a..c67c405b43 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 @@ -35,8 +35,6 @@ import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.io.compress.CompressionCodec; -import org.apache.hadoop.io.compress.DefaultCodec; -import org.apache.hadoop.util.ReflectionUtils; import org.apache.tez.common.TezRuntimeFrameworkConfigs; import org.apache.tez.common.counters.TaskCounter; import org.apache.tez.common.counters.TezCounter; @@ -46,14 +44,13 @@ import org.apache.tez.runtime.api.InputContext; import org.apache.tez.runtime.library.api.KeyValueReader; import org.apache.tez.runtime.library.api.TezRuntimeConfiguration; -import org.apache.tez.runtime.library.common.ConfigUtils; import org.apache.tez.runtime.library.common.MemoryUpdateCallbackHandler; import org.apache.tez.runtime.library.common.readers.UnorderedKVReader; import org.apache.tez.runtime.library.common.shuffle.ShuffleEventHandler; import org.apache.tez.runtime.library.common.shuffle.impl.ShuffleInputEventHandlerImpl; import org.apache.tez.runtime.library.common.shuffle.impl.ShuffleManager; import org.apache.tez.runtime.library.common.shuffle.impl.SimpleFetchedInputAllocator; - +import org.apache.tez.runtime.library.utils.CodecUtils; import org.apache.tez.common.Preconditions; /** @@ -114,14 +111,7 @@ public synchronized void start() throws IOException { if (!isStarted.get()) { ////// Initial configuration memoryUpdateCallbackHandler.validateUpdateReceived(); - CompressionCodec codec; - if (ConfigUtils.isIntermediateInputCompressed(conf)) { - Class codecClass = ConfigUtils - .getIntermediateInputCompressorClass(conf, DefaultCodec.class); - codec = ReflectionUtils.newInstance(codecClass, conf); - } else { - codec = null; - } + CompressionCodec codec = CodecUtils.getCodec(conf); boolean compositeFetch = ShuffleUtils.isTezShuffleHandler(conf); boolean ifileReadAhead = conf.getBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_IFILE_READAHEAD, diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/utils/CodecUtils.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/utils/CodecUtils.java new file mode 100644 index 0000000000..99d22c5dcc --- /dev/null +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/utils/CodecUtils.java @@ -0,0 +1,127 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.utils; + +import java.io.IOException; +import java.io.InputStream; + +import org.apache.hadoop.conf.Configurable; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.io.compress.CompressionCodec; +import org.apache.hadoop.io.compress.CompressionInputStream; +import org.apache.hadoop.io.compress.Compressor; +import org.apache.hadoop.io.compress.Decompressor; +import org.apache.hadoop.io.compress.DefaultCodec; +import org.apache.hadoop.util.ReflectionUtils; +import org.apache.tez.runtime.library.api.TezRuntimeConfiguration; +import org.apache.tez.runtime.library.common.ConfigUtils; +import org.apache.tez.runtime.library.common.TezRuntimeUtils; +import org.apache.tez.runtime.library.common.sort.impl.IFile; +import org.apache.tez.runtime.library.common.sort.impl.IFileInputStream; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public final class CodecUtils { + + private static final Logger LOG = LoggerFactory.getLogger(IFile.class); + private static final int DEFAULT_BUFFER_SIZE = 128 * 1024; + + private CodecUtils() { + } + + public static CompressionCodec getCodec(Configuration conf) throws IOException { + if (ConfigUtils.shouldCompressIntermediateOutput(conf)) { + Class codecClass = + ConfigUtils.getIntermediateOutputCompressorClass(conf, DefaultCodec.class); + CompressionCodec codec = ReflectionUtils.newInstance(codecClass, conf); + + if (codec != null) { + Class compressorType = null; + Throwable cause = null; + try { + compressorType = codec.getCompressorType(); + } catch (RuntimeException e) { + cause = e; + } + if (compressorType == null) { + String errMsg = String.format( + "Unable to get CompressorType for codec (%s). This is most" + + " likely due to missing native libraries for the codec.", + conf.get(TezRuntimeConfiguration.TEZ_RUNTIME_COMPRESS_CODEC)); + throw new IOException(errMsg, cause); + } + } + return codec; + } else { + return null; + } + } + + public static InputStream getDecompressedInputStreamWithBufferSize(CompressionCodec codec, + IFileInputStream checksumIn, Decompressor decompressor, int compressedLength) + throws IOException { + String bufferSizeProp = TezRuntimeUtils.getBufferSizeProperty(codec); + Configurable configurableCodec = (Configurable) codec; + int originalSize = configurableCodec.getConf().getInt(bufferSizeProp, DEFAULT_BUFFER_SIZE); + + CompressionInputStream in = null; + + if (bufferSizeProp != null) { + Configuration conf = configurableCodec.getConf(); + int newBufSize = Math.min(compressedLength, DEFAULT_BUFFER_SIZE); + LOG.trace("buffer size was set according to min(compressedLength, {}): {}={}", + DEFAULT_BUFFER_SIZE, bufferSizeProp, newBufSize); + + synchronized (codec) { + conf.setInt(bufferSizeProp, newBufSize); + + in = codec.createInputStream(checksumIn, decompressor); + /* + * We would better reset the original buffer size into the codec. Basically the buffer size + * is used at 2 places. + * + * 1. It can tell the inputstream/outputstream buffersize (which is created by + * codec.createInputStream/codec.createOutputStream). This is something which might and + * should be optimized in config, as inputstreams instantiate and use their own buffer and + * won't reuse buffers from previous streams (TEZ-4135). + * + * 2. The same buffersize is used when a codec creates a new Compressor/Decompressor. The + * fundamental difference is that Compressor/Decompressor instances are expensive and reused + * by hadoop's CodecPool. Here is a hidden mismatch, which can happen when a codec is + * created with a small buffersize config. Once it creates a Compressor/Decompressor + * instance from its config field, the reused Compressor/Decompressor instance will be + * reused later, even when application handles large amount of data. This way we can end up + * in large stream buffers + small compressor/decompressor buffers, which can be suboptimal, + * moreover, it can lead to strange errors, when a compressed output exceeds the size of the + * buffer (TEZ-4234). + * + * An interesting outcome is that - as the codec buffersize config affects both + * compressor(output) and decompressor(input) paths - an altered codec config can cause the + * issues above for Compressor instances as well, even when we tried to leverage from + * smaller buffer size only on decompression paths. + */ + configurableCodec.getConf().setInt(bufferSizeProp, originalSize); + } + } else { + in = codec.createInputStream(checksumIn, decompressor); + } + + return in; + } +} \ No newline at end of file diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/TestShuffleUtils.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/TestShuffleUtils.java index 520dec7e70..446801ac2f 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/TestShuffleUtils.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/TestShuffleUtils.java @@ -290,6 +290,7 @@ public void testInternalErrorTranslation() throws Exception { .thenThrow(new InternalError(codecErrorMsg)); Decompressor mockDecoder = mock(Decompressor.class); CompressionCodec mockCodec = mock(ConfigurableCodecForTest.class); + when(((ConfigurableCodecForTest) mockCodec).getConf()).thenReturn(mock(Configuration.class)); when(mockCodec.createDecompressor()).thenReturn(mockDecoder); when(mockCodec.createInputStream(any(InputStream.class), any(Decompressor.class))) .thenReturn(mockCodecStream); @@ -312,6 +313,7 @@ public void testExceptionTranslation() throws Exception { .thenThrow(new IllegalArgumentException(codecErrorMsg)); Decompressor mockDecoder = mock(Decompressor.class); CompressionCodec mockCodec = mock(ConfigurableCodecForTest.class); + when(((ConfigurableCodecForTest) mockCodec).getConf()).thenReturn(mock(Configuration.class)); when(mockCodec.createDecompressor()).thenReturn(mockDecoder); when(mockCodec.createInputStream(any(InputStream.class), any(Decompressor.class))) .thenReturn(mockCodecStream); @@ -327,7 +329,8 @@ public void testExceptionTranslation() throws Exception { CompressionInputStream mockCodecStream1 = mock(CompressionInputStream.class); when(mockCodecStream1.read(any(byte[].class), anyInt(), anyInt())) .thenThrow(new SocketTimeoutException(codecErrorMsg)); - CompressionCodec mockCodec1 = mock(CompressionCodec.class); + CompressionCodec mockCodec1 = mock(ConfigurableCodecForTest.class); + when(((ConfigurableCodecForTest) mockCodec1).getConf()).thenReturn(mock(Configuration.class)); when(mockCodec1.createDecompressor()).thenReturn(mockDecoder); when(mockCodec1.createInputStream(any(InputStream.class), any(Decompressor.class))) .thenReturn(mockCodecStream1); @@ -342,7 +345,8 @@ public void testExceptionTranslation() throws Exception { CompressionInputStream mockCodecStream2 = mock(CompressionInputStream.class); when(mockCodecStream2.read(any(byte[].class), anyInt(), anyInt())) .thenThrow(new InternalError(codecErrorMsg)); - CompressionCodec mockCodec2 = mock(CompressionCodec.class); + CompressionCodec mockCodec2 = mock(ConfigurableCodecForTest.class); + when(((ConfigurableCodecForTest) mockCodec2).getConf()).thenReturn(mock(Configuration.class)); when(mockCodec2.createDecompressor()).thenReturn(mockDecoder); when(mockCodec2.createInputStream(any(InputStream.class), any(Decompressor.class))) .thenReturn(mockCodecStream2); diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestIFile.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestIFile.java index c74496e46b..bf35955625 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestIFile.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestIFile.java @@ -50,9 +50,11 @@ import org.apache.hadoop.io.Text; import org.apache.hadoop.io.compress.CompressionCodec; import org.apache.hadoop.io.compress.CompressionCodecFactory; +import org.apache.hadoop.io.compress.lz4.Lz4Compressor; import org.apache.hadoop.io.serializer.Deserializer; import org.apache.hadoop.io.serializer.SerializationFactory; import org.apache.hadoop.io.serializer.WritableSerialization; +import org.apache.hadoop.util.NativeCodeLoader; import org.apache.tez.common.TezRuntimeFrameworkConfigs; import org.apache.tez.runtime.library.common.InputAttemptIdentifier; import org.apache.tez.runtime.library.common.TezRuntimeUtils; @@ -66,6 +68,7 @@ import org.apache.tez.runtime.library.utils.BufferUtils; import org.junit.After; import org.junit.Assert; +import org.junit.Assume; import org.junit.Before; import org.junit.Test; import org.slf4j.Logger; @@ -729,13 +732,16 @@ public void testReadToDisk() throws IOException { @Test public void testInMemoryBufferSize() throws IOException { + Configurable configurableCodec = (Configurable) codec; + int originalCodecBufferSize = + configurableCodec.getConf().getInt(TezRuntimeUtils.getBufferSizeProperty(codec), -1); + // for smaller amount of data, codec buffer should be sized according to compressed data length List data = KVDataGen.generateTestData(false, rnd.nextInt(100)); Writer writer = writeTestFile(false, false, data, codec); readAndVerifyData(writer.getRawLength(), writer.getCompressedLength(), data, codec); - Configurable configurableCodec = (Configurable) codec; - Assert.assertEquals(writer.getCompressedLength(), + Assert.assertEquals(originalCodecBufferSize, // original size is repaired configurableCodec.getConf().getInt(TezRuntimeUtils.getBufferSizeProperty(codec), 0)); // buffer size cannot grow infinitely with compressed data size @@ -743,10 +749,57 @@ public void testInMemoryBufferSize() throws IOException { writer = writeTestFile(false, false, data, codec); readAndVerifyData(writer.getRawLength(), writer.getCompressedLength(), data, codec); - Assert.assertEquals(128*1024, + Assert.assertEquals(originalCodecBufferSize, // original size is repaired configurableCodec.getConf().getInt(TezRuntimeUtils.getBufferSizeProperty(codec), 0)); } + @Test(expected = IllegalArgumentException.class) + public void testSmallDataCompression() throws IOException { + Assume.assumeTrue(NativeCodeLoader.isNativeCodeLoaded()); + + tryWriteFileWithBufferSize(17, "org.apache.hadoop.io.compress.Lz4Codec"); + tryWriteFileWithBufferSize(32, "org.apache.hadoop.io.compress.Lz4Codec"); + } + + private void tryWriteFileWithBufferSize(int bufferSize, String codecClassName) + throws IOException { + Configuration conf = new Configuration(); + + System.out.println("trying with buffer size: " + bufferSize); + conf.set(TezRuntimeUtils.getBufferSizeProperty(codecClassName), Integer.toString(bufferSize)); + CompressionCodecFactory codecFactory = new CompressionCodecFactory(conf); + CompressionCodec codecToTest = + codecFactory.getCodecByClassName(codecClassName); + List data = KVDataGen.generateTestDataOfKeySize(false, 1, 0); + writeTestFile(false, false, data, codecToTest); + } + + @Test(expected = IllegalArgumentException.class) + public void testLz4CompressedDataIsLargerThanOriginal() throws IOException { + Assume.assumeTrue(NativeCodeLoader.isNativeCodeLoaded()); + + // this one succeeds + byte[] buf = new byte[32]; + initBufWithNumbers(buf, 24, 45, 55, 49, 54, 55, 55, 54, 49, 48, 50, 55, 49, 56, 54, 48, 57, 48); + Lz4Compressor comp = new Lz4Compressor(32, false); + comp.setInput(buf, 0, 32); + comp.compress(buf, 0, 32); + + // adding 1 more element makes that fail + buf = new byte[32]; + initBufWithNumbers(buf, 24, 45, 55, 49, 54, 55, 55, 54, 49, 48, 50, 55, 49, 56, 54, 48, 57, 48, + 50); + comp = new Lz4Compressor(32, false); + comp.setInput(buf, 0, 32); + comp.compress(buf, 0, 32); + } + + private void initBufWithNumbers(byte[] buf, int... args) { + for (int i = 0; i < args.length; i++) { + buf[i] = (byte) args[i]; + } + } + /** * Test different options (RLE, repeat keys, compression) on reader/writer * From 6e9c1b2d605f3a4197a5185cab81cc66d13a1437 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?L=C3=A1szl=C3=B3=20Bodor?= Date: Tue, 6 Oct 2020 12:00:17 +0200 Subject: [PATCH 005/137] =?UTF-8?q?TEZ-4238:=20Check=20null=20mrReader=20i?= =?UTF-8?q?n=20MRInput.close=20(L=C3=A1szl=C3=B3=20Bodor=20reviewed=20by?= =?UTF-8?q?=20Hadoop=20QA,=20Jonathan=20Turner=20Eagles,=20Hadoop=20QA)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Laszlo Bodor --- .../apache/tez/mapreduce/input/MRInput.java | 5 ++++- .../tez/mapreduce/input/TestMRInput.java | 22 +++++++++++++++++++ 2 files changed, 26 insertions(+), 1 deletion(-) diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/MRInput.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/MRInput.java index 5c8ad4ea68..891249b63e 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/MRInput.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/MRInput.java @@ -592,7 +592,10 @@ public void handleEvents(List inputEvents) throws Exception { @Override public List close() throws IOException { - mrReader.close(); + if (mrReader != null) { + mrReader.close(); + mrReader = null; + } long inputRecords = getContext().getCounters() .findCounter(TaskCounter.INPUT_RECORDS_PROCESSED).getValue(); getContext().getStatisticsReporter().reportItemsProcessed(inputRecords); diff --git a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/input/TestMRInput.java b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/input/TestMRInput.java index 5ca5c26619..844ea51cf1 100644 --- a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/input/TestMRInput.java +++ b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/input/TestMRInput.java @@ -47,6 +47,7 @@ import org.apache.tez.mapreduce.protos.MRRuntimeProtos; import org.apache.tez.runtime.api.Event; import org.apache.tez.runtime.api.InputContext; +import org.apache.tez.runtime.api.InputStatisticsReporter; import org.apache.tez.runtime.api.events.InputDataInformationEvent; import org.junit.Test; @@ -186,6 +187,16 @@ public void testConfigMerge() throws Exception { assertEquals("payload-value", mergedConfig.get("payload-key")); } + @Test + public void testMRInputCloseWithUnintializedReader() throws IOException { + InputContext inputContext = mock(InputContext.class); + doReturn(new TezCounters()).when(inputContext).getCounters(); + doReturn(new InputStatisticsReporterImplForTest()).when(inputContext).getStatisticsReporter(); + + MRInput mrInput = new MRInput(inputContext, 0); + mrInput.close(); // shouldn't throw NPE + } + /** * Test class to verify */ @@ -276,4 +287,15 @@ public void readFields(DataInput in) throws IOException { } } + + public static class InputStatisticsReporterImplForTest implements InputStatisticsReporter { + + @Override + public synchronized void reportDataSize(long size) { + } + + @Override + public void reportItemsProcessed(long items) { + } + } } From 16a0050909ac5928a131ede8ffdb9a4d12c82cf7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?L=C3=A1szl=C3=B3=20Bodor?= Date: Tue, 13 Oct 2020 07:48:36 +0200 Subject: [PATCH 006/137] =?UTF-8?q?TEZ-4229:=20Improve=20TezLocalCacheMana?= =?UTF-8?q?ger=20to=20use=20configured=20root=20directory=20(L=C3=A1szl?= =?UTF-8?q?=C3=B3=20Bodor=20reviewed=20by=20Panagiotis=20Garefalakis,=20As?= =?UTF-8?q?hutosh=20Chauhan)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Laszlo Bodor --- .../apache/tez/dag/api/TezConfiguration.java | 9 ++++ .../app/launcher/TezLocalCacheManager.java | 10 ++++- .../launcher/TestTezLocalCacheManager.java | 41 +++++++++++++++++++ 3 files changed, 58 insertions(+), 2 deletions(-) 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 d5e5e73d45..2af08a9044 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 @@ -1748,6 +1748,15 @@ public TezConfiguration(boolean loadDefaults) { public static final boolean TEZ_LOCAL_MODE_DEFAULT = false; + /** + * String value. TezLocalCacheManager uses this folder as a root for temp and localized files. + */ + @ConfigurationScope(Scope.VERTEX) + @ConfigurationProperty + public static final String TEZ_LOCAL_CACHE_ROOT_FOLDER = TEZ_PREFIX + "local.cache.root.folder"; + + public static final String TEZ_LOCAL_CACHE_ROOT_FOLDER_DEFAULT = "."; + /** * Tez AM Inline Mode flag. Not valid till Tez-684 get checked-in */ diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/TezLocalCacheManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/TezLocalCacheManager.java index 9bcbb1530a..f4892ab81e 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/TezLocalCacheManager.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/TezLocalCacheManager.java @@ -26,6 +26,7 @@ import org.apache.hadoop.yarn.api.records.LocalResource; import org.apache.hadoop.yarn.api.records.LocalResourceType; import org.apache.hadoop.yarn.util.FSDownload; +import org.apache.tez.dag.api.TezConfiguration; import java.io.IOException; import java.nio.file.Files; @@ -63,7 +64,7 @@ public TezLocalCacheManager(Map resources, Configuration this.fileContext = FileContext.getLocalFSFileContext(); this.resources = resources; this.conf = conf; - this.tempDir = Files.createTempDirectory(Paths.get("."), "tez-local-cache"); + this.tempDir = Files.createTempDirectory(getLocalCacheRoot(), "tez-local-cache"); } /** @@ -72,7 +73,7 @@ public TezLocalCacheManager(Map resources, Configuration * @throws IOException when an error occurs in download or link */ public void localize() throws IOException { - String absPath = Paths.get(".").toAbsolutePath().normalize().toString(); + String absPath = getLocalCacheRoot().toAbsolutePath().normalize().toString(); Path cwd = fileContext.makeQualified(new Path(absPath)); ExecutorService threadPool = null; @@ -181,6 +182,11 @@ private boolean createSymlink(Path target, Path link) throws IOException { } } + private java.nio.file.Path getLocalCacheRoot() { + return Paths.get(conf.get(TezConfiguration.TEZ_LOCAL_CACHE_ROOT_FOLDER, + TezConfiguration.TEZ_LOCAL_CACHE_ROOT_FOLDER_DEFAULT)); + } + /** * Wrapper to keep track of download path and link path. */ diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/launcher/TestTezLocalCacheManager.java b/tez-dag/src/test/java/org/apache/tez/dag/app/launcher/TestTezLocalCacheManager.java index beca047349..5596dc8feb 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/launcher/TestTezLocalCacheManager.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/launcher/TestTezLocalCacheManager.java @@ -28,6 +28,7 @@ import org.apache.hadoop.yarn.factories.RecordFactory; import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; import org.apache.hadoop.yarn.util.ConverterUtils; +import org.apache.tez.dag.api.TezConfiguration; import org.junit.Assert; import org.junit.Test; @@ -110,4 +111,44 @@ private static LocalResource createFile(String content) throws IOException { ret.setTimestamp(fs.getFileStatus(p).getModificationTime()); return ret; } + + @Test + public void testLocalizeRootDirectory() throws URISyntaxException, IOException { + // default directory + Map resources = new HashMap<>(); + + LocalResource resourceOne = createFile("content-one"); + resources.put("file-one", resourceOne); + + TezLocalCacheManager manager = new TezLocalCacheManager(resources, new Configuration()); + + try { + Assert.assertFalse(Files.exists(Paths.get("./file-one"))); + manager.localize(); + Assert.assertTrue(Files.exists(Paths.get("./file-one"))); + + } finally { + manager.cleanup(); + Assert.assertFalse(Files.exists(Paths.get("./file-one"))); + } + + // configured directory + Configuration conf = new Configuration(); + conf.set(TezConfiguration.TEZ_LOCAL_CACHE_ROOT_FOLDER, "target"); + manager = new TezLocalCacheManager(resources, conf); + + try { + // files don't exist at all + Assert.assertFalse(Files.exists(Paths.get("./file-one"))); + Assert.assertFalse(Files.exists(Paths.get("./target/file-one"))); + manager.localize(); + // file appears only at configured location + Assert.assertFalse(Files.exists(Paths.get("./file-one"))); + Assert.assertTrue(Files.exists(Paths.get("./target/file-one"))); + + } finally { + manager.cleanup(); + Assert.assertFalse(Files.exists(Paths.get("./target/file-one"))); + } + } } From 970d46b5f8be72cc9abf1769f8c7c80307737fc5 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?L=C3=A1szl=C3=B3=20Bodor?= Date: Wed, 14 Oct 2020 17:24:57 +0200 Subject: [PATCH 007/137] =?UTF-8?q?TEZ-4070:=20SSLFactory=20not=20closed?= =?UTF-8?q?=20in=20DAGClientTimelineImpl=20caused=20native=20memory=20issu?= =?UTF-8?q?es=20(L=C3=A1szl=C3=B3=20Bodor=20reviewed=20by=20Jonathan=20Tur?= =?UTF-8?q?ner=20Eagles)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Laszlo Bodor --- tez-api/pom.xml | 5 ++ .../dag/api/client/DAGClientTimelineImpl.java | 3 ++ .../dag/api/client/TimelineReaderFactory.java | 36 ++++++++++--- .../tez/dag/api/client/rpc/TestDAGClient.java | 52 +++++++++++++++++++ 4 files changed, 88 insertions(+), 8 deletions(-) diff --git a/tez-api/pom.xml b/tez-api/pom.xml index 6659f4f9e4..95a7610ee4 100644 --- a/tez-api/pom.xml +++ b/tez-api/pom.xml @@ -110,6 +110,11 @@ mockito-all test + + org.bouncycastle + bcprov-jdk15on + test + 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 d34dbf008d..17d2386860 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 @@ -213,6 +213,9 @@ public void close() throws IOException { httpClient.destroy(); httpClient = null; } + if (timelineReaderStrategy != null) { + timelineReaderStrategy.close(); + } } private DAGStatusProto.Builder parseDagStatus(JSONObject jsonRoot, Set statusOptions) 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 index c0569dda98..40340cc44e 100644 --- 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 @@ -133,6 +133,7 @@ public static boolean isTimelineClientSupported() { public interface TimelineReaderStrategy { Client getHttpClient() throws IOException; + void close(); } /* @@ -142,6 +143,8 @@ private static class TimelineReaderTokenAuthenticatedStrategy implements Timelin private final Configuration conf; private final boolean useHttps; private final int connTimeout; + private ConnectionConfigurator connectionConfigurator; + private SSLFactory sslFactory; public TimelineReaderTokenAuthenticatedStrategy(final Configuration conf, final boolean useHttps, @@ -150,6 +153,7 @@ public TimelineReaderTokenAuthenticatedStrategy(final Configuration conf, this.conf = conf; this.useHttps = useHttps; this.connTimeout = connTimeout; + this.sslFactory = useHttps ? new SSLFactory(CLIENT, conf) : null; } @Override @@ -160,8 +164,8 @@ public Client getHttpClient() throws IOException { UserGroupInformation authUgi; String doAsUser; ClientConfig clientConfig = new DefaultClientConfig(JSONRootElementProvider.App.class); - ConnectionConfigurator connectionConfigurator = getNewConnectionConf(conf, useHttps, - connTimeout); + connectionConfigurator = getNewConnectionConf(conf, useHttps, + connTimeout, sslFactory); try { authenticator = getTokenAuthenticator(); @@ -238,6 +242,13 @@ public HttpURLConnection getHttpURLConnection(URL url) throws IOException { } } } + + @Override + public void close() { + if (sslFactory != null) { + sslFactory.destroy(); + } + } } /* @@ -247,11 +258,13 @@ public HttpURLConnection getHttpURLConnection(URL url) throws IOException { protected static class TimelineReaderPseudoAuthenticatedStrategy implements TimelineReaderStrategy { private final ConnectionConfigurator connectionConf; + private final SSLFactory sslFactory; public TimelineReaderPseudoAuthenticatedStrategy(final Configuration conf, final boolean useHttps, final int connTimeout) { - connectionConf = getNewConnectionConf(conf, useHttps, connTimeout); + sslFactory = useHttps ? new SSLFactory(CLIENT, conf) : null; + connectionConf = getNewConnectionConf(conf, useHttps, connTimeout, sslFactory); } @Override @@ -282,15 +295,23 @@ public HttpURLConnection getHttpURLConnection(URL url) throws IOException { return httpURLConnection; } } + + @Override + public void close() { + if (sslFactory != null) { + sslFactory.destroy(); + } + } } private static ConnectionConfigurator getNewConnectionConf(final Configuration conf, final boolean useHttps, - final int connTimeout) { + final int connTimeout, + final SSLFactory sslFactory) { ConnectionConfigurator connectionConf = null; if (useHttps) { try { - connectionConf = getNewSSLConnectionConf(conf, connTimeout); + connectionConf = getNewSSLConnectionConf(conf, connTimeout, sslFactory); } catch (IOException e) { if (LOG.isDebugEnabled()) { LOG.debug("Cannot load customized ssl related configuration." @@ -313,13 +334,12 @@ public HttpURLConnection configure(HttpURLConnection httpURLConnection) throws I } private static ConnectionConfigurator getNewSSLConnectionConf(final Configuration conf, - final int connTimeout) + final int connTimeout, + final SSLFactory sslFactory) throws IOException { - final SSLFactory sslFactory; final SSLSocketFactory sslSocketFactory; final HostnameVerifier hostnameVerifier; - sslFactory = new SSLFactory(CLIENT, conf); try { sslFactory.init(); sslSocketFactory = sslFactory.createSSLSocketFactory(); diff --git a/tez-api/src/test/java/org/apache/tez/dag/api/client/rpc/TestDAGClient.java b/tez-api/src/test/java/org/apache/tez/dag/api/client/rpc/TestDAGClient.java index 211baf257e..6a5e817d44 100644 --- a/tez-api/src/test/java/org/apache/tez/dag/api/client/rpc/TestDAGClient.java +++ b/tez-api/src/test/java/org/apache/tez/dag/api/client/rpc/TestDAGClient.java @@ -28,11 +28,15 @@ import static org.mockito.Mockito.when; import javax.annotation.Nullable; + +import java.io.File; import java.io.IOException; +import java.lang.reflect.Field; import java.util.EnumSet; import java.util.Set; import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.security.ssl.KeyStoreTestUtil; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ApplicationReport; import org.apache.hadoop.yarn.exceptions.YarnException; @@ -46,6 +50,7 @@ import org.apache.tez.dag.api.client.DagStatusSource; import org.apache.tez.dag.api.client.StatusGetOpts; import org.apache.tez.dag.api.client.VertexStatus; +import org.apache.tez.dag.api.client.TimelineReaderFactory.TimelineReaderStrategy; import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.GetDAGStatusRequestProto; import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.GetDAGStatusResponseProto; import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.GetVertexStatusRequestProto; @@ -62,6 +67,7 @@ import org.apache.tez.dag.api.records.DAGProtos.TezCountersProto; import org.apache.tez.dag.api.records.DAGProtos.VertexStatusProto; import org.apache.tez.dag.api.records.DAGProtos.VertexStatusStateProto; +import org.junit.Assert; import org.junit.Before; import org.junit.Test; import org.mockito.ArgumentCaptor; @@ -525,4 +531,50 @@ public Object answer(InvocationOnMock invocation) throws Throwable { }).when(mock).getDAGStatus(isNull(RpcController.class), any(GetDAGStatusRequestProto.class)); return mock; } + + @Test + /* testing idea is borrowed from YARN-5309 */ + public void testTimelineClientCleanup() throws Exception { + TezConfiguration tezConf = new TezConfiguration(); + tezConf.set("yarn.http.policy", "HTTPS_ONLY"); + + File testDir = new File(System.getProperty("java.io.tmpdir")); + String sslConfDir = KeyStoreTestUtil.getClasspathDir(TestDAGClient.class); + KeyStoreTestUtil.setupSSLConfig(testDir.getAbsolutePath(), sslConfDir, tezConf, false); + + DAGClientTimelineImpl dagClient = + new DAGClientTimelineImpl(mockAppId, dagIdStr, tezConf, mock(FrameworkClient.class), 10000); + Field field = DAGClientTimelineImpl.class.getDeclaredField("timelineReaderStrategy"); + field.setAccessible(true); + TimelineReaderStrategy strategy = (TimelineReaderStrategy) field.get(dagClient); + strategy.getHttpClient(); // calls SSLFactory.init + + ThreadGroup threadGroup = Thread.currentThread().getThreadGroup(); + + while (threadGroup.getParent() != null) { + threadGroup = threadGroup.getParent(); + } + + Thread[] threads = new Thread[threadGroup.activeCount()]; + + threadGroup.enumerate(threads); + Thread reloaderThread = null; + for (Thread thread : threads) { + if ((thread.getName() != null) && (thread.getName().contains("Truststore reloader thread"))) { + reloaderThread = thread; + } + } + Assert.assertTrue("Reloader is not alive", reloaderThread.isAlive()); + + dagClient.close(); + boolean reloaderStillAlive = true; + for (int i = 0; i < 10; i++) { + reloaderStillAlive = reloaderThread.isAlive(); + if (!reloaderStillAlive) { + break; + } + Thread.sleep(1000); + } + Assert.assertFalse("Reloader is still alive", reloaderStillAlive); + } } From 6c53307adb0ec1e31d3c20ab599f38e6df449300 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?L=C3=A1szl=C3=B3=20Bodor?= Date: Thu, 22 Oct 2020 09:13:46 +0200 Subject: [PATCH 008/137] =?UTF-8?q?TEZ-4243:=20Changes=20for=200.10.0=20re?= =?UTF-8?q?lease=20(L=C3=A1szl=C3=B3=20Bodor=20reviewed=20by=20Jonathan=20?= =?UTF-8?q?Turner=20Eagles)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Laszlo Bodor --- Tez_DOAP.rdf | 7 +++++ .../markdown/releases/apache-tez-0-10-0.md | 30 +++++++++++++++++++ docs/src/site/markdown/releases/index.md | 1 + docs/src/site/site.xml | 1 + 4 files changed, 39 insertions(+) create mode 100644 docs/src/site/markdown/releases/apache-tez-0-10-0.md diff --git a/Tez_DOAP.rdf b/Tez_DOAP.rdf index 68142ec096..055a45cb5f 100644 --- a/Tez_DOAP.rdf +++ b/Tez_DOAP.rdf @@ -34,6 +34,13 @@ Java + + + Version 0.10.0 + 2020-10-15 + 0.10.0 + + Version 0.9.2 diff --git a/docs/src/site/markdown/releases/apache-tez-0-10-0.md b/docs/src/site/markdown/releases/apache-tez-0-10-0.md new file mode 100644 index 0000000000..0ba6f7e243 --- /dev/null +++ b/docs/src/site/markdown/releases/apache-tez-0-10-0.md @@ -0,0 +1,30 @@ + + +Apache TEZ® 0.10.0 + +Apache TEZ® 0.10.0 +---------------------- + +- [Download Release Artifacts](http://www.apache.org/dyn/closer.lua/tez/0.10.0/) +- [Release Notes](0.10.0/release-notes.txt) +- Documentation + - [API Javadocs](0.10.0/tez-api-javadocs/index.html) : Documentation for the Tez APIs + - [Runtime Library Javadocs](0.10.0/tez-runtime-library-javadocs/index.html) : Documentation for built-in implementations of useful Inputs, Outputs, Processors etc. written based on the Tez APIs + - [Tez Mapreduce Javadocs](0.10.0/tez-mapreduce-javadocs/index.html) : Documentation for built-in implementations of Mapreduce compatible Inputs, Outputs, Processors etc. written based on the Tez APIs + - [Tez Configuration](0.10.0/tez-api-javadocs/configs/TezConfiguration.html) : Documentation for configurations of Tez. These configurations are typically specified in tez-site.xml. + - [Tez Runtime Configuration](0.10.0/tez-runtime-library-javadocs/configs/TezRuntimeConfiguration.html) : Documentation for runtime configurations of Tez. These configurations are typically specified by job submitters. diff --git a/docs/src/site/markdown/releases/index.md b/docs/src/site/markdown/releases/index.md index 23ddd43f67..26c18b0017 100644 --- a/docs/src/site/markdown/releases/index.md +++ b/docs/src/site/markdown/releases/index.md @@ -19,6 +19,7 @@ Releases ------------ +- [Apache TEZ® 0.10.0](./apache-tez-0-10-0.html) (Oct 15, 2020) - [Apache TEZ® 0.9.2](./apache-tez-0-9-2.html) (Mar 29, 2019) - [Apache TEZ® 0.9.1](./apache-tez-0-9-1.html) (Jan 04, 2018) - [Apache TEZ® 0.9.0](./apache-tez-0-9-0.html) (Jul 27, 2017) diff --git a/docs/src/site/site.xml b/docs/src/site/site.xml index b4fbfbf878..04187dd3f7 100644 --- a/docs/src/site/site.xml +++ b/docs/src/site/site.xml @@ -129,6 +129,7 @@ + From 4c339011383b0501e8080729c015ef642b7a2bbc Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?L=C3=A1szl=C3=B3=20Bodor?= Date: Wed, 11 Nov 2020 14:58:24 +0100 Subject: [PATCH 009/137] =?UTF-8?q?TEZ-4237:=20Upgrade=20async-http-client?= =?UTF-8?q?-1.9.40=20due=20to=20CVE-2017-14063=20(L=C3=A1szl=C3=B3=20Bodor?= =?UTF-8?q?=20reviewed=20by=20Ashutosh=20Chauhan)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Laszlo Bodor --- pom.xml | 6 +- tez-plugins/tez-aux-services/pom.xml | 12 +++ tez-runtime-library/pom.xml | 2 +- .../java/org/apache/tez/http/SSLFactory.java | 24 ++++-- .../http/async/netty/AsyncHttpConnection.java | 27 +++---- .../netty/TezBodyDeferringAsyncHandler.java | 30 ++++---- .../common/shuffle/impl/ShuffleManager.java | 3 +- .../orderedgrouped/ShuffleScheduler.java | 1 + .../apache/tez/http/TestHttpConnection.java | 24 ++---- .../apache/tez/test/TestSecureShuffle.java | 75 +++++++++++++++++-- 10 files changed, 140 insertions(+), 64 deletions(-) diff --git a/pom.xml b/pom.xml index 26b20d8094..04088bd9f8 100644 --- a/pom.xml +++ b/pom.xml @@ -233,9 +233,9 @@ ${slf4j.version} - com.ning - async-http-client - 1.9.40 + org.asynchttpclient + async-http-client + 2.12.1 org.slf4j diff --git a/tez-plugins/tez-aux-services/pom.xml b/tez-plugins/tez-aux-services/pom.xml index 0a80788d2a..97096c8332 100644 --- a/tez-plugins/tez-aux-services/pom.xml +++ b/tez-plugins/tez-aux-services/pom.xml @@ -55,6 +55,12 @@ org.apache.hadoop hadoop-hdfs provided + + + io.netty + * + + org.apache.hadoop @@ -126,6 +132,12 @@ hadoop-hdfs test test-jar + + + io.netty + * + + org.apache.tez diff --git a/tez-runtime-library/pom.xml b/tez-runtime-library/pom.xml index b59d2fb774..ecabf8976a 100644 --- a/tez-runtime-library/pom.xml +++ b/tez-runtime-library/pom.xml @@ -30,7 +30,7 @@ RoaringBitmap - com.ning + org.asynchttpclient async-http-client 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 203eb40dcd..4147be8980 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 @@ -18,7 +18,12 @@ package org.apache.tez.http; -import com.ning.http.client.AsyncHttpClientConfig; +import org.asynchttpclient.DefaultAsyncHttpClientConfig; + +import io.netty.handler.ssl.ClientAuth; +import io.netty.handler.ssl.JdkSslContext; +import io.netty.handler.ssl.SupportedCipherSuiteFilter; + import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; @@ -223,17 +228,20 @@ public HttpURLConnection configure(HttpURLConnection conn) throws IOException { } /** - * Set ssl context for {@link com.ning.http.client.AsyncHttpClientConfig.Builder} + * Set ssl context for {@link org.asynchttpclient.DefaultAsyncHttpClientConfig.Builder} * - * @param asyncNingBuilder {@link com.ning.http.client.AsyncHttpClientConfig.Builder} instance to + * @param builder {@link org.asynchttpclient.DefaultAsyncHttpClientConfig.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()); + public void configure(DefaultAsyncHttpClientConfig.Builder builder) throws IOException { + if (builder != null) { + JdkSslContext jdkSslContext = + new JdkSslContext(context, mode.equals(Mode.CLIENT), /* ciphers */null, + SupportedCipherSuiteFilter.INSTANCE, /* ApplicationProtocolConfig */ null, + requireClientCert ? ClientAuth.REQUIRE : ClientAuth.OPTIONAL, enabledProtocols, + /* startTls */ true); + builder.setSslContext(jdkSslContext); } } - } \ No newline at end of file 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 index ac0a49c401..43f64b82b3 100644 --- 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 @@ -20,12 +20,13 @@ import com.google.common.annotations.VisibleForTesting; import org.apache.tez.common.Preconditions; -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.asynchttpclient.AsyncHttpClient; +import org.asynchttpclient.DefaultAsyncHttpClient; +import org.asynchttpclient.DefaultAsyncHttpClientConfig; +import org.asynchttpclient.ListenableFuture; +import org.asynchttpclient.Request; +import org.asynchttpclient.RequestBuilder; +import org.asynchttpclient.Response; import org.apache.commons.io.IOUtils; import org.apache.tez.http.BaseHttpConnection; import org.apache.tez.http.HttpConnectionParams; @@ -76,7 +77,7 @@ private void initClient(HttpConnectionParams httpConnParams) throws IOException synchronized (AsyncHttpConnection.class) { if (httpAsyncClient == null) { LOG.info("Initializing AsyncClient (TezBodyDeferringAsyncHandler)"); - AsyncHttpClientConfig.Builder builder = new AsyncHttpClientConfig.Builder(); + DefaultAsyncHttpClientConfig.Builder builder = new DefaultAsyncHttpClientConfig.Builder(); if (httpConnParams.isSslShuffle()) { //Configure SSL SSLFactory sslFactory = httpConnParams.getSslFactory(); @@ -91,16 +92,16 @@ private void initClient(HttpConnectionParams httpConnParams) throws IOException * setMaxConnections & addRequestFilter. */ builder - .setAllowPoolingConnections(httpConnParams.isKeepAlive()) - .setAllowPoolingSslConnections(httpConnParams.isKeepAlive()) + .setKeepAlive(httpConnParams.isKeepAlive()) .setCompressionEnforced(false) //.setExecutorService(applicationThreadPool) - //.addRequestFilter(new ThrottleRequestFilter()) + //.addRequestFilter(new ThrottleRequestFilter(1)) .setMaxConnectionsPerHost(1) .setConnectTimeout(httpConnParams.getConnectionTimeout()) - .setDisableUrlEncodingForBoundedRequests(true) + .setDisableUrlEncodingForBoundRequests(true) .build(); - httpAsyncClient = new AsyncHttpClient(builder.build()); + DefaultAsyncHttpClientConfig config = builder.build(); + httpAsyncClient = new DefaultAsyncHttpClient(config); } } } @@ -208,7 +209,7 @@ public DataInputStream getInputStream() throws IOException, InterruptedException } @VisibleForTesting - public void close() { + public void close() throws IOException { httpAsyncClient.close(); httpAsyncClient = null; } 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 index 8e83eaca64..acbdab7df9 100644 --- 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 @@ -17,15 +17,16 @@ */ 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.asynchttpclient.AsyncHandler; +import org.asynchttpclient.HttpResponseBodyPart; +import org.asynchttpclient.HttpResponseStatus; +import org.asynchttpclient.Response; import org.apache.hadoop.classification.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import io.netty.handler.codec.http.HttpHeaders; + import java.io.FilterInputStream; import java.io.IOException; import java.io.InputStream; @@ -38,8 +39,8 @@ 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 + * Same as {@link org.asynchttpclient.BodyDeferringAsyncHandler} with additional checks handle + * errors in getResponse(). Based on testing, at very high load {@link org.asynchttpclient * .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. @@ -92,27 +93,28 @@ public void onThrowable(Throwable t) { } } - public AsyncHandler.STATE onStatusReceived(HttpResponseStatus responseStatus) throws Exception { + public AsyncHandler.State onStatusReceived(HttpResponseStatus responseStatus) throws Exception { responseBuilder.reset(); responseBuilder.accumulate(responseStatus); statusReceived = true; - return AsyncHandler.STATE.CONTINUE; + return AsyncHandler.State.CONTINUE; } - public AsyncHandler.STATE onHeadersReceived(HttpResponseHeaders headers) throws Exception { + @Override + public AsyncHandler.State onHeadersReceived(HttpHeaders headers) throws Exception { responseBuilder.accumulate(headers); - return AsyncHandler.STATE.CONTINUE; + return AsyncHandler.State.CONTINUE; } - public AsyncHandler.STATE onBodyPartReceived(HttpResponseBodyPart bodyPart) throws Exception { + 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; + output.write(bodyPart.getBodyPartBytes()); + return AsyncHandler.State.CONTINUE; } protected void closeOut() throws IOException { 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 2b83ad8920..901ee08a7a 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 @@ -337,7 +337,8 @@ public ShuffleManager(InputContext inputContext, Configuration conf, int numInpu + ifileReadAhead + ", ifileReadAheadLength=" + ifileReadAheadLength +", " + "localDiskFetchEnabled=" + localDiskFetchEnabled + ", " + "sharedFetchEnabled=" + sharedFetchEnabled + ", " - + httpConnectionParams.toString() + ", maxTaskOutputAtOnce=" + maxTaskOutputAtOnce); + + httpConnectionParams.toString() + ", maxTaskOutputAtOnce=" + maxTaskOutputAtOnce + + ", asyncHttp=" + asyncHttp); } public void updateApproximateInputRecords(int delta) { 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 e7f63ab7ad..416041e005 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 @@ -441,6 +441,7 @@ public ShuffleScheduler(InputContext inputContext, + ", maxStallTimeFraction=" + maxStallTimeFraction + ", minReqProgressFraction=" + minReqProgressFraction + ", checkFailedFetchSinceLastCompletion=" + checkFailedFetchSinceLastCompletion + + ", asyncHttp=" + asyncHttp ); } 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 index ed4ed54c2e..afeb6e561f 100644 --- 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 @@ -24,8 +24,6 @@ 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.ConnectException; @@ -42,7 +40,7 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; -import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.any; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.when; @@ -78,6 +76,7 @@ public Thread newThread(Runnable r) { }); url = new URL(NOT_HOSTED_URL); tokenSecretManager = mock(JobTokenSecretManager.class); + when(tokenSecretManager.computeHash(any())).thenReturn("1234".getBytes()); } @AfterClass @@ -89,7 +88,7 @@ public void baseTest(Callable worker, CountDownLatch latch, String message InterruptedException { long startTime = System.currentTimeMillis(); try { - Future future = executorService.submit(worker); + Future future = executorService.submit(worker); future.get(); } catch (ExecutionException e) { assertTrue(e.getCause().getCause() instanceof IOException); @@ -117,14 +116,13 @@ public void testConnectionTimeout() throws IOException, InterruptedException { } @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)); + Future future = executorService.submit(new Worker(latch, asyncHttpConn, true)); while(currentThread == null) { synchronized (this) { @@ -153,24 +151,14 @@ HttpConnectionParams getConnectionParams() { 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(); + realConn.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(); + realConn.computeEncHash(); return connection; } 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 6d34464c04..aed240997e 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 @@ -25,8 +25,13 @@ import java.io.File; import java.io.IOException; import java.io.OutputStreamWriter; +import java.net.InetAddress; +import java.security.KeyPair; +import java.security.cert.X509Certificate; import java.util.ArrayList; import java.util.Collection; +import java.util.HashMap; +import java.util.Map; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataOutputStream; @@ -37,7 +42,7 @@ 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.hadoop.security.ssl.SSLFactory; import org.apache.tez.dag.api.TezConfiguration; import org.apache.tez.mapreduce.examples.TestOrderedWordCount; import org.apache.tez.runtime.library.api.TezRuntimeConfiguration; @@ -133,7 +138,7 @@ public void setupTezCluster() throws Exception { conf.setInt(TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_FETCH_FAILURES_LIMIT, 2); conf.setLong(TezConfiguration.TEZ_AM_SLEEP_TIME_BEFORE_EXIT_MILLIS, 500); - + conf.setBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_USE_ASYNC_HTTP, asyncHttp); String sslConf = conf.get(SSL_CLIENT_CONF_KEY, "ssl-client.xml"); conf.addResource(sslConf); @@ -205,10 +210,68 @@ private static void createSampleFile(Path inputLoc) throws IOException { */ private static void setupKeyStores() throws Exception { keysStoresDir.mkdirs(); - String sslConfsDir = - KeyStoreTestUtil.getClasspathDir(TestSecureShuffle.class); + String sslConfsDir = KeyStoreTestUtil.getClasspathDir(TestSecureShuffle.class); + + setupSSLConfig(keysStoresDir.getAbsolutePath(), sslConfsDir, conf, true, true, ""); + } + + /** + * This is a copied version of hadoop's KeyStoreTestUtil.setupSSLConfig which was needed to create + * server certs with actual hostname in CN instead of "localhost". While upgrading async http + * client in TEZ-4237, it turned out that netty doesn't support custom hostname verifiers anymore + * (as discussed in https://github.com/AsyncHttpClient/async-http-client/issues/928), that's why + * it cannot be set for an async http connection. So instead of hacking an ALLOW_ALL verifier + * somehow (which cannot be propagated to netty), a valid certificate with the actual hostname + * should be generated in setupSSLConfig, so the only change is the usage of + * "InetAddress.getLocalHost().getHostName()". + */ + public static void setupSSLConfig(String keystoresDir, String sslConfDir, Configuration config, + boolean useClientCert, boolean trustStore, String excludeCiphers) throws Exception { + String clientKS = keystoresDir + "/clientKS.jks"; + String clientPassword = "clientP"; + String serverKS = keystoresDir + "/serverKS.jks"; + String serverPassword = "serverP"; + String trustKS = null; + String trustPassword = "trustP"; + + File sslClientConfFile = new File(sslConfDir, KeyStoreTestUtil.getClientSSLConfigFileName()); + File sslServerConfFile = new File(sslConfDir, KeyStoreTestUtil.getServerSSLConfigFileName()); + + Map certs = new HashMap(); + + if (useClientCert) { + KeyPair cKP = KeyStoreTestUtil.generateKeyPair("RSA"); + X509Certificate cCert = + KeyStoreTestUtil.generateCertificate("CN=localhost, O=client", cKP, 30, "SHA1withRSA"); + KeyStoreTestUtil.createKeyStore(clientKS, clientPassword, "client", cKP.getPrivate(), cCert); + certs.put("client", cCert); + } + + String localhostName = InetAddress.getLocalHost().getHostName(); + KeyPair sKP = KeyStoreTestUtil.generateKeyPair("RSA"); + X509Certificate sCert = + KeyStoreTestUtil.generateCertificate("CN="+localhostName+", O=server", sKP, 30, "SHA1withRSA"); + KeyStoreTestUtil.createKeyStore(serverKS, serverPassword, "server", sKP.getPrivate(), sCert); + certs.put("server", sCert); + + if (trustStore) { + trustKS = keystoresDir + "/trustKS.jks"; + KeyStoreTestUtil.createTrustStore(trustKS, trustPassword, certs); + } + + Configuration clientSSLConf = KeyStoreTestUtil.createClientSSLConfig(clientKS, clientPassword, + clientPassword, trustKS, excludeCiphers); + Configuration serverSSLConf = KeyStoreTestUtil.createServerSSLConfig(serverKS, serverPassword, + serverPassword, trustKS, excludeCiphers); + + KeyStoreTestUtil.saveConfig(sslClientConfFile, clientSSLConf); + KeyStoreTestUtil.saveConfig(sslServerConfFile, serverSSLConf); + + // this will be ignored for AsyncHttpConnection, see method comments above + config.set(SSLFactory.SSL_HOSTNAME_VERIFIER_KEY, "ALLOW_ALL"); - KeyStoreTestUtil.setupSSLConfig(keysStoresDir.getAbsolutePath(), - sslConfsDir, conf, true); + config.set(SSLFactory.SSL_CLIENT_CONF_KEY, sslClientConfFile.getName()); + config.set(SSLFactory.SSL_SERVER_CONF_KEY, sslServerConfFile.getName()); + config.setBoolean(SSLFactory.SSL_REQUIRE_CLIENT_CERT_KEY, useClientCert); } } From 989d286d09cac7c4e4c5a0e06dd75ea5a6f15478 Mon Sep 17 00:00:00 2001 From: Marton Bod Date: Sat, 21 Nov 2020 07:45:38 +0100 Subject: [PATCH 010/137] =?UTF-8?q?TEZ-4248:=20MRReaderMapred=20should=20p?= =?UTF-8?q?ropagate=20properties=20based=20on=20config=20(Marton=20Bod=20v?= =?UTF-8?q?ia=20L=C3=A1szl=C3=B3=20Bodor)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Laszlo Bodor --- .../apache/tez/dag/api/TezConfiguration.java | 7 +++ .../tez/mapreduce/lib/MRReaderMapred.java | 25 +++++++++-- .../mapreduce/lib/TestKVReadersWithMR.java | 45 +++++++++++++++++++ 3 files changed, 73 insertions(+), 4 deletions(-) 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 2af08a9044..85f85181ca 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 @@ -2114,4 +2114,11 @@ static Set getPropertySet() { @ConfigurationProperty public static final String TEZ_JOB_FS_SERVERS_TOKEN_RENEWAL_EXCLUDE = "tez.job.fs-servers.token-renewal.exclude"; + /** + * Comma-separated list of properties that MRReaderMapred should return (if present) when calling for config updates. + */ + @ConfigurationScope(Scope.VERTEX) + @ConfigurationProperty + public static final String TEZ_MRREADER_CONFIG_UPDATE_PROPERTIES = "tez.mrreader.config.update.properties"; + } diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/lib/MRReaderMapred.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/lib/MRReaderMapred.java index 21a4f96df4..e04ae7f046 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/lib/MRReaderMapred.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/lib/MRReaderMapred.java @@ -30,6 +30,7 @@ import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapred.JobContext; import org.apache.hadoop.mapred.RecordReader; +import org.apache.tez.dag.api.TezConfiguration; import org.apache.tez.common.counters.TezCounter; import org.apache.tez.common.counters.TezCounters; import org.apache.tez.mapreduce.hadoop.mapred.MRReporter; @@ -145,6 +146,13 @@ public Object getCurrentValue() throws IOException { * @return the additional fields set by {@link MRInput} */ public Configuration getConfigUpdates() { + String propertyList = jobConf.get(TezConfiguration.TEZ_MRREADER_CONFIG_UPDATE_PROPERTIES); + if (propertyList != null) { + String[] properties = propertyList.split(","); + for (String prop : properties) { + addToIncrementalConfFromJobConf(prop); + } + } if (incrementalConf != null) { return new Configuration(incrementalConf); } @@ -161,15 +169,24 @@ private void setupOldRecordReader() throws IOException { setupComplete = true; } - private void setIncrementalConfigParams(InputSplit inputSplit) { - if (inputSplit instanceof FileSplit) { - FileSplit fileSplit = (FileSplit) inputSplit; + private void setIncrementalConfigParams(InputSplit split) { + if (split instanceof FileSplit) { + FileSplit fileSplit = (FileSplit) split; this.incrementalConf = new Configuration(false); this.incrementalConf.set(JobContext.MAP_INPUT_FILE, fileSplit.getPath().toString()); this.incrementalConf.setLong(JobContext.MAP_INPUT_START, fileSplit.getStart()); this.incrementalConf.setLong(JobContext.MAP_INPUT_PATH, fileSplit.getLength()); } - LOG.info("Processing split: " + inputSplit); + LOG.info("Processing split: " + split); + } + + private void addToIncrementalConfFromJobConf(String property) { + if (jobConf.get(property) != null) { + if (incrementalConf == null) { + incrementalConf = new Configuration(false); + } + incrementalConf.set(property, jobConf.get(property)); + } } } diff --git a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/lib/TestKVReadersWithMR.java b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/lib/TestKVReadersWithMR.java index dad18deeca..21a92469e2 100644 --- a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/lib/TestKVReadersWithMR.java +++ b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/lib/TestKVReadersWithMR.java @@ -18,6 +18,7 @@ package org.apache.tez.mapreduce.lib; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapred.RecordReader; import org.apache.hadoop.mapreduce.InputSplit; @@ -25,12 +26,15 @@ 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.TezConfiguration; import org.apache.tez.runtime.api.InputContext; import org.junit.Before; import org.junit.Test; import java.io.IOException; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; import static org.mockito.Mockito.mock; @@ -107,6 +111,47 @@ public void testWithSpecificNumberOfKV_MapReduce(int kvPairs) throws IOException } } + @Test + public void testIncrementalConfigWithMultipleProperties() throws IOException { + InputContext mockContext = mock(InputContext.class); + MRReaderMapred reader = new MRReaderMapred(conf, counters, inputRecordCounter, mockContext); + conf.set(TezConfiguration.TEZ_MRREADER_CONFIG_UPDATE_PROPERTIES, "column.names,does_not_exist,column.ids"); + conf.set("column.names", "first_name,last_name,id"); + conf.set("column.ids", "1,2,3"); + conf.set("random", "value"); + + Configuration incrementalConf = reader.getConfigUpdates(); + + assertEquals(2, incrementalConf.size()); + assertEquals("first_name,last_name,id", incrementalConf.get("column.names")); + assertEquals("1,2,3", incrementalConf.get("column.ids")); + } + + @Test + public void testIncrementalConfigWithSingleProperty() throws IOException { + InputContext mockContext = mock(InputContext.class); + MRReaderMapred reader = new MRReaderMapred(conf, counters, inputRecordCounter, mockContext); + conf.set(TezConfiguration.TEZ_MRREADER_CONFIG_UPDATE_PROPERTIES, "column.names"); + conf.set("column.names", "first_name,last_name,id"); + conf.set("random", "value"); + + Configuration incrementalConf = reader.getConfigUpdates(); + + assertEquals(1, incrementalConf.size()); + assertEquals("first_name,last_name,id", incrementalConf.get("column.names")); + } + + @Test + public void testIncrementalConfigWithZeroProperty() throws IOException { + InputContext mockContext = mock(InputContext.class); + MRReaderMapred reader = new MRReaderMapred(conf, counters, inputRecordCounter, mockContext); + conf.set("random", "value"); + + Configuration incrementalConf = reader.getConfigUpdates(); + + assertNull(incrementalConf); + } + static class DummyRecordReader implements RecordReader { int records; From 938d6a105c398531774a8c341dc851b7ceee4eeb Mon Sep 17 00:00:00 2001 From: Rajesh Balamohan Date: Tue, 24 Nov 2020 13:10:15 +0530 Subject: [PATCH 011/137] TEZ-4251: Acquiring locks for getInputVertices and getOutputVertices is not consistent (Krisztian Kasa reviewed by Rajesh Balamohan) --- .../java/org/apache/tez/dag/app/dag/impl/VertexImpl.java | 7 +------ 1 file changed, 1 insertion(+), 6 deletions(-) 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 711d02827e..f86dc987b3 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 @@ -4460,12 +4460,7 @@ public int hashCode() { @Override public Map getInputVertices() { - readLock.lock(); - try { - return Collections.unmodifiableMap(this.sourceVertices); - } finally { - readLock.unlock(); - } + return Collections.unmodifiableMap(this.sourceVertices); } @Override From dadc09f5a44c1cb61af00efecb3d27b92c92aa8f Mon Sep 17 00:00:00 2001 From: Rajesh Balamohan Date: Wed, 2 Dec 2020 03:58:42 +0530 Subject: [PATCH 012/137] =?UTF-8?q?TEZ-4244:=20Consider=20using=20RawLocal?= =?UTF-8?q?FileSystem=20in=20LocalDiskFetchedInput=20(Rajesh=20Balamohan?= =?UTF-8?q?=20reviewed=20by=20L=C3=A1szl=C3=B3=20Bodor)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../runtime/library/common/shuffle/LocalDiskFetchedInput.java | 2 +- .../apache/tez/runtime/library/common/shuffle/TestFetcher.java | 3 ++- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/LocalDiskFetchedInput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/LocalDiskFetchedInput.java index cb6ea0599f..d7ec265048 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/LocalDiskFetchedInput.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/LocalDiskFetchedInput.java @@ -49,7 +49,7 @@ public LocalDiskFetchedInput(long startOffset, long compressedSize, this.size = compressedSize; this.startOffset = startOffset; this.inputFile = inputFile; - localFS = FileSystem.getLocal(conf); + localFS = FileSystem.getLocal(conf).getRaw(); } @Override 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 05d4eb4145..cea3272818 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 @@ -18,6 +18,7 @@ package org.apache.tez.runtime.library.common.shuffle; +import org.apache.hadoop.fs.RawLocalFileSystem; import org.apache.tez.runtime.library.common.CompositeInputAttemptIdentifier; import static org.mockito.Matchers.anyInt; import static org.mockito.Matchers.anyLong; @@ -258,7 +259,7 @@ protected void verifyFetchSucceeded(FetcherCallback callback, CompositeInputAtte LocalDiskFetchedInput f = capturedFetchedInput.getValue(); Assert.assertEquals("success callback filename", f.getInputFile().toString(), SHUFFLE_INPUT_FILE_PREFIX + pathComponent); - Assert.assertTrue("success callback fs", f.getLocalFS() instanceof LocalFileSystem); + Assert.assertTrue("success callback fs", f.getLocalFS() instanceof RawLocalFileSystem); Assert.assertEquals("success callback filesystem", f.getStartOffset(), p * 10); Assert.assertEquals("success callback compressed size", f.getSize(), p * 100); Assert.assertEquals("success callback input id", f.getInputAttemptIdentifier(), srcAttempId.expand(0)); From 51953a67641392eb5cf4b2c4fb035baab0badf27 Mon Sep 17 00:00:00 2001 From: Rajesh Balamohan Date: Mon, 7 Dec 2020 12:35:26 -0800 Subject: [PATCH 013/137] TEZ-4256 : Reduce key comparisons in reducer side (Rajesh Balamohan via Gopal V) Signed-off-by: Ashutosh Chauhan --- .../library/common/shuffle/orderedgrouped/MergeManager.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/MergeManager.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/MergeManager.java index 8565e7111c..59ff577fed 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/MergeManager.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/MergeManager.java @@ -1128,7 +1128,7 @@ public KeyState readRawKey(DataInputBuffer key) throws IOException { final int klen = kb.getLength() - kp; key.reset(kb.getData(), kp, klen); bytesRead += klen; - return KeyState.NEW_KEY; + return kvIter.isSameKey() ? KeyState.SAME_KEY : KeyState.NEW_KEY; } return KeyState.NO_KEY; } From b2a05115dcce28a2b188435b445b6d7e43103ab1 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Mustafa=20=C4=B0man?= Date: Mon, 7 Dec 2020 22:40:07 +0100 Subject: [PATCH 014/137] =?UTF-8?q?TEZ-4253:=20Revert=20TEZ-4170=20(Mustaf?= =?UTF-8?q?a=20=C4=B0man=20via=20Attila=20Magyar,=20Ashutosh=20Chauhan)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Laszlo Bodor --- .../app/dag/RootInputInitializerManager.java | 274 +++++++++--------- .../tez/dag/app/dag/impl/VertexImpl.java | 25 +- .../dag/TestRootInputInitializerManager.java | 116 +------- .../tez/dag/app/dag/impl/TestDAGRecovery.java | 6 +- .../dag/impl/TestRootInputVertexManager.java | 5 +- .../tez/dag/app/dag/impl/TestVertexImpl.java | 99 +------ 6 files changed, 180 insertions(+), 345 deletions(-) diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/RootInputInitializerManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/RootInputInitializerManager.java index bd4bcd89a8..3c4a05eb84 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/RootInputInitializerManager.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/RootInputInitializerManager.java @@ -18,33 +18,35 @@ package org.apache.tez.dag.app.dag; -import static org.apache.tez.dag.app.dag.VertexState.FAILED; +import javax.annotation.Nullable; import java.io.IOException; import java.lang.reflect.UndeclaredThrowableException; import java.security.PrivilegedExceptionAction; -import java.util.ArrayList; import java.util.Collection; -import java.util.Collections; import java.util.HashMap; import java.util.Iterator; import java.util.LinkedList; import java.util.List; import java.util.Map; -import java.util.Objects; import java.util.Set; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.Future; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.Objects; -import javax.annotation.Nullable; +import org.apache.tez.common.Preconditions; +import com.google.common.collect.LinkedListMultimap; +import com.google.common.collect.ListMultimap; +import com.google.common.collect.Lists; -import org.apache.commons.lang.exception.ExceptionUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.yarn.event.EventHandler; -import org.apache.tez.common.Preconditions; +import org.apache.tez.common.GuavaShim; import org.apache.tez.common.ReflectionUtils; import org.apache.tez.common.TezUtilsInternal; import org.apache.tez.dag.api.InputDescriptor; @@ -52,38 +54,38 @@ import org.apache.tez.dag.api.RootInputLeafOutput; import org.apache.tez.dag.api.TezException; import org.apache.tez.dag.api.TezUncheckedException; +import org.apache.tez.dag.api.event.*; import org.apache.tez.dag.api.event.VertexState; -import org.apache.tez.dag.api.event.VertexStateUpdate; import org.apache.tez.dag.api.oldrecords.TaskState; import org.apache.tez.dag.app.AppContext; import org.apache.tez.dag.app.dag.event.VertexEventRootInputFailed; import org.apache.tez.dag.app.dag.event.VertexEventRootInputInitialized; import org.apache.tez.dag.app.dag.impl.AMUserCodeException; -import org.apache.tez.dag.app.dag.impl.AMUserCodeException.Source; import org.apache.tez.dag.app.dag.impl.TezRootInputInitializerContextImpl; -import org.apache.tez.dag.app.dag.impl.VertexImpl; +import org.apache.tez.dag.app.dag.impl.AMUserCodeException.Source; import org.apache.tez.dag.records.TezTaskID; import org.apache.tez.dag.records.TezVertexID; import org.apache.tez.runtime.api.Event; import org.apache.tez.runtime.api.InputInitializer; import org.apache.tez.runtime.api.InputInitializerContext; -import org.apache.tez.runtime.api.events.InputInitializerEvent; -import org.apache.tez.runtime.api.impl.TezEvent; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import com.google.common.annotations.VisibleForTesting; -import com.google.common.collect.LinkedListMultimap; -import com.google.common.collect.ListMultimap; -import com.google.common.collect.Lists; +import com.google.common.util.concurrent.FutureCallback; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ListeningExecutorService; +import com.google.common.util.concurrent.MoreExecutors; +import com.google.common.util.concurrent.ThreadFactoryBuilder; + +import org.apache.tez.runtime.api.events.InputInitializerEvent; +import org.apache.tez.runtime.api.impl.TezEvent; public class RootInputInitializerManager { private static final Logger LOG = LoggerFactory.getLogger(RootInputInitializerManager.class); - @VisibleForTesting - protected ListeningExecutorService executor; + private final ExecutorService rawExecutor; + private final ListeningExecutorService executor; @SuppressWarnings("rawtypes") private final EventHandler eventHandler; private volatile boolean isStopped = false; @@ -94,135 +96,51 @@ public class RootInputInitializerManager { private final AppContext appContext; @VisibleForTesting - final Map initializerMap = new ConcurrentHashMap<>(); + final Map initializerMap = new HashMap(); public RootInputInitializerManager(Vertex vertex, AppContext appContext, UserGroupInformation dagUgi, StateChangeNotifier stateTracker) { this.appContext = appContext; this.vertex = vertex; this.eventHandler = appContext.getEventHandler(); - this.executor = appContext.getExecService(); + this.rawExecutor = Executors.newCachedThreadPool(new ThreadFactoryBuilder() + .setDaemon(true).setNameFormat("InputInitializer {" + this.vertex.getName() + "} #%d").build()); + this.executor = MoreExecutors.listeningDecorator(rawExecutor); this.dagUgi = dagUgi; this.entityStateTracker = stateTracker; } + public void runInputInitializers(List> + inputs) throws TezException { + for (RootInputLeafOutput input : inputs) { - public void runInputInitializers(List> inputs, - List pendingInitializerEvents) { - List initWrappers = createInitializerWrappers(inputs); - if (!initWrappers.isEmpty()) { - executor.submit(() -> createAndStartInitializing(pendingInitializerEvents, initWrappers)); - } - } - - /** - * Create input wrappers for all inputs in parallel. - * - * @param inputs - * @return - */ - protected List createInitializerWrappers( - List> inputs) { - String current = null; - final List result = Collections.synchronizedList(new ArrayList<>()); - try { - final List> fResults = new ArrayList<>(); - for (RootInputLeafOutput each : inputs) { - current = each.getName(); - fResults.add(executor.submit(() -> { - InitializerWrapper initializer = createInitializerWrapper(each); - initializerMap.put(each.getName(), initializer); - registerPendingVertex(each, initializer); - result.add(initializer); - return null; - })); - } - for(Future f : fResults) { - f.get(); - } - } catch (InterruptedException | ExecutionException t) { - failVertex(t, current); - } - return result; - } - - void failVertex(Throwable t, String inputName) { - VertexImpl vertexImpl = (VertexImpl) vertex; - String msg = "Fail to create InputInitializerManager, " + ExceptionUtils.getStackTrace(t); - LOG.info(msg); - vertexImpl.finished(FAILED, VertexTerminationCause.INIT_FAILURE, msg); - eventHandler.handle(new VertexEventRootInputFailed(vertex.getVertexId(), inputName, - new AMUserCodeException(AMUserCodeException.Source.InputInitializer, t))); - } + InputInitializerContext context = + new TezRootInputInitializerContextImpl(input, vertex, appContext, this); - /** - * Start initializers in parallel. - * - * @param pendingInitializerEvents - * @param result - */ - protected void createAndStartInitializing(List pendingInitializerEvents, List result) { - handleInitializerEvents(pendingInitializerEvents); - pendingInitializerEvents.clear(); - for (InitializerWrapper inputWrapper : result) { - executor.submit(() -> runInitializerAndProcessResult(inputWrapper)); - } - } - - private void runInitializerAndProcessResult(InitializerWrapper initializer) { - try { - List result = runInitializer(initializer); - LOG.info("Succeeded InputInitializer for Input: " + initializer.getInput().getName() + - " on vertex " + initializer.getVertexLogIdentifier()); - eventHandler.handle(new VertexEventRootInputInitialized(vertex.getVertexId(), - initializer.getInput().getName(), result)); - } catch (Throwable t) { - if (t instanceof UndeclaredThrowableException) { - t = t.getCause(); - } - LOG.info("Failed InputInitializer for Input: " + initializer.getInput().getName() + - " on vertex " + initializer.getVertexLogIdentifier()); - eventHandler.handle(new VertexEventRootInputFailed(vertex.getVertexId(), initializer.getInput().getName(), - new AMUserCodeException(Source.InputInitializer,t))); - } finally { - initializer.setComplete(); - } - } - - private List runInitializer(InitializerWrapper initializer) throws IOException, InterruptedException { - return dagUgi.doAs((PrivilegedExceptionAction>) () -> { - LOG.info( - "Starting InputInitializer for Input: " + initializer.getInput().getName() + - " on vertex " + initializer.getVertexLogIdentifier()); + InputInitializer initializer; try { - TezUtilsInternal.setHadoopCallerContext(appContext.getHadoopShim(), - initializer.vertexId); - return initializer.getInitializer().initialize(); + TezUtilsInternal.setHadoopCallerContext(appContext.getHadoopShim(), vertex.getVertexId()); + initializer = createInitializer(input, context); } finally { appContext.getHadoopShim().clearHadoopCallerContext(); } - }); - } - private InitializerWrapper createInitializerWrapper(RootInputLeafOutput input) throws TezException { - InputInitializerContext context = - new TezRootInputInitializerContextImpl(input, vertex, appContext, this); - try { - TezUtilsInternal.setHadoopCallerContext(appContext.getHadoopShim(), vertex.getVertexId()); - InputInitializer initializer = createInitializer(input, context); - return new InitializerWrapper(input, initializer, context, vertex, entityStateTracker, appContext); - } finally { - appContext.getHadoopShim().clearHadoopCallerContext(); - } - } + InitializerWrapper initializerWrapper = + new InitializerWrapper(input, initializer, context, vertex, entityStateTracker, appContext); - private void registerPendingVertex(RootInputLeafOutput input, InitializerWrapper initializerWrapper) { - // Register pending vertex update registrations - List vertexUpdateRegistrations = pendingVertexRegistrations.removeAll(input.getName()); - if (vertexUpdateRegistrations != null) { - for (VertexUpdateRegistrationHolder h : vertexUpdateRegistrations) { - initializerWrapper.registerForVertexStateUpdates(h.vertexName, h.stateSet); + // Register pending vertex update registrations + List vertexUpdateRegistrations = + pendingVertexRegistrations.removeAll(input.getName()); + if (vertexUpdateRegistrations != null) { + for (VertexUpdateRegistrationHolder h : vertexUpdateRegistrations) { + initializerWrapper.registerForVertexStateUpdates(h.vertexName, h.stateSet); + } } + + initializerMap.put(input.getName(), initializerWrapper); + ListenableFuture> future = executor + .submit(new InputInitializerCallable(initializerWrapper, dagUgi, appContext)); + Futures.addCallback(future, createInputInitializerCallback(initializerWrapper), GuavaShim.directExecutor()); } } @@ -315,6 +233,11 @@ public void registerForVertexUpdates(String vertexName, String inputName, } } + @VisibleForTesting + protected InputInitializerCallback createInputInitializerCallback(InitializerWrapper initializer) { + return new InputInitializerCallback(initializer, eventHandler, vertex.getVertexId()); + } + @VisibleForTesting @InterfaceAudience.Private public InitializerWrapper getInitializerWrapper(String inputName) { @@ -322,7 +245,92 @@ public InitializerWrapper getInitializerWrapper(String inputName) { } public void shutdown() { - isStopped = true; + if (executor != null && !isStopped) { + // Don't really care about what is running if an error occurs. If no error + // occurs, all execution is complete. + executor.shutdownNow(); + isStopped = true; + } + } + + private static class InputInitializerCallable implements + Callable> { + + private final InitializerWrapper initializerWrapper; + private final UserGroupInformation ugi; + private final AppContext appContext; + + InputInitializerCallable(InitializerWrapper initializer, UserGroupInformation ugi, + AppContext appContext) { + this.initializerWrapper = initializer; + this.ugi = ugi; + this.appContext = appContext; + } + + @Override + public List call() throws Exception { + List events = ugi.doAs(new PrivilegedExceptionAction>() { + @Override + public List run() throws Exception { + LOG.info( + "Starting InputInitializer for Input: " + initializerWrapper.getInput().getName() + + " on vertex " + initializerWrapper.getVertexLogIdentifier()); + try { + TezUtilsInternal.setHadoopCallerContext(appContext.getHadoopShim(), + initializerWrapper.vertexId); + return initializerWrapper.getInitializer().initialize(); + } finally { + appContext.getHadoopShim().clearHadoopCallerContext(); + } + } + }); + return events; + } + } + + @SuppressWarnings("rawtypes") + @VisibleForTesting + private static class InputInitializerCallback implements + FutureCallback> { + + private final InitializerWrapper initializer; + private final EventHandler eventHandler; + private final TezVertexID vertexID; + + InputInitializerCallback(InitializerWrapper initializer, + EventHandler eventHandler, TezVertexID vertexID) { + this.initializer = initializer; + this.eventHandler = eventHandler; + this.vertexID = vertexID; + } + + @SuppressWarnings("unchecked") + @Override + public void onSuccess(List result) { + initializer.setComplete(); + LOG.info( + "Succeeded InputInitializer for Input: " + initializer.getInput().getName() + + " on vertex " + initializer.getVertexLogIdentifier()); + eventHandler.handle(new VertexEventRootInputInitialized(vertexID, + initializer.getInput().getName(), result)); + } + + @SuppressWarnings("unchecked") + @Override + public void onFailure(Throwable t) { + // catch real root cause of failure, it would throw UndeclaredThrowableException + // if using UGI.doAs + if (t instanceof UndeclaredThrowableException) { + t = t.getCause(); + } + initializer.setComplete(); + LOG.info( + "Failed InputInitializer for Input: " + initializer.getInput().getName() + + " on vertex " + initializer.getVertexLogIdentifier()); + eventHandler + .handle(new VertexEventRootInputFailed(vertexID, initializer.getInput().getName(), + new AMUserCodeException(Source.InputInitializer, t))); + } } @VisibleForTesting 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 f86dc987b3..6ae3ba55a8 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 @@ -2416,7 +2416,7 @@ void tryEnactKill(VertexTerminationCause trigger, } } - public VertexState finished(VertexState finalState, + VertexState finished(VertexState finalState, VertexTerminationCause termCause, String diag) { if (finishTime == 0) setFinishTime(); if (termCause != null) { @@ -3073,7 +3073,13 @@ private VertexState handleInitEvent(VertexImpl vertex) { if (vertex.inputsWithInitializers != null) { if (vertex.recoveryData == null || !vertex.recoveryData.shouldSkipInit()) { LOG.info("Vertex will initialize from input initializer. " + vertex.logIdentifier); - vertex.setupInputInitializerManager(); + try { + vertex.setupInputInitializerManager(); + } catch (TezException e) { + String msg = "Fail to create InputInitializerManager, " + ExceptionUtils.getStackTrace(e); + LOG.info(msg); + return vertex.finished(VertexState.FAILED, VertexTerminationCause.INIT_FAILURE, msg); + } } return VertexState.INITIALIZING; } else { @@ -3106,7 +3112,13 @@ private VertexState handleInitEvent(VertexImpl vertex) { if (vertex.inputsWithInitializers != null && (vertex.recoveryData == null || !vertex.recoveryData.shouldSkipInit())) { LOG.info("Vertex will initialize from input initializer. " + vertex.logIdentifier); - vertex.setupInputInitializerManager(); + try { + vertex.setupInputInitializerManager(); + } catch (TezException e) { + String msg = "Fail to create InputInitializerManager, " + ExceptionUtils.getStackTrace(e); + LOG.error(msg); + return vertex.finished(VertexState.FAILED, VertexTerminationCause.INIT_FAILURE, msg); + } return VertexState.INITIALIZING; } if (!vertex.uninitializedEdges.isEmpty()) { @@ -4243,7 +4255,7 @@ public void transition(VertexImpl vertex, VertexEvent event) { } } - private void setupInputInitializerManager() { + private void setupInputInitializerManager() throws TezException { rootInputInitializerManager = createRootInputInitializerManager( getDAG().getName(), getName(), getVertexId(), eventHandler, getTotalTasks(), @@ -4258,7 +4270,10 @@ eventHandler, getTotalTasks(), LOG.info("Starting " + inputsWithInitializers.size() + " inputInitializers for vertex " + logIdentifier); initWaitsForRootInitializers = true; - rootInputInitializerManager.runInputInitializers(inputList, pendingInitializerEvents); + rootInputInitializerManager.runInputInitializers(inputList); + // Send pending rootInputInitializerEvents + rootInputInitializerManager.handleInitializerEvents(pendingInitializerEvents); + pendingInitializerEvents.clear(); } private static class VertexStateChangedCallback diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/TestRootInputInitializerManager.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/TestRootInputInitializerManager.java index a197e54e6e..ffa4309602 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/TestRootInputInitializerManager.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/TestRootInputInitializerManager.java @@ -14,9 +14,7 @@ package org.apache.tez.dag.app.dag; -import static org.apache.tez.dag.app.TestDAGAppMaster.DAGAppMasterForTest.createCredentials; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; import static org.mockito.Matchers.any; import static org.mockito.Mockito.RETURNS_DEEP_STUBS; import static org.mockito.Mockito.doReturn; @@ -27,43 +25,20 @@ import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; -import java.io.File; import java.io.IOException; import java.util.Collections; -import java.util.LinkedList; import java.util.List; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.ThreadPoolExecutor; import com.google.common.collect.Lists; -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.FSDataOutputStream; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.FileUtil; -import org.apache.hadoop.fs.Path; import org.apache.hadoop.security.UserGroupInformation; -import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.ApplicationId; -import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.event.EventHandler; -import org.apache.hadoop.yarn.util.SystemClock; -import org.apache.tez.client.TezApiVersionInfo; -import org.apache.tez.common.TezCommonUtils; import org.apache.tez.dag.api.InputDescriptor; import org.apache.tez.dag.api.InputInitializerDescriptor; import org.apache.tez.dag.api.RootInputLeafOutput; -import org.apache.tez.dag.api.TezConfiguration; -import org.apache.tez.dag.api.TezConstants; import org.apache.tez.dag.api.TezException; import org.apache.tez.dag.api.oldrecords.TaskState; -import org.apache.tez.dag.api.records.DAGProtos; import org.apache.tez.dag.app.AppContext; -import org.apache.tez.dag.app.DAGAppMaster; import org.apache.tez.dag.records.TezDAGID; import org.apache.tez.dag.records.TezTaskAttemptID; import org.apache.tez.dag.records.TezTaskID; @@ -75,34 +50,11 @@ import org.apache.tez.runtime.api.events.InputInitializerEvent; import org.apache.tez.runtime.api.impl.EventMetaData; import org.apache.tez.runtime.api.impl.TezEvent; -import org.junit.After; -import org.junit.Before; import org.junit.Test; import org.mockito.ArgumentCaptor; public class TestRootInputInitializerManager { - private static final File TEST_DIR = new File(System.getProperty("test.build.data"), - TestRootInputInitializerManager.class.getName()).getAbsoluteFile(); - private static ListeningExecutorService execService; - - @Before - public void setUp() { - ExecutorService rawExecutor = Executors.newCachedThreadPool(new ThreadFactoryBuilder() - .setDaemon(true).setNameFormat("Test App Shared Pool - " + "#%d").build()); - execService = MoreExecutors.listeningDecorator(rawExecutor); - FileUtil.fullyDelete(TEST_DIR); - TEST_DIR.mkdirs(); - } - - @After - public void tearDown() { - if (execService != null) { - execService.shutdownNow(); - } - FileUtil.fullyDelete(TEST_DIR); - } - // Simple testing. No events if task doesn't succeed. // Also exercises path where two attempts are reported as successful via the stateChangeNotifier. // Primarily a failure scenario, when a Task moves back to running from success @@ -262,7 +214,6 @@ public void testCorrectUgiUsage() throws TezException, InterruptedException { AppContext appContext = mock(AppContext.class); doReturn(new DefaultHadoopShim()).when(appContext).getHadoopShim(); doReturn(mock(EventHandler.class)).when(appContext).getEventHandler(); - when(appContext.getExecService()).thenReturn(execService); UserGroupInformation dagUgi = UserGroupInformation.createRemoteUser("fakeuser"); StateChangeNotifier stateChangeNotifier = mock(StateChangeNotifier.class); RootInputInitializerManager rootInputInitializerManager = new RootInputInitializerManager(vertex, appContext, dagUgi, stateChangeNotifier); @@ -271,73 +222,12 @@ public void testCorrectUgiUsage() throws TezException, InterruptedException { InputInitializerDescriptor iid = InputInitializerDescriptor.create(InputInitializerForUgiTest.class.getName()); RootInputLeafOutput rootInput = new RootInputLeafOutput<>("InputName", id, iid); - rootInputInitializerManager.runInputInitializers(Collections.singletonList(rootInput), Collections.emptyList()); + rootInputInitializerManager.runInputInitializers(Collections.singletonList(rootInput)); InputInitializerForUgiTest.awaitInitialize(); assertEquals(dagUgi, InputInitializerForUgiTest.ctorUgi); - assertEquals(dagUgi.getRealUser(), InputInitializerForUgiTest.initializeUgi.getRealUser()); - } - - @Test (timeout = 10000) - public synchronized void testParallelInputInitialization() throws InterruptedException, IOException { - // Create Local DAGAppMaster with default conf - Configuration conf = new Configuration(true); - conf.setBoolean(TezConfiguration.TEZ_LOCAL_MODE, true); - conf.set(TezConfiguration.TEZ_AM_STAGING_DIR, TEST_DIR.toString()); - - FileSystem fs = FileSystem.getLocal(conf); - FSDataOutputStream sessionJarsPBOutStream = - TezCommonUtils.createFileForAM(fs, new Path(TEST_DIR.toString(), - TezConstants.TEZ_AM_LOCAL_RESOURCES_PB_FILE_NAME)); - DAGProtos.PlanLocalResourcesProto.getDefaultInstance() - .writeDelimitedTo(sessionJarsPBOutStream); - sessionJarsPBOutStream.close(); - - ApplicationId appId = ApplicationId.newInstance(1, 1); - ApplicationAttemptId attemptId = ApplicationAttemptId.newInstance(appId, 1); - - DAGAppMaster am = new DAGAppMaster(attemptId, - ContainerId.newContainerId(attemptId, 1), - "127.0.0.1", 0, 0, new SystemClock(), 1, true, - TEST_DIR.toString(), new String[] {TEST_DIR.toString()}, - new String[] {TEST_DIR.toString()}, - new TezApiVersionInfo().getVersion(), createCredentials(), - "someuser", null); - am.init(conf); - - Vertex vertex = mock(Vertex.class); - doReturn(mock(TezVertexID.class)).when(vertex).getVertexId(); - UserGroupInformation dagUgi = UserGroupInformation.createRemoteUser("fakeuser"); - StateChangeNotifier stateChangeNotifier = mock(StateChangeNotifier.class); - RootInputInitializerManager rootInputInitializerManager = - new RootInputInitializerManager(vertex, am.getContext(), dagUgi, stateChangeNotifier); - - List> inlist = new LinkedList(); - // Make sure we dont have any OOM issue by controlling the capacity of the thread pool - // and also block producer (createInitializerWrapper when resources are saturated) - InputDescriptor id = mock(InputDescriptor.class); - InputInitializerDescriptor iid = InputInitializerDescriptor.create(InputInitializerForUgiTest.class.getName()); - for (int i=0; i < 10000; i++) { - RootInputLeafOutput rootInput = - new RootInputLeafOutput<>("InputName"+i, id, iid); - inlist.add(rootInput); - } - - List initWrappers = - rootInputInitializerManager.createInitializerWrappers(inlist); - - int maxThreadSize = conf.getInt(TezConfiguration.TEZ_AM_DAG_APPCONTEXT_THREAD_COUNT_LIMIT, - TezConfiguration.TEZ_AM_DAG_APPCONTEXT_THREAD_COUNT_LIMIT_DEFAULT); - ThreadPoolExecutor amThreadPool = am.getContext().getThreadPool(); - - rootInputInitializerManager.executor.submit(() -> rootInputInitializerManager - .createAndStartInitializing(Collections.emptyList(), initWrappers)); - - while (am.getContext().getThreadPool().getQueue().size() > 0) { - assertTrue(amThreadPool.getPoolSize() <= maxThreadSize); - Thread.sleep(100); - } + assertEquals(dagUgi, InputInitializerForUgiTest.initializeUgi); } public static class InputInitializerForUgiTest extends InputInitializer { @@ -379,4 +269,4 @@ static void awaitInitialize() throws InterruptedException { } } } -} +} \ No newline at end of file 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 9636329da3..95ea8a030e 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 @@ -330,10 +330,8 @@ public void setup() { Mockito.doAnswer(new Answer() { public ListenableFuture answer(InvocationOnMock invocation) { Object[] args = invocation.getArguments(); - if (args[0] instanceof CallableEvent) { - CallableEvent e = (CallableEvent) args[0]; - dispatcher.getEventHandler().handle(e); - } + CallableEvent e = (CallableEvent) args[0]; + dispatcher.getEventHandler().handle(e); return mockFuture; } }).when(execService).submit((Callable) any()); diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestRootInputVertexManager.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestRootInputVertexManager.java index 16a97d4374..39a291ebbd 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestRootInputVertexManager.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestRootInputVertexManager.java @@ -24,7 +24,7 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; import static org.mockito.Matchers.any; -import static org.mockito.Matchers.anyList; +import static org.mockito.Matchers.anyListOf; import static org.mockito.Matchers.eq; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doReturn; @@ -176,7 +176,7 @@ public void testRootInputVertexManagerSlowStart() { final List scheduledTasks = Lists.newLinkedList(); doAnswer(new ScheduledTasksAnswer(scheduledTasks)).when( - mockContext).scheduleTasks(anyList()); + mockContext).scheduleTasks(anyListOf(VertexManagerPluginContext.ScheduleTaskRequest.class)); // source vertices have 0 tasks. immediate start of all managed tasks when(mockContext.getVertexNumTasks(mockSrcVertexId1)).thenReturn(0); @@ -568,6 +568,7 @@ public ScheduledTasksAnswer(List scheduledTasks) { public Object answer(InvocationOnMock invocation) throws IOException { Object[] args = invocation.getArguments(); scheduledTasks.clear(); + @SuppressWarnings("unchecked") List tasks = (List)args[0]; for (VertexManagerPluginContext.ScheduleTaskRequest task : tasks) { 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 5ae95560fa..fb7872ff9a 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 @@ -43,15 +43,11 @@ import java.util.Map; import java.util.Set; import java.util.concurrent.Callable; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.locks.Condition; import java.util.concurrent.locks.ReentrantLock; -import com.google.common.util.concurrent.MoreExecutors; -import com.google.common.util.concurrent.ThreadFactoryBuilder; import com.google.protobuf.ByteString; import org.apache.commons.lang.StringUtils; @@ -2425,10 +2421,10 @@ private void setupVertices() { dagConf); } } else { - v = new VertexImplWithRunningInputInitializerWithExecutor(vertexId, vPlan, vPlan.getName(), conf, - dispatcher.getEventHandler(), taskCommunicatorManagerInterface, - clock, thh, appContext, locationHint, dispatcher, customInitializer, updateTracker, - dagConf, vertexGroups); + v = new VertexImpl(vertexId, vPlan, vPlan.getName(), conf, + dispatcher.getEventHandler(), taskCommunicatorManagerInterface, + clock, thh, true, appContext, locationHint, vertexGroups, taskSpecificLaunchCmdOption, + updateTracker, dagConf); } vertices.put(vName, v); vertexIdMap.put(vertexId, v); @@ -2532,10 +2528,8 @@ public void setupPostDagCreation() throws TezException { Mockito.doAnswer(new Answer() { public ListenableFuture answer(InvocationOnMock invocation) { Object[] args = invocation.getArguments(); - if (args[0] instanceof CallableEvent) { - CallableEvent e = (CallableEvent) args[0]; - dispatcher.getEventHandler().handle(e); - } + CallableEvent e = (CallableEvent) args[0]; + dispatcher.getEventHandler().handle(e); return mockFuture; }}) .when(execService).submit((Callable) any()); @@ -2766,13 +2760,12 @@ public void testNonExistVertexManager() throws TezException { } @Test(timeout=5000) - public void testNonExistInputInitializer() throws Exception { + public void testNonExistInputInitializer() throws TezException { setupPreDagCreation(); dagPlan = createDAGPlanWithNonExistInputInitializer(); setupPostDagCreation(); VertexImpl v1 = vertices.get("vertex1"); v1.handle(new VertexEvent(v1.getVertexId(), VertexEventType.V_INIT)); - while (v1.getTerminationCause() == null) Thread.sleep(10); Assert.assertEquals(VertexState.FAILED, v1.getState()); Assert.assertEquals(VertexTerminationCause.INIT_FAILURE, v1.getTerminationCause()); Assert.assertTrue(StringUtils.join(v1.getDiagnostics(),"") @@ -5849,43 +5842,6 @@ protected RootInputInitializerManager createRootInputInitializerManager( } } - @SuppressWarnings("rawtypes") - private static class VertexImplWithRunningInputInitializerWithExecutor extends VertexImpl { - private RootInputInitializerManagerWithExecutor rootInputInitializerManager; - - public VertexImplWithRunningInputInitializerWithExecutor(TezVertexID vertexId, - VertexPlan vertexPlan, String vertexName, - Configuration conf, - EventHandler eventHandler, - TaskCommunicatorManagerInterface taskCommunicatorManagerInterface, - Clock clock, TaskHeartbeatHandler thh, - AppContext appContext, - VertexLocationHint vertexLocationHint, - DrainDispatcher dispatcher, - InputInitializer presetInitializer, - StateChangeNotifier updateTracker, - Configuration dagConf, - Map vertexGroups) { - super(vertexId, vertexPlan, vertexName, conf, eventHandler, - taskCommunicatorManagerInterface, clock, thh, true, - appContext, vertexLocationHint, vertexGroups, taskSpecificLaunchCmdOption, - updateTracker, dagConf); - } - - @Override - protected RootInputInitializerManager createRootInputInitializerManager( - String dagName, String vertexName, TezVertexID vertexID, - EventHandler eventHandler, int numTasks, int numNodes, - Resource taskResource, Resource totalResource) { - try { - rootInputInitializerManager = new RootInputInitializerManagerWithExecutor(this, this.getAppContext(), stateChangeNotifier); - } catch (IOException e) { - throw new RuntimeException(e); - } - return rootInputInitializerManager; - } - } - @SuppressWarnings("rawtypes") private static class VertexImplWithControlledInitializerManager extends VertexImpl { @@ -5942,11 +5898,9 @@ public RootInputInitializerManagerWithRunningInitializer(Vertex vertex, AppConte IOException { super(vertex, appContext, UserGroupInformation.getCurrentUser(), tracker); this.presetInitializer = presetInitializer; - ExecutorService rawExecutor = Executors.newCachedThreadPool(new ThreadFactoryBuilder() - .setDaemon(true).setNameFormat("Test App Shared Pool - " + "#%d").build()); - this.executor = MoreExecutors.listeningDecorator(rawExecutor); } + @Override protected InputInitializer createInitializer( RootInputLeafOutput input, @@ -5956,31 +5910,6 @@ protected InputInitializer createInitializer( } return presetInitializer; } - - @Override - public void shutdown() { - super.shutdown(); - if (executor != null) { - executor.shutdown(); - } - } - } - - private static class RootInputInitializerManagerWithExecutor extends RootInputInitializerManager { - public RootInputInitializerManagerWithExecutor(Vertex vertex, AppContext appContext, StateChangeNotifier tracker) throws IOException { - super(vertex, appContext, UserGroupInformation.getCurrentUser(), tracker); - ExecutorService rawExecutor = Executors.newCachedThreadPool(new ThreadFactoryBuilder() - .setDaemon(true).setNameFormat("Test App Shared Pool - " + "#%d").build()); - this.executor = MoreExecutors.listeningDecorator(rawExecutor); - } - - @Override - public void shutdown() { - super.shutdown(); - if (executor != null) { - executor.shutdown(); - } - } } @SuppressWarnings({"rawtypes", "unchecked"}) @@ -6002,15 +5931,12 @@ public RootInputInitializerManagerControlled(Vertex vertex, AppContext appContex this.eventHandler = eventHandler; this.dispatcher = dispatcher; this.vertexID = vertex.getVertexId(); - ExecutorService rawExecutor = Executors.newCachedThreadPool(new ThreadFactoryBuilder() - .setDaemon(true).setNameFormat("Test App Shared Pool - " + "#%d").build()); - this.executor = MoreExecutors.listeningDecorator(rawExecutor); } @Override public void runInputInitializers( - List> inputs, List pendingInitializerEvents) { - this.inputs = inputs; + List> inputList) { + this.inputs = inputList; } @Override @@ -6035,13 +5961,10 @@ public void handleInputInitializerEvent(List events) thro @Override public void shutdown() { hasShutDown = true; - if (executor != null) { - executor.shutdown(); - } } public void failInputInitialization() throws TezException { - super.runInputInitializers(inputs, Collections.emptyList()); + super.runInputInitializers(inputs); eventHandler.handle(new VertexEventRootInputFailed(vertexID, inputs .get(0).getName(), new AMUserCodeException(Source.InputInitializer, From 1f1a177ef70b6e4fba514a02e027fe3b071117e4 Mon Sep 17 00:00:00 2001 From: Lewis John McGibbney Date: Thu, 10 Dec 2020 23:13:01 -0600 Subject: [PATCH 015/137] TEZ-4258. Fix travisci build Signed-off-by: Jonathan Eagles --- .travis.yml | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/.travis.yml b/.travis.yml index bfc6b34bbf..0437531837 100644 --- a/.travis.yml +++ b/.travis.yml @@ -20,12 +20,10 @@ sudo: required env: MAVEN_OPTS="-Xmx2G -XX:MaxPermSize=512M" -jdk: - - oraclejdk8 - before_install: - ./build-tools/install-protobuf.sh script: + - jdk_switcher use oraclejdk8 - mvn -B clean install package -DskipTests=true -Dmaven.javadoc.skip=true From 849e1d7694cdfd2432d631830940bc95c6f26ead Mon Sep 17 00:00:00 2001 From: Lewis John McGibbney Date: Fri, 11 Dec 2020 09:42:25 -0600 Subject: [PATCH 016/137] TEZ-4258. Fix travisci build (addendum 1) Signed-off-by: Jonathan Eagles --- .travis.yml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/.travis.yml b/.travis.yml index 0437531837..3637a0defe 100644 --- a/.travis.yml +++ b/.travis.yml @@ -14,6 +14,8 @@ # See the License for the specific language governing permissions and # limitations under the License. +dist: trusty + language: java sudo: required From 979bf5d7d7a1d3d66839e330388d3c9ec975b1c3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?L=C3=A1szl=C3=B3=20Bodor?= Date: Fri, 15 Jan 2021 10:08:50 -0600 Subject: [PATCH 017/137] TEZ-3706. add option to skip Tez UI build Signed-off-by: Jonathan Eagles --- BUILDING.txt | 10 ++ pom.xml | 4 +- tez-ui/pom.xml | 277 +++++++++++++++++++++++++++++++++++++++++-------- 3 files changed, 245 insertions(+), 46 deletions(-) diff --git a/BUILDING.txt b/BUILDING.txt index f2231ea088..875bf3e097 100644 --- a/BUILDING.txt +++ b/BUILDING.txt @@ -104,6 +104,16 @@ Issue with PhantomJS on building in PowerPC. please try installing PhantomJS manually and rerun. Refer https://github.com/ibmsoe/phantomjs-1/blob/v2.1.1-ppc64/README.md and install it globally for the build to work. +---------------------------------------------------------------------------------- +Skip UI build: + +In case you want to completely skip UI build, you can use 'noui' profile. +For instance, a full build without tests and tez-ui looks like: + + $ mvn clean install -DskipTests -Pnoui + +It's important to note that maven will still include tez-ui project, but all of the maven plugins are skipped. + ---------------------------------------------------------------------------------- Protocol Buffer compiler: diff --git a/pom.xml b/pom.xml index 04088bd9f8..59ec193dfb 100644 --- a/pom.xml +++ b/pom.xml @@ -825,7 +825,7 @@ org.apache.maven.plugins maven-war-plugin - 2.5 + 3.2.3 org.apache.maven.plugins @@ -921,7 +921,7 @@ org.apache.maven.plugins maven-resources-plugin - 2.6 + 3.1.0 UTF-8 diff --git a/tez-ui/pom.xml b/tez-ui/pom.xml index 500d21af87..e52eb52dfa 100644 --- a/tez-ui/pom.xml +++ b/tez-ui/pom.xml @@ -40,6 +40,173 @@ + + + + default + + + !test + + + + + + exec-maven-plugin + org.codehaus.mojo + + + + ember build + generate-resources + + exec + + + ${webappDir} + ${nodeExecutable} + + ${packageManagerScript} + run + build:mvn + + + + + + + ember test + test + + exec + + + ${skipTests} + ${webappDir} + ${nodeExecutable} + + ${packageManagerScript} + run + test:mvn + + + + + + + + + + + + + ui-tests + + + test + ember + + + + + + exec-maven-plugin + org.codehaus.mojo + + + + ember build + generate-resources + + exec + + + ${webappDir} + ${nodeExecutable} + + ${packageManagerScript} + run + build:mvn + + + + + + + ember test + test + + exec + + + ${skipTests} + ${webappDir} + ${nodeExecutable} + + ${packageManagerScript} + run + test:mvn + + + + + + + + + + + + + no-ui-tests + + + test + .* + + + + + + exec-maven-plugin + org.codehaus.mojo + + + + ember build + generate-resources + + exec + + + ${webappDir} + ${nodeExecutable} + + ${packageManagerScript} + run + build:mvn + + + + + + + ember test + test + + exec + + + ${nodeExecutable} + true + + + + + + + + cleanUICache @@ -86,6 +253,72 @@ --allow-root=true + + noui + + false + + + + + org.apache.rat + apache-rat-plugin + + true + + + + com.github.eirslett + frontend-maven-plugin + + true + + + + exec-maven-plugin + org.codehaus.mojo + + true + + + + ro.isdc.wro4j + wro4j-maven-plugin + + true + + + + org.apache.maven.plugins + maven-war-plugin + + true + + + + org.apache.maven.plugins + maven-jar-plugin + + true + + + + org.apache.maven.plugins + maven-resources-plugin + + true + + + + org.apache.maven.plugins + maven-install-plugin + + true + + + + + @@ -167,50 +400,6 @@ - - exec-maven-plugin - org.codehaus.mojo - - - - - ember build - generate-resources - - exec - - - ${webappDir} - ${nodeExecutable} - - ${packageManagerScript} - run - build:mvn - - - - - - - ember test - test - - exec - - - ${skipTests} - ${webappDir} - ${nodeExecutable} - - ${packageManagerScript} - run - test:mvn - - - - - - ro.isdc.wro4j From fb81c11583ea7c31978e2417e0e31f4a141dd7a7 Mon Sep 17 00:00:00 2001 From: Lewis John McGibbney Date: Fri, 15 Jan 2021 16:08:49 -0600 Subject: [PATCH 018/137] TEZ-4261. Fix Javadoc warnings Signed-off-by: Jonathan Eagles --- .../tez/mapreduce/hadoop/DeprecatedKeys.java | 6 ----- .../tez/mapreduce/hadoop/MRInputHelpers.java | 22 +++++++++---------- 2 files changed, 11 insertions(+), 17 deletions(-) diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/DeprecatedKeys.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/DeprecatedKeys.java index b8d491a4b0..098057ba17 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/DeprecatedKeys.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/DeprecatedKeys.java @@ -22,7 +22,6 @@ import java.util.HashMap; import java.util.Map; -import org.apache.hadoop.conf.Configuration; import org.apache.tez.dag.api.TezConfiguration; import org.apache.tez.dag.library.vertexmanager.ShuffleVertexManager; import org.apache.tez.runtime.library.api.TezRuntimeConfiguration; @@ -184,11 +183,6 @@ private static void registerMRToRuntimeKeyTranslation(String mrKey, String tezKey) { mrParamToTezRuntimeParamMap.put(mrKey, tezKey); } - - @SuppressWarnings("unused") - private static void _(String mrKey, String tezKey) { - Configuration.addDeprecation(mrKey, tezKey); - } public static Map getMRToDAGParamMap() { return Collections.unmodifiableMap(mrParamToDAGParamMap); diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MRInputHelpers.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MRInputHelpers.java index 9c42cdaf4e..19d64a5abb 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MRInputHelpers.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MRInputHelpers.java @@ -782,7 +782,7 @@ private static int getIntProperty(Configuration conf, String propertyName) { } /** - * @see {@link InputContext#getDagIdentifier} + * @see InputContext#getDagIdentifier() * @param conf configuration instance * @return dag index */ @@ -802,7 +802,7 @@ public static String getDagIdString(Configuration conf) { } /** - * * @see {@link InputContext#getTaskVertexIndex} + * @see InputContext#getTaskVertexIndex * @param conf configuration instance * @return vertex index */ @@ -822,7 +822,7 @@ public static String getVertexIdString(Configuration conf) { } /** - * @see {@link InputContext#getTaskIndex} + * @see InputContext#getTaskIndex * @param conf configuration instance * @return task index */ @@ -842,7 +842,7 @@ public static String getTaskIdString(Configuration conf) { } /** - * @see {@link InputContext#getTaskAttemptNumber} + * @see InputContext#getTaskAttemptNumber * @param conf configuration instance * @return task attempt index */ @@ -862,7 +862,7 @@ public static String getTaskAttemptIdString(Configuration conf) { } /** - * @see {@link InputContext#getInputIndex} + * @see InputContext#getInputIndex * @param conf configuration instance * @return input index */ @@ -872,7 +872,7 @@ public static int getInputIndex(Configuration conf) { } /** - * @see {@link InputContext#getDAGName} + * @see InputContext#getDAGName * @param conf configuration instance * @return dag name */ @@ -882,7 +882,7 @@ public static String getDagName(Configuration conf) { } /** - * @see {@link InputContext#getTaskVertexName} + * @see InputContext#getTaskVertexName * @param conf configuration instance * @return vertex name */ @@ -892,7 +892,7 @@ public static String getVertexName(Configuration conf) { } /** - * @see {@link InputContext#getSourceVertexName} + * @see InputContext#getSourceVertexName * @param conf configuration instance * @return source name */ @@ -902,7 +902,7 @@ public static String getInputName(Configuration conf) { } /** - * @see {@link InputContext#getApplicationId} + * @see InputContext#getApplicationId * @param conf configuration instance * @return applicationId as a string */ @@ -912,7 +912,7 @@ public static String getApplicationIdString(Configuration conf) { } /** - * @see {@link InputContext#getUniqueIdentifier} + * @see InputContext#getUniqueIdentifier * @param conf configuration instance * @return unique identifier for the input */ @@ -922,7 +922,7 @@ public static String getUniqueIdentifier(Configuration conf) { } /** - * @see {@link InputContext#getDAGAttemptNumber} + * @see InputContext#getDAGAttemptNumber * @param conf configuration instance * @return attempt number */ From c2d48a688e041e2b5689d26908a7de7d135bff3e Mon Sep 17 00:00:00 2001 From: belugabehr <12578579+belugabehr@users.noreply.github.com> Date: Wed, 20 Jan 2021 15:53:15 -0500 Subject: [PATCH 019/137] TEZ-4095: Review of Debug Logging (#87) --- .../org/apache/tez/client/TezClientUtils.java | 13 ++---- .../apache/tez/common/JavaOptsChecker.java | 11 ++--- .../org/apache/tez/common/ProgressHelper.java | 12 ++--- .../apache/tez/dag/api/TezConfiguration.java | 4 +- .../tez/dag/api/client/DAGClientImpl.java | 16 ++----- .../dag/api/client/TimelineReaderFactory.java | 6 +-- .../dag/api/client/rpc/DAGClientRPCImpl.java | 8 +--- .../tez/dag/api/client/DAGClientHandler.java | 4 +- .../org/apache/tez/dag/app/DAGAppMaster.java | 36 +++++---------- .../apache/tez/dag/app/RecoveryParser.java | 7 +-- .../tez/dag/app/TaskCommunicatorManager.java | 13 ++---- .../dag/app/TezLocalTaskCommunicatorImpl.java | 4 +- .../tez/dag/app/TezTaskCommunicatorImpl.java | 17 ++----- .../apache/tez/dag/app/dag/impl/DAGImpl.java | 5 +- .../tez/dag/app/dag/impl/TaskAttemptImpl.java | 31 ++++--------- .../apache/tez/dag/app/dag/impl/TaskImpl.java | 12 ++--- .../tez/dag/app/dag/impl/VertexImpl.java | 46 +++++-------------- .../speculation/legacy/LegacySpeculator.java | 4 +- .../launcher/TezContainerLauncherImpl.java | 4 +- .../dag/app/rm/DagAwareYarnTaskScheduler.java | 2 +- .../tez/dag/app/rm/TaskSchedulerManager.java | 8 +--- .../dag/app/rm/YarnTaskSchedulerService.java | 31 ++++--------- .../app/rm/container/AMContainerHelpers.java | 4 +- .../tez/dag/app/web/AMWebController.java | 8 +--- .../apache/tez/dag/app/web/WebUIService.java | 4 +- .../dag/history/recovery/RecoveryService.java | 22 +++------ .../split/TezGroupedSplitsInputFormat.java | 12 ++--- .../split/TezGroupedSplitsInputFormat.java | 12 ++--- .../tez/mapreduce/client/YARNRunner.java | 4 +- .../tez/mapreduce/hadoop/MRInputHelpers.java | 8 +--- .../apache/tez/mapreduce/output/MROutput.java | 4 +- .../tez/mapreduce/processor/MRTask.java | 5 +- .../apache/tez/auxservices/IndexCache.java | 17 ++----- .../tez/auxservices/ShuffleHandler.java | 24 +++------- .../LogicalIOProcessorRuntimeTask.java | 24 +++------- .../runtime/api/impl/TezInputContextImpl.java | 4 +- .../api/impl/TezOutputContextImpl.java | 4 +- .../api/impl/TezProcessorContextImpl.java | 4 +- .../common/resources/ScalingAllocator.java | 9 +--- .../apache/tez/runtime/task/TaskReporter.java | 8 +--- .../org/apache/tez/runtime/task/TezChild.java | 4 +- .../org/apache/tez/http/HttpConnection.java | 8 +--- .../http/async/netty/AsyncHttpConnection.java | 4 +- .../library/common/TezRuntimeUtils.java | 4 +- .../library/common/shuffle/ShuffleUtils.java | 8 +--- .../impl/ShuffleInputEventHandlerImpl.java | 12 ++--- .../common/shuffle/impl/ShuffleManager.java | 20 ++------ .../orderedgrouped/FetcherOrderedGrouped.java | 38 ++++----------- .../shuffle/orderedgrouped/MergeManager.java | 8 +--- ...huffleInputEventHandlerOrderedGrouped.java | 17 ++----- .../orderedgrouped/ShuffleScheduler.java | 12 ++--- .../library/common/sort/impl/IFile.java | 4 +- .../task/local/output/TezTaskOutputFiles.java | 6 +-- .../library/input/OrderedGroupedKVInput.java | 4 +- .../WeightedScalingMemoryDistributor.java | 4 +- .../org/apache/tez/tools/TFileLoader.java | 5 +- 56 files changed, 167 insertions(+), 462 deletions(-) diff --git a/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java b/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java index d34d31e4cf..1e09489564 100644 --- a/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java +++ b/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java @@ -479,9 +479,7 @@ public static ApplicationSubmissionContext createApplicationSubmissionContext( capability.setVirtualCores( amConfig.getTezConfiguration().getInt(TezConfiguration.TEZ_AM_RESOURCE_CPU_VCORES, TezConfiguration.TEZ_AM_RESOURCE_CPU_VCORES_DEFAULT)); - if (LOG.isDebugEnabled()) { - LOG.debug("AppMaster capability = " + capability); - } + LOG.debug("AppMaster capability = {}", capability); // Setup required Credentials for the AM launch. DAG specific credentials // are handled separately. @@ -531,10 +529,7 @@ public static ApplicationSubmissionContext createApplicationSubmissionContext( } vargsFinal.add(mergedCommand.toString()); - if (LOG.isDebugEnabled()) { - LOG.debug("Command to launch container for ApplicationMaster is : " - + mergedCommand); - } + LOG.debug("Command to launch container for ApplicationMaster is : {}", mergedCommand); Map environment = new TreeMap(); TezYARNUtils.setupDefaultEnv(environment, conf, @@ -968,9 +963,7 @@ public static DAGClientAMProtocolBlockingPB getAMProxy(final Configuration conf, serviceAddr); userUgi.addToken(token); } - if (LOG.isDebugEnabled()) { - LOG.debug("Connecting to Tez AM at " + serviceAddr); - } + LOG.debug("Connecting to Tez AM at {}", serviceAddr); DAGClientAMProtocolBlockingPB proxy = null; try { proxy = userUgi.doAs(new PrivilegedExceptionAction() { diff --git a/tez-api/src/main/java/org/apache/tez/common/JavaOptsChecker.java b/tez-api/src/main/java/org/apache/tez/common/JavaOptsChecker.java index 6de402aae9..3e93446149 100644 --- a/tez-api/src/main/java/org/apache/tez/common/JavaOptsChecker.java +++ b/tez-api/src/main/java/org/apache/tez/common/JavaOptsChecker.java @@ -38,9 +38,8 @@ public class JavaOptsChecker { public void checkOpts(String opts) throws TezException { Set gcOpts = new TreeSet(); - if (LOG.isDebugEnabled()) { - LOG.debug("Checking JVM GC opts: " + opts); - } + LOG.debug("Checking JVM GC opts: {}", opts); + Matcher matcher = pattern.matcher(opts); while (matcher.find()) { if (matcher.groupCount() != 3) { @@ -74,10 +73,8 @@ public void checkOpts(String opts) throws TezException { } } - if (LOG.isDebugEnabled()) { - LOG.debug("Found clashing GC opts" - + ", conflicting GC Values=" + gcOpts); - } + LOG.debug("Found clashing GC opts, conflicting GC Values={}", gcOpts); + throw new TezException("Invalid/conflicting GC options found," + " cmdOpts=\"" + opts + "\""); } diff --git a/tez-api/src/main/java/org/apache/tez/common/ProgressHelper.java b/tez-api/src/main/java/org/apache/tez/common/ProgressHelper.java index 1518ccdaf1..289847a96a 100644 --- a/tez-api/src/main/java/org/apache/tez/common/ProgressHelper.java +++ b/tez-api/src/main/java/org/apache/tez/common/ProgressHelper.java @@ -117,10 +117,8 @@ public void run() { // Report progress as 0.0f when if are errors. processorContext.setProgress(progressVal); } catch (Throwable th) { - if (LOG.isDebugEnabled()) { - LOG.debug("progress update: Encountered InterruptedException during" - + " Processor={}", processorName, th); - } + LOG.debug("progress update: Encountered InterruptedException during" + + " Processor={}", processorName, th); if (th instanceof InterruptedException) { // set interrupt flag to true sand exit Thread.currentThread().interrupt(); @@ -161,10 +159,8 @@ public void shutDownProgressTaskService() { scheduledExecutorService.shutdownNow(); } } catch (InterruptedException e) { - if (LOG.isDebugEnabled()) { - LOG.debug("Interrupted exception while shutting down the " - + "executor service for the processor name={}", processorName); - } + LOG.debug("Interrupted exception while shutting down the " + + "executor service for the processor name={}", processorName); } scheduledExecutorService.shutdownNow(); } 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 85f85181ca..179b1957e8 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 @@ -1899,9 +1899,7 @@ public TezConfiguration(boolean loadDefaults) { public static void validateProperty(String property, Scope usedScope) { Scope validScope = PropertyScope.get(property); if (validScope == null) { - if (LOG.isDebugEnabled()) { - LOG.debug(property + " is not standard configuration property of tez, can not been validated"); - } + LOG.debug("{} is not standard configuration property of tez, can not been validated", property); } else { if (usedScope.ordinal() > validScope.ordinal()) { throw new IllegalStateException(property + " is set at the scope of " + usedScope 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 01a10b269c..b54db324a9 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 @@ -247,9 +247,7 @@ private DAGStatus getDAGStatusInternal(@Nullable Set statusOption LOG.info("Failed to fetch DAG data for completed DAG from YARN Timeline" + " - Application not found by YARN", e); } catch (TezException e) { - if (LOG.isDebugEnabled()) { - LOG.info("DAGStatus fetch failed." + e.getMessage()); - } + LOG.debug("DAGStatus fetch failed", e); } } @@ -302,9 +300,7 @@ public VertexStatus getVertexStatus(String vertexName, Set status + " - Application not found by YARN", e); return null; } catch (TezException e) { - if (LOG.isDebugEnabled()) { - LOG.debug("ERROR fetching vertex data from Yarn Timeline. " + e.getMessage()); - } + LOG.debug("ERROR fetching vertex data from Yarn Timeline", e); } } @@ -425,9 +421,7 @@ private VertexStatus getVertexStatusViaAM(String vertexName, Set */ @VisibleForTesting protected DAGStatus getDAGStatusViaRM() throws TezException, IOException { - if(LOG.isDebugEnabled()) { - LOG.debug("GetDAGStatus via AM for app: " + appId + " dag:" + dagId); - } + LOG.debug("GetDAGStatus via AM for app: {} dag:{}", appId, dagId); ApplicationReport appReport; try { appReport = frameworkClient.getApplicationReport(appId); @@ -638,9 +632,7 @@ private void switchToTimelineClient() throws IOException, TezException { realClient.close(); realClient = new DAGClientTimelineImpl(appId, dagId, conf, frameworkClient, (int) (2 * PRINT_STATUS_INTERVAL_MILLIS)); - if (LOG.isDebugEnabled()) { - LOG.debug("dag completed switching to DAGClientTimelineImpl"); - } + LOG.debug("dag completed switching to DAGClientTimelineImpl"); } @VisibleForTesting 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 index 40340cc44e..fc1595fa49 100644 --- 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 @@ -313,10 +313,8 @@ private static ConnectionConfigurator getNewConnectionConf(final Configuration c try { connectionConf = getNewSSLConnectionConf(conf, connTimeout, sslFactory); } catch (IOException e) { - if (LOG.isDebugEnabled()) { - LOG.debug("Cannot load customized ssl related configuration." - + " Falling back to system-generic settings.", e); - } + LOG.debug("Cannot load customized ssl related configuration." + + " Falling back to system-generic settings.", e); } } diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientRPCImpl.java b/tez-api/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientRPCImpl.java index c54058be8a..5d5752e6e2 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientRPCImpl.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientRPCImpl.java @@ -146,9 +146,7 @@ public String getSessionIdentifierString() { @Override public void tryKillDAG() throws TezException, IOException { - if(LOG.isDebugEnabled()) { - LOG.debug("TryKill for app: " + appId + " dag:" + dagId); - } + LOG.debug("TryKill for app: {} dag:{}", appId, dagId); try { if (createAMProxyIfNeeded()) { TryKillDAGRequestProto requestProto = @@ -186,9 +184,7 @@ void resetProxy(Exception e) { DAGStatus getDAGStatusViaAM(Set statusOptions, long timeout) throws IOException, TezException { - if(LOG.isDebugEnabled()) { - LOG.debug("GetDAGStatus via AM for app: " + appId + " dag:" + dagId); - } + LOG.debug("GetDAGStatus via AM for app: {} dag:{}", appId, dagId); GetDAGStatusRequestProto.Builder requestProtoBuilder = GetDAGStatusRequestProto.newBuilder() .setDagId(dagId).setTimeout(timeout); diff --git a/tez-dag/src/main/java/org/apache/tez/dag/api/client/DAGClientHandler.java b/tez-dag/src/main/java/org/apache/tez/dag/api/client/DAGClientHandler.java index 618676d978..4cdd1ec9d1 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/api/client/DAGClientHandler.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/api/client/DAGClientHandler.java @@ -100,9 +100,7 @@ DAG getDAG(String dagIdStr) throws TezException { final String currentDAGIdStr = currentDAG.getID().toString(); if (!currentDAGIdStr.equals(dagIdStr)) { if (getAllDagIDs().contains(dagIdStr)) { - if (LOG.isDebugEnabled()) { - LOG.debug("Looking for finished dagId " + dagIdStr + " current dag is " + currentDAGIdStr); - } + LOG.debug("Looking for finished dagId {} current dag is {}", dagIdStr, currentDAGIdStr); throw new DAGNotRunningException("DAG " + dagIdStr + " Not running, current dag is " + currentDAGIdStr); } else { 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 f4a8923d4a..cde77b3bf6 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 @@ -576,9 +576,7 @@ public synchronized void serviceInit(final Configuration conf) throws Exception this.webUIService = new WebUIService(context); addIfService(webUIService, false); } else { - if (LOG.isDebugEnabled()) { - LOG.debug("Web UI Service is not enabled."); - } + LOG.debug("Web UI Service is not enabled."); } this.taskSchedulerManager = createTaskSchedulerManager(taskSchedulerDescriptors); @@ -1335,9 +1333,8 @@ public String submitDAGToAppMaster(DAGPlan dagPlan, // the job user's UGI context LOG.info("Starting DAG submitted via RPC: " + dagPlan.getName()); - if (LOG.isDebugEnabled()) { - LOG.debug("Invoked with additional local resources: " + additionalResources); - } + LOG.debug("Invoked with additional local resources: {}", additionalResources); + if (!dagPlan.getName().startsWith(TezConstants.TEZ_PREWARM_DAG_NAME_PREFIX)) { submittedDAGs.incrementAndGet(); } @@ -1863,9 +1860,8 @@ void startServices() { try { Throwable firstError = null; List threads = new ArrayList(); - if(LOG.isDebugEnabled()) { - LOG.debug("Begin parallel start"); - } + LOG.debug("Begin parallel start"); + for(ServiceWithDependency sd : services.values()) { // start the service. If this fails that service // will be stopped and an exception raised @@ -1889,9 +1885,7 @@ void startServices() { if(firstError != null) { throw ServiceStateException.convert(firstError); } - if(LOG.isDebugEnabled()) { - LOG.debug("End parallel start"); - } + LOG.debug("End parallel start"); } catch (InterruptedException e) { e.printStackTrace(); } @@ -1899,9 +1893,7 @@ void startServices() { void initServices(Configuration conf) { for (ServiceWithDependency sd : services.values()) { - if (LOG.isDebugEnabled()) { - LOG.debug("Initing service : " + sd.service); - } + LOG.debug("Initing service : {}", sd.service); sd.service.init(conf); } } @@ -1919,9 +1911,7 @@ void stopServices() { for (int i = services.size() - 1; i >= 0; i--) { Service service = serviceList.get(i); - if (LOG.isDebugEnabled()) { - LOG.debug("Stopping service : " + service); - } + LOG.debug("Stopping service : {}", service); Exception ex = ServiceOperations.stopQuietly(service); if (ex != null && firstException == null) { LOG.warn("Failed to stop service, name=" + service.getName(), ex); @@ -2163,10 +2153,8 @@ public void serviceStop() throws Exception { boolean deleteTezScratchData = this.amConf.getBoolean( TezConfiguration.TEZ_AM_STAGING_SCRATCH_DATA_AUTO_DELETE, TezConfiguration.TEZ_AM_STAGING_SCRATCH_DATA_AUTO_DELETE_DEFAULT); - if (LOG.isDebugEnabled()) { - LOG.debug("Checking whether tez scratch data dir should be deleted, deleteTezScratchData=" - + deleteTezScratchData); - } + LOG.debug("Checking whether tez scratch data dir should be deleted, deleteTezScratchData={}", + deleteTezScratchData); if (deleteTezScratchData && this.taskSchedulerManager != null && this.taskSchedulerManager.hasUnregistered()) { // Delete tez scratch data dir @@ -2443,9 +2431,7 @@ static class DAGAppMasterShutdownHook implements Runnable { public void run() { LOG.info("DAGAppMasterShutdownHook invoked"); if(appMaster.getServiceState() == STATE.STOPPED) { - if(LOG.isDebugEnabled()) { - LOG.debug("DAGAppMaster already stopped. Ignoring signal"); - } + LOG.debug("DAGAppMaster already stopped. Ignoring signal"); synchronized (appMaster.shutdownHandlerRunning) { try { if (appMaster.shutdownHandlerRunning.get()) { diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/RecoveryParser.java b/tez-dag/src/main/java/org/apache/tez/dag/app/RecoveryParser.java index dfb7f61e6e..19c24f300c 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/RecoveryParser.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/RecoveryParser.java @@ -346,11 +346,8 @@ private static HistoryEvent getNextEvent(CodedInputStream inputStream) } catch (EOFException eof) { return null; } - if (LOG.isDebugEnabled()) { - LOG.debug("Parsed event from input stream" - + ", eventType=" + eventType - + ", event=" + event.toString()); - } + LOG.debug("Parsed event from input stream, eventType={}, event={}", + eventType, event); return event; } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorManager.java index 55b2d1b021..3a99456ed7 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorManager.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorManager.java @@ -220,10 +220,7 @@ 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); - } + LOG.debug("Received heartbeat from container, request={}", request); if (!registeredContainers.containsKey(containerId)) { LOG.warn("Received task heartbeat from unknown container with id: " + containerId + @@ -488,9 +485,7 @@ public void dagSubmitted() { @Override public void registerRunningContainer(ContainerId containerId, int taskCommId) { - if (LOG.isDebugEnabled()) { - LOG.debug("ContainerId: " + containerId + " registered with TaskAttemptListener"); - } + LOG.debug("ContainerId: {} registered with TaskAttemptListener", containerId); ContainerInfo oldInfo = registeredContainers.put(containerId, NULL_CONTAINER_INFO); if (oldInfo != null) { throw new TezUncheckedException( @@ -515,9 +510,7 @@ public void registerRunningContainer(ContainerId containerId, int taskCommId) { @Override public void unregisterRunningContainer(ContainerId containerId, int taskCommId, ContainerEndReason endReason, String diagnostics) { - if (LOG.isDebugEnabled()) { - LOG.debug("Unregistering Container from TaskAttemptListener: " + containerId); - } + LOG.debug("Unregistering Container from TaskAttemptListener: {}", containerId); ContainerInfo containerInfo = registeredContainers.remove(containerId); if (containerInfo.taskAttemptId != null) { registeredAttempts.remove(containerInfo.taskAttemptId); 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 15d90d3832..b5749591c1 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 @@ -39,8 +39,6 @@ protected void startRpcServer() { } catch (UnknownHostException e) { throw new TezUncheckedException(e); } - if (LOG.isDebugEnabled()) { - LOG.debug("Not starting TaskAttemptListener RPC in LocalMode"); - } + LOG.debug("Not starting TaskAttemptListener RPC in LocalMode"); } } 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 9c8fb6c67e..6d69d36014 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 @@ -310,18 +310,14 @@ public ContainerTask getTask(ContainerContext containerContext) throws IOExcepti } else { ContainerId containerId = ConverterUtils.toContainerId(containerContext .getContainerIdentifier()); - if (LOG.isDebugEnabled()) { - LOG.debug("Container with id: " + containerId + " asked for a task"); - } + LOG.debug("Container with id: {} asked for a task", containerId); task = getContainerTask(containerId); if (task != null && !task.shouldDie()) { getContext().taskSubmitted(task.getTaskSpec().getTaskAttemptID(), containerId); getContext().taskStartedRemotely(task.getTaskSpec().getTaskAttemptID()); } } - if (LOG.isDebugEnabled()) { - LOG.debug("getTask returning task: " + task); - } + LOG.debug("getTask returning task: {}", task); return task; } @@ -335,10 +331,7 @@ public TezHeartbeatResponse heartbeat(TezHeartbeatRequest request) throws IOExce TezException { ContainerId containerId = ConverterUtils.toContainerId(request.getContainerIdentifier()); long requestId = request.getRequestId(); - if (LOG.isDebugEnabled()) { - LOG.debug("Received heartbeat from container" - + ", request=" + request); - } + LOG.debug("Received heartbeat from container, request={}", request); ContainerInfo containerInfo = registeredContainers.get(containerId); if (containerInfo == null) { @@ -436,9 +429,7 @@ private ContainerTask getContainerTask(ContainerId containerId) throws IOExcepti } } else { task = null; - if (LOG.isDebugEnabled()) { - LOG.debug("No task assigned yet for running container: " + containerId); - } + LOG.debug("No task assigned yet for running container: {}", containerId); } } } 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 bb5c1aa966..09e9e71b92 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 @@ -1651,10 +1651,7 @@ DAGState initializeDAG() { if (!groupInfo.outputs.isEmpty()) { // shared outputs for (String vertexName : groupInfo.groupMembers) { - if (LOG.isDebugEnabled()) { - LOG.debug("Setting shared outputs for group: " + groupName + - " on vertex: " + vertexName); - } + LOG.debug("Setting shared outputs for group: {} on vertex: {}", groupName, vertexName); Vertex v = getVertex(vertexName); v.addSharedOutputs(groupInfo.outputs); } 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 9a5e73de5a..8b16b2e8b7 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 @@ -1267,10 +1267,8 @@ public TaskAttemptStateInternal transition(TaskAttemptImpl ta, TaskAttemptEvent TaskAttemptFinishedEvent taFinishedEvent = ta.recoveryData.getTaskAttemptFinishedEvent(); if (taFinishedEvent == null) { - if (LOG.isDebugEnabled()) { - LOG.debug("Only TaskAttemptStartedEvent but no TaskAttemptFinishedEvent, " - + "send out TaskAttemptEventAttemptKilled to move it to KILLED"); - } + LOG.debug("Only TaskAttemptStartedEvent but no TaskAttemptFinishedEvent, " + + "send out TaskAttemptEventAttemptKilled to move it to KILLED"); ta.sendEvent(new TaskAttemptEventAttemptKilled(ta.getID(), "Task Attempt killed in recovery due to can't recover the running task attempt", TaskAttemptTerminationCause.TERMINATED_AT_RECOVERY, true)); @@ -1285,30 +1283,21 @@ public TaskAttemptStateInternal transition(TaskAttemptImpl ta, TaskAttemptEvent + "taskAttemptId=" + ta.getID()); switch (taFinishedEvent.getState()) { case FAILED: - if (LOG.isDebugEnabled()) { - LOG.debug("TaskAttemptFinishedEvent is seen with state of FAILED" - + ", send TA_FAILED to itself" - + ", attemptId=" + ta.attemptId); - } + LOG.debug("TaskAttemptFinishedEvent is seen with state of FAILED, " + + "send TA_FAILED to itself, attemptId={}", ta.attemptId); ta.sendEvent(new TaskAttemptEventAttemptFailed(ta.getID(), TaskAttemptEventType.TA_FAILED, taFinishedEvent.getTaskFailureType(), taFinishedEvent.getDiagnostics(), taFinishedEvent.getTaskAttemptError(), true)); break; case KILLED: - if (LOG.isDebugEnabled()) { - LOG.debug("TaskAttemptFinishedEvent is seen with state of KILLED" - + ", send TA_KILLED to itself" - + ", attemptId=" + ta.attemptId); - } + LOG.debug("TaskAttemptFinishedEvent is seen with state of KILLED, " + + "send TA_KILLED to itself, attemptId={}", ta.attemptId); ta.sendEvent(new TaskAttemptEventAttemptKilled(ta.getID(), taFinishedEvent.getDiagnostics(), taFinishedEvent.getTaskAttemptError(), true)); break; case SUCCEEDED: - if (LOG.isDebugEnabled()) { - LOG.debug("TaskAttemptFinishedEvent is seen with state of SUCCEEDED" - + ", send TA_DONE to itself" - + ", attemptId=" + ta.attemptId); - } + LOG.debug("TaskAttemptFinishedEvent is seen with state of SUCCEEDED, " + + "send TA_DONE to itself, attemptId={}", ta.attemptId); ta.sendEvent(new TaskAttemptEvent(ta.getID(), TaskAttemptEventType.TA_DONE)); break; default: @@ -1671,9 +1660,7 @@ public void transition(TaskAttemptImpl ta, TaskAttemptEvent event) { if (ta.recoveryData != null && ta.recoveryData.isTaskAttemptSucceeded()) { TaskAttemptFinishedEvent taFinishedEvent = ta.recoveryData .getTaskAttemptFinishedEvent(); - if (LOG.isDebugEnabled()) { - LOG.debug("TaskAttempt is recovered to SUCCEEDED, attemptId=" + ta.attemptId); - } + LOG.debug("TaskAttempt is recovered to SUCCEEDED, attemptId={}", ta.attemptId); ta.reportedStatus.counters = taFinishedEvent.getCounters(); List tezEvents = taFinishedEvent.getTAGeneratedEvents(); if (tezEvents != null && !tezEvents.isEmpty()) { 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 0b4b1160f2..cb8545f8ca 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 @@ -690,9 +690,7 @@ private TaskAttempt selectBestAttempt() { public boolean canCommit(TezTaskAttemptID taskAttemptID) { writeLock.lock(); try { - if (LOG.isDebugEnabled()) { - LOG.debug("Commit go/no-go request from " + taskAttemptID); - } + LOG.debug("Commit go/no-go request from {}", taskAttemptID); TaskState state = getState(); if (state == TaskState.SCHEDULED) { // the actual running task ran and is done and asking for commit. we are still stuck @@ -730,9 +728,7 @@ public boolean canCommit(TezTaskAttemptID taskAttemptID) { } } else { if (commitAttempt.equals(taskAttemptID)) { - if (LOG.isDebugEnabled()) { - LOG.debug(taskAttemptID + " already given a go for committing the task output."); - } + LOG.debug("{} already given a go for committing the task output.", taskAttemptID); return true; } // Don't think this can be a pluggable decision, so simply raise an @@ -740,9 +736,7 @@ public boolean canCommit(TezTaskAttemptID taskAttemptID) { // Wait for commit attempt to succeed. Dont kill this. If commit // attempt fails then choose a different committer. When commit attempt // succeeds then this and others will be killed - if (LOG.isDebugEnabled()) { - LOG.debug(commitAttempt + " is current committer. Commit waiting for: " + taskAttemptID); - } + LOG.debug("{} is current committer. Commit waiting for: {}", commitAttempt, taskAttemptID); return false; } 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 6ae3ba55a8..e21add0e3b 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 @@ -880,10 +880,8 @@ void resetCompletedTaskStatsCache(boolean recompute) { @Override public void initServices() { if (servicesInited.get()) { - if (LOG.isDebugEnabled()) { - LOG.debug("Skipping Initing services for vertex because already" - + " Initialized, name=" + this.vertexName); - } + LOG.debug("Skipping Initing services for vertex because already" + + " Initialized, name={}", this.vertexName); return; } writeLock.lock(); @@ -891,10 +889,7 @@ public void initServices() { List servicesToAdd = new ArrayList<>(); if (isSpeculationEnabled()) { // Initialize the speculator - if (LOG.isDebugEnabled()) { - LOG.debug( - "Initing service vertex speculator, name=" + this.vertexName); - } + LOG.debug("Initing service vertex speculator, name={}", this.vertexName); speculator = new LegacySpeculator(vertexConf, getAppContext(), this); speculator.init(vertexConf); servicesToAdd.add(speculator); @@ -904,9 +899,7 @@ public void initServices() { } finally { writeLock.unlock(); } - if (LOG.isDebugEnabled()) { - LOG.debug("Initing service vertex, name=" + this.vertexName); - } + LOG.debug("Initing service vertex, name={}", this.vertexName); } @Override @@ -936,9 +929,7 @@ public void stopServices() { try { if (servicesInited.get()) { for (AbstractService srvc : services) { - if (LOG.isDebugEnabled()) { - LOG.debug("Stopping service : " + srvc); - } + LOG.debug("Stopping service : {}", srvc); Exception ex = ServiceOperations.stopQuietly(srvc); if (ex != null && firstException == null) { LOG.warn(String.format( @@ -1591,7 +1582,7 @@ private void computeProgress() { if (LOG.isDebugEnabled()) { if (!ProgressHelper.isProgressWithinRange(taskProg)) { LOG.debug("progress update: vertex={}, task={} incorrect; range={}", - getName(), task.getTaskId().toString(), taskProg); + getName(), task.getTaskId(), taskProg); } } accProg += ProgressHelper.processProgress(taskProg); @@ -2523,11 +2514,8 @@ private void initializeCommitters() throws Exception { final RootInputLeafOutput od = entry.getValue(); if (od.getControllerDescriptor() == null || od.getControllerDescriptor().getClassName() == null) { - if (LOG.isDebugEnabled()) { - LOG.debug("Ignoring committer as none specified for output=" - + outputName - + ", vertexId=" + logIdentifier); - } + LOG.debug("Ignoring committer as none specified for output={}, vertexId={}", + outputName, logIdentifier); continue; } LOG.info("Instantiating committer for output=" + outputName @@ -2548,19 +2536,13 @@ public Void run() throws Exception { .createClazzInstance(od.getControllerDescriptor().getClassName(), new Class[]{OutputCommitterContext.class}, new Object[]{outputCommitterContext}); - if (LOG.isDebugEnabled()) { - LOG.debug("Invoking committer init for output=" + outputName - + ", vertex=" + logIdentifier); - } + LOG.debug("Invoking committer init for output={}, vertex={}", outputName, logIdentifier); try { TezUtilsInternal.setHadoopCallerContext(appContext.getHadoopShim(), vertexId); outputCommitter.initialize(); outputCommitters.put(outputName, outputCommitter); - if (LOG.isDebugEnabled()) { - LOG.debug("Invoking committer setup for output=" + outputName - + ", vertex=" + logIdentifier); - } + LOG.debug("Invoking committer setup for output={}, vertex={}", outputName, logIdentifier); outputCommitter.setupOutput(); } finally { appContext.getHadoopShim().clearHadoopCallerContext(); @@ -4741,9 +4723,7 @@ public NoOpVertexManager(VertexManagerPluginContext context) { @Override public void initialize() throws Exception { - if (LOG.isDebugEnabled()) { - LOG.debug("initialize NoOpVertexManager"); - } + LOG.debug("initialize NoOpVertexManager"); configurationDoneEvent = new VertexConfigurationDoneEvent(); configurationDoneEvent.fromProtoStream(CodedInputStream.newInstance(getContext().getUserPayload().deepCopyAsArray())); String vertexName = getContext().getVertexName(); @@ -4769,9 +4749,7 @@ public void onVertexStarted(List completions) } getContext().doneReconfiguringVertex(); int numTasks = getContext().getVertexNumTasks(getContext().getVertexName()); - if (LOG.isDebugEnabled()) { - LOG.debug("Schedule all the tasks, numTask=" + numTasks); - } + LOG.debug("Schedule all the tasks, numTask={}", numTasks); List tasks = new ArrayList(); for (int i=0;i statuses) { // being released // completion of a container we had released earlier // an allocated container completed. notify app - if (LOG.isDebugEnabled()) { - LOG.debug("Released container completed:" + completedId + - " last allocated to task: " + task); - } + LOG.debug("Released container completed:{} last allocated to task: {}", + completedId, task); appContainerStatus.put(task, containerStatus); continue; } @@ -1216,12 +1214,9 @@ boolean preemptIfNeeded() { if(!preemptionWaitDeadlineCrossed && fitsIn(highestPriRequest.getCapability(), freeResources)) { - if (LOG.isDebugEnabled()) { - LOG.debug(highestPriRequest + " fits in free resources"); - } else { - if (numHeartbeats % 50 == 1) { - LOG.info(highestPriRequest + " fits in free resources"); - } + LOG.debug("{} fits in free resources", highestPriRequest); + if (numHeartbeats % 50 == 1) { + LOG.info(highestPriRequest + " fits in free resources"); } return true; } @@ -1509,10 +1504,8 @@ private CookieContainerRequest getMatchingRequestWithoutPriority( if (container.getId().equals( cookieContainerRequest.getAffinitizedContainer())) { // container level match - if (LOG.isDebugEnabled()) { - LOG.debug("Matching with affinity for request: " - + cookieContainerRequest + " container: " + affCId); - } + LOG.debug("Matching with affinity for request: {} container: {}", + cookieContainerRequest, affCId); return cookieContainerRequest; } if (LOG.isDebugEnabled()) { @@ -2025,10 +2018,7 @@ private void mainLoop() { if (delayedContainer == null) { continue; } - if (LOG.isDebugEnabled()) { - LOG.debug("Considering HeldContainer: " - + delayedContainer + " for assignment"); - } + LOG.debug("Considering HeldContainer: {} for assignment", delayedContainer); long currentTs = System.currentTimeMillis(); long nextScheduleTs = delayedContainer.getNextScheduleTime(); if (currentTs >= nextScheduleTs) { @@ -2091,10 +2081,7 @@ private void doAssignAll() { // honor reuse-locality flags (container not timed out yet), Don't queue // (already in queue), don't release (release happens when containers // time-out) - if (LOG.isDebugEnabled()) { - LOG.debug("Trying to assign all delayed containers to newly received" - + " tasks"); - } + LOG.debug("Trying to assign all delayed containers to newly received tasks"); Iterator iter = delayedContainers.iterator(); while(iter.hasNext()) { HeldContainer delayedContainer = iter.next(); diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerHelpers.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerHelpers.java index ee322655af..19cf5b7e21 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerHelpers.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerHelpers.java @@ -126,9 +126,7 @@ private static ContainerLaunchContext createCommonContainerLaunchContext( containerTokens_dob.getLength()); // Add shuffle token - if (LOG.isDebugEnabled()) { - LOG.debug("Putting shuffle token in serviceData in common CLC"); - } + LOG.debug("Putting shuffle token in serviceData in common CLC"); serviceData.put(auxiliaryService, TezCommonUtils.serializeServiceData(TokenCache.getSessionToken(containerCredentials))); } catch (IOException e) { diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/web/AMWebController.java b/tez-dag/src/main/java/org/apache/tez/dag/app/web/AMWebController.java index 2115dac247..08d754d8a0 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/web/AMWebController.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/web/AMWebController.java @@ -144,9 +144,7 @@ public void setCorsHeaders() { URL url = new URL(historyUrlBase); origin = url.getProtocol() + "://" + url.getAuthority(); } catch (MalformedURLException e) { - if (LOG.isDebugEnabled()) { - LOG.debug("Invalid url set for tez history url base: " + historyUrlBase, e); - } + LOG.debug("Invalid url set for tez history url base: {}", historyUrlBase, e); } } @@ -161,9 +159,7 @@ public void setCorsHeaders() { } void sendErrorResponse(int sc, String msg, Exception e) { - if (LOG.isDebugEnabled()) { - LOG.debug(msg, e); - } + LOG.debug(msg, e); try { response().sendError(sc, msg); diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/web/WebUIService.java b/tez-dag/src/main/java/org/apache/tez/dag/app/web/WebUIService.java index b1560a5ead..1670370187 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/web/WebUIService.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/web/WebUIService.java @@ -119,9 +119,7 @@ protected void serviceStart() throws Exception { @Override protected void serviceStop() throws Exception { if (this.webApp != null) { - if (LOG.isDebugEnabled()) { - LOG.debug("Stopping WebApp"); - } + LOG.debug("Stopping WebApp"); this.webApp.stop(); } super.serviceStop(); diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/recovery/RecoveryService.java b/tez-dag/src/main/java/org/apache/tez/dag/history/recovery/RecoveryService.java index a0a152c81c..45e7d2fc9c 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/recovery/RecoveryService.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/recovery/RecoveryService.java @@ -405,11 +405,8 @@ private void createFatalErrorFlagDir() throws IOException { protected void handleSummaryEvent(TezDAGID dagID, HistoryEventType eventType, SummaryEvent summaryEvent) throws IOException { - if (LOG.isDebugEnabled()) { - LOG.debug("Handling summary event" - + ", dagID=" + dagID - + ", eventType=" + eventType); - } + LOG.debug("Handling summary event, dagID={}, eventType={}", dagID, eventType); + if (summaryStream == null) { Path summaryPath = TezCommonUtils.getSummaryRecoveryPath(recoveryPath); if (LOG.isDebugEnabled()) { @@ -470,11 +467,8 @@ protected void handleRecoveryEvent(DAGHistoryEvent event) throws IOException { } - if (LOG.isDebugEnabled()) { - LOG.debug("Writing recovery event to output stream" - + ", dagId=" + dagID - + ", eventType=" + eventType); - } + LOG.debug("Writing recovery event to output stream, dagId={}, eventType={}", + dagID, eventType); ++unflushedEventsCount; recoveryStream.codedOutputStream.writeFixed32NoTag(event.getHistoryEvent().getEventType().ordinal()); event.getHistoryEvent().toProtoStream(recoveryStream.codedOutputStream); @@ -489,11 +483,9 @@ private void maybeFlush(RecoveryStream recoveryStream) throws IOException { boolean doFlush = false; if (maxUnflushedEvents >=0 && unflushedEventsCount >= maxUnflushedEvents) { - if (LOG.isDebugEnabled()) { - LOG.debug("Max unflushed events count reached. Flushing recovery data" - + ", unflushedEventsCount=" + unflushedEventsCount - + ", maxUnflushedEvents=" + maxUnflushedEvents); - } + LOG.debug("Max unflushed events count reached. Flushing recovery data, " + + "unflushedEventsCount={}, maxUnflushedEvents={}", unflushedEventsCount, + maxUnflushedEvents); doFlush = true; } else if (flushInterval >= 0 && ((currentTime - lastFlushTime) >= (flushInterval*1000))) { diff --git a/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/split/TezGroupedSplitsInputFormat.java b/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/split/TezGroupedSplitsInputFormat.java index bce16eee56..61e1f6c431 100644 --- a/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/split/TezGroupedSplitsInputFormat.java +++ b/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/split/TezGroupedSplitsInputFormat.java @@ -68,25 +68,19 @@ public void setInputFormat(InputFormat wrappedInputFormat) { public void setSplitSizeEstimator(SplitSizeEstimator estimator) { Preconditions.checkArgument(estimator != null); this.estimator = estimator; - if (LOG.isDebugEnabled()) { - LOG.debug("Split size estimator : " + estimator); - } + LOG.debug("Split size estimator : {}", estimator); } public void setSplitLocationProvider(SplitLocationProvider locationProvider) { Preconditions.checkArgument(locationProvider != null); this.locationProvider = locationProvider; - if (LOG.isDebugEnabled()) { - LOG.debug("Split size location provider: " + locationProvider); - } + LOG.debug("Split size location provider: {}", locationProvider); } public void setDesiredNumberOfSplits(int num) { Preconditions.checkArgument(num >= 0); this.desiredNumSplits = num; - if (LOG.isDebugEnabled()) { - LOG.debug("desiredNumSplits: " + desiredNumSplits); - } + LOG.debug("desiredNumSplits: {}", desiredNumSplits); } @Override diff --git a/tez-mapreduce/src/main/java/org/apache/hadoop/mapreduce/split/TezGroupedSplitsInputFormat.java b/tez-mapreduce/src/main/java/org/apache/hadoop/mapreduce/split/TezGroupedSplitsInputFormat.java index 226425cd57..863f9aa792 100644 --- a/tez-mapreduce/src/main/java/org/apache/hadoop/mapreduce/split/TezGroupedSplitsInputFormat.java +++ b/tez-mapreduce/src/main/java/org/apache/hadoop/mapreduce/split/TezGroupedSplitsInputFormat.java @@ -70,25 +70,19 @@ public void setInputFormat(InputFormat wrappedInputFormat) { public void setDesiredNumberOfSplits(int num) { Preconditions.checkArgument(num >= 0); this.desiredNumSplits = num; - if (LOG.isDebugEnabled()) { - LOG.debug("desiredNumSplits: " + desiredNumSplits); - } + LOG.debug("desiredNumSplits: {}", desiredNumSplits); } public void setSplitSizeEstimator(SplitSizeEstimator estimator) { Preconditions.checkArgument(estimator != null); this.estimator = estimator; - if (LOG.isDebugEnabled()) { - LOG.debug("Split size estimator : " + estimator); - } + LOG.debug("Split size estimator : {}", estimator); } public void setSplitLocationProvider(SplitLocationProvider locationProvider) { Preconditions.checkArgument(locationProvider != null); this.locationProvider = locationProvider; - if (LOG.isDebugEnabled()) { - LOG.debug("Split location provider : " + locationProvider); - } + LOG.debug("Split location provider : {}", locationProvider); } @Override diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/client/YARNRunner.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/client/YARNRunner.java index 5a7d754ba5..9dba357951 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/client/YARNRunner.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/client/YARNRunner.java @@ -616,9 +616,7 @@ public JobStatus submitJob(JobID jobId, String jobSubmitDir, Credentials ts) dagAMConf.set(TezConfiguration.TEZ_AM_LAUNCH_CMD_OPTS, javaOpts.toString()); if (envStr.length() > 0) { dagAMConf.set(TezConfiguration.TEZ_AM_LAUNCH_ENV, envStr); - if (LOG.isDebugEnabled()) { - LOG.debug("Setting MR AM env to : " + envStr); - } + LOG.debug("Setting MR AM env to : {}", envStr); } // Submit to ResourceManager diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MRInputHelpers.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MRInputHelpers.java index 19d64a5abb..a8e85a34e4 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MRInputHelpers.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MRInputHelpers.java @@ -322,18 +322,14 @@ public static InputSplitInfoMem generateInputSplitsToMem(Configuration conf, InputSplitInfoMem splitInfoMem = null; JobConf jobConf = new JobConf(conf); if (jobConf.getUseNewMapper()) { - if (LOG.isDebugEnabled()) { - LOG.debug("Generating mapreduce api input splits"); - } + LOG.debug("Generating mapreduce api input splits"); Job job = Job.getInstance(conf); org.apache.hadoop.mapreduce.InputSplit[] splits = generateNewSplits(job, groupSplits, sortSplits, targetTasks); splitInfoMem = new InputSplitInfoMem(splits, createTaskLocationHintsFromSplits(splits), splits.length, job.getCredentials(), job.getConfiguration()); } else { - if (LOG.isDebugEnabled()) { - LOG.debug("Generating mapred api input splits"); - } + LOG.debug("Generating mapred api input splits"); org.apache.hadoop.mapred.InputSplit[] splits = generateOldSplits(jobConf, groupSplits, sortSplits, targetTasks); splitInfoMem = new InputSplitInfoMem(splits, createTaskLocationHintsFromSplits(splits), diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/output/MROutput.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/output/MROutput.java index 950e629907..19ece5a0f6 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/output/MROutput.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/output/MROutput.java @@ -488,9 +488,7 @@ public void initCommitter(JobConf job, boolean useNewApi) throws IOException, InterruptedException { if (useNewApi) { - if (LOG.isDebugEnabled()) { - LOG.debug("using new api for output committer"); - } + LOG.debug("using new api for output committer"); this.committer = newOutputFormat.getOutputCommitter( newApiTaskAttemptContext); diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/MRTask.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/MRTask.java index b79f19cfa0..1a13168cb7 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/MRTask.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/MRTask.java @@ -509,10 +509,7 @@ public void close() throws IOException { comparator, keyClass, valueClass); - if (LOG.isDebugEnabled()) { - LOG.debug("Using key class: " + keyClass - + ", valueClass: " + valueClass); - } + LOG.debug("Using key class: {}, valueClass: {}", keyClass, valueClass); org.apache.hadoop.mapreduce.Reducer.Context reducerContext = diff --git a/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/IndexCache.java b/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/IndexCache.java index 625f7ab8ba..54db975292 100644 --- a/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/IndexCache.java +++ b/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/IndexCache.java @@ -84,9 +84,7 @@ public TezSpillRecord getSpillRecord(String mapId, Path fileName, String expecte } } } - if (LOG.isDebugEnabled()) { - LOG.debug("IndexCache HIT: MapId " + mapId + " found"); - } + LOG.debug("IndexCache HIT: MapId {} found", mapId); } if (info.mapSpillRecord.size() == 0) { @@ -125,9 +123,7 @@ public TezIndexRecord getIndexInformation(String mapId, int reduce, } } } - if (LOG.isDebugEnabled()) { - LOG.debug("IndexCache HIT: MapId " + mapId + " found"); - } + LOG.debug("IndexCache HIT: MapId {} found", mapId); } if (info.mapSpillRecord.size() == 0 || @@ -161,14 +157,11 @@ private IndexInformation readIndexFileToCache(Path indexFileName, } } } - if (LOG.isDebugEnabled()) { - LOG.debug("IndexCache HIT: MapId " + mapId + " found"); - } + LOG.debug("IndexCache HIT: MapId {} found", mapId); return info; } - if (LOG.isDebugEnabled()) { - LOG.debug("IndexCache MISS: MapId " + mapId + " not found"); - } + LOG.debug("IndexCache MISS: MapId {} not found", mapId); + TezSpillRecord tmp = null; try { tmp = new TezSpillRecord(indexFileName, fs, expectedIndexOwner); diff --git a/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java b/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java index b67883dfcf..55389ea78e 100644 --- a/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java +++ b/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java @@ -933,9 +933,7 @@ public AttemptPathInfo load(AttemptPathIdentifier key) throws Path mapOutputFileName = lDirAlloc.getLocalPathToRead( attemptBase + Path.SEPARATOR + DATA_FILE_NAME, conf); - if (LOG.isDebugEnabled()) { - LOG.debug("Loaded : " + key + " via loader"); - } + LOG.debug("Loaded : {} via loader", key); return new AttemptPathInfo(indexFileName, mapOutputFileName); } }); @@ -1011,10 +1009,7 @@ public void messageReceived(ChannelHandlerContext ctx, MessageEvent evt) boolean keepAliveParam = false; if (keepAliveList != null && keepAliveList.size() == 1) { keepAliveParam = Boolean.parseBoolean(keepAliveList.get(0)); - if (LOG.isDebugEnabled()) { - LOG.debug("KeepAliveParam : " + keepAliveList - + " : " + keepAliveParam); - } + LOG.debug("KeepAliveParam : {} : {}", keepAliveList, keepAliveParam); } final List mapIds = splitMaps(q.get("map")); final Range reduceRange = splitReduces(q.get("reduce")); @@ -1226,11 +1221,8 @@ protected MapOutputInfo getMapOutputInfo(String dagId, String mapId, AttemptPathIdentifier identifier = new AttemptPathIdentifier( jobId, dagId, user, mapId); pathInfo = pathCache.get(identifier); - if (LOG.isDebugEnabled()) { - LOG.debug("Retrieved pathInfo for " + identifier + - " check for corresponding loaded messages to determine whether" + - " it was loaded or cached"); - } + LOG.debug("Retrieved pathInfo for {} check for corresponding loaded " + + "messages to determine whether it was loaded or cached", identifier); } catch (ExecutionException e) { if (e.getCause() instanceof IOException) { throw (IOException) e.getCause(); @@ -1303,13 +1295,9 @@ protected void setResponseHeaders(HttpResponse response, boolean keepAliveParam, response.headers().set(HttpHeaders.Names.CONTENT_LENGTH, String.valueOf(contentLength)); response.headers().set(HttpHeaders.Names.CONNECTION, HttpHeaders.Values.KEEP_ALIVE); response.headers().set(HttpHeaders.Values.KEEP_ALIVE, "timeout=" + connectionKeepAliveTimeOut); - if (LOG.isDebugEnabled()) { - LOG.debug("Content Length in shuffle : " + contentLength); - } + LOG.debug("Content Length in shuffle : {}", contentLength); } else { - if (LOG.isDebugEnabled()) { - LOG.debug("Setting connection close header..."); - } + LOG.debug("Setting connection close header..."); response.headers().set(HttpHeaders.Names.CONNECTION, CONNECTION_CLOSE); } } 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 f8a3de2242..583cc0099a 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 @@ -467,9 +467,7 @@ protected Void callInternal() throws Exception { } protected Void _callInternal() throws Exception { - if (LOG.isDebugEnabled()) { - LOG.debug("Initializing Input using InputSpec: " + inputSpec); - } + LOG.debug("Initializing Input using InputSpec: {}", inputSpec); String edgeName = inputSpec.getSourceVertexName(); InputContext inputContext = createInputContext(inputsMap, inputSpec, inputIndex); LogicalInput input = createInput(inputSpec, inputContext); @@ -483,9 +481,7 @@ protected Void _callInternal() throws Exception { inputContext.getTaskVertexName(), inputContext.getSourceVertexName(), taskSpec.getTaskAttemptID()); initializedInputs.put(edgeName, input); - if (LOG.isDebugEnabled()) { - LOG.debug("Initialized Input with src edge: " + edgeName); - } + LOG.debug("Initialized Input with src edge: {}", edgeName); initializedInputs.put(edgeName, input); return null; } @@ -512,9 +508,7 @@ protected Void callInternal() throws Exception { } protected Void _callInternal() throws Exception { - if (LOG.isDebugEnabled()) { - LOG.debug("Starting Input with src edge: " + srcVertexName); - } + LOG.debug("Starting Input with src edge: {}", srcVertexName); input.start(); LOG.info("Started Input with src edge: " + srcVertexName); @@ -544,9 +538,7 @@ protected Void callInternal() throws Exception { } protected Void _callInternal() throws Exception { - if (LOG.isDebugEnabled()) { - LOG.debug("Initializing Output using OutputSpec: " + outputSpec); - } + LOG.debug("Initializing Output using OutputSpec: {}", outputSpec); String edgeName = outputSpec.getDestinationVertexName(); OutputContext outputContext = createOutputContext(outputSpec, outputIndex); LogicalOutput output = createOutput(outputSpec, outputContext); @@ -559,9 +551,7 @@ protected Void _callInternal() throws Exception { outputContext.getTaskVertexName(), outputContext.getDestinationVertexName(), taskSpec.getTaskAttemptID()); initializedOutputs.put(edgeName, output); - if (LOG.isDebugEnabled()) { - LOG.debug("Initialized Output with dest edge: " + edgeName); - } + LOG.debug("Initialized Output with dest edge: {}", edgeName); initializedOutputs.put(edgeName, output); return null; } @@ -579,9 +569,7 @@ private void initializeGroupInputs() throws TezException { if (groupInputSpecs != null && !groupInputSpecs.isEmpty()) { groupInputsMap = new ConcurrentHashMap(groupInputSpecs.size()); for (GroupInputSpec groupInputSpec : groupInputSpecs) { - if (LOG.isDebugEnabled()) { - LOG.debug("Initializing GroupInput using GroupInputSpec: " + groupInputSpec); - } + LOG.debug("Initializing GroupInput using GroupInputSpec: {}", groupInputSpec); MergedInputContext mergedInputContext = new TezMergedInputContextImpl(groupInputSpec.getMergedInputDescriptor().getUserPayload(), groupInputSpec.getGroupName(), groupInputsMap, inputReadyTracker, localDirs, this); diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezInputContextImpl.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezInputContextImpl.java index 9ff284d5bf..f28573a649 100644 --- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezInputContextImpl.java +++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezInputContextImpl.java @@ -187,8 +187,6 @@ public void close() throws IOException { super.close(); this.userPayload = null; this.inputReadyTracker = null; - if (LOG.isDebugEnabled()) { - LOG.debug("Cleared TezInputContextImpl related information"); - } + LOG.debug("Cleared TezInputContextImpl related information"); } } diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezOutputContextImpl.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezOutputContextImpl.java index db3212280f..ec8280a239 100644 --- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezOutputContextImpl.java +++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezOutputContextImpl.java @@ -167,8 +167,6 @@ public OutputStatisticsReporter getStatisticsReporter() { public void close() throws IOException { super.close(); this.userPayload = null; - if (LOG.isDebugEnabled()) { - LOG.debug("Cleared TezOutputContextImpl related information"); - } + LOG.debug("Cleared TezOutputContextImpl related information"); } } diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezProcessorContextImpl.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezProcessorContextImpl.java index 71ed077c50..e09aa8377e 100644 --- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezProcessorContextImpl.java +++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezProcessorContextImpl.java @@ -143,9 +143,7 @@ public void close() throws IOException { super.close(); this.userPayload = null; this.inputReadyTracker = null; - if (LOG.isDebugEnabled()) { - LOG.debug("Cleared TezProcessorContextImpl related information"); - } + LOG.debug("Cleared TezProcessorContextImpl related information"); } } diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/common/resources/ScalingAllocator.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/common/resources/ScalingAllocator.java index 872632e1ea..e045abd9d2 100644 --- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/common/resources/ScalingAllocator.java +++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/common/resources/ScalingAllocator.java @@ -80,16 +80,11 @@ public Long apply(InitialMemoryRequestContext requestContext) { long requestedSize = request.getRequestedSize(); if (requestedSize == 0) { allocations.add(0l); - if (LOG.isDebugEnabled()) { - LOG.debug("Scaling requested: 0 to allocated: 0"); - } + LOG.debug("Scaling requested: 0 to allocated: 0"); } else { long allocated = (long) ((requestedSize / (double) totalRequested) * availableForAllocation); allocations.add(allocated); - if (LOG.isDebugEnabled()) { - LOG.debug("Scaling requested: " + requestedSize + " to allocated: " + allocated); - } - + LOG.debug("Scaling requested: {} to allocated: {}", requestedSize, allocated); } } return allocations; 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 fb066fd2bd..978942d4e7 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 @@ -264,16 +264,12 @@ private synchronized ResponseWrapper heartbeat(Collection eventsArg) t int maxEvents = Math.min(maxEventsToGet, task.getMaxEventsToHandle()); TezHeartbeatRequest request = new TezHeartbeatRequest(requestId, events, fromPreRoutedEventId, containerIdStr, task.getTaskAttemptID(), fromEventId, maxEvents); - if (LOG.isDebugEnabled()) { - LOG.debug("Sending heartbeat to AM, request=" + request); - } + LOG.debug("Sending heartbeat to AM, request={}", request); maybeLogCounters(); TezHeartbeatResponse response = umbilical.heartbeat(request); - if (LOG.isDebugEnabled()) { - LOG.debug("Received heartbeat response from AM, response=" + response); - } + LOG.debug("Received heartbeat response from AM, response={}", response); if (response.shouldDie()) { LOG.info("Received should die response from AM"); 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 86ceb12d7d..c82355a9fa 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 @@ -337,9 +337,7 @@ private void handleNewTaskLocalResources(ContainerTask containerTask, UserGroupInformation ugi) throws IOException, TezException { final Map additionalResources = containerTask.getAdditionalResources(); - if (LOG.isDebugEnabled()) { - LOG.debug("Additional Resources added to container: " + additionalResources); - } + LOG.debug("Additional Resources added to container: {}", additionalResources); if (additionalResources != null && !additionalResources.isEmpty()) { LOG.info("Localizing additional local resources for Task : " + additionalResources); 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 index 3b45cdd709..e12331c250 100644 --- 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 @@ -77,9 +77,7 @@ public HttpConnection(URL url, HttpConnectionParams connParams, this.url = url; this.stopWatch = new StopWatch(); this.urlLogCount = new AtomicLong(); - if (LOG.isDebugEnabled()) { - LOG.debug("MapOutput URL :" + url.toString()); - } + LOG.debug("MapOutput URL :{}", url); } @VisibleForTesting @@ -278,9 +276,7 @@ public void cleanup(boolean disconnect) throws IOException { stopWatch.reset().start(); try { if (input != null) { - if (LOG.isDebugEnabled()) { - LOG.debug("Closing input on " + logIdentifier); - } + LOG.debug("Closing input on {}", logIdentifier); input.close(); input = null; } 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 index 43f64b82b3..63b8934821 100644 --- 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 @@ -113,9 +113,7 @@ public AsyncHttpConnection(URL url, HttpConnectionParams connParams, this.httpConnParams = connParams; this.url = url; this.stopWatch = new StopWatch(); - if (LOG.isDebugEnabled()) { - LOG.debug("MapOutput URL :" + url.toString()); - } + LOG.debug("MapOutput URL :{}", url); initClient(httpConnParams); pos = new PipedOutputStream(); diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/TezRuntimeUtils.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/TezRuntimeUtils.java index daeafbc6fe..9ff3d1c1e5 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/TezRuntimeUtils.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/TezRuntimeUtils.java @@ -80,9 +80,7 @@ public static Combiner instantiateCombiner(Configuration conf, TaskContext taskC if (className == null) { return null; } - if (LOG.isDebugEnabled()) { - LOG.debug("Using Combiner class: " + className); - } + LOG.debug("Using Combiner class: {}", className); try { clazz = (Class) conf.getClassByName(className); } catch (ClassNotFoundException e) { 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 40909d4715..6a61474a5a 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 @@ -121,10 +121,7 @@ public static void shuffleToMemory(byte[] shuffleData, IFile.Reader.readToMemory(shuffleData, input, compressedLength, codec, ifileReadAhead, ifileReadAheadLength); // metrics.inputBytes(shuffleData.length); - if (LOG.isDebugEnabled()) { - LOG.debug("Read " + shuffleData.length + " bytes from input for " - + identifier); - } + LOG.debug("Read {} bytes from input for {}", shuffleData.length, identifier); } catch (InternalError | Exception e) { // Close the streams LOG.info("Failed to read data to memory for " + identifier + ". len=" + compressedLength + @@ -200,8 +197,7 @@ public static void ioCleanup(Closeable... closeables) { try { c.close(); } catch (IOException e) { - if (LOG.isDebugEnabled()) - LOG.debug("Exception in closing " + c, e); + LOG.debug("Exception in closing {}", c, e); } } } diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleInputEventHandlerImpl.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleInputEventHandlerImpl.java index e924876628..bcb7bb58ea 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleInputEventHandlerImpl.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleInputEventHandlerImpl.java @@ -183,10 +183,8 @@ private void processDataMovementEvent(DataMovementEvent dme, DataMovementEventPa if (emptyPartitionsBitSet.get(srcIndex)) { CompositeInputAttemptIdentifier srcAttemptIdentifier = constructInputAttemptIdentifier(dme.getTargetIndex(), 1, dme.getVersion(), shufflePayload, false); - if (LOG.isDebugEnabled()) { - LOG.debug("Source partition: " + srcIndex + " did not generate any data. SrcAttempt: [" - + srcAttemptIdentifier + "]. Not fetching."); - } + LOG.debug("Source partition: {} did not generate any data. SrcAttempt: [{}]. Not fetching.", + srcIndex, srcAttemptIdentifier); numDmeEventsNoData.getAndIncrement(); shuffleManager.addCompletedInputWithNoData(srcAttemptIdentifier.expand(0)); return; @@ -261,10 +259,8 @@ private void processCompositeRoutedDataMovementEvent(CompositeRoutedDataMovement allPartitionsEmpty &= emptyPartitionsBitSet.get(srcPartitionId); if (emptyPartitionsBitSet.get(srcPartitionId)) { InputAttemptIdentifier srcAttemptIdentifier = compositeInputAttemptIdentifier.expand(i); - if (LOG.isDebugEnabled()) { - LOG.debug("Source partition: " + srcPartitionId + " did not generate any data. SrcAttempt: [" - + srcAttemptIdentifier + "]. Not fetching."); - } + LOG.debug("Source partition: {} did not generate any data. SrcAttempt: [{}]. Not fetching.", + srcPartitionId, srcAttemptIdentifier); numDmeEventsNoData.getAndIncrement(); shuffleManager.addCompletedInputWithNoData(srcAttemptIdentifier); } 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 901ee08a7a..56195a8641 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 @@ -441,9 +441,7 @@ protected Void callInternal() throws Exception { break; } - if (LOG.isDebugEnabled()) { - LOG.debug(srcNameTrimmed + ": " + "NumCompletedInputs: " + numCompletedInputs); - } + LOG.debug("{}: NumCompletedInputs: {}", srcNameTrimmed, numCompletedInputs); if (numCompletedInputs.get() < numInputs && !isShutdown.get()) { lock.lock(); try { @@ -661,9 +659,7 @@ public void addKnownInput(String hostName, int port, public void addCompletedInputWithNoData( InputAttemptIdentifier srcAttemptIdentifier) { int inputIdentifier = srcAttemptIdentifier.getInputIdentifier(); - if (LOG.isDebugEnabled()) { - LOG.debug("No input data exists for SrcTask: " + inputIdentifier + ". Marking as complete."); - } + LOG.debug("No input data exists for SrcTask: {}. Marking as complete.", inputIdentifier); lock.lock(); try { if (!completedInputSet.get(inputIdentifier)) { @@ -1144,9 +1140,7 @@ public void onSuccess(Void result) { @Override public void onFailure(Throwable t) { if (isShutdown.get()) { - if (LOG.isDebugEnabled()) { - LOG.debug(srcNameTrimmed + ": " + "Already shutdown. Ignoring error: " + t); - } + LOG.debug("{}: Already shutdown. Ignoring error.", srcNameTrimmed, t); } else { LOG.error(srcNameTrimmed + ": " + "Scheduler failed with error: ", t); inputContext.reportFailure(TaskFailureType.NON_FATAL, t, "Shuffle Scheduler Failed"); @@ -1177,9 +1171,7 @@ private void doBookKeepingForFetcherComplete() { public void onSuccess(FetchResult result) { fetcher.shutdown(); if (isShutdown.get()) { - if (LOG.isDebugEnabled()) { - LOG.debug(srcNameTrimmed + ": " + "Already shutdown. Ignoring event from fetcher"); - } + LOG.debug("{}: Already shutdown. Ignoring event from fetcher", srcNameTrimmed); } else { Iterable pendingInputs = result.getPendingInputs(); if (pendingInputs != null && pendingInputs.iterator().hasNext()) { @@ -1202,9 +1194,7 @@ public void onFailure(Throwable t) { // Unsuccessful - the fetcher may not have shutdown correctly. Try shutting it down. fetcher.shutdown(); if (isShutdown.get()) { - if (LOG.isDebugEnabled()) { - LOG.debug(srcNameTrimmed + ": " + "Already shutdown. Ignoring error from fetcher: " + t); - } + LOG.debug("{}: Already shutdown. Ignoring error from fetcher.", srcNameTrimmed, t); } else { LOG.error(srcNameTrimmed + ": " + "Fetcher failed with error: ", t); shuffleError = t; 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 327232710b..c9bd092f05 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 @@ -208,9 +208,7 @@ public Void callInternal() { public void shutDown() { if (!stopped) { - if (LOG.isDebugEnabled()) { - LOG.debug("Fetcher stopped for host " + mapHost); - } + LOG.debug("Fetcher stopped for host {}", mapHost); stopped = true; // An interrupt will come in while shutting down the thread. cleanupCurrentConnection(false); @@ -288,19 +286,14 @@ protected void copyFromHost(MapHost host) throws IOException { // Setup connection again if disconnected cleanupCurrentConnection(true); if (stopped) { - if (LOG.isDebugEnabled()) { - LOG.debug("Not re-establishing connection since Fetcher has been stopped"); - } + LOG.debug("Not re-establishing connection since Fetcher has been stopped"); return; } // Connect with retry if (!setupConnection(host, remaining.values())) { if (stopped) { cleanupCurrentConnection(true); - if (LOG.isDebugEnabled()) { - LOG.debug( - "Not reporting connection re-establishment failure since fetcher is stopped"); - } + LOG.debug("Not reporting connection re-establishment failure since fetcher is stopped"); return; } failedTasks = new InputAttemptFetchFailure[] { @@ -354,9 +347,7 @@ boolean setupConnection(MapHost host, Collection attempt connectSucceeded = httpConnection.connect(); if (stopped) { - if (LOG.isDebugEnabled()) { - LOG.debug("Detected fetcher has been shutdown after connection establishment. Returning"); - } + LOG.debug("Detected fetcher has been shutdown after connection establishment. Returning"); return false; } input = httpConnection.getInputStream(); @@ -367,9 +358,7 @@ boolean setupConnection(MapHost host, Collection attempt Thread.currentThread().interrupt(); //reset status } if (stopped) { - if (LOG.isDebugEnabled()) { - LOG.debug("Not reporting fetch failure, since an Exception was caught after shutdown"); - } + LOG.debug("Not reporting fetch failure, since an Exception was caught after shutdown"); return false; } ioErrs.increment(1); @@ -471,9 +460,7 @@ protected InputAttemptFetchFailure[] copyMapOutput(MapHost host, DataInputStream return new InputAttemptFetchFailure[] { InputAttemptFetchFailure.fromAttempt(getNextRemainingAttempt()) }; } else { - if (LOG.isDebugEnabled()) { - LOG.debug("Already shutdown. Ignoring invalid map id error"); - } + LOG.debug("Already shutdown. Ignoring invalid map id error"); return EMPTY_ATTEMPT_ID_ARRAY; } } @@ -518,9 +505,7 @@ protected InputAttemptFetchFailure[] copyMapOutput(MapHost host, DataInputStream return new InputAttemptFetchFailure[] { new InputAttemptFetchFailure(getNextRemainingAttempt()) }; } else { - if (LOG.isDebugEnabled()) { - LOG.debug("Already stopped. Ignoring verification failure."); - } + LOG.debug("Already stopped. Ignoring verification failure."); return EMPTY_ATTEMPT_ID_ARRAY; } } @@ -544,9 +529,7 @@ protected InputAttemptFetchFailure[] copyMapOutput(MapHost host, DataInputStream ioErrs.increment(1); scheduler.reportLocalError(e); } else { - if (LOG.isDebugEnabled()) { - LOG.debug("Already stopped. Ignoring error from merger.reserve"); - } + LOG.debug("Already stopped. Ignoring error from merger.reserve"); } return EMPTY_ATTEMPT_ID_ARRAY; } @@ -760,10 +743,7 @@ protected void setupLocalDiskFetch(MapHost host) throws InterruptedException { LOG.warn("Failed to read local disk output of " + srcAttemptId + " from " + host.getHostIdentifier(), e); } else { - if (LOG.isDebugEnabled()) { - LOG.debug( - "Ignoring fetch error during local disk copy since fetcher has already been stopped"); - } + LOG.debug("Ignoring fetch error during local disk copy since fetcher has already been stopped"); return; } diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/MergeManager.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/MergeManager.java index 59ff577fed..46360e1287 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/MergeManager.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/MergeManager.java @@ -388,9 +388,7 @@ public void waitForInMemoryMerge() throws InterruptedException { } if (triggerAdditionalMerge) { inMemoryMerger.waitForMerge(); - if (LOG.isDebugEnabled()) { - LOG.debug("Additional in-memory merge triggered"); - } + LOG.debug("Additional in-memory merge triggered"); } } @@ -700,9 +698,7 @@ static void cleanup(FileSystem fs, Path path) { } try { - if (LOG.isDebugEnabled()) { - LOG.debug("Deleting " + path); - } + LOG.debug("Deleting {}", path); fs.delete(path, true); } catch (IOException e) { LOG.info("Error in deleting " + path); 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 116098fe26..0c55a3a388 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 @@ -156,11 +156,8 @@ private void processDataMovementEvent(DataMovementEvent dmEvent, DataMovementEve if (shufflePayload.hasEmptyPartitions()) { try { if (emptyPartitionsBitSet.get(partitionId)) { - if (LOG.isDebugEnabled()) { - LOG.debug( - "Source partition: " + partitionId + " did not generate any data. SrcAttempt: [" - + srcAttemptIdentifier + "]. Not fetching."); - } + LOG.debug("Source partition: {} did not generate any data. SrcAttempt: [{}]. Not fetching.", + partitionId, srcAttemptIdentifier); numDmeEventsNoData.getAndIncrement(); scheduler.copySucceeded(srcAttemptIdentifier.expand(0), null, 0, 0, 0, null, true); return; @@ -191,10 +188,8 @@ private void processCompositeRoutedDataMovementEvent(CompositeRoutedDataMovement allPartitionsEmpty &= emptyPartitionsBitSet.get(srcPartitionId); if (emptyPartitionsBitSet.get(srcPartitionId)) { InputAttemptIdentifier srcInputAttemptIdentifier = compositeInputAttemptIdentifier.expand(i); - if (LOG.isDebugEnabled()) { - LOG.debug("Source partition: " + srcPartitionId + " did not generate any data. SrcAttempt: [" - + srcInputAttemptIdentifier + "]. Not fetching."); - } + LOG.debug("Source partition: {} did not generate any data. SrcAttempt: [{}]. Not fetching.", + srcPartitionId, srcInputAttemptIdentifier); numDmeEventsNoData.getAndIncrement(); scheduler.copySucceeded(srcInputAttemptIdentifier, null, 0, 0, 0, null, true); } @@ -212,9 +207,7 @@ private void processCompositeRoutedDataMovementEvent(CompositeRoutedDataMovement private void processTaskFailedEvent(InputFailedEvent ifEvent) { InputAttemptIdentifier taIdentifier = new InputAttemptIdentifier(ifEvent.getTargetIndex(), ifEvent.getVersion()); scheduler.obsoleteInput(taIdentifier); - if (LOG.isDebugEnabled()) { - LOG.debug("Obsoleting output of src-task: " + taIdentifier); - } + LOG.debug("Obsoleting output of src-task: {}", taIdentifier); } /** 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 416041e005..67681cedc7 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 @@ -1104,9 +1104,7 @@ public void obsoleteInput(InputAttemptIdentifier srcAttempt) { if (eventInfo.eventsProcessed.isEmpty() && !eventInfo.scheduledForDownload) { // obsoleted anyways; no point tracking if nothing is started pipelinedShuffleInfoEventsMap.remove(srcAttempt.getInputIdentifier()); - if (LOG.isDebugEnabled()) { - LOG.debug("Removing " + eventInfo + " from tracking"); - } + LOG.debug("Removing {} from tracking", eventInfo); return; } IOException exception = new IOException(srcAttempt + " is marked as obsoleteInput, but it " @@ -1128,9 +1126,7 @@ public synchronized void putBackKnownMapOutput(MapHost host, public synchronized MapHost getHost() throws InterruptedException { while (pendingHosts.isEmpty() && remainingMaps.get() > 0) { - if (LOG.isDebugEnabled()) { - LOG.debug("PendingHosts=" + pendingHosts); - } + LOG.debug("PendingHosts={}", pendingHosts); waitAndNotifyProgress(); } @@ -1436,9 +1432,7 @@ protected Void callInternal() throws InterruptedException { if (mapHost == null) { break; // Check for the exit condition. } - if (LOG.isDebugEnabled()) { - LOG.debug(srcNameTrimmed + ": " + "Processing pending host: " + mapHost.toString()); - } + LOG.debug("{}: Processing pending host: {}", srcNameTrimmed, mapHost); if (!isShutdown.get()) { count++; if (LOG.isDebugEnabled()) { diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFile.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFile.java index 1b2aefff41..a4bbf5aabf 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFile.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFile.java @@ -844,9 +844,7 @@ public static void readToMemory(byte[] buffer, InputStream in, int compressedLen try { in.close(); } catch(IOException e) { - if(LOG.isDebugEnabled()) { - LOG.debug("Exception in closing " + in, e); - } + LOG.debug("Exception in closing {}", in, e); } } throw ioe; diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/task/local/output/TezTaskOutputFiles.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/task/local/output/TezTaskOutputFiles.java index 88474f99f4..3fb90865d1 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/task/local/output/TezTaskOutputFiles.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/task/local/output/TezTaskOutputFiles.java @@ -65,11 +65,7 @@ public TezTaskOutputFiles(Configuration conf, String uniqueId, int dagID) { * if service_id = tez_shuffle then "${appDir}/dagId/output/${uniqueId}" */ private Path getAttemptOutputDir() { - if (LOG.isDebugEnabled()) { - LOG.debug("getAttemptOutputDir: " - + Constants.TEZ_RUNTIME_TASK_OUTPUT_DIR + "/" - + uniqueId); - } + LOG.debug("getAttemptOutputDir: {}/{}", Constants.TEZ_RUNTIME_TASK_OUTPUT_DIR, uniqueId); String dagPath = getDagOutputDir(Constants.TEZ_RUNTIME_TASK_OUTPUT_DIR); return new Path(dagPath, uniqueId); } 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 2b405bb343..313c13d188 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 @@ -130,9 +130,7 @@ public synchronized void start() throws IOException { // Start the shuffle - copy and merge shuffle = createShuffle(); shuffle.run(); - if (LOG.isDebugEnabled()) { - LOG.debug("Initialized the handlers in shuffle..Safe to start processing.."); - } + LOG.debug("Initialized the handlers in shuffle..Safe to start processing.."); List pending = new LinkedList(); pendingEvents.drainTo(pending); if (pending.size() > 0) { diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/resources/WeightedScalingMemoryDistributor.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/resources/WeightedScalingMemoryDistributor.java index b82e6d3d8f..52f3d44b4f 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/resources/WeightedScalingMemoryDistributor.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/resources/WeightedScalingMemoryDistributor.java @@ -244,9 +244,7 @@ private RequestType getRequestTypeForClass(String className) { requestType = RequestType.PARTITIONED_UNSORTED_OUTPUT; } else { requestType = RequestType.OTHER; - if (LOG.isDebugEnabled()) { - LOG.debug("Falling back to RequestType.OTHER for class: " + className); - } + LOG.debug("Falling back to RequestType.OTHER for class: {}", className); } return requestType; } diff --git a/tez-tools/tez-tfile-parser/src/main/java/org/apache/tez/tools/TFileLoader.java b/tez-tools/tez-tfile-parser/src/main/java/org/apache/tez/tools/TFileLoader.java index 18e9940d1d..320428b8cd 100644 --- a/tez-tools/tez-tfile-parser/src/main/java/org/apache/tez/tools/TFileLoader.java +++ b/tez-tools/tez-tfile-parser/src/main/java/org/apache/tez/tools/TFileLoader.java @@ -66,10 +66,7 @@ public Tuple getNext() throws IOException { currentKey = recReader.getCurrentKey(); String line = recReader.getCurrentValue().toString(); - if (LOG.isDebugEnabled()) { - LOG.debug("currentKey: " + currentKey - + ", line=" + line); - } + LOG.debug("currentKey: {}, line={}", currentKey, line); //Tuple would be of format: machine, key, line Tuple tuple = tupleFactory.newTuple(3); if (currentKey != null) { From f481d3899dceda668b74d993cd37f72da3009127 Mon Sep 17 00:00:00 2001 From: Eric Badger Date: Fri, 22 Jan 2021 16:48:15 -0600 Subject: [PATCH 020/137] TEZ-4265. Tez UI doesn't show Load Counters button if any of the Dags have them Signed-off-by: Jonathan Eagles --- tez-ui/src/main/webapp/app/components/home-table-controls.js | 4 ++-- .../tests/integration/components/home-table-controls-test.js | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/tez-ui/src/main/webapp/app/components/home-table-controls.js b/tez-ui/src/main/webapp/app/components/home-table-controls.js index 70c3863416..0804ede998 100644 --- a/tez-ui/src/main/webapp/app/components/home-table-controls.js +++ b/tez-ui/src/main/webapp/app/components/home-table-controls.js @@ -25,8 +25,8 @@ export default Ember.Component.extend({ var processedRows = this.get("dataProcessor.processedRows"), countersLoaded = true; if(processedRows) { - countersLoaded = processedRows.some(function (row) { - return Object.keys(row.get("counterGroupsHash")).length !== 0; + countersLoaded = !processedRows.any(function (row) { + return Object.keys(row.get("counterGroupsHash")).length === 0; }); } return countersLoaded; diff --git a/tez-ui/src/main/webapp/tests/integration/components/home-table-controls-test.js b/tez-ui/src/main/webapp/tests/integration/components/home-table-controls-test.js index ccce2dbe29..c35c339e3c 100644 --- a/tez-ui/src/main/webapp/tests/integration/components/home-table-controls-test.js +++ b/tez-ui/src/main/webapp/tests/integration/components/home-table-controls-test.js @@ -66,7 +66,7 @@ test('countersLoaded test', function(assert) { })] }); this.render(hbs`{{home-table-controls dataProcessor=dataProcessor}}`); - assert.equal(this.$().find("button").attr("class").split(" ").indexOf("no-visible"), 2); + assert.equal(this.$().find("button").attr("class").split(" ").indexOf("no-visible"), -1); this.set("dataProcessor", { processedRows: [Ember.Object.create({ From 7c71780ae6546ea5a1703d25675c1f8f94f384e6 Mon Sep 17 00:00:00 2001 From: Marton Bod Date: Mon, 25 Jan 2021 11:27:24 +0100 Subject: [PATCH 021/137] =?UTF-8?q?TEZ-4264:=20Make=20vertex=20id=20availa?= =?UTF-8?q?ble=20from=20MROutputCommitter=20(Marton=20Bod=20via=20L=C3=A1s?= =?UTF-8?q?zl=C3=B3=20Bodor)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Laszlo Bodor --- .../org/apache/tez/mapreduce/committer/MROutputCommitter.java | 1 + .../main/java/org/apache/tez/mapreduce/hadoop/MRJobConfig.java | 1 + 2 files changed, 2 insertions(+) diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/committer/MROutputCommitter.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/committer/MROutputCommitter.java index b7ebc4c05d..71e5681cbf 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/committer/MROutputCommitter.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/committer/MROutputCommitter.java @@ -78,6 +78,7 @@ public void initialize() throws IOException { jobConf.getCredentials().mergeAll(UserGroupInformation.getCurrentUser().getCredentials()); jobConf.setInt(MRJobConfig.APPLICATION_ATTEMPT_ID, getContext().getDAGAttemptNumber()); + jobConf.setInt(MRJobConfig.VERTEX_ID, getContext().getVertexIndex()); committer = getOutputCommitter(getContext()); jobContext = getJobContextFromVertexContext(getContext()); initialized = true; diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MRJobConfig.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MRJobConfig.java index ca954d92ff..e162460773 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MRJobConfig.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MRJobConfig.java @@ -645,6 +645,7 @@ public interface MRJobConfig { "mrr.vertex."; public static final String VERTEX_NAME = "mapreduce.task.vertex.name"; + public static final String VERTEX_ID = "mapreduce.task.vertex.id"; public static final String MR_TEZ_SPLITS_VIA_EVENTS = MR_TEZ_PREFIX + "splits.via.events"; public static final boolean MR_TEZ_SPLITS_VIA_EVENTS_DEFAULT = true; From 7374b69ed6c4af4e7c9127f1b05473c2166abf6d Mon Sep 17 00:00:00 2001 From: belugabehr <12578579+belugabehr@users.noreply.github.com> Date: Mon, 25 Jan 2021 17:44:11 -0500 Subject: [PATCH 022/137] TEZ-4268: Buffer File Read of TEZ AM Local Resources PB File (#91) --- .../org/apache/tez/dag/app/DAGAppMaster.java | 16 +++++++--------- 1 file changed, 7 insertions(+), 9 deletions(-) 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 cde77b3bf6..ba072a9b5e 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 @@ -20,6 +20,7 @@ +import java.io.BufferedInputStream; import java.io.File; import java.io.FileInputStream; import java.io.FileNotFoundException; @@ -604,17 +605,14 @@ public synchronized void serviceInit(final Configuration conf) throws Exception if (!versionMismatch) { if (isSession) { - FileInputStream sessionResourcesStream = null; - try { - sessionResourcesStream = new FileInputStream( - new File(workingDirectory, TezConstants.TEZ_AM_LOCAL_RESOURCES_PB_FILE_NAME)); + try (BufferedInputStream sessionResourcesStream = + new BufferedInputStream( + new FileInputStream(new File(workingDirectory, + TezConstants.TEZ_AM_LOCAL_RESOURCES_PB_FILE_NAME)))) { PlanLocalResourcesProto amLocalResourceProto = PlanLocalResourcesProto .parseDelimitedFrom(sessionResourcesStream); - amResources.putAll(DagTypeConverters.convertFromPlanLocalResources(amLocalResourceProto)); - } finally { - if (sessionResourcesStream != null) { - sessionResourcesStream.close(); - } + amResources.putAll(DagTypeConverters + .convertFromPlanLocalResources(amLocalResourceProto)); } } } From 04597c7b591917bbe995a9e182c3be60e33816c9 Mon Sep 17 00:00:00 2001 From: Rajesh Balamohan Date: Thu, 28 Jan 2021 05:20:25 +0530 Subject: [PATCH 023/137] TEZ-3985: Correctness: Throw a clear exception for DMEs sent during cleanup (Contributed by Jaume M, reviewed by Rajesh Balamohan, Ashutosh Chauhan) --- .../apache/tez/runtime/api/OutputContext.java | 10 ++ .../api/impl/TezOutputContextImpl.java | 30 +++++- .../runtime/internals/api/TezTrapEvent.java | 52 ++++++++++ .../internals/api/TezTrapEventType.java | 29 ++++++ .../tez/runtime/task/TaskRunner2Callable.java | 12 ++- .../tez/runtime/task/TezTaskRunner2.java | 3 +- .../tez/runtime/task/TezTrapEventHandler.java | 92 ++++++++++++++++++ .../TestLogicalIOProcessorRuntimeTask.java | 97 ++++++++++++++++++- 8 files changed, 318 insertions(+), 7 deletions(-) create mode 100644 tez-runtime-internals/src/main/java/org/apache/tez/runtime/internals/api/TezTrapEvent.java create mode 100644 tez-runtime-internals/src/main/java/org/apache/tez/runtime/internals/api/TezTrapEventType.java create mode 100644 tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTrapEventHandler.java diff --git a/tez-api/src/main/java/org/apache/tez/runtime/api/OutputContext.java b/tez-api/src/main/java/org/apache/tez/runtime/api/OutputContext.java index 882eb4be2a..33fe772f85 100644 --- a/tez-api/src/main/java/org/apache/tez/runtime/api/OutputContext.java +++ b/tez-api/src/main/java/org/apache/tez/runtime/api/OutputContext.java @@ -19,6 +19,7 @@ package org.apache.tez.runtime.api; import org.apache.hadoop.classification.InterfaceAudience.Public; +import org.apache.hadoop.yarn.event.EventHandler; /** * Context handle for the Output to initialize itself. @@ -48,4 +49,13 @@ public interface OutputContext extends TaskContext { */ public OutputStatisticsReporter getStatisticsReporter(); + /** + * Notify the context that at this point no more events should be sent. + * This is used as a safety measure to prevent events being sent after close + * or in cleanup. After this is called events being queued to be sent to the + * AM will instead be passed to the event handler. + * @param eventHandler should handle the events after the call. + */ + void trapEvents(EventHandler eventHandler); + } diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezOutputContextImpl.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezOutputContextImpl.java index ec8280a239..20ec0622c7 100644 --- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezOutputContextImpl.java +++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezOutputContextImpl.java @@ -29,7 +29,9 @@ import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.yarn.event.EventHandler; import org.apache.tez.common.TezExecutors; +import org.apache.tez.runtime.internals.api.TezTrapEvent; import org.apache.tez.common.counters.TezCounters; import org.apache.tez.dag.api.OutputDescriptor; import org.apache.tez.dag.api.TezConfiguration; @@ -47,17 +49,28 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +@SuppressWarnings("unchecked") public class TezOutputContextImpl extends TezTaskContextImpl implements OutputContext { private static final Logger LOG = LoggerFactory.getLogger(TezOutputContextImpl.class); private volatile UserPayload userPayload; + + /** + * Holds whether we can accept more events to send to the AM. + */ + private volatile boolean trapEvents; private final String destinationVertexName; private final EventMetaData sourceInfo; private final int outputIndex; private final OutputStatisticsReporterImpl statsReporter; + /** + * Handler for the events after the trap flag is set. + */ + private EventHandler trapEventHandler; + class OutputStatisticsReporterImpl implements OutputStatisticsReporter { @Override @@ -71,7 +84,7 @@ public synchronized void reportDataSize(long size) { public void reportItemsProcessed(long items) { // this is a concurrent map. Plus we are not adding/deleting entries runtimeTask.getTaskStatistics().getIOStatistics().get(destinationVertexName) - .setItemsProcessed(items);; + .setItemsProcessed(items); } } @@ -124,7 +137,11 @@ public void sendEvents(List events) { TezEvent tEvt = new TezEvent(e, sourceInfo); tezEvents.add(tEvt); } - tezUmbilical.addEvents(tezEvents); + if (trapEvents) { + trapEventHandler.handle(new TezTrapEvent(tezEvents)); + } else { + tezUmbilical.addEvents(tezEvents); + } } @Override @@ -163,6 +180,15 @@ public OutputStatisticsReporter getStatisticsReporter() { return statsReporter; } + /** + * This will monitor some of the events that will be sent. + */ + @Override + public final void trapEvents(final EventHandler eventHandler) { + trapEvents = true; + this.trapEventHandler = eventHandler; + } + @Override public void close() throws IOException { super.close(); diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/internals/api/TezTrapEvent.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/internals/api/TezTrapEvent.java new file mode 100644 index 0000000000..8c63b74007 --- /dev/null +++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/internals/api/TezTrapEvent.java @@ -0,0 +1,52 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tez.runtime.internals.api; + +import org.apache.tez.common.TezAbstractEvent; +import org.apache.tez.runtime.api.impl.TezEvent; + +import java.util.List; + +import static org.apache.tez.runtime.internals.api.TezTrapEventType.TRAP_EVENT_TYPE; + +/** + * Event sent when no more events should be sent to the AM. + */ +public class TezTrapEvent extends TezAbstractEvent { + /** + * Events that were reported. + */ + private final List tezEvents; + + /** + * Create a tez trap event. + * @param events events tried to be sent to the AM. + */ + public TezTrapEvent(final List events) { + super(TRAP_EVENT_TYPE); + this.tezEvents = events; + } + + /** + * @return events. + */ + public final List getTezEvents() { + return tezEvents; + } +} \ No newline at end of file diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/internals/api/TezTrapEventType.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/internals/api/TezTrapEventType.java new file mode 100644 index 0000000000..89cb78ea06 --- /dev/null +++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/internals/api/TezTrapEventType.java @@ -0,0 +1,29 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tez.runtime.internals.api; + +/** + * Enum type with only one value representing this event. + */ +public enum TezTrapEventType { + /** + * Single value for this event type. + */ + TRAP_EVENT_TYPE +} 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 0e6dfda080..810a806228 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 @@ -24,6 +24,7 @@ import org.apache.tez.common.TezUtilsInternal; import org.apache.tez.common.counters.TezCounters; import org.apache.tez.runtime.LogicalIOProcessorRuntimeTask; +import org.apache.tez.runtime.api.impl.TezUmbilical; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -46,10 +47,16 @@ public class TaskRunner2Callable extends CallableWithNdc outputContext.trapEvents(new TezTrapEventHandler(outputContext, + this.tezUmbilical))); task.cleanup(); } } 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 ae81769aca..bbf037b8bf 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 @@ -177,7 +177,8 @@ public TaskRunner2Result run() { // 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); + taskRunnerCallable = new TaskRunner2Callable(task, ugi, + umbilicalAndErrorHandler); taskReporter.registerTask(task, umbilicalAndErrorHandler); future = executor.submit(taskRunnerCallable); } diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTrapEventHandler.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTrapEventHandler.java new file mode 100644 index 0000000000..b35dbb0162 --- /dev/null +++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezTrapEventHandler.java @@ -0,0 +1,92 @@ +/* + * 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 com.google.common.base.Preconditions; +import org.apache.hadoop.yarn.event.EventHandler; +import org.apache.tez.runtime.api.OutputContext; +import org.apache.tez.runtime.api.impl.TezEvent; +import org.apache.tez.runtime.api.impl.TezOutputContextImpl; +import org.apache.tez.runtime.api.impl.TezUmbilical; +import org.apache.tez.runtime.internals.api.TezTrapEvent; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.List; + +/** + * Class that handles the events after the trap has been activated. At + * this point no more events of some types shouldn't be sent and it's + * a bug to do so. If the events arrive here probably the task will be + * restarted because it has failed. + */ +public class TezTrapEventHandler implements EventHandler { + /** + * logger. + */ + private static final Logger + LOG = LoggerFactory.getLogger(TezOutputContextImpl.class); + + /** + * Output context that will report the events. + */ + private final OutputContext outputContext; + + /** + * Protocol to send the events. + */ + private final TezUmbilical tezUmbilical; + + /** + * @param output context that will report the events. + * @param umbilical used to send the events to the AM. + */ + TezTrapEventHandler(final OutputContext output, + final TezUmbilical umbilical) { + this.outputContext = output; + this.tezUmbilical = umbilical; + } + + /** + * Decide what to do with the events. + * @param tezTrapEvent event holding the tez events. + */ + @Override + public final void handle(final TezTrapEvent tezTrapEvent) { + Preconditions.checkArgument(tezTrapEvent.getTezEvents() != null); + List tezEvents = new ArrayList( + tezTrapEvent.getTezEvents().size()); + for (TezEvent tezEvent: tezTrapEvent.getTezEvents()) { + switch (tezEvent.getEventType()) { + case COMPOSITE_DATA_MOVEMENT_EVENT: + case DATA_MOVEMENT_EVENT: + String errorMsg = "Some events won't be sent to the AM because all" + + " the events should have been sent at this point. Most likely" + + " this would result in a bug. " + + " event:" + tezEvent.toString(); + Throwable throwable = new Throwable(errorMsg); + LOG.error(errorMsg, throwable); + break; + default: + LOG.info("Event of type " + tezEvent.getEventType() + " will be sent" + + " to the AM after the task was closed "); + tezEvents.add(tezEvent); + } + } + tezUmbilical.addEvents(tezEvents); + } +} \ No newline at end of file diff --git a/tez-runtime-internals/src/test/java/org/apache/tez/runtime/TestLogicalIOProcessorRuntimeTask.java b/tez-runtime-internals/src/test/java/org/apache/tez/runtime/TestLogicalIOProcessorRuntimeTask.java index 599f98f666..ba9a66d0f1 100644 --- a/tez-runtime-internals/src/test/java/org/apache/tez/runtime/TestLogicalIOProcessorRuntimeTask.java +++ b/tez-runtime-internals/src/test/java/org/apache/tez/runtime/TestLogicalIOProcessorRuntimeTask.java @@ -24,13 +24,18 @@ import static org.junit.Assert.fail; import static org.mockito.Mockito.mock; +import java.io.IOException; import java.nio.ByteBuffer; import java.util.Arrays; +import java.util.Collections; import java.util.HashMap; import java.util.LinkedList; import java.util.List; import java.util.Map; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.tez.common.TezExecutors; import org.apache.tez.common.TezSharedExecutor; import org.apache.tez.dag.api.InputDescriptor; import org.apache.tez.dag.api.OutputDescriptor; @@ -41,12 +46,15 @@ import org.apache.tez.dag.records.TezTaskID; import org.apache.tez.dag.records.TezVertexID; import org.apache.tez.hadoop.shim.DefaultHadoopShim; +import org.apache.tez.hadoop.shim.HadoopShim; import org.apache.tez.runtime.api.AbstractLogicalIOProcessor; import org.apache.tez.runtime.api.AbstractLogicalInput; import org.apache.tez.runtime.api.AbstractLogicalOutput; import org.apache.tez.runtime.api.Event; +import org.apache.tez.runtime.api.ExecutionContext; import org.apache.tez.runtime.api.LogicalInput; import org.apache.tez.runtime.api.LogicalOutput; +import org.apache.tez.runtime.api.ObjectRegistry; import org.apache.tez.runtime.api.Reader; import org.apache.tez.runtime.api.InputContext; import org.apache.tez.runtime.api.OutputContext; @@ -57,13 +65,16 @@ import org.apache.tez.runtime.api.impl.InputSpec; import org.apache.tez.runtime.api.impl.OutputSpec; import org.apache.tez.runtime.api.impl.TaskSpec; +import org.apache.tez.runtime.api.impl.TezEvent; import org.apache.tez.runtime.api.impl.TezUmbilical; import org.apache.tez.runtime.common.resources.ScalingAllocator; +import org.apache.tez.runtime.task.TaskRunner2Callable; import org.junit.Test; import com.google.common.collect.HashMultimap; import com.google.common.collect.Lists; import com.google.common.collect.Multimap; + import org.mockito.Mockito; public class TestLogicalIOProcessorRuntimeTask { @@ -149,6 +160,39 @@ public void testAutoStart() throws Exception { } + @Test + public void testEventsCantBeSentInCleanup() throws Exception { + TezDAGID dagId = createTezDagId(); + TezVertexID vertexId = createTezVertexId(dagId); + Map serviceConsumerMetadata = new HashMap<>(); + Multimap startedInputsMap = HashMultimap.create(); + TezUmbilical umbilical = mock(TezUmbilical.class); + TezConfiguration tezConf = new TezConfiguration(); + tezConf.set(TezConfiguration.TEZ_TASK_SCALE_MEMORY_ALLOCATOR_CLASS, + ScalingAllocator.class.getName()); + + TezTaskAttemptID taId1 = createTaskAttemptID(vertexId, 1); + TaskSpec task1 = createTaskSpec(taId1, "dag1", "vertex1", 30, + RunExceptionProcessor.class.getName(), + TestOutputWithEvents.class.getName()); + + TezSharedExecutor sharedExecutor = new TezSharedExecutor(tezConf); + LogicalIOProcessorRuntimeTask lio = + new CleanupLogicalIOProcessorRuntimeTask(task1, 0, tezConf, null, + umbilical, serviceConsumerMetadata, new HashMap(), + startedInputsMap, null, "", new ExecutionContextImpl("localhost"), + Runtime.getRuntime().maxMemory(), true, new DefaultHadoopShim(), + sharedExecutor); + + TaskRunner2Callable runner = + new TaskRunner2Callable(lio, UserGroupInformation.getCurrentUser(), umbilical); + + runner.call(); + + // We verify that no events were sent + Mockito.verify(umbilical, Mockito.only()).addEvents(Collections. emptyList()); + } + /** * We should expect no events being sent to the AM if an * exception happens in the close method of the processor @@ -167,7 +211,7 @@ public void testExceptionHappensInClose() throws Exception { TezTaskAttemptID taId1 = createTaskAttemptID(vertexId, 1); TaskSpec task1 = createTaskSpec(taId1, "dag1", "vertex1", 30, - FaultyTestProcessor.class.getName(), + CloseExceptionProcessor.class.getName(), TestOutputWithEvents.class.getName()); TezSharedExecutor sharedExecutor = new TezSharedExecutor(tezConf); @@ -281,6 +325,31 @@ private TezDAGID createTezDagId() { return TezDAGID.getInstance("2000", 100, 1); } + private static class CleanupLogicalIOProcessorRuntimeTask + extends LogicalIOProcessorRuntimeTask { + CleanupLogicalIOProcessorRuntimeTask(TaskSpec taskSpec, + int appAttemptNumber, Configuration tezConf, String[] localDirs, + TezUmbilical tezUmbilical, + Map serviceConsumerMetadata, + Map envMap, Multimap startedInputsMap, + ObjectRegistry objectRegistry, String pid, + org.apache.tez.runtime.api.ExecutionContext ExecutionContext, + long memAvailable, boolean updateSysCounters, HadoopShim hadoopShim, + TezExecutors sharedExecutor) throws IOException { + super(taskSpec, appAttemptNumber, tezConf, localDirs, tezUmbilical, + serviceConsumerMetadata, envMap, startedInputsMap, objectRegistry, + pid, ExecutionContext, memAvailable, updateSysCounters, hadoopShim, + sharedExecutor); + } + + @Override public void cleanup() throws InterruptedException { + getOutputContexts().forEach(context + -> context.sendEvents(Arrays.asList( + CompositeDataMovementEvent.create(0, 0, null) + ))); + } + } + public static class TestProcessor extends AbstractLogicalIOProcessor { public static volatile int runCount = 0; @@ -310,8 +379,30 @@ public void close() throws Exception { } - public static class FaultyTestProcessor extends TestProcessor { - public FaultyTestProcessor(ProcessorContext context) { + public static class RunExceptionProcessor + extends TestProcessor { + + public RunExceptionProcessor(ProcessorContext context) { + super(context); + } + + public void run(Map inputs, + Map outputs) + throws Exception { + // This exception is thrown in purpose because we want to test this + throw new RuntimeException(); + } + + @Override + public void close() throws Exception { + // This exception is thrown because this method shouldn't be called + // if run has thrown an exception. + throw new RuntimeException(); + } + } + + public static class CloseExceptionProcessor extends TestProcessor { + public CloseExceptionProcessor(ProcessorContext context) { super(context); } From 033ba916099acd46e51a99bc839469864a950f45 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?L=C3=A1szl=C3=B3=20Bodor?= Date: Thu, 28 Jan 2021 10:16:39 +0100 Subject: [PATCH 024/137] =?UTF-8?q?TEZ-4240:=20Remove=20SHA-256=20from=20T?= =?UTF-8?q?ez=20(L=C3=A1szl=C3=B3=20Bodor=20reviewed=20by=20Jonathan=20Tur?= =?UTF-8?q?ner=20Eagles)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Laszlo Bodor --- .../src/main/java/org/apache/tez/client/TezClientUtils.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java b/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java index 1e09489564..8e7ccaf550 100644 --- a/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java +++ b/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java @@ -1124,7 +1124,7 @@ public static byte[] getLocalSha(Path path, Configuration conf) throws IOExcepti InputStream is = null; try { is = FileSystem.getLocal(conf).open(path); - return DigestUtils.sha256(is); + return DigestUtils.sha384(is); } finally { if (is != null) { is.close(); @@ -1136,7 +1136,7 @@ public static byte[] getResourceSha(URI uri, Configuration conf) throws IOExcept InputStream is = null; try { is = FileSystem.get(uri, conf).open(new Path(uri)); - return DigestUtils.sha256(is); + return DigestUtils.sha384(is); } finally { if (is != null) { is.close(); From 41ff94583e6433402e2a7cc183269d2582f0302f Mon Sep 17 00:00:00 2001 From: belugabehr <12578579+belugabehr@users.noreply.github.com> Date: Thu, 28 Jan 2021 15:33:42 -0500 Subject: [PATCH 025/137] TEZ-4269: Re-Work Threadpool in DAGAppMaster (#92) --- .../java/org/apache/tez/dag/app/AppContext.java | 5 +---- .../org/apache/tez/dag/app/DAGAppMaster.java | 17 ++++------------- 2 files changed, 5 insertions(+), 17 deletions(-) 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 fc4ddcfcdc..c9a7083c1d 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 @@ -20,7 +20,7 @@ import java.util.Map; import java.util.Set; -import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.ExecutorService; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; @@ -74,9 +74,6 @@ public interface AppContext { DAG getCurrentDAG(); - // For testing only! - ThreadPoolExecutor getThreadPool(); - ListeningExecutorService getExecService(); void setDAG(DAG dag); 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 ba072a9b5e..dbcefe97a5 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 @@ -48,10 +48,10 @@ import java.util.Set; import java.util.Timer; import java.util.TimerTask; +import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; @@ -303,7 +303,6 @@ public class DAGAppMaster extends AbstractService { private Path tezSystemStagingDir; private FileSystem recoveryFS; - private ThreadPoolExecutor rawExecutor; private ListeningExecutorService execService; // TODO May not need to be a bidi map @@ -621,9 +620,9 @@ public synchronized void serviceInit(final Configuration conf) throws Exception TezConfiguration.TEZ_AM_DAG_APPCONTEXT_THREAD_COUNT_LIMIT_DEFAULT); // NOTE: LinkedBlockingQueue does not have a capacity Limit and can thus // occupy large memory chunks when numerous Runables are pending for execution - rawExecutor = new ThreadPoolExecutor(threadCount, threadCount, - 60L, TimeUnit.SECONDS, new LinkedBlockingQueue(), - new ThreadFactoryBuilder().setDaemon(true).setNameFormat("App Shared Pool - " + "#%d").build()); + ExecutorService rawExecutor = + Executors.newFixedThreadPool(threadCount, new ThreadFactoryBuilder() + .setDaemon(true).setNameFormat("App Shared Pool - #%d").build()); execService = MoreExecutors.listeningDecorator(rawExecutor); initServices(conf); @@ -1503,14 +1502,6 @@ public DAG getCurrentDAG() { return dag; } - @Override - // For Testing only! - public ThreadPoolExecutor getThreadPool() { - synchronized (DAGAppMaster.this) { - return rawExecutor; - } - } - @Override public ListeningExecutorService getExecService() { return execService; From 61a28c75c3885f0a043d304091c2a303dde5b617 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?L=C3=A1szl=C3=B3=20Bodor?= Date: Thu, 28 Jan 2021 22:17:32 +0100 Subject: [PATCH 026/137] =?UTF-8?q?TEZ-4236:=20DAGClientServer=20is=20not?= =?UTF-8?q?=20really=20needed=20to=20be=20started/used=20in=20local=20mode?= =?UTF-8?q?=20(L=C3=A1szl=C3=B3=20Bodor=20reviewed=20by=20Ashutosh=20Chauh?= =?UTF-8?q?an)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Laszlo Bodor --- .../apache/tez/client/FrameworkClient.java | 109 +++++++++++++ .../java/org/apache/tez/client/TezClient.java | 143 +++++------------- .../org/apache/tez/client/TezClientUtils.java | 14 +- .../apache/tez/dag/api/TezConfiguration.java | 13 ++ .../tez/dag/api/client/DAGClientImpl.java | 2 +- .../dag/api/client/DAGClientImplLocal.java | 53 +++++++ .../org/apache/tez/client/TestTezClient.java | 55 ++++--- .../org/apache/tez/client/LocalClient.java | 79 +++++++++- .../org/apache/tez/dag/app/DAGAppMaster.java | 8 +- .../apache/tez/dag/app/LocalDAGAppMaster.java | 51 +++++++ .../tez/dag/app/rm/TaskSchedulerManager.java | 4 +- .../apache/tez/dag/app/MockLocalClient.java | 4 +- .../org/apache/tez/test/TestLocalMode.java | 10 +- 13 files changed, 394 insertions(+), 151 deletions(-) create mode 100644 tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClientImplLocal.java create mode 100644 tez-dag/src/main/java/org/apache/tez/dag/app/LocalDAGAppMaster.java diff --git a/tez-api/src/main/java/org/apache/tez/client/FrameworkClient.java b/tez-api/src/main/java/org/apache/tez/client/FrameworkClient.java index 7c60ec100d..2ec6d2864d 100644 --- a/tez-api/src/main/java/org/apache/tez/client/FrameworkClient.java +++ b/tez-api/src/main/java/org/apache/tez/client/FrameworkClient.java @@ -21,20 +21,39 @@ import java.io.IOException; 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.api.records.ApplicationSubmissionContext; import org.apache.hadoop.yarn.client.api.YarnClient; import org.apache.hadoop.yarn.client.api.YarnClientApplication; import org.apache.hadoop.yarn.exceptions.YarnException; +import org.apache.tez.common.RPCUtil; import org.apache.tez.common.ReflectionUtils; +import org.apache.tez.dag.api.DAG; +import org.apache.tez.dag.api.DAGSubmissionTimedOut; +import org.apache.tez.dag.api.DagTypeConverters; import org.apache.tez.dag.api.TezConfiguration; import org.apache.tez.dag.api.TezException; import org.apache.tez.dag.api.TezReflectionException; import org.apache.tez.dag.api.TezUncheckedException; +import org.apache.tez.dag.api.client.DAGClient; +import org.apache.tez.dag.api.client.DAGClientImpl; +import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolBlockingPB; +import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.GetAMStatusRequestProto; +import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.GetAMStatusResponseProto; +import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.ShutdownSessionRequestProto; +import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.SubmitDAGRequestProto; +import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.SubmitDAGResponseProto; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.protobuf.ServiceException; @Private public abstract class FrameworkClient { + protected static final Logger LOG = LoggerFactory.getLogger(FrameworkClient.class); public static FrameworkClient createFrameworkClient(TezConfiguration tezConf) { @@ -77,4 +96,94 @@ public abstract ApplicationId submitApplication(ApplicationSubmissionContext app public abstract boolean isRunning() throws IOException; + public TezAppMasterStatus getAMStatus(Configuration conf, ApplicationId appId, + UserGroupInformation ugi) throws TezException, ServiceException, IOException { + DAGClientAMProtocolBlockingPB proxy = getProxy(conf, appId, ugi); + + if (proxy == null) { + return TezAppMasterStatus.INITIALIZING; + } + GetAMStatusResponseProto response = + proxy.getAMStatus(null, GetAMStatusRequestProto.newBuilder().build()); + return DagTypeConverters.convertTezAppMasterStatusFromProto(response.getStatus()); + } + + public DAGClient submitDag(DAG dag, SubmitDAGRequestProto request, String clientName, + ApplicationId sessionAppId, long clientTimeout, UserGroupInformation ugi, TezConfiguration tezConf) + throws IOException, TezException, DAGSubmissionTimedOut { + DAGClientAMProtocolBlockingPB proxy = null; + try { + proxy = waitForProxy(clientTimeout, tezConf, sessionAppId, ugi); + } catch (InterruptedException e) { + throw new IOException("Interrupted while trying to create a connection to the AM", e); + } + if (proxy == null) { + try { + LOG.warn("DAG submission to session timed out, stopping session"); + stop(); + } catch (Throwable t) { + LOG.info("Got an exception when trying to stop session", t); + } + throw new DAGSubmissionTimedOut("Could not submit DAG to Tez Session" + + ", timed out after " + clientTimeout + " seconds"); + } + + String dagId = null; + try { + SubmitDAGResponseProto response = proxy.submitDAG(null, request); + // the following check is only for testing since the final class + // SubmitDAGResponseProto cannot be mocked + if (response != null) { + dagId = response.getDagId(); + } + } catch (ServiceException e) { + RPCUtil.unwrapAndThrowException(e); + } + + LOG.info("Submitted dag to TezSession" + + ", sessionName=" + clientName + + ", applicationId=" + sessionAppId + + ", dagId=" + dagId + + ", dagName=" + dag.getName()); + return new DAGClientImpl(sessionAppId, dagId, tezConf, this, ugi); + } + + protected DAGClientAMProtocolBlockingPB waitForProxy(long clientTimeout, Configuration conf, + ApplicationId sessionAppId, UserGroupInformation ugi) + throws IOException, TezException, InterruptedException { + long startTime = System.currentTimeMillis(); + long endTime = startTime + (clientTimeout * 1000); + DAGClientAMProtocolBlockingPB proxy = null; + while (true) { + proxy = TezClientUtils.getAMProxy(this, conf, sessionAppId, ugi); + if (proxy != null) { + break; + } + Thread.sleep(100L); + if (clientTimeout != -1 && System.currentTimeMillis() > endTime) { + break; + } + } + return proxy; + } + + /** + * Shuts down session and returns a boolean=true if a proxy was successfully created and through + * that proxy a shutdownSession was called. + */ + public boolean shutdownSession(Configuration conf, ApplicationId sessionAppId, + UserGroupInformation ugi) throws TezException, IOException, ServiceException { + DAGClientAMProtocolBlockingPB proxy = getProxy(conf, sessionAppId, ugi); + if (proxy != null) { + ShutdownSessionRequestProto request = ShutdownSessionRequestProto.newBuilder().build(); + proxy.shutdownSession(null, request); + return true; + } + return false; + } + + protected DAGClientAMProtocolBlockingPB getProxy(Configuration conf, ApplicationId sessionAppId, + UserGroupInformation ugi) throws TezException, IOException { + return TezClientUtils.getAMProxy(this, conf, sessionAppId, ugi); + } } diff --git a/tez-api/src/main/java/org/apache/tez/client/TezClient.java b/tez-api/src/main/java/org/apache/tez/client/TezClient.java index fbe35094f0..da213b8564 100644 --- a/tez-api/src/main/java/org/apache/tez/client/TezClient.java +++ b/tez-api/src/main/java/org/apache/tez/client/TezClient.java @@ -39,7 +39,6 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.security.UserGroupInformation; import org.apache.tez.common.JavaOptsChecker; -import org.apache.tez.common.RPCUtil; import org.apache.tez.common.TezCommonUtils; import org.apache.tez.common.counters.Limits; import org.apache.tez.dag.api.TezConfigurationConstants; @@ -75,10 +74,7 @@ import org.apache.tez.dag.api.client.DAGClient; import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolBlockingPB; import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.GetAMStatusRequestProto; -import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.GetAMStatusResponseProto; -import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.ShutdownSessionRequestProto; import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.SubmitDAGRequestProto; -import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.SubmitDAGResponseProto; import org.apache.tez.dag.api.client.DAGClientImpl; import org.apache.tez.dag.api.records.DAGProtos.DAGPlan; @@ -123,7 +119,8 @@ public class TezClient { private ApplicationId lastSubmittedAppId; @VisibleForTesting final AMConfiguration amConfig; - private FrameworkClient frameworkClient; + @VisibleForTesting + FrameworkClient frameworkClient; private String diagnostics; @VisibleForTesting final boolean isSession; @@ -158,7 +155,7 @@ public class TezClient { private TezClient(String name, TezConfiguration tezConf) { this(name, tezConf, tezConf.getBoolean( - TezConfiguration.TEZ_AM_SESSION_MODE, TezConfiguration.TEZ_AM_SESSION_MODE_DEFAULT)); + TezConfiguration.TEZ_AM_SESSION_MODE, TezConfiguration.TEZ_AM_SESSION_MODE_DEFAULT)); } @Private @@ -549,7 +546,8 @@ public DAGClientAMProtocolBlockingPB sendAMHeartbeat(DAGClientAMProtocolBlocking try { if (proxy == null) { try { - proxy = waitForProxy(); + proxy = frameworkClient.waitForProxy(clientTimeout, amConfig.getTezConfiguration(), + sessionAppId, getUgi()); } catch (InterruptedException e) { LOG.debug("Interrupted while trying to create a connection to the AM", e); } catch (SessionNotRunning e) { @@ -629,7 +627,6 @@ private DAGClient submitDAGSession(DAG dag) throws TezException, IOException { verifySessionStateForSubmission(); - String dagId = null; String callerContextStr = ""; if (dag.getCallerContext() != null) { callerContextStr = ", callerContext=" + dag.getCallerContext().contextAsSimpleString(); @@ -678,42 +675,8 @@ private DAGClient submitDAGSession(DAG dag) throws TezException, IOException { } } - DAGClientAMProtocolBlockingPB proxy = null; - try { - proxy = waitForProxy(); - } catch (InterruptedException e) { - throw new IOException("Interrupted while trying to create a connection to the AM", e); - } - if (proxy == null) { - try { - LOG.warn("DAG submission to session timed out, stopping session"); - stop(); - } catch (Throwable t) { - LOG.info("Got an exception when trying to stop session", t); - } - throw new DAGSubmissionTimedOut("Could not submit DAG to Tez Session" - + ", timed out after " + clientTimeout + " seconds"); - } - - try { - SubmitDAGResponseProto response = proxy.submitDAG(null, request); - // the following check is only for testing since the final class - // SubmitDAGResponseProto cannot be mocked - if (response != null) { - dagId = response.getDagId(); - } - } catch (ServiceException e) { - RPCUtil.unwrapAndThrowException(e); - } - - LOG.info("Submitted dag to TezSession" - + ", sessionName=" + clientName - + ", applicationId=" + sessionAppId - + ", dagId=" + dagId - + ", dagName=" + dag.getName()); - return new DAGClientImpl(sessionAppId, dagId, - amConfig.getTezConfiguration(), - frameworkClient, getUgi()); + return frameworkClient.submitDag(dag, request, clientName, sessionAppId, clientTimeout, + getUgi(), amConfig.getTezConfiguration()); } private UserGroupInformation getUgi() throws IOException { @@ -746,39 +709,34 @@ public synchronized void stop() throws TezException, IOException { sessionStopped.set(true); boolean sessionShutdownSuccessful = false; try { - DAGClientAMProtocolBlockingPB proxy = getAMProxy(sessionAppId); - if (proxy != null) { - ShutdownSessionRequestProto request = - ShutdownSessionRequestProto.newBuilder().build(); - proxy.shutdownSession(null, request); - sessionShutdownSuccessful = true; - boolean asynchronousStop = amConfig.getTezConfiguration().getBoolean( - TezConfiguration.TEZ_CLIENT_ASYNCHRONOUS_STOP, - TezConfiguration.TEZ_CLIENT_ASYNCHRONOUS_STOP_DEFAULT); - if (!asynchronousStop) { - LOG.info("Waiting until application is in a final state"); - long currentTimeMillis = System.currentTimeMillis(); - long timeKillIssued = currentTimeMillis; - long killTimeOut = amConfig.getTezConfiguration().getLong( - TezConfiguration.TEZ_CLIENT_HARD_KILL_TIMEOUT_MS, - TezConfiguration.TEZ_CLIENT_HARD_KILL_TIMEOUT_MS_DEFAULT); - ApplicationReport appReport = frameworkClient - .getApplicationReport(sessionAppId); - while ((currentTimeMillis < timeKillIssued + killTimeOut) - && !isJobInTerminalState(appReport.getYarnApplicationState())) { - try { - Thread.sleep(1000L); - } catch (InterruptedException ie) { - /** interrupted, just break */ - break; - } - currentTimeMillis = System.currentTimeMillis(); - appReport = frameworkClient.getApplicationReport(sessionAppId); + sessionShutdownSuccessful = frameworkClient + .shutdownSession(amConfig.getTezConfiguration(), sessionAppId, getUgi()); + boolean asynchronousStop = amConfig.getTezConfiguration().getBoolean( + TezConfiguration.TEZ_CLIENT_ASYNCHRONOUS_STOP, + TezConfiguration.TEZ_CLIENT_ASYNCHRONOUS_STOP_DEFAULT); + if (!asynchronousStop && sessionShutdownSuccessful) { + LOG.info("Waiting until application is in a final state"); + long currentTimeMillis = System.currentTimeMillis(); + long timeKillIssued = currentTimeMillis; + long killTimeOut = amConfig.getTezConfiguration().getLong( + TezConfiguration.TEZ_CLIENT_HARD_KILL_TIMEOUT_MS, + TezConfiguration.TEZ_CLIENT_HARD_KILL_TIMEOUT_MS_DEFAULT); + ApplicationReport appReport = frameworkClient + .getApplicationReport(sessionAppId); + while ((currentTimeMillis < timeKillIssued + killTimeOut) + && !isJobInTerminalState(appReport.getYarnApplicationState())) { + try { + Thread.sleep(1000L); + } catch (InterruptedException ie) { + /** interrupted, just break */ + break; } + currentTimeMillis = System.currentTimeMillis(); + appReport = frameworkClient.getApplicationReport(sessionAppId); + } - if (!isJobInTerminalState(appReport.getYarnApplicationState())) { - frameworkClient.killApplication(sessionAppId); - } + if (!isJobInTerminalState(appReport.getYarnApplicationState())) { + frameworkClient.killApplication(sessionAppId); } } } catch (TezException e) { @@ -873,14 +831,7 @@ public synchronized TezAppMasterStatus getAppMasterStatus() throws TezException, return TezAppMasterStatus.SHUTDOWN; case RUNNING: try { - DAGClientAMProtocolBlockingPB proxy = getAMProxy(appId); - if (proxy == null) { - return TezAppMasterStatus.INITIALIZING; - } - GetAMStatusResponseProto response = proxy.getAMStatus(null, - GetAMStatusRequestProto.newBuilder().build()); - return DagTypeConverters.convertTezAppMasterStatusFromProto( - response.getStatus()); + return frameworkClient.getAMStatus(amConfig.getTezConfiguration(), appId, getUgi()); } catch (TezException e) { LOG.info("Failed to retrieve AM Status via proxy", e); } catch (ServiceException e) { @@ -1059,32 +1010,6 @@ protected FrameworkClient createFrameworkClient() { return FrameworkClient.createFrameworkClient(amConfig.getTezConfiguration()); } - @VisibleForTesting - // for testing - protected DAGClientAMProtocolBlockingPB getAMProxy(ApplicationId appId) - throws TezException, IOException { - return TezClientUtils.getAMProxy( - frameworkClient, amConfig.getTezConfiguration(), appId, getUgi()); - } - - private DAGClientAMProtocolBlockingPB waitForProxy() - throws IOException, TezException, InterruptedException { - long startTime = System.currentTimeMillis(); - long endTime = startTime + (clientTimeout * 1000); - DAGClientAMProtocolBlockingPB proxy = null; - while (true) { - proxy = getAMProxy(sessionAppId); - if (proxy != null) { - break; - } - Thread.sleep(100l); - if (clientTimeout != -1 && System.currentTimeMillis() > endTime) { - break; - } - } - return proxy; - } - private void verifySessionStateForSubmission() throws SessionNotRunning { Preconditions.checkState(isSession, "Invalid without session mode"); if (!sessionStarted.get()) { diff --git a/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java b/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java index 8e7ccaf550..79069ede7d 100644 --- a/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java +++ b/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java @@ -912,13 +912,12 @@ private static Path localizeDagPlanAsText(DAGPlan dagPB, FileSystem fs, AMConfig return textPath; } - static DAGClientAMProtocolBlockingPB getAMProxy(FrameworkClient yarnClient, - Configuration conf, - ApplicationId applicationId, UserGroupInformation ugi) throws TezException, IOException { + static DAGClientAMProtocolBlockingPB getAMProxy(FrameworkClient frameworkClient, + Configuration conf, ApplicationId applicationId, UserGroupInformation ugi) + throws TezException, IOException { ApplicationReport appReport; try { - appReport = yarnClient.getApplicationReport( - applicationId); + appReport = frameworkClient.getApplicationReport(applicationId); if(appReport == null) { throw new TezUncheckedException("Could not retrieve application report" @@ -948,8 +947,9 @@ static DAGClientAMProtocolBlockingPB getAMProxy(FrameworkClient yarnClient, } catch (YarnException e) { throw new TezException(e); } - return getAMProxy(conf, appReport.getHost(), - appReport.getRpcPort(), appReport.getClientToAMToken(), ugi); + + return getAMProxy(conf, appReport.getHost(), appReport.getRpcPort(), + appReport.getClientToAMToken(), ugi); } @Private 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 179b1957e8..05eb4b286c 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 @@ -1748,6 +1748,19 @@ public TezConfiguration(boolean loadDefaults) { public static final boolean TEZ_LOCAL_MODE_DEFAULT = false; + /** + * Boolean value. Enable local mode execution in Tez without using network for communicating with + * DAGAppMaster. This option only makes sense when {@link #TEZ_LOCAL_MODE} is true. When + * TEZ_LOCAL_MODE_WITHOUT_NETWORK is turned on, LocalClient will call DAGAppMaster's methods + * directly. + */ + @ConfigurationScope(Scope.AM) + @ConfigurationProperty(type = "boolean") + public static final String TEZ_LOCAL_MODE_WITHOUT_NETWORK = + TEZ_PREFIX + "local.mode.without.network"; + + public static final boolean TEZ_LOCAL_MODE_WITHOUT_NETWORK_DEFAULT = false; + /** * String value. TezLocalCacheManager uses this folder as a root for temp and localized files. */ 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 b54db324a9..e58863f614 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 @@ -209,7 +209,7 @@ public DAGStatus getDAGStatus(@Nullable Set statusOptions, } } - private DAGStatus getDAGStatusInternal(@Nullable Set statusOptions, + protected DAGStatus getDAGStatusInternal(@Nullable Set statusOptions, long timeout) throws TezException, IOException { if (!dagCompleted) { diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClientImplLocal.java b/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClientImplLocal.java new file mode 100644 index 0000000000..a0509cdc5f --- /dev/null +++ b/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClientImplLocal.java @@ -0,0 +1,53 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.tez.dag.api.client; + +import java.io.IOException; +import java.util.HashSet; +import java.util.Set; +import java.util.function.BiFunction; + +import javax.annotation.Nullable; + +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.tez.client.FrameworkClient; +import org.apache.tez.dag.api.TezConfiguration; +import org.apache.tez.dag.api.TezException; + +/** + * A DAGClientImpl which is typically used for tez.local.mode.without.network=true. + */ +public class DAGClientImplLocal extends DAGClientImpl { + + private BiFunction, Long, DAGStatus> dagStatusFunction; + + public DAGClientImplLocal(ApplicationId appId, String dagId, TezConfiguration conf, + FrameworkClient frameworkClient, UserGroupInformation ugi, + BiFunction, Long, DAGStatus> dagStatusFunction) { + super(appId, dagId, conf, frameworkClient, ugi); + this.dagStatusFunction = dagStatusFunction; + } + + @Override + protected DAGStatus getDAGStatusInternal(@Nullable Set statusOptions, long timeout) + throws TezException, IOException { + return dagStatusFunction.apply(statusOptions == null ? new HashSet<>() : statusOptions, + timeout); + } +} diff --git a/tez-api/src/test/java/org/apache/tez/client/TestTezClient.java b/tez-api/src/test/java/org/apache/tez/client/TestTezClient.java index 7316452df3..67c4a604b6 100644 --- a/tez-api/src/test/java/org/apache/tez/client/TestTezClient.java +++ b/tez-api/src/test/java/org/apache/tez/client/TestTezClient.java @@ -56,6 +56,7 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.security.Credentials; +import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.util.Time; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ApplicationReport; @@ -93,7 +94,6 @@ import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.ShutdownSessionRequestProto; import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.SubmitDAGRequestProto; import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.TezAppMasterStatusProto; -import org.apache.tez.dag.api.records.DAGProtos.ConfigurationProto; import org.apache.tez.dag.api.records.DAGProtos.DAGStatusProto; import org.apache.tez.dag.api.records.DAGProtos.DAGStatusStateProto; import org.apache.tez.dag.api.records.DAGProtos.ProgressProto; @@ -112,7 +112,6 @@ public class TestTezClient { TestTezClient.class.getName()).getAbsoluteFile(); class TezClientForTest extends TezClient { - TezYarnClient mockTezYarnClient; DAGClientAMProtocolBlockingPB sessionAmProxy; YarnClient mockYarnClient; ApplicationId mockAppId; @@ -120,23 +119,13 @@ class TezClientForTest extends TezClient { Long prewarmTimeoutMs; public TezClientForTest(String name, TezConfiguration tezConf, - @Nullable Map localResources, - @Nullable Credentials credentials) { + @Nullable Map localResources, @Nullable Credentials credentials) { super(name, tezConf, localResources, credentials); } - + @Override protected FrameworkClient createFrameworkClient() { - return mockTezYarnClient; - } - - @Override - protected DAGClientAMProtocolBlockingPB getAMProxy(ApplicationId appId) - throws TezException, IOException { - if (!callRealGetSessionAMProxy) { - return sessionAmProxy; - } - return super.getAMProxy(appId); + return frameworkClient; // already initialized } public void setPrewarmTimeoutMs(Long prewarmTimeoutMs) { @@ -148,7 +137,34 @@ protected long getPrewarmWaitTimeMs() { return prewarmTimeoutMs == null ? super.getPrewarmWaitTimeMs() : prewarmTimeoutMs; } } - + + class TezYarnClientForTest extends TezYarnClient { + private TezClientForTest client; + + protected TezYarnClientForTest(YarnClient yarnClient, TezClientForTest client) { + super(yarnClient); + this.client = client; + } + + @Override + protected DAGClientAMProtocolBlockingPB waitForProxy(long clientTimeout, Configuration conf, + ApplicationId sessionAppId, UserGroupInformation ugi) throws TezException, IOException { + if (!client.callRealGetSessionAMProxy) { + return client.sessionAmProxy; + } + return super.getProxy(conf, sessionAppId, ugi); + } + + @Override + protected DAGClientAMProtocolBlockingPB getProxy(Configuration conf, ApplicationId sessionAppId, + UserGroupInformation ugi) throws TezException, IOException { + if (!client.callRealGetSessionAMProxy) { + return client.sessionAmProxy; + } + return super.getProxy(conf, sessionAppId, ugi); + } + } + TezClientForTest configureAndCreateTezClient() throws YarnException, IOException, ServiceException { return configureAndCreateTezClient(null); } @@ -179,11 +195,11 @@ TezClientForTest configureAndCreateTezClient(Map lrs, boo .thenReturn(GetAMStatusResponseProto.newBuilder().setStatus(TezAppMasterStatusProto.RUNNING).build()); client.sessionAmProxy = sessionAmProxy; - client.mockTezYarnClient = new TezYarnClient(yarnClient); + client.frameworkClient = new TezYarnClientForTest(yarnClient, client); client.mockYarnClient = yarnClient; client.mockAppId = appId1; - - return client; + + return client; } @Test (timeout = 5000) @@ -987,7 +1003,6 @@ public void testYarnZkDeprecatedConf() { String val = "hostname:2181"; conf.set("yarn.resourcemanager.zk-address", val); - ConfigurationProto confProto = null; //Test that Exception is not thrown by createFinalConfProtoForApp TezClientUtils.createFinalConfProtoForApp(conf, null); } diff --git a/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java b/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java index 5a6bb9a22c..c76bd6bace 100644 --- a/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java +++ b/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java @@ -22,11 +22,14 @@ import java.net.InetAddress; import java.nio.ByteBuffer; import java.util.List; - +import java.util.Map; +import java.util.Set; +import java.util.function.BiFunction; import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse; import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.FinalApplicationStatus; +import org.apache.hadoop.yarn.api.records.LocalResource; import org.apache.hadoop.yarn.client.api.YarnClientApplication; import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.exceptions.YarnException; @@ -46,18 +49,28 @@ import org.apache.hadoop.yarn.util.SystemClock; import org.apache.tez.common.TezCommonUtils; import org.apache.tez.common.TezUtilsInternal; +import org.apache.tez.dag.api.DAGSubmissionTimedOut; +import org.apache.tez.dag.api.DagTypeConverters; import org.apache.tez.dag.api.TezConfiguration; import org.apache.tez.dag.api.TezException; +import org.apache.tez.dag.api.client.DAGClient; import org.apache.tez.dag.api.client.DAGClientHandler; +import org.apache.tez.dag.api.client.DAGClientImpl; +import org.apache.tez.dag.api.client.DAGClientImplLocal; +import org.apache.tez.dag.api.client.DAGStatus; +import org.apache.tez.dag.api.client.StatusGetOpts; +import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.SubmitDAGRequestProto; import org.apache.tez.dag.api.records.DAGProtos.AMPluginDescriptorProto; import org.apache.tez.dag.app.AppContext; import org.apache.tez.dag.app.DAGAppMaster; import org.apache.tez.dag.app.DAGAppMasterState; +import org.apache.tez.dag.app.LocalDAGAppMaster; import org.apache.tez.dag.app.dag.DAG; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import com.google.common.annotations.VisibleForTesting; +import com.google.protobuf.ServiceException; public class LocalClient extends FrameworkClient { public static final Logger LOG = LoggerFactory.getLogger(LocalClient.class); @@ -72,6 +85,8 @@ public class LocalClient extends FrameworkClient { private boolean isSession; private TezApiVersionInfo versionInfo = new TezApiVersionInfo(); private volatile Throwable amFailException = null; + private boolean isLocalWithoutNetwork; + private static final String localModeDAGSchedulerClassName = "org.apache.tez.dag.app.dag.impl.DAGSchedulerNaturalOrderControlled"; @@ -89,6 +104,10 @@ public void init(TezConfiguration tezConf) { // disable web service for local mode. this.conf.setBoolean(TezConfiguration.TEZ_AM_WEBSERVICE_ENABLE, false); + + this.isLocalWithoutNetwork = + tezConf.getBoolean(TezConfiguration.TEZ_LOCAL_MODE_WITHOUT_NETWORK, + TezConfiguration.TEZ_LOCAL_MODE_WITHOUT_NETWORK_DEFAULT); } @@ -170,7 +189,6 @@ public ApplicationReport getApplicationReport(ApplicationId appId) { report.setYarnApplicationState(convertDAGAppMasterState(dagAppMaster.getState())); report.setFinalApplicationStatus(convertDAGAppMasterStateToFinalYARNState(dagAppMaster.getState())); - List diagnostics = dagAppMaster.getDiagnostics(); if (diagnostics != null) { report.setDiagnostics(diagnostics.toString()); @@ -333,7 +351,7 @@ public void run() { dagAppMaster = createDAGAppMaster(applicationAttemptId, cId, currentHost, nmPort, nmHttpPort, - new SystemClock(), appSubmitTime, isSession, userDir.toUri().getPath(), + SystemClock.getInstance(), appSubmitTime, isSession, userDir.toUri().getPath(), new String[] {localDir.toUri().getPath()}, new String[] {logDir.toUri().getPath()}, amCredentials, UserGroupInformation.getCurrentUser().getShortUserName()); DAGAppMaster.initAndStartAppMaster(dagAppMaster, conf); @@ -372,8 +390,57 @@ protected DAGAppMaster createDAGAppMaster(ApplicationAttemptId applicationAttemp TezUtilsInternal.readUserSpecifiedTezConfiguration(userDir) .getAmPluginDescriptor(); - return new DAGAppMaster(applicationAttemptId, cId, currentHost, nmPort, nmHttpPort, - new SystemClock(), appSubmitTime, isSession, userDir, localDirs, logDirs, - versionInfo.getVersion(), credentials, jobUserName, amPluginDescriptorProto); + return isLocalWithoutNetwork + ? new LocalDAGAppMaster(applicationAttemptId, cId, currentHost, nmPort, nmHttpPort, + SystemClock.getInstance(), appSubmitTime, isSession, userDir, localDirs, logDirs, + versionInfo.getVersion(), credentials, jobUserName, amPluginDescriptorProto) + : new DAGAppMaster(applicationAttemptId, cId, currentHost, nmPort, nmHttpPort, + SystemClock.getInstance(), appSubmitTime, isSession, userDir, localDirs, logDirs, + versionInfo.getVersion(), credentials, jobUserName, amPluginDescriptorProto); + } + + @Override + public TezAppMasterStatus getAMStatus(Configuration configuration, ApplicationId appId, + UserGroupInformation ugi) throws TezException, ServiceException, IOException { + return clientHandler.getTezAppMasterStatus(); + } + + @Override + public DAGClient submitDag(org.apache.tez.dag.api.DAG dag, SubmitDAGRequestProto request, + String clientName, ApplicationId sessionAppId, long clientTimeout, UserGroupInformation ugi, + TezConfiguration tezConf) throws IOException, TezException, DAGSubmissionTimedOut { + + Map additionalResources = null; + if (request.hasAdditionalAmResources()) { + additionalResources = + DagTypeConverters.convertFromPlanLocalResources(request.getAdditionalAmResources()); + } + + String dagId = dagAppMaster.submitDAGToAppMaster(request.getDAGPlan(), additionalResources); + + return isLocalWithoutNetwork + ? new DAGClientImplLocal(sessionAppId, dagId, tezConf, this, + ugi, new BiFunction, Long, DAGStatus>() { + @Override + public DAGStatus apply(Set statusOpts, Long timeout) { + try { + return clientHandler.getDAGStatus(dagId, statusOpts, timeout); + } catch (TezException e) { + throw new RuntimeException(e); + } + } + }) + : new DAGClientImpl(sessionAppId, dagId, tezConf, this, ugi); + } + + @Override + public boolean shutdownSession(Configuration configuration, ApplicationId sessionAppId, + UserGroupInformation ugi) throws TezException, IOException, ServiceException { + if (isLocalWithoutNetwork) { + clientHandler.shutdownAM(); + return true; + } else { + return super.shutdownSession(configuration, sessionAppId, ugi); + } } } 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 dbcefe97a5..395e84a105 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 @@ -508,8 +508,7 @@ public synchronized void serviceInit(final Configuration conf) throws Exception recoveryEnabled = conf.getBoolean(TezConfiguration.DAG_RECOVERY_ENABLED, TezConfiguration.DAG_RECOVERY_ENABLED_DEFAULT); - clientRpcServer = new DAGClientServer(clientHandler, appAttemptID, recoveryFS); - addIfService(clientRpcServer, true); + initClientRpcServer(); taskHeartbeatHandler = createTaskHeartbeatHandler(context, conf); addIfService(taskHeartbeatHandler, true); @@ -647,6 +646,11 @@ public synchronized void serviceInit(final Configuration conf) throws Exception } } + protected void initClientRpcServer() { + clientRpcServer = new DAGClientServer(clientHandler, appAttemptID, recoveryFS); + addIfService(clientRpcServer, true); + } + @VisibleForTesting protected DAGAppMasterShutdownHandler createShutdownHandler() { return new DAGAppMasterShutdownHandler(); diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/LocalDAGAppMaster.java b/tez-dag/src/main/java/org/apache/tez/dag/app/LocalDAGAppMaster.java new file mode 100644 index 0000000000..e0c8443577 --- /dev/null +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/LocalDAGAppMaster.java @@ -0,0 +1,51 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.tez.dag.app; + +import org.apache.hadoop.security.Credentials; +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.api.records.DAGProtos.AMPluginDescriptorProto; + +/** + * A DAGAppMaster implementation which is really local in a sense that it doesn't start an RPC + * server for handling dag requests. It is typically used by LocalClient, which already has an + * embedded DAGAppMaster, but by default, it calls RPC methods. With + * tez.local.mode.without.network=true, LocalClient will call the DAGAppMaster's methods directly. + */ +public class LocalDAGAppMaster extends DAGAppMaster { + + public LocalDAGAppMaster(ApplicationAttemptId applicationAttemptId, ContainerId containerId, + String nmHost, int nmPort, int nmHttpPort, Clock clock, long appSubmitTime, boolean isSession, + String workingDirectory, String[] localDirs, String[] logDirs, String clientVersion, + Credentials credentials, String jobUserName, AMPluginDescriptorProto pluginDescriptorProto) { + super(applicationAttemptId, containerId, nmHost, nmPort, nmHttpPort, clock, appSubmitTime, + isSession, workingDirectory, localDirs, logDirs, clientVersion, credentials, jobUserName, + pluginDescriptorProto); + } + + @Override + protected void initClientRpcServer() { + // nothing to do, in case of LocalDAGAppMaster clientRpcServer is not supposed to be used by clients + } + + public int getRpcPort() { + return 0; + } +} diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerManager.java index cc2e163720..8e6bfe79a5 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerManager.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerManager.java @@ -641,7 +641,9 @@ protected void instantiateSchedulers(String host, int port, String trackingUrl, @Override public synchronized void serviceStart() throws Exception { - InetSocketAddress serviceAddr = clientService.getBindAddress(); + // clientService is null in case of LocalDAGAppMaster + InetSocketAddress serviceAddr = clientService == null ? new InetSocketAddress("127.0.0.1", 0) + : clientService.getBindAddress(); dagAppMaster = appContext.getAppMaster(); // if web service is enabled then set tracking url. else disable it (value = ""). // the actual url set on the rm web ui will be the proxy url set by WebAppProxyServlet, which diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/MockLocalClient.java b/tez-dag/src/test/java/org/apache/tez/dag/app/MockLocalClient.java index 552651616e..c3355473d1 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/MockLocalClient.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/MockLocalClient.java @@ -38,7 +38,7 @@ public class MockLocalClient extends LocalClient { public MockLocalClient(AtomicBoolean mockAppLauncherGoFlag, Clock clock) { this(mockAppLauncherGoFlag, clock, false, false, 1, 1); } - + public MockLocalClient(AtomicBoolean mockAppLauncherGoFlag, Clock clock, boolean initFailFlag, boolean startFailFlag, int concurrency, int containers) { this.mockAppLauncherGoFlag = mockAppLauncherGoFlag; @@ -60,7 +60,7 @@ protected DAGAppMaster createDAGAppMaster(ApplicationAttemptId applicationAttemp concurrency, containers); return mockApp; } - + public MockDAGAppMaster getMockApp() { return mockApp; } diff --git a/tez-tests/src/test/java/org/apache/tez/test/TestLocalMode.java b/tez-tests/src/test/java/org/apache/tez/test/TestLocalMode.java index e6ef8c9cb7..bdb71ad177 100644 --- a/tez-tests/src/test/java/org/apache/tez/test/TestLocalMode.java +++ b/tez-tests/src/test/java/org/apache/tez/test/TestLocalMode.java @@ -68,14 +68,16 @@ public class TestLocalMode { private static FileSystem remoteFs; private final boolean useDfs; + private final boolean useLocalModeWithoutNetwork; - @Parameterized.Parameters(name = "useDFS:{0}") + @Parameterized.Parameters(name = "useDFS:{0} useLocalModeWithoutNetwork:{1}") public static Collection params() { - return Arrays.asList(new Object[][]{{false}, {true}}); + return Arrays.asList(new Object[][]{{false, false}, {true, false}, {false, true}, {true, true}}); } - public TestLocalMode(boolean useDfs) { + public TestLocalMode(boolean useDfs, boolean useLocalModeWithoutNetwork) { this.useDfs = useDfs; + this.useLocalModeWithoutNetwork = useLocalModeWithoutNetwork; } @BeforeClass @@ -105,6 +107,8 @@ public static void afterClass() throws InterruptedException { private TezConfiguration createConf() { TezConfiguration conf = new TezConfiguration(); conf.setBoolean(TezConfiguration.TEZ_LOCAL_MODE, true); + conf.setBoolean(TezConfiguration.TEZ_LOCAL_MODE_WITHOUT_NETWORK, useLocalModeWithoutNetwork); + if (useDfs) { conf.set("fs.defaultFS", remoteFs.getUri().toString()); } else { From 0d3b428ecbd5b3c240191a6d1907010aa549c189 Mon Sep 17 00:00:00 2001 From: belugabehr <12578579+belugabehr@users.noreply.github.com> Date: Thu, 28 Jan 2021 16:53:24 -0500 Subject: [PATCH 027/137] TEZ-4266: Make Tez Inherit From Apache POM (#89) --- pom.xml | 121 ++++-------------- .../src/main/assembly/tez-dist-minimal.xml | 1 + 2 files changed, 25 insertions(+), 97 deletions(-) diff --git a/pom.xml b/pom.xml index 59ec193dfb..892a8a2d66 100644 --- a/pom.xml +++ b/pom.xml @@ -16,6 +16,11 @@ 4.0.0 + + org.apache + apache + 23 + org.apache.tez tez pom @@ -35,33 +40,35 @@ + + 1.8 + 1.8 true + 3.0.0-M4 + false + UTF-8 + ${maven.build.timestamp} + + apache.snapshots.https + Apache Development Snapshot Repository + https://repository.apache.org/content/repositories/snapshots + apache.staging.https + Apache Release Distribution Repository + https://repository.apache.org/service/local/staging/deploy/maven2 + ${user.home}/clover.license 27.0-jre 3.1.3 3.10.5.Final 0.13.0 1.19 - 1.8 1.7.10 - [${javac.version},) - apache.snapshots.https - Apache Development Snapshot Repository - https://repository.apache.org/content/repositories/snapshots - apache.staging.https - Apache Release Distribution Repository - https://repository.apache.org/service/local/staging/deploy/maven2 - false 2.5.0 ${env.PROTOC_PATH} - UTF-8 scm:git:https://gitbox.apache.org/repos/asf/tez.git - ${maven.build.timestamp} 1.4 3.0.5 - 2.10.4 - 2.4.3 - 3.1.1 + 3.1.1 8.35 1.3.6 ${project.build.directory}/tmp @@ -744,24 +751,9 @@ - - org.apache.maven.plugins - maven-compiler-plugin - 3.1 - - ${javac.version} - ${javac.version} - - - - org.apache.maven.plugins - maven-enforcer-plugin - 1.4 - org.apache.maven.plugins maven-jar-plugin - 2.4 @@ -789,17 +781,11 @@ - - org.apache.maven.plugins - maven-source-plugin - 2.2.1 - org.apache.maven.plugins maven-javadoc-plugin - ${javadoc-maven-plugin.version} - -Xdoclint:none + none @@ -807,11 +793,6 @@ hadoop-maven-plugins ${hadoop.version} - - org.apache.maven.plugins - maven-assembly-plugin - 2.4 - com.github.eirslett frontend-maven-plugin @@ -822,20 +803,9 @@ org.codehaus.mojo 1.3.2 - - org.apache.maven.plugins - maven-war-plugin - 3.2.3 - - - org.apache.maven.plugins - maven-shade-plugin - ${shade-maven-plugin.version} - org.apache.rat apache-rat-plugin - 0.10 CHANGES.txt @@ -865,7 +835,6 @@ org.apache.maven.plugins maven-surefire-plugin - 3.0.0-M4 1 false @@ -918,29 +887,6 @@ findbugs-maven-plugin ${findbugs-maven-plugin.version} - - org.apache.maven.plugins - maven-resources-plugin - 3.1.0 - - UTF-8 - - - - org.apache.maven.plugins - maven-site-plugin - 3.4 - - - org.apache.maven.plugins - maven-deploy-plugin - 2.8.1 - - - org.apache.maven.plugins - maven-gpg-plugin - 1.4 - org.codehaus.mojo build-helper-maven-plugin @@ -996,20 +942,7 @@ - - org.apache.maven.plugins - maven-project-info-reports-plugin - 2.9 - - false - - - - org.apache.maven.plugins - maven-clean-plugin - 3.0.0 - - + ro.isdc.wro4j wro4j-maven-plugin 1.7.9 @@ -1029,7 +962,6 @@ org.apache.maven.plugins maven-enforcer-plugin - 1.4 false @@ -1043,7 +975,7 @@ [3.0.2,) - ${enforced.java.version} + [${maven.compiler.target},) @@ -1090,7 +1022,6 @@ --> org.owasp dependency-check-maven - ${dependency-check-maven.version} @@ -1242,7 +1173,6 @@ org.apache.maven.plugins maven-javadoc-plugin - ${javadoc-maven-plugin.version} true @@ -1270,8 +1200,6 @@ maven-compiler-plugin true - ${javac.version} - ${javac.version} 9999 @@ -1350,7 +1278,6 @@ org.apache.maven.plugins maven-javadoc-plugin - ${javadoc-maven-plugin.version} aggregate diff --git a/tez-dist/src/main/assembly/tez-dist-minimal.xml b/tez-dist/src/main/assembly/tez-dist-minimal.xml index 35f9be76e8..4c95d37efa 100644 --- a/tez-dist/src/main/assembly/tez-dist-minimal.xml +++ b/tez-dist/src/main/assembly/tez-dist-minimal.xml @@ -26,6 +26,7 @@ org.apache.tez:tez-aux-services org.apache.tez:tez-ext-service-tests org.apache.tez:tez-ui + org.apache.tez:tez-docs / From 73bcabd2bca2536bf4f3673443a8dcdaaf79a4eb Mon Sep 17 00:00:00 2001 From: Bodor Laszlo Date: Mon, 1 Feb 2021 11:33:19 +0100 Subject: [PATCH 028/137] TEZ-4277: AsyncDispatcher can hang on serviceStop if the eventhandling thread is in BLOCKED state (#97) (Laszlo Bodor reviewed by Rajesh Balamohan) TEZ-4277: AsyncDispatcher can hang on serviceStop if the eventhandling thread is in BLOCKED state --- .../apache/tez/dag/api/TezConfiguration.java | 11 ++++++++ .../apache/tez/common/AsyncDispatcher.java | 28 ++++++++++++++++--- .../org/apache/tez/client/LocalClient.java | 3 +- 3 files changed, 37 insertions(+), 5 deletions(-) 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 05eb4b286c..eef0d65424 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 @@ -193,6 +193,17 @@ public TezConfiguration(boolean loadDefaults) { @Private public static final int TEZ_AM_CONCURRENT_DISPATCHER_CONCURRENCY_DEFAULT = 10; + /** + * Integer value. Milliseconds while AsyncDispatcher should wait for events to be processed on + * serviceStop. The idea is borrowed from YARN-3999. + */ + @Private + @ConfigurationScope(Scope.AM) + public static final String TEZ_AM_DISPATCHER_DRAIN_EVENTS_TIMEOUT = TEZ_AM_PREFIX + + "dispatcher.drain-events.timeout"; + @Private + public static final int TEZ_AM_DISPATCHER_DRAIN_EVENTS_TIMEOUT_DEFAULT = 10000; + /** * Boolean value. Execution mode for the Tez application. True implies session mode. If the client * code is written according to best practices then the same code can execute in either mode based diff --git a/tez-common/src/main/java/org/apache/tez/common/AsyncDispatcher.java b/tez-common/src/main/java/org/apache/tez/common/AsyncDispatcher.java index c197f1d32e..f9f21ca313 100644 --- a/tez-common/src/main/java/org/apache/tez/common/AsyncDispatcher.java +++ b/tez-common/src/main/java/org/apache/tez/common/AsyncDispatcher.java @@ -33,6 +33,7 @@ import org.apache.hadoop.yarn.event.Event; import org.apache.hadoop.yarn.event.EventHandler; import org.apache.hadoop.yarn.exceptions.YarnRuntimeException; +import org.apache.tez.dag.api.TezConfiguration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -142,19 +143,34 @@ protected void serviceStop() throws Exception { if (drainEventsOnStop) { blockNewEvents = true; LOG.info("AsyncDispatcher is draining to stop, ignoring any new events."); + long endTime = System.currentTimeMillis() + getConfig() + .getInt(TezConfiguration.TEZ_AM_DISPATCHER_DRAIN_EVENTS_TIMEOUT, + TezConfiguration.TEZ_AM_DISPATCHER_DRAIN_EVENTS_TIMEOUT_DEFAULT); + synchronized (waitForDrained) { - while (!drained && eventHandlingThread.isAlive()) { + while (!drained && eventHandlingThread.isAlive() && System.currentTimeMillis() < endTime) { waitForDrained.wait(1000); - LOG.info("Waiting for AsyncDispatcher to drain."); + LOG.info( + "Waiting for AsyncDispatcher to drain. Current queue size: {}, handler thread state: {}", + eventQueue.size(), eventHandlingThread.getState()); } } - } stopped = true; if (eventHandlingThread != null) { eventHandlingThread.interrupt(); try { - eventHandlingThread.join(); + /* + * The event handling thread can be alive at this point, but in BLOCKED state, which leads + * to app hang, as a BLOCKED thread might never finish under some circumstances + */ + if (eventHandlingThread.getState() == Thread.State.BLOCKED) { + LOG.warn( + "eventHandlingThread is in BLOCKED state, let's not wait for it in order to prevent app hang"); + } else { + eventHandlingThread.join(); + LOG.info("joined event handling thread, state: {}", eventHandlingThread.getState()); + } } catch (InterruptedException ie) { LOG.warn("Interrupted Exception while stopping", ie); } @@ -181,6 +197,10 @@ protected void dispatch(Event event) { throw new Exception("No handler for registered for " + type); } } catch (Throwable t) { + if (t instanceof InterruptedException) { + LOG.warn("Interrupted Exception while handling event: " + event.getType(), t); + Thread.currentThread().interrupt(); + } LOG.error("Error in dispatcher thread", t); // If serviceStop is called, we should exit this thread gracefully. if (exitOnDispatchException diff --git a/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java b/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java index c76bd6bace..f6d9587734 100644 --- a/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java +++ b/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java @@ -47,6 +47,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationReport; import org.apache.hadoop.yarn.api.records.YarnApplicationState; import org.apache.hadoop.yarn.util.SystemClock; +import org.apache.tez.common.AsyncDispatcher; import org.apache.tez.common.TezCommonUtils; import org.apache.tez.common.TezUtilsInternal; import org.apache.tez.dag.api.DAGSubmissionTimedOut; @@ -356,7 +357,7 @@ public void run() { amCredentials, UserGroupInformation.getCurrentUser().getShortUserName()); DAGAppMaster.initAndStartAppMaster(dagAppMaster, conf); clientHandler = new DAGClientHandler(dagAppMaster); - + ((AsyncDispatcher)dagAppMaster.getDispatcher()).setDrainEventsOnStop(); } catch (Throwable t) { LOG.error("Error starting DAGAppMaster", t); if (dagAppMaster != null) { From 26759e06bd3a9317fe439d7ad81b60e1dabc932e Mon Sep 17 00:00:00 2001 From: Rajesh Balamohan Date: Mon, 8 Feb 2021 09:44:22 +0100 Subject: [PATCH 029/137] =?UTF-8?q?TEZ-4273:=20Clear=20off=20staging=20fil?= =?UTF-8?q?es=20when=20TezYarnClient=20is=20unable=20to=20submit=20applica?= =?UTF-8?q?tions=20(Rajesh=20Balamohan=20via=20L=C3=A1szl=C3=B3=20Bodor)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Laszlo Bodor --- .../java/org/apache/tez/client/TezClient.java | 24 +++++++++++++++++++ 1 file changed, 24 insertions(+) diff --git a/tez-api/src/main/java/org/apache/tez/client/TezClient.java b/tez-api/src/main/java/org/apache/tez/client/TezClient.java index da213b8564..378017b364 100644 --- a/tez-api/src/main/java/org/apache/tez/client/TezClient.java +++ b/tez-api/src/main/java/org/apache/tez/client/TezClient.java @@ -33,6 +33,7 @@ import javax.annotation.Nullable; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.CommonConfigurationKeys; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; @@ -403,6 +404,10 @@ public synchronized void start() throws TezException, IOException { LOG.info("The url to track the Tez Session: " + appReport.getTrackingUrl()); sessionStarted.set(true); } catch (YarnException e) { + cleanStagingDir(); + throw new TezException(e); + } catch (IOException e) { + cleanStagingDir(); throw new TezException(e); } @@ -411,6 +416,21 @@ public synchronized void start() throws TezException, IOException { } } + private void cleanStagingDir() { + Configuration conf = amConfig.getTezConfiguration(); + String appId = sessionAppId.toString(); + Path stagingDir = TezCommonUtils.getTezSystemStagingPath(conf, appId); + boolean isStgDeleted = false; + try { + FileSystem fs = stagingDir.getFileSystem(conf); + isStgDeleted = fs.delete(stagingDir, true); + } catch (IOException ioe) { + LOG.error("Error deleting staging dir " + stagingDir, ioe); + } finally { + LOG.info("Staging dir {}, deleted:{} ", stagingDir, isStgDeleted); + } + } + public synchronized TezClient getClient(String appIdStr) throws IOException, TezException { return getClient(appIdfromString(appIdStr)); } @@ -450,6 +470,10 @@ public synchronized TezClient getClient(ApplicationId appId) throws TezException LOG.info("The url to track the Tez Session: " + appReport.getTrackingUrl()); sessionStarted.set(true); } catch (YarnException e) { + cleanStagingDir(); + throw new TezException(e); + } catch (IOException e) { + cleanStagingDir(); throw new TezException(e); } From 601d1f3578e50d7212477cd3919e9ccc7cf117cf Mon Sep 17 00:00:00 2001 From: Bodor Laszlo Date: Mon, 8 Feb 2021 19:34:34 +0100 Subject: [PATCH 030/137] TEZ-4283: Docker fails to build on master - upgrade to nodejs10 (#104) --- build-tools/docker/Dockerfile | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/build-tools/docker/Dockerfile b/build-tools/docker/Dockerfile index 143861d25a..ae26e41934 100644 --- a/build-tools/docker/Dockerfile +++ b/build-tools/docker/Dockerfile @@ -171,12 +171,13 @@ RUN pip2 install pylint==1.9.2 RUN pip2 install python-dateutil==2.7.3 ### -# Install node.js for web UI framework (4.2.6 ships with Xenial) +# Install node.js for web UI framework (8.10.0 ships with Bionic, let's override with 10.x) ### -# hadolint ignore=DL3008, DL3016 -RUN apt-get -q update \ - && apt-get install -y --no-install-recommends nodejs npm \ - && apt-get clean \ +RUN curl -sL https://deb.nodesource.com/setup_10.x | bash - +# hadolint ignore=DL3008, DL3015 +RUN apt-get install -y nodejs +# hadolint ignore=DL3016 +RUN apt-get clean \ && rm -rf /var/lib/apt/lists/* \ && npm install npm@latest -g \ && npm install -g jshint From cd0548631dd10be5f0d6c13d295f2726d180d239 Mon Sep 17 00:00:00 2001 From: Laszlo Bodor Date: Mon, 8 Feb 2021 20:21:39 +0100 Subject: [PATCH 031/137] =?UTF-8?q?TEZ-4273:=20Clear=20off=20staging=20fil?= =?UTF-8?q?es=20when=20TezYarnClient=20is=20unable=20to=20submit=20applica?= =?UTF-8?q?tions=20(Rajesh=20Balamohan=20via=20L=C3=A1szl=C3=B3=20Bodor)?= =?UTF-8?q?=20-=20addendum=20test=20fix?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../java/org/apache/tez/dag/app/TestMockDAGAppMaster.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/TestMockDAGAppMaster.java b/tez-dag/src/test/java/org/apache/tez/dag/app/TestMockDAGAppMaster.java index 1a4659593f..5dca6940ea 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/TestMockDAGAppMaster.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/TestMockDAGAppMaster.java @@ -958,7 +958,7 @@ public void testInitFailed() throws Exception { tezClient.start(); } catch (Exception e) { e.printStackTrace(); - Assert.assertEquals("FailInit", e.getCause().getCause().getMessage()); + Assert.assertEquals("FailInit", e.getCause().getCause().getCause().getMessage()); MockDAGAppMaster mockApp = tezClient.getLocalClient().getMockApp(); // will timeout if DAGAppMasterShutdownHook is not invoked mockApp.waitForServiceToStop(Integer.MAX_VALUE); @@ -974,7 +974,7 @@ public void testStartFailed() { tezClient.start(); } catch (Exception e) { e.printStackTrace(); - Assert.assertEquals("FailStart", e.getCause().getCause().getMessage()); + Assert.assertEquals("FailStart", e.getCause().getCause().getCause().getMessage()); MockDAGAppMaster mockApp = tezClient.getLocalClient().getMockApp(); // will timeout if DAGAppMasterShutdownHook is not invoked mockApp.waitForServiceToStop(Integer.MAX_VALUE); From 1847817c2a92f26b5e9aa71c250f4559a564bf0a Mon Sep 17 00:00:00 2001 From: Bodor Laszlo Date: Mon, 8 Feb 2021 21:29:49 +0100 Subject: [PATCH 032/137] TEZ-4281: dag_*_priority.dot files should go into a valid log directory (#102) (Laszlo Bodor reviewed by Rajesh Balamohan) --- tez-dag/findbugs-exclude.xml | 12 ++++++++++++ .../src/main/java/org/apache/tez/Utils.java | 18 +++--------------- .../org/apache/tez/dag/app/DAGAppMaster.java | 2 +- .../apache/tez/dag/app/dag/impl/DAGImpl.java | 18 +++++++++++++++++- 4 files changed, 33 insertions(+), 17 deletions(-) diff --git a/tez-dag/findbugs-exclude.xml b/tez-dag/findbugs-exclude.xml index a6ce38053b..50422ff0e0 100644 --- a/tez-dag/findbugs-exclude.xml +++ b/tez-dag/findbugs-exclude.xml @@ -252,4 +252,16 @@ + + + + + + + + + + + + diff --git a/tez-dag/src/main/java/org/apache/tez/Utils.java b/tez-dag/src/main/java/org/apache/tez/Utils.java index dbde327107..b352334f44 100644 --- a/tez-dag/src/main/java/org/apache/tez/Utils.java +++ b/tez-dag/src/main/java/org/apache/tez/Utils.java @@ -16,9 +16,7 @@ import javax.annotation.Nullable; import org.apache.hadoop.classification.InterfaceAudience; -import org.apache.hadoop.yarn.api.ApplicationConstants; import org.apache.hadoop.yarn.event.Event; -import org.apache.tez.common.TezCommonUtils; import org.apache.tez.dag.api.records.DAGProtos; import org.apache.tez.dag.app.AppContext; import org.apache.tez.dag.app.dag.DAG; @@ -109,19 +107,6 @@ public static void processNonFatalServiceErrorReport(String entityString, } } - /** - * Generate a visualization file. - * @param dag DAG. - * @param dagPB DAG plan. - * @param scheduler scheduler that provide the priorities of the vertexes. - */ - public static void generateDAGVizFile(final DAG dag, - final DAGProtos.DAGPlan dagPB, @Nullable final DAGScheduler scheduler) { - generateDAGVizFile(dag, dagPB, TezCommonUtils.getTrimmedStrings( - System.getenv(ApplicationConstants.Environment.LOG_DIRS.name())), - scheduler); - } - /** * Generate a visualization file. * @param dag DAG. @@ -217,6 +202,9 @@ public static void generateDAGVizFile(final DAG dag, if (logDirs != null && logDirs.length != 0) { outputFile += logDirs[0]; outputFile += File.separator; + } else { + LOG.warn("DAGVizFile will be created under current (.) directory: {}," + + " which is not expected and recommended", new File(".").getAbsolutePath()); } outputFile += dagId.toString(); // Means we have set the priorities 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 395e84a105..1352b68f26 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 @@ -1030,7 +1030,7 @@ DAGImpl createDAG(DAGPlan dagPB, TezDAGID dagId) { new DAGImpl(dagId, amConf, dagPB, dispatcher.getEventHandler(), taskCommunicatorManager, dagCredentials, clock, appMasterUgi.getShortUserName(), - taskHeartbeatHandler, context); + taskHeartbeatHandler, context).setLogDirs(logDirs); try { if (LOG.isDebugEnabled()) { 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 09e9e71b92..026ca29b7a 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 @@ -57,6 +57,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.security.Credentials; import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.yarn.api.ApplicationConstants; import org.apache.hadoop.yarn.api.records.LocalResource; import org.apache.hadoop.yarn.event.EventHandler; import org.apache.hadoop.yarn.state.InvalidStateTransitonException; @@ -67,6 +68,7 @@ import org.apache.hadoop.yarn.util.Clock; import org.apache.tez.common.ATSConstants; import org.apache.tez.common.ReflectionUtils; +import org.apache.tez.common.TezCommonUtils; import org.apache.tez.common.counters.AggregateTezCounters; import org.apache.tez.common.counters.DAGCounter; import org.apache.tez.common.counters.TezCounters; @@ -225,6 +227,7 @@ public class DAGImpl implements org.apache.tez.dag.app.dag.DAG, private TaskSpecificLaunchCmdOption taskSpecificLaunchCmdOption; private static final DagStateChangedCallback STATE_CHANGED_CALLBACK = new DagStateChangedCallback(); + private String[] logDirs; @VisibleForTesting Map> commitFutures @@ -1662,7 +1665,7 @@ DAGState initializeDAG() { // which didn't have the priorities if (getConf().getBoolean(TezConfiguration.TEZ_GENERATE_DEBUG_ARTIFACTS, TezConfiguration.TEZ_GENERATE_DEBUG_ARTIFACTS_DEFAULT)) { - Utils.generateDAGVizFile(this, jobPlan, dagScheduler); + Utils.generateDAGVizFile(this, jobPlan, logDirs, dagScheduler); } return DAGState.INITED; } @@ -2510,4 +2513,17 @@ public void onFailure(Throwable t) { eventHandler.handle(new DAGEventCommitCompleted(dagId, outputKey, false, t)); } } + + public String[] getLogDirs() { + if (logDirs == null) { + logDirs = TezCommonUtils + .getTrimmedStrings(System.getenv(ApplicationConstants.Environment.LOG_DIRS.name())); + } + return logDirs; + } + + public DAGImpl setLogDirs(String[] logDirs) { + this.logDirs = logDirs; + return this; + } } From 84b5bd6e7dfb554ed3a4e9a3642f6cfbb0bb9df7 Mon Sep 17 00:00:00 2001 From: Bodor Laszlo Date: Tue, 9 Feb 2021 10:51:56 +0100 Subject: [PATCH 033/137] TEZ-4282: Possible NPE in LocalClient after TEZ-4236 (#103) (Laszlo Bodor reviewed by Panagiotis Garefalakis, Ashutosh Chauhan) --- .../org/apache/tez/client/LocalClient.java | 19 ++++++++++++++----- 1 file changed, 14 insertions(+), 5 deletions(-) diff --git a/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java b/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java index f6d9587734..d0580bbae9 100644 --- a/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java +++ b/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java @@ -157,7 +157,9 @@ public ApplicationId submitApplication(ApplicationSubmissionContext appContext) @Override public void killApplication(ApplicationId appId) { try { - clientHandler.shutdownAM(); + if (clientHandler != null){ + clientHandler.shutdownAM(); + } } catch (TezException e) { throw new RuntimeException(e); } @@ -403,7 +405,13 @@ protected DAGAppMaster createDAGAppMaster(ApplicationAttemptId applicationAttemp @Override public TezAppMasterStatus getAMStatus(Configuration configuration, ApplicationId appId, UserGroupInformation ugi) throws TezException, ServiceException, IOException { - return clientHandler.getTezAppMasterStatus(); + if (isLocalWithoutNetwork) { + if (clientHandler == null) { + return TezAppMasterStatus.INITIALIZING; + } + return clientHandler.getTezAppMasterStatus(); + } + return super.getAMStatus(configuration, appId, ugi); } @Override @@ -438,10 +446,11 @@ public DAGStatus apply(Set statusOpts, Long timeout) { public boolean shutdownSession(Configuration configuration, ApplicationId sessionAppId, UserGroupInformation ugi) throws TezException, IOException, ServiceException { if (isLocalWithoutNetwork) { - clientHandler.shutdownAM(); + if (clientHandler != null){ + clientHandler.shutdownAM(); + } return true; - } else { - return super.shutdownSession(configuration, sessionAppId, ugi); } + return super.shutdownSession(configuration, sessionAppId, ugi); } } From 851e9d8b5efa962dc879cf6507de7cb495cfdad5 Mon Sep 17 00:00:00 2001 From: Bodor Laszlo Date: Tue, 9 Feb 2021 18:55:32 +0100 Subject: [PATCH 034/137] TEZ-4278: Show PR link in Jira (#99) --- .asf.yaml | 38 ++++++++++++++++++++++++++++++++++++++ 1 file changed, 38 insertions(+) create mode 100644 .asf.yaml diff --git a/.asf.yaml b/.asf.yaml new file mode 100644 index 0000000000..4fac240422 --- /dev/null +++ b/.asf.yaml @@ -0,0 +1,38 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +github: + description: "Apache Tez" + homepage: https://tez.apache.org/ + labels: + - tez + - java + - apache + - big-data + - hadoop + features: + wiki: false + issues: false + projects: false + enabled_merge_buttons: + squash: true + merge: false + rebase: false +notifications: + commits: commits@tez.apache.org + issues: dev@tez.apache.org + pullrequests: dev@tez.apache.org + jira_options: link label worklog From a42e1700e6020eab43d42abbfc649a50798a1e92 Mon Sep 17 00:00:00 2001 From: Roman Date: Wed, 10 Feb 2021 22:35:13 +0300 Subject: [PATCH 035/137] TEZ-3966: Fix config file mime type (#26) * Fix file mime type, to avoid error when "X-Content-Type-Options: nosniff" header enabled * Update ember-cli-build.js to include only 'configs.js' --- tez-ui/README.md | 4 ++-- tez-ui/src/main/webapp/README.md | 2 +- tez-ui/src/main/webapp/app/index.html | 2 +- tez-ui/src/main/webapp/config/{configs.env => configs.js} | 0 tez-ui/src/main/webapp/ember-cli-build.js | 2 +- 5 files changed, 5 insertions(+), 5 deletions(-) rename tez-ui/src/main/webapp/config/{configs.env => configs.js} (100%) diff --git a/tez-ui/README.md b/tez-ui/README.md index cf62cdcd38..2716ca2823 100644 --- a/tez-ui/README.md +++ b/tez-ui/README.md @@ -32,8 +32,8 @@ For more information on Tez and the Tez UI - Check the [Tez homepage](http://tez * `yarn.timeline-service.enabled` : Enabled the timeline server for logging details * `yarn.timeline-service.webapp.address` : Value must be the IP:PORT on which timeline server is running -### In configs.env - This environment configuration file can be found at `./src/main/webapp/config/configs.env` +### In configs.js + This environment configuration file can be found at `./src/main/webapp/config/configs.js` * `ENV.hosts.timeline` : Timeline Server Address. By default TEZ UI looks for timeline server at http://localhost:8188. * `ENV.hosts.rm` : Resource Manager Address. By default RM REST APIs are expected to be at http://localhost:8088. diff --git a/tez-ui/src/main/webapp/README.md b/tez-ui/src/main/webapp/README.md index 0443df49ad..0ee7b46c10 100644 --- a/tez-ui/src/main/webapp/README.md +++ b/tez-ui/src/main/webapp/README.md @@ -24,7 +24,7 @@ For more information on Tez and the Tez UI - Check the [Tez homepage](http://tez ## Configurations * By default timeline is expected at localhost:8188 & RM at localhost:8088 -* You can point the UI to custom locations by setting the environment variables in `src/main/webapp/config/configs.env` +* You can point the UI to custom locations by setting the environment variables in `src/main/webapp/config/configs.js` ## Development diff --git a/tez-ui/src/main/webapp/app/index.html b/tez-ui/src/main/webapp/app/index.html index 757d66330f..f90c5c4cf3 100644 --- a/tez-ui/src/main/webapp/app/index.html +++ b/tez-ui/src/main/webapp/app/index.html @@ -37,7 +37,7 @@ {{content-for 'body'}} - + diff --git a/tez-ui/src/main/webapp/config/configs.env b/tez-ui/src/main/webapp/config/configs.js similarity index 100% rename from tez-ui/src/main/webapp/config/configs.env rename to tez-ui/src/main/webapp/config/configs.js diff --git a/tez-ui/src/main/webapp/ember-cli-build.js b/tez-ui/src/main/webapp/ember-cli-build.js index 9b52210af9..7bbc77d334 100644 --- a/tez-ui/src/main/webapp/ember-cli-build.js +++ b/tez-ui/src/main/webapp/ember-cli-build.js @@ -44,7 +44,7 @@ module.exports = function(defaults) { var configEnv = new Funnel('config', { srcDir: '/', - include: ['*.env'], + include: ['configs.js'], destDir: '/config' }); var zipWorker = new Funnel('bower_components/zip-js', { From f88aa6f21980ba85160b6338515efc54e974c267 Mon Sep 17 00:00:00 2001 From: Bodor Laszlo Date: Fri, 12 Feb 2021 16:28:48 +0100 Subject: [PATCH 036/137] TEZ-4290: Dockerfile improvements (#111) (Laszlo Bodor reviewed by Jonathan Turner Eagles) --- build-tools/docker/Dockerfile | 368 +++++++++++++++------------ build-tools/docker/tez_env_checks.sh | 117 --------- 2 files changed, 205 insertions(+), 280 deletions(-) delete mode 100644 build-tools/docker/tez_env_checks.sh diff --git a/build-tools/docker/Dockerfile b/build-tools/docker/Dockerfile index ae26e41934..f51cc3bd2b 100644 --- a/build-tools/docker/Dockerfile +++ b/build-tools/docker/Dockerfile @@ -1,4 +1,4 @@ - +# # Licensed to the Apache Software Foundation (ASF) under one # or more contributor license agreements. See the NOTICE file # distributed with this work for additional information @@ -15,219 +15,261 @@ # See the License for the specific language governing permissions and # limitations under the License. -# Dockerfile for installing the necessary dependencies for building Hadoop. -# See BUILDING.txt. +############### +# +# Apache Yetus Dockerfile for Apache Tez +# NOTE: This file is compatible with Docker BuildKit. It will work +# with standard docker build, but it is a lot faster +# if BuildKit is enabled. +# +############### -FROM ubuntu:bionic +FROM ubuntu:focal AS tezbase WORKDIR /root - SHELL ["/bin/bash", "-o", "pipefail", "-c"] -##### -# Disable suggests/recommends -##### -RUN echo APT::Install-Recommends "0"\; > /etc/apt/apt.conf.d/10disableextras -RUN echo APT::Install-Suggests "0"\; >> /etc/apt/apt.conf.d/10disableextras - ENV DEBIAN_FRONTEND noninteractive ENV DEBCONF_TERSE true ###### -# Install common dependencies from packages. Versions here are either -# sufficient or irrelevant. -# -# WARNING: DO NOT PUT JAVA APPS HERE! Otherwise they will install default -# Ubuntu Java. See Java section below! +# Install some basic Apache Yetus requirements +# some git repos need ssh-client so do it too +# Adding libffi-dev for all the programming languages +# that take advantage of it. ###### # hadolint ignore=DL3008 -RUN apt-get -q update \ - && apt-get -q install -y --no-install-recommends \ - apt-utils \ - build-essential \ - bzip2 \ - clang \ - curl \ - doxygen \ - fuse \ - g++ \ - gcc \ - git \ - gnupg-agent \ - libbz2-dev \ - libfuse-dev \ - libprotobuf-dev \ - libprotoc-dev \ - libsasl2-dev \ - libsnappy-dev \ - libssl-dev \ - libtool \ - locales \ - make \ - pinentry-curses \ - pkg-config \ - python \ - python2.7 \ - python-pip \ - python-pkg-resources \ - python-setuptools \ - python-wheel \ - rsync \ - software-properties-common \ - sudo \ - valgrind \ - zlib1g-dev \ +RUN apt-get -q update && apt-get -q install --no-install-recommends -y \ + apt-transport-https \ + apt-utils \ + ca-certificates \ + curl \ + dirmngr \ + git \ + gpg \ + gpg-agent \ + libffi-dev \ + locales \ + make \ + pkg-config \ + rsync \ + software-properties-common \ + ssh-client \ + xz-utils \ && apt-get clean \ && rm -rf /var/lib/apt/lists/* +### +# Set the locale +### +RUN locale-gen en_US.UTF-8 +ENV LANG en_US.UTF-8 +ENV LANGUAGE en_US:en +ENV LC_ALL en_US.UTF-8 -####### -# OpenJDK 8 -####### +#### +# Install GNU C/C++ (everything generally needs this) +#### # hadolint ignore=DL3008 -RUN apt-get -q update \ - && apt-get -q install -y --no-install-recommends openjdk-8-jdk libbcprov-java \ +RUN apt-get -q update && apt-get -q install --no-install-recommends -y \ + g++ \ + gcc \ + libc-dev \ && apt-get clean \ && rm -rf /var/lib/apt/lists/* +### +# Install golang as part of base so we can do each +# helper utility in parallel. go bins are typically +# statically linked, so this is perfectly safe. +### +# hadolint ignore=DL3008 +RUN add-apt-repository -y ppa:longsleep/golang-backports \ + && apt-get -q update \ + && apt-get -q install --no-install-recommends -y golang-go \ + && apt-get clean \ + && rm -rf /var/lib/apt/lists/* -###### -# Install cmake 3.1.0 (3.5.1 ships with Xenial) -###### -RUN mkdir -p /opt/cmake \ - && curl -L -s -S \ - https://cmake.org/files/v3.1/cmake-3.1.0-Linux-x86_64.tar.gz \ - -o /opt/cmake.tar.gz \ - && tar xzf /opt/cmake.tar.gz --strip-components 1 -C /opt/cmake -ENV CMAKE_HOME /opt/cmake -ENV PATH "${PATH}:/opt/cmake/bin" +############ +# Fetch all of the non-conflicting bits in parallel +############# ###### -# Install Google Protobuf 2.5.0 (2.6.0 ships with Xenial) +# Install Google Protobuf 2.5.0 ###### -# hadolint ignore=DL3003 +FROM tezbase AS protobuf250 +SHELL ["/bin/bash", "-o", "pipefail", "-c"] RUN mkdir -p /opt/protobuf-src \ && curl -L -s -S \ https://github.com/google/protobuf/releases/download/v2.5.0/protobuf-2.5.0.tar.gz \ -o /opt/protobuf.tar.gz \ - && tar xzf /opt/protobuf.tar.gz --strip-components 1 -C /opt/protobuf-src \ - && cd /opt/protobuf-src \ - && ./configure --prefix=/opt/protobuf \ - && make install \ - && cd /root \ - && rm -rf /opt/protobuf-src -ENV PROTOBUF_HOME /opt/protobuf -ENV PATH "${PATH}:/opt/protobuf/bin" + && tar xzf /opt/protobuf.tar.gz --strip-components 1 -C /opt/protobuf-src +WORKDIR /opt/protobuf-src +RUN ./configure --prefix=/opt/protobuf \ + && make install +WORKDIR /root +RUN rm -rf /opt/protobuf-src -###### -# Install Apache Maven 3.3.9 (3.3.9 ships with Xenial) -###### +#### +# Install shellcheck (shell script lint) +#### +FROM tezbase AS shellcheck +SHELL ["/bin/bash", "-o", "pipefail", "-c"] +RUN curl -sSL \ + https://github.com/koalaman/shellcheck/releases/download/v0.7.1/shellcheck-v0.7.1.linux.x86_64.tar.xz \ + | tar --strip-components 1 --wildcards -xJf - '*/shellcheck' \ + && chmod a+rx shellcheck \ + && mv shellcheck /bin/shellcheck \ + && shasum -a 512 /bin/shellcheck \ + | awk '$1!="aae813283d49f18f95a205dca1c5184267d07534a08abc952ebea1958fee06f8a0207373b6770a083079ba875458ea9da443f2b9910a50dcd93b935048bc14f5" {exit(1)}' + +#### +# Install hadolint (dockerfile lint) +#### +FROM tezbase AS hadolint +SHELL ["/bin/bash", "-o", "pipefail", "-c"] +RUN curl -sSL \ + https://github.com/hadolint/hadolint/releases/download/v1.18.0/hadolint-Linux-x86_64 \ + -o /bin/hadolint \ + && chmod a+rx /bin/hadolint \ + && shasum -a 512 /bin/hadolint \ + | awk '$1!="df27253d374c143a606483b07a26234ac7b4bca40b4eba53e79609c81aa70146e7d5c145f90dcec71d6d1aad1048b7d9d2de68d92284f48a735d04d19c5c5559" {exit(1)}' + +#### +# Install buf (protobuf lint) +#### +FROM tezbase AS buf +SHELL ["/bin/bash", "-o", "pipefail", "-c"] +RUN curl -sSL \ + https://github.com/bufbuild/buf/releases/download/v0.21.0/buf-Linux-x86_64.tar.gz \ + -o buf.tar.gz \ + && shasum -a 256 buf.tar.gz \ + | awk '$1!="95aba62ac0ecc5a9120cc58c65cdcc85038633a816bddfe8398c5ae3b32803f1" {exit(1)}' \ + && tar -xzf buf.tar.gz -C /usr/local --strip-components 1 \ + && rm buf.tar.gz + +######## +# +# +# Content that needs to be installed in order due to packages... +# +# +######## + +FROM tezbase +SHELL ["/bin/bash", "-o", "pipefail", "-c"] + +#### +# Install java (first, since we want to dicate what form of Java) +#### + +#### +# OpenJDK 8 +#### # hadolint ignore=DL3008 -RUN apt-get -q update \ - && apt-get -q install -y --no-install-recommends maven \ +RUN apt-get -q update && apt-get -q install --no-install-recommends -y openjdk-8-jdk-headless \ && apt-get clean \ && rm -rf /var/lib/apt/lists/* -ENV MAVEN_HOME /usr + +#### +# OpenJDK 11 (but keeps default to JDK8) +# NOTE: This default only works when Apache Yetus is launched +# _in_ the container and not outside of it! +#### +# hadolint ignore=DL3008 +RUN apt-get -q update && apt-get -q install --no-install-recommends -y default-jre-headless openjdk-11-jdk-headless \ + && apt-get clean \ + && rm -rf /var/lib/apt/lists/* \ + && update-java-alternatives -s java-1.8.0-openjdk-amd64 \ + && rm -f /usr/lib/jvm/default-java \ + && ln -s java-8-openjdk-amd64 /usr/lib/jvm/default-java +ENV JAVA_HOME /usr/lib/jvm/java-8-openjdk-amd64 ###### -# Install findbugs 3.0.1 (3.0.1 ships with Xenial) -# Ant is needed for findbugs +# Install findbugs ###### # hadolint ignore=DL3008 -RUN apt-get -q update \ - && apt-get -q install -y --no-install-recommends findbugs ant \ +RUN apt-get -q update && apt-get -q install --no-install-recommends -y findbugs \ && apt-get clean \ && rm -rf /var/lib/apt/lists/* ENV FINDBUGS_HOME /usr -#### -# Install shellcheck (0.4.6, the latest as of 2017-09-26) -#### +###### +# Install maven +###### # hadolint ignore=DL3008 -RUN add-apt-repository -y ppa:hvr/ghc \ - && apt-get -q update \ - && apt-get -q install -y --no-install-recommends shellcheck ghc-8.0.2 \ +RUN apt-get -q update && apt-get -q install --no-install-recommends -y maven \ && apt-get clean \ && rm -rf /var/lib/apt/lists/* -#### -# Install bats (0.4.0, the latest as of 2017-09-26, ships with Xenial) -#### -# hadolint ignore=DL3008 -RUN apt-get -q update \ - && apt-get -q install -y --no-install-recommends bats \ +###### +# Install python3 and pylint3 +# astroid and pylint go hand-in-hand. Upgrade both at the same time. +###### +# hadolint ignore=DL3008,DL3013 +RUN apt-get -q update && apt-get -q install --no-install-recommends -y \ + python3 \ + python3-bcrypt \ + python3-cffi \ + python3-cryptography \ + python3-dateutil \ + python3-dev \ + python3-dev \ + python3-isort \ + python3-dockerpty \ + python3-nacl \ + python3-pyrsistent \ + python3-setuptools \ + python3-setuptools \ + python3-singledispatch \ + python3-six \ + python3-wheel \ + python3-wrapt \ + python3-yaml \ && apt-get clean \ - && rm -rf /var/lib/apt/lists/* - -#### -# Install pylint at fixed version (2.0.0 removed python2 support) -# https://github.com/PyCQA/pylint/issues/2294 -#### -RUN pip2 install pylint==1.9.2 - -#### -# Install dateutil.parser -#### -RUN pip2 install python-dateutil==2.7.3 + && rm -rf /var/lib/apt/lists/* \ + && curl -sSL https://bootstrap.pypa.io/get-pip.py -o /tmp/get-pip.py \ + && python3 /tmp/get-pip.py \ + && rm /usr/local/bin/pip /tmp/get-pip.py \ + && pip3 install -v \ + astroid==2.4.2 \ + codespell==2.0 \ + pylint==2.5.3 \ + yamllint==1.24.2 \ + && rm -rf /root/.cache \ + && mv /usr/local/bin/pylint /usr/local/bin/pylint3 +RUN ln -s /usr/local/bin/pylint3 /usr/local/bin/pylint +RUN ln -s /usr/local/bin/pip3 /usr/local/bin/pip ### -# Install node.js for web UI framework (8.10.0 ships with Bionic, let's override with 10.x) +# Install npm and JSHint ### -RUN curl -sL https://deb.nodesource.com/setup_10.x | bash - -# hadolint ignore=DL3008, DL3015 -RUN apt-get install -y nodejs -# hadolint ignore=DL3016 -RUN apt-get clean \ +# hadolint ignore=DL3008 +RUN curl -sSL https://deb.nodesource.com/setup_14.x | bash - \ + && apt-get -q install --no-install-recommends -y nodejs \ + && apt-get clean \ && rm -rf /var/lib/apt/lists/* \ - && npm install npm@latest -g \ - && npm install -g jshint + && npm install -g \ + jshint@2.12.0 \ + markdownlint-cli@0.23.2 \ + && rm -rf /root/.npm -### -# Install hadolint -#### -RUN curl -L -s -S \ - https://github.com/hadolint/hadolint/releases/download/v1.11.1/hadolint-Linux-x86_64 \ - -o /bin/hadolint \ - && chmod a+rx /bin/hadolint \ - && shasum -a 512 /bin/hadolint | \ - awk '$1!="734e37c1f6619cbbd86b9b249e69c9af8ee1ea87a2b1ff71dccda412e9dac35e63425225a95d71572091a3f0a11e9a04c2fc25d9e91b840530c26af32b9891ca" {exit(1)}' - -### -# Avoid out of memory errors in builds -### -ENV MAVEN_OPTS -Xms256m -Xmx1536m +##### +# Now all the stuff that was built in parallel +##### +COPY --from=shellcheck /bin/shellcheck /bin/shellcheck +COPY --from=hadolint /bin/hadolint /bin/hadolint +COPY --from=buf /usr/local/bin/buf /usr/local/bin/buf +COPY --from=protobuf250 /opt/protobuf /opt/protobuf -### -# Everything past this point is either not needed for testing or breaks Yetus. -# So tell Yetus not to read the rest of the file: -# YETUS CUT HERE -### +ENV PROTOBUF_HOME /opt/protobuf +ENV PROTOC_PATH /opt/protobuf/bin/protoc +ENV PATH "${PATH}:/opt/protobuf/bin" #### -# Install svn & Forrest (for Apache Hadoop website) +# YETUS CUT HERE +# Magic text above! Everything from here on is ignored +# by Yetus, so could include anything not needed +# by your testing environment ### -# hadolint ignore=DL3008 -RUN apt-get -q update \ - && apt-get -q install -y --no-install-recommends subversion \ - && apt-get clean \ - && rm -rf /var/lib/apt/lists/* - -RUN mkdir -p /opt/apache-forrest \ - && curl -L -s -S \ - https://archive.apache.org/dist/forrest/0.8/apache-forrest-0.8.tar.gz \ - -o /opt/forrest.tar.gz \ - && tar xzf /opt/forrest.tar.gz --strip-components 1 -C /opt/apache-forrest -RUN echo 'forrest.home=/opt/apache-forrest' > build.properties -ENV FORREST_HOME=/opt/apache-forrest - -# Hugo static website generator (for new tez site and Ozone docs) -RUN curl -L -o hugo.deb https://github.com/gohugoio/hugo/releases/download/v0.30.2/hugo_0.30.2_Linux-64bit.deb \ - && dpkg --install hugo.deb \ - && rm hugo.deb - -# Add a welcome message and environment checks. -COPY tez_env_checks.sh /root/tez_env_checks.sh -RUN chmod 755 /root/tez_env_checks.sh -# hadolint ignore=SC2016 -RUN echo '${HOME}/tez_env_checks.sh' >> /root/.bashrc diff --git a/build-tools/docker/tez_env_checks.sh b/build-tools/docker/tez_env_checks.sh deleted file mode 100644 index c9420e79e9..0000000000 --- a/build-tools/docker/tez_env_checks.sh +++ /dev/null @@ -1,117 +0,0 @@ -#!/usr/bin/env bash - -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -# SHELLDOC-IGNORE - -# ------------------------------------------------------- -function showWelcome { -cat < Date: Fri, 12 Feb 2021 17:21:36 +0100 Subject: [PATCH 037/137] TEZ-4259: Create Jenkinsfile for use with tez-multibranch (github yetus integration) (Jonathan Turner Eagles reviewed by Laszlo Bodor) --- Jenkinsfile | 216 ++++++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 216 insertions(+) create mode 100644 Jenkinsfile diff --git a/Jenkinsfile b/Jenkinsfile new file mode 100644 index 0000000000..1d6264658b --- /dev/null +++ b/Jenkinsfile @@ -0,0 +1,216 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +pipeline { + + agent { + label 'Hadoop' + } + + options { + buildDiscarder(logRotator(numToKeepStr: '5')) + timeout (time: 20, unit: 'HOURS') + timestamps() + checkoutToSubdirectory('src') + } + + environment { + SOURCEDIR = 'src' + // will also need to change notification section below + PATCHDIR = 'out' + DOCKERFILE = "${SOURCEDIR}/build-tools/docker/Dockerfile" + YETUS='yetus' + // Branch or tag name. Yetus release tags are 'rel/X.Y.Z' + YETUS_VERSION='rel/0.12.0' + + } + + parameters { + string(name: 'JIRA_ISSUE_KEY', + defaultValue: '', + description: 'The JIRA issue that has a patch needing pre-commit testing. Example: HADOOP-1234') + } + + stages { + stage ('install yetus') { + steps { + dir("${WORKSPACE}/${YETUS}") { + checkout([ + $class: 'GitSCM', + branches: [[name: "${env.YETUS_VERSION}"]], + userRemoteConfigs: [[ url: 'https://github.com/jteagles/yetus']]] + ) + } + } + } + + stage ('precommit-run') { + steps { + withCredentials( + [usernamePassword(credentialsId: 'apache-tez-at-github.com', + passwordVariable: 'GITHUB_PASSWD', + usernameVariable: 'GITHUB_USER'), + usernamePassword(credentialsId: 'tez-ci', + passwordVariable: 'JIRA_PASSWORD', + usernameVariable: 'JIRA_USER')]) { + sh '''#!/usr/bin/env bash + + set -e + + TESTPATCHBIN="${WORKSPACE}/${YETUS}/precommit/src/main/shell/test-patch.sh" + + # this must be clean for every run + if [[ -d "${WORKSPACE}/${PATCHDIR}" ]]; then + rm -rf "${WORKSPACE}/${PATCHDIR}" + fi + mkdir -p "${WORKSPACE}/${PATCHDIR}" + + # if given a JIRA issue, process it. If CHANGE_URL is set + # (e.g., Github Branch Source plugin), process it. + # otherwise exit, because we don't want Hadoop to do a + # full build. We wouldn't normally do this check for smaller + # projects. :) + if [[ -n "${JIRA_ISSUE_KEY}" ]]; then + YETUS_ARGS+=("${JIRA_ISSUE_KEY}") + elif [[ -z "${CHANGE_URL}" ]]; then + echo "Full build skipped" > "${WORKSPACE}/${PATCHDIR}/report.html" + exit 0 + fi + + YETUS_ARGS+=("--patch-dir=${WORKSPACE}/${PATCHDIR}") + + # where the source is located + YETUS_ARGS+=("--basedir=${WORKSPACE}/${SOURCEDIR}") + + # our project defaults come from a personality file + YETUS_ARGS+=("--project=tez") + + # lots of different output formats + YETUS_ARGS+=("--brief-report-file=${WORKSPACE}/${PATCHDIR}/brief.txt") + YETUS_ARGS+=("--console-report-file=${WORKSPACE}/${PATCHDIR}/console.txt") + YETUS_ARGS+=("--html-report-file=${WORKSPACE}/${PATCHDIR}/report.html") + + # enable writing back to Github + YETUS_ARGS+=(--github-user="${GITHUB_USER}") + YETUS_ARGS+=(--github-password="${GITHUB_PASSWD}") + + # enable writing back to ASF JIRA + YETUS_ARGS+=(--jira-password="${JIRA_PASSWORD}") + YETUS_ARGS+=(--jira-user="${JIRA_USER}") + + # auto-kill any surefire stragglers during unit test runs + YETUS_ARGS+=("--reapermode=kill") + + # set relatively high limits for ASF machines + # changing these to higher values may cause problems + # with other jobs on systemd-enabled machines + YETUS_ARGS+=("--proclimit=5500") + YETUS_ARGS+=("--dockermemlimit=20g") + + # -1 findbugs issues that show up prior to the patch being applied + # YETUS_ARGS+=("--findbugs-strict-precheck") + + # rsync these files back into the archive dir + YETUS_ARGS+=("--archive-list=checkstyle-errors.xml,findbugsXml.xml") + + # URL for user-side presentation in reports and such to our artifacts + # (needs to match the archive bits below) + YETUS_ARGS+=("--build-url-artifacts=artifact/out") + + # plugins to enable + YETUS_ARGS+=("--plugins=all") + + # use Hadoop's bundled shelldocs + YETUS_ARGS+=("--shelldocs=${WORKSPACE}/${SOURCEDIR}/dev-support/bin/shelldocs") + + # don't let these tests cause -1s because we aren't really paying that + # much attention to them + YETUS_ARGS+=("--tests-filter=checkstyle") + + # run in docker mode and specifically point to our + # Dockerfile since we don't want to use the auto-pulled version. + YETUS_ARGS+=("--docker") + YETUS_ARGS+=("--dockerfile=${DOCKERFILE}") + YETUS_ARGS+=("--mvn-custom-repos") + + # effectively treat dev-suport as a custom maven module + YETUS_ARGS+=("--skip-dirs=dev-support") + + # help keep the ASF boxes clean + YETUS_ARGS+=("--sentinel") + + # test with Java 8 and 11 + YETUS_ARGS+=("--java-home=/usr/lib/jvm/java-8-openjdk-amd64") + YETUS_ARGS+=("--multijdkdirs=/usr/lib/jvm/java-11-openjdk-amd64") + YETUS_ARGS+=("--multijdktests=compile") + YETUS_ARGS+=("--debug") + + "${TESTPATCHBIN}" "${YETUS_ARGS[@]}" + ''' + } + } + } + + } + + post { + always { + script { + // Yetus output + archiveArtifacts "${env.PATCHDIR}/**" + // Publish the HTML report so that it can be looked at + // Has to be relative to WORKSPACE. + publishHTML (target: [ + allowMissing: true, + keepAll: true, + alwaysLinkToLastBuild: true, + // Has to be relative to WORKSPACE + reportDir: "${env.PATCHDIR}", + reportFiles: 'report.html', + reportName: 'Yetus Report' + ]) + // Publish JUnit results + try { + junit "${env.SOURCEDIR}/**/target/surefire-reports/*.xml" + } catch(e) { + echo 'junit processing: ' + e.toString() + } + } + } + + // Jenkins pipeline jobs fill slaves on PRs without this :( + cleanup() { + script { + sh ''' + # See YETUS-764 + if [ -f "${WORKSPACE}/${PATCHDIR}/pidfile.txt" ]; then + echo "test-patch process appears to still be running: killing" + kill `cat "${WORKSPACE}/${PATCHDIR}/pidfile.txt"` || true + sleep 10 + fi + if [ -f "${WORKSPACE}/${PATCHDIR}/cidfile.txt" ]; then + echo "test-patch container appears to still be running: killing" + docker kill `cat "${WORKSPACE}/${PATCHDIR}/cidfile.txt"` || true + fi + # See HADOOP-13951 + chmod -R u+rxw "${WORKSPACE}" + ''' + deleteDir() + } + } + } +} \ No newline at end of file From f9c96b2be3d1ff0b81ab397984f99b85cdcf7047 Mon Sep 17 00:00:00 2001 From: Jonathan Eagles Date: Fri, 12 Feb 2021 10:48:35 -0600 Subject: [PATCH 038/137] TEZ-4241. UI: containerLogs testWrapper test fails (review by Bodor Laszlo) Signed-off-by: Jonathan Eagles --- tez-ui/src/main/webapp/tests/unit/serializers/dag-test.js | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tez-ui/src/main/webapp/tests/unit/serializers/dag-test.js b/tez-ui/src/main/webapp/tests/unit/serializers/dag-test.js index 6334569491..80f41b6bbb 100644 --- a/tez-ui/src/main/webapp/tests/unit/serializers/dag-test.js +++ b/tez-ui/src/main/webapp/tests/unit/serializers/dag-test.js @@ -110,8 +110,8 @@ test('containerLogs test', function(assert) { }), [], "No logs"); assert.deepEqual(mapper({ - otherinfo: {inProgressLogsURL_1: "foo", inProgressLogsURL_2: "bar"}, - }), [{text: "1", href: "http://foo"}, {text: "2", href: "http://bar"}], "2 logs"); + otherinfo: {inProgressLogsURL_1: "http://foo", inProgressLogsURL_2: "https://bar"}, + }), [{text: "1", href: "http://foo"}, {text: "2", href: "https://bar"}], "2 logs"); }); test('vertexIdNameMap test', function(assert) { From b4152fc85316eb95a4f8a6b2e41a151a89f772da Mon Sep 17 00:00:00 2001 From: belugabehr <12578579+belugabehr@users.noreply.github.com> Date: Fri, 12 Feb 2021 12:55:37 -0500 Subject: [PATCH 039/137] TEZ-4288: Upgrade SLF4J to 1.7.30 (#109) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 892a8a2d66..9fa2cb4723 100644 --- a/pom.xml +++ b/pom.xml @@ -62,7 +62,7 @@ 3.10.5.Final 0.13.0 1.19 - 1.7.10 + 1.7.30 2.5.0 ${env.PROTOC_PATH} scm:git:https://gitbox.apache.org/repos/asf/tez.git From 7d32807d858dad37bb31637eb04a80c8436f605a Mon Sep 17 00:00:00 2001 From: Jonathan Eagles Date: Sat, 13 Feb 2021 22:56:28 -0600 Subject: [PATCH 040/137] TEZ-4025. javadoc compilation is broken in jdk11 (review by Bodor Laszlo) Signed-off-by: Jonathan Eagles --- tez-tools/tez-javadoc-tools/pom.xml | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/tez-tools/tez-javadoc-tools/pom.xml b/tez-tools/tez-javadoc-tools/pom.xml index b17604f207..e0405b685b 100644 --- a/tez-tools/tez-javadoc-tools/pom.xml +++ b/tez-tools/tez-javadoc-tools/pom.xml @@ -94,6 +94,12 @@ true + + maven-javadoc-plugin + + org.apache.tez.tools.javadoc.doclet + + From 3089d478f70f139139fea68028712d5fc32012d1 Mon Sep 17 00:00:00 2001 From: Bodor Laszlo Date: Sun, 14 Feb 2021 20:10:05 +0100 Subject: [PATCH 041/137] TEZ-4286: Consider sending fewer emails from github to dev@tez.apache.org (#107) (Laszlo Bodor reviewed by Jonathan Turner Eagles) --- .asf.yaml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/.asf.yaml b/.asf.yaml index 4fac240422..776f8a6f7c 100644 --- a/.asf.yaml +++ b/.asf.yaml @@ -33,6 +33,6 @@ github: rebase: false notifications: commits: commits@tez.apache.org - issues: dev@tez.apache.org - pullrequests: dev@tez.apache.org + issues: issues@tez.apache.org + pullrequests: issues@tez.apache.org jira_options: link label worklog From 70623f0d8e9de3c59b61c10abe5442bfd8f66e2c Mon Sep 17 00:00:00 2001 From: belugabehr <12578579+belugabehr@users.noreply.github.com> Date: Wed, 17 Feb 2021 16:47:00 -0500 Subject: [PATCH 042/137] TEZ-4289: Remove Dependency on commons-math (#110) * TEZ-4289: Remove Dependency on commons-math * Remove stray reference to commons-math * Fix checkstyle and missing parenthesis --- pom.xml | 5 ----- .../src/main/java/org/apache/tez/client/TezClientUtils.java | 5 ++--- tez-dag/pom.xml | 4 ---- .../org/apache/tez/dag/app/rm/DagAwareYarnTaskScheduler.java | 5 ++--- .../org/apache/tez/dag/app/rm/YarnTaskSchedulerService.java | 5 ++--- 5 files changed, 6 insertions(+), 18 deletions(-) diff --git a/pom.xml b/pom.xml index 9fa2cb4723..19fd5167cd 100644 --- a/pom.xml +++ b/pom.xml @@ -668,11 +668,6 @@ commons-collections4 4.1 - - org.apache.commons - commons-math3 - 3.1.1 - commons-cli commons-cli diff --git a/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java b/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java index 79069ede7d..389b58400b 100644 --- a/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java +++ b/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java @@ -43,7 +43,6 @@ import com.google.common.base.Strings; import org.apache.commons.codec.digest.DigestUtils; import org.apache.commons.lang.StringUtils; -import org.apache.commons.math3.util.Precision; import org.apache.tez.common.JavaOptsChecker; import org.apache.tez.dag.api.records.DAGProtos.AMPluginDescriptorProto; import org.apache.tez.serviceplugins.api.ServicePluginsDescriptor; @@ -1007,11 +1006,11 @@ public static String maybeAddDefaultMemoryJavaOpts(String javaOpts, Resource res return javaOpts; } - if ((maxHeapFactor <= 0 && !Precision.equals(maxHeapFactor, -1, 0.01)) || maxHeapFactor >= 1) { + if ((maxHeapFactor <= 0 && Double.valueOf("-1") != maxHeapFactor) || maxHeapFactor >= 1) { return javaOpts; } - if (Precision.equals(maxHeapFactor, -1, 0.01)) { + if (Double.valueOf("-1") == maxHeapFactor) { maxHeapFactor = resource.getMemory() < TezConstants.TEZ_CONTAINER_SMALL_SLAB_BOUND_MB ? TezConstants.TEZ_CONTAINER_MAX_JAVA_HEAP_FRACTION_SMALL_SLAB : TezConstants.TEZ_CONTAINER_MAX_JAVA_HEAP_FRACTION_LARGE_SLAB; diff --git a/tez-dag/pom.xml b/tez-dag/pom.xml index 89033859c6..c0dccc8642 100644 --- a/tez-dag/pom.xml +++ b/tez-dag/pom.xml @@ -87,10 +87,6 @@ org.apache.hadoop hadoop-yarn-server-web-proxy - - org.apache.commons - commons-math3 - org.slf4j slf4j-api diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/DagAwareYarnTaskScheduler.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/DagAwareYarnTaskScheduler.java index d8b3397c70..29f350f9d0 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/DagAwareYarnTaskScheduler.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/DagAwareYarnTaskScheduler.java @@ -22,7 +22,6 @@ import org.apache.tez.common.Preconditions; import com.google.common.util.concurrent.ThreadFactoryBuilder; import org.apache.commons.lang.mutable.MutableInt; -import org.apache.commons.math3.random.RandomDataGenerator; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.util.Time; @@ -80,6 +79,7 @@ import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ScheduledThreadPoolExecutor; +import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; /** @@ -95,7 +95,6 @@ public class DagAwareYarnTaskScheduler extends TaskScheduler private static final Logger LOG = LoggerFactory.getLogger(DagAwareYarnTaskScheduler.class); private static final Comparator PREEMPT_ORDER_COMPARATOR = new PreemptOrderComparator(); - private final RandomDataGenerator random = new RandomDataGenerator(); private AMRMClientAsyncWrapper client; private ScheduledExecutorService reuseExecutor; private ResourceCalculator resourceCalculator; @@ -1544,7 +1543,7 @@ long getIdleExpirationTimestamp(long now) { if (idleExpirationTimestamp == 0) { if (idleContainerTimeoutMin > 0) { idleExpirationTimestamp = now + (idleContainerTimeoutMin == idleContainerTimeoutMax ? idleContainerTimeoutMin - : random.nextLong(idleContainerTimeoutMin, idleContainerTimeoutMax)); + : ThreadLocalRandom.current().nextLong(idleContainerTimeoutMin, idleContainerTimeoutMax)); } else { idleExpirationTimestamp = Long.MAX_VALUE; } 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 b4aa561fbe..6f51566f75 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 @@ -32,6 +32,7 @@ import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.PriorityBlockingQueue; +import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; @@ -44,7 +45,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.commons.lang.exception.ExceptionUtils; -import org.apache.commons.math3.random.RandomDataGenerator; import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.util.ExitUtil; @@ -153,7 +153,6 @@ public class YarnTaskSchedulerService extends TaskScheduler Set sessionMinHeldContainers = Sets.newHashSet(); - RandomDataGenerator random = new RandomDataGenerator(); private final Configuration conf; @VisibleForTesting @@ -593,7 +592,7 @@ long getHeldContainerExpireTime(long startTime) { long expireTime = (startTime + idleContainerTimeoutMin); if (idleContainerTimeoutMin != -1 && idleContainerTimeoutMin < idleContainerTimeoutMax) { long expireTimeMax = startTime + idleContainerTimeoutMax; - expireTime = random.nextLong(expireTime, expireTimeMax); + expireTime = ThreadLocalRandom.current().nextLong(expireTime, expireTimeMax); } return expireTime; From 2fefe4a6b16bea8f02169ec2e2a6e89575440328 Mon Sep 17 00:00:00 2001 From: Bodor Laszlo Date: Thu, 18 Feb 2021 17:35:19 +0100 Subject: [PATCH 043/137] TEZ-4291: Fancy precommit report on PR (#112) --- Jenkinsfile | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/Jenkinsfile b/Jenkinsfile index 1d6264658b..835a0523a4 100644 --- a/Jenkinsfile +++ b/Jenkinsfile @@ -52,7 +52,7 @@ pipeline { checkout([ $class: 'GitSCM', branches: [[name: "${env.YETUS_VERSION}"]], - userRemoteConfigs: [[ url: 'https://github.com/jteagles/yetus']]] + userRemoteConfigs: [[ url: 'https://github.com/apache/yetus']]] ) } } @@ -61,8 +61,8 @@ pipeline { stage ('precommit-run') { steps { withCredentials( - [usernamePassword(credentialsId: 'apache-tez-at-github.com', - passwordVariable: 'GITHUB_PASSWD', + [usernamePassword(credentialsId: 'apache-hadoop-at-github.com', + passwordVariable: 'GITHUB_TOKEN', usernameVariable: 'GITHUB_USER'), usernamePassword(credentialsId: 'tez-ci', passwordVariable: 'JIRA_PASSWORD', @@ -106,7 +106,7 @@ pipeline { # enable writing back to Github YETUS_ARGS+=(--github-user="${GITHUB_USER}") - YETUS_ARGS+=(--github-password="${GITHUB_PASSWD}") + YETUS_ARGS+=(--github-token="${GITHUB_TOKEN}") # enable writing back to ASF JIRA YETUS_ARGS+=(--jira-password="${JIRA_PASSWORD}") @@ -153,6 +153,9 @@ pipeline { # help keep the ASF boxes clean YETUS_ARGS+=("--sentinel") + # use emoji vote so it is easier to find the broken line + YETUS_ARGS+=("--github-use-emoji-vote") + # test with Java 8 and 11 YETUS_ARGS+=("--java-home=/usr/lib/jvm/java-8-openjdk-amd64") YETUS_ARGS+=("--multijdkdirs=/usr/lib/jvm/java-11-openjdk-amd64") From 2dcbe0bca22be89a797acd5f2228d91c4c112069 Mon Sep 17 00:00:00 2001 From: jacobtolar Date: Fri, 5 Mar 2021 09:44:09 -0600 Subject: [PATCH 044/137] fix typo: missing space (#59) I'm happy to file a JIRA if necessary; I wasn't sure if that was useful or not for a 1-character change. :) The log message looks like this currently: ``` 2020-02-10 20:12:58,267 [INFO] [Dispatcher thread {Central}] |node.AMNodeImpl|: Attempt failedon node: ... ``` --- .../main/java/org/apache/tez/dag/app/rm/node/AMNodeImpl.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 f4ad032514..df19534bfc 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 @@ -304,7 +304,7 @@ protected static class TaskAttemptFailedTransition implements @Override public AMNodeState transition(AMNodeImpl node, AMNodeEvent nEvent) { AMNodeEventTaskAttemptEnded event = (AMNodeEventTaskAttemptEnded) nEvent; - LOG.info("Attempt " + (event.failed() ? "failed" : "killed") + "on node: " + node.getNodeId() + LOG.info("Attempt " + (event.failed() ? "failed" : "killed") + " on node: " + node.getNodeId() + " TA: " + event.getTaskAttemptId() + ", container: " + event.getContainerId() + ", numFailedTAs: " + node.numFailedTAs); From 2b671d6211ad8bdebaa16889bf4f1f4121c4f1f9 Mon Sep 17 00:00:00 2001 From: Bodor Laszlo Date: Mon, 5 Apr 2021 11:55:20 +0200 Subject: [PATCH 045/137] TEZ-4299: Default java opts cause jdk11 to fail (#116) (Laszlo Bodor reviewed by Jonathan Turner Eagles) --- .../org/apache/tez/common/TezCommonUtils.java | 6 ++++++ .../apache/tez/dag/api/TezConfiguration.java | 21 +++++++++++++++++-- 2 files changed, 25 insertions(+), 2 deletions(-) diff --git a/tez-api/src/main/java/org/apache/tez/common/TezCommonUtils.java b/tez-api/src/main/java/org/apache/tez/common/TezCommonUtils.java index 27259580d4..3163968908 100644 --- a/tez-api/src/main/java/org/apache/tez/common/TezCommonUtils.java +++ b/tez-api/src/main/java/org/apache/tez/common/TezCommonUtils.java @@ -566,4 +566,10 @@ public static long getDAGSessionTimeout(Configuration conf) { return 1000l * timeoutSecs; } + public static int getJavaVersion() { + String javaVersionString = System.getProperty("java.version"); + return javaVersionString.split("\\.")[0].equals("1") + ? Integer.parseInt(javaVersionString.split("\\.")[1]) // "1.8" -> 8 + : Integer.parseInt(javaVersionString.split("\\.")[0]); // "9.x" -> 9, "11.x" -> 11 + } } 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 eef0d65424..17a826e7e7 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 @@ -33,6 +33,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.util.Shell; import org.apache.hadoop.yarn.api.records.LocalResource; +import org.apache.tez.common.TezCommonUtils; import org.apache.tez.common.annotation.ConfigurationClass; import org.apache.tez.common.annotation.ConfigurationProperty; import org.apache.tez.dag.api.EdgeProperty.ConcurrentEdgeTriggerType; @@ -343,8 +344,11 @@ public TezConfiguration(boolean loadDefaults) { @ConfigurationScope(Scope.AM) @ConfigurationProperty public static final String TEZ_AM_LAUNCH_CMD_OPTS = TEZ_AM_PREFIX + "launch.cmd-opts"; - public static final String TEZ_AM_LAUNCH_CMD_OPTS_DEFAULT = + public static final String TEZ_AM_LAUNCH_CMD_OPTS_JDK8_DEFAULT = "-XX:+PrintGCDetails -verbose:gc -XX:+PrintGCTimeStamps -XX:+UseNUMA -XX:+UseParallelGC"; + public static final String TEZ_AM_LAUNCH_CMD_OPTS_JDK9_DEFAULT = + "-verbose:gc -Xlog:gc*,safepoint::time,uptime -XX:+UseNUMA -XX:+UseParallelGC"; + public static final String TEZ_AM_LAUNCH_CMD_OPTS_DEFAULT; /** * String value. Command line options which will be prepended to {@link @@ -368,8 +372,21 @@ public TezConfiguration(boolean loadDefaults) { @ConfigurationProperty public static final String TEZ_TASK_LAUNCH_CMD_OPTS = TEZ_TASK_PREFIX + "launch.cmd-opts"; - public static final String TEZ_TASK_LAUNCH_CMD_OPTS_DEFAULT = + public static final String TEZ_TASK_LAUNCH_CMD_OPTS_JDK8_DEFAULT = "-XX:+PrintGCDetails -verbose:gc -XX:+PrintGCTimeStamps -XX:+UseNUMA -XX:+UseParallelGC"; + public static final String TEZ_TASK_LAUNCH_CMD_OPTS_JDK9_DEFAULT = + "-verbose:gc -Xlog:gc*,safepoint::time,uptime -XX:+UseNUMA -XX:+UseParallelGC"; + public static final String TEZ_TASK_LAUNCH_CMD_OPTS_DEFAULT; + + static { + if (TezCommonUtils.getJavaVersion() >= 9) { + TEZ_AM_LAUNCH_CMD_OPTS_DEFAULT = TEZ_AM_LAUNCH_CMD_OPTS_JDK9_DEFAULT; + TEZ_TASK_LAUNCH_CMD_OPTS_DEFAULT = TEZ_TASK_LAUNCH_CMD_OPTS_JDK9_DEFAULT; + } else { + TEZ_AM_LAUNCH_CMD_OPTS_DEFAULT = TEZ_AM_LAUNCH_CMD_OPTS_JDK8_DEFAULT; + TEZ_TASK_LAUNCH_CMD_OPTS_DEFAULT = TEZ_TASK_LAUNCH_CMD_OPTS_JDK8_DEFAULT; + } + } /** * Double value. Tez automatically determines the Xmx for the JVMs used to run From b934644871027722b40d83822bf6a0978141d6bf Mon Sep 17 00:00:00 2001 From: jshmchenxi Date: Sat, 10 Apr 2021 16:11:48 +0800 Subject: [PATCH 046/137] TEZ-4302: NullPointerException in CodecUtils with GzipCodec (#117) (Xi Chen reviewed by Laszlo Bodor) --- .../apache/tez/runtime/library/common/TezRuntimeUtils.java | 5 ++++- .../org/apache/tez/runtime/library/utils/CodecUtils.java | 3 ++- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/TezRuntimeUtils.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/TezRuntimeUtils.java index 9ff3d1c1e5..a1df131121 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/TezRuntimeUtils.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/TezRuntimeUtils.java @@ -38,6 +38,7 @@ import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.CommonConfigurationKeys; +import org.apache.hadoop.fs.CommonConfigurationKeysPublic; import org.apache.tez.dag.api.TezUncheckedException; import org.apache.tez.runtime.api.OutputContext; import org.apache.tez.runtime.api.TaskContext; @@ -269,7 +270,9 @@ public static String getBufferSizeProperty(CompressionCodec codec) { public static String getBufferSizeProperty(String className) { switch (className) { case "org.apache.hadoop.io.compress.DefaultCodec": - return "io.file.buffer.size"; + case "org.apache.hadoop.io.compress.BZip2Codec": + case "org.apache.hadoop.io.compress.GzipCodec": + return CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY; case "org.apache.hadoop.io.compress.SnappyCodec": return CommonConfigurationKeys.IO_COMPRESSION_CODEC_SNAPPY_BUFFERSIZE_KEY; case "org.apache.hadoop.io.compress.ZStandardCodec": diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/utils/CodecUtils.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/utils/CodecUtils.java index 99d22c5dcc..8e5154f3b0 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/utils/CodecUtils.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/utils/CodecUtils.java @@ -78,7 +78,8 @@ public static InputStream getDecompressedInputStreamWithBufferSize(CompressionCo throws IOException { String bufferSizeProp = TezRuntimeUtils.getBufferSizeProperty(codec); Configurable configurableCodec = (Configurable) codec; - int originalSize = configurableCodec.getConf().getInt(bufferSizeProp, DEFAULT_BUFFER_SIZE); + int originalSize = bufferSizeProp == null ? DEFAULT_BUFFER_SIZE : + configurableCodec.getConf().getInt(bufferSizeProp, DEFAULT_BUFFER_SIZE); CompressionInputStream in = null; From aea070714cd464a7403bf13164eb841f3f791f20 Mon Sep 17 00:00:00 2001 From: Bodor Laszlo Date: Thu, 15 Apr 2021 07:58:33 +0200 Subject: [PATCH 047/137] TEZ-4279: Add vertexId into vertex status for dag clients (#101) (Laszlo Bodor reviewed by Ashutosh Chauhan) --- .../apache/tez/dag/api/DagTypeConverters.java | 19 ++++++++----------- .../tez/dag/api/client/VertexStatus.java | 4 ++++ tez-api/src/main/proto/DAGApiRecords.proto | 9 +++++---- .../tez/dag/api/TestDagTypeConverters.java | 19 +++++++++++++++++++ .../tez/dag/api/client/rpc/TestDAGClient.java | 1 + .../dag/api/client/VertexStatusBuilder.java | 5 +++++ .../tez/dag/app/dag/impl/VertexImpl.java | 1 + 7 files changed, 43 insertions(+), 15 deletions(-) diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/DagTypeConverters.java b/tez-api/src/main/java/org/apache/tez/dag/api/DagTypeConverters.java index acc5f12933..5a2cb64cc3 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/DagTypeConverters.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/DagTypeConverters.java @@ -624,23 +624,20 @@ public static TezCountersProto convertTezCountersToProto( } public static DAGProtos.StatusGetOptsProto convertStatusGetOptsToProto( - StatusGetOpts statusGetOpts) { + StatusGetOpts statusGetOpts) { switch (statusGetOpts) { - case GET_COUNTERS: - return DAGProtos.StatusGetOptsProto.GET_COUNTERS; + case GET_COUNTERS: + return DAGProtos.StatusGetOptsProto.GET_COUNTERS; } - throw new TezUncheckedException("Could not convert StatusGetOpts to" - + " proto"); + throw new TezUncheckedException("Could not convert StatusGetOpts to" + " proto"); } - public static StatusGetOpts convertStatusGetOptsFromProto( - DAGProtos.StatusGetOptsProto proto) { + public static StatusGetOpts convertStatusGetOptsFromProto(DAGProtos.StatusGetOptsProto proto) { switch (proto) { - case GET_COUNTERS: - return StatusGetOpts.GET_COUNTERS; + case GET_COUNTERS: + return StatusGetOpts.GET_COUNTERS; } - throw new TezUncheckedException("Could not convert to StatusGetOpts from" - + " proto"); + throw new TezUncheckedException("Could not convert to StatusGetOpts from" + " proto"); } public static List convertStatusGetOptsToProto( diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/client/VertexStatus.java b/tez-api/src/main/java/org/apache/tez/dag/api/client/VertexStatus.java index dfb9bbe8cd..9efb12d612 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/client/VertexStatus.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/client/VertexStatus.java @@ -59,6 +59,10 @@ public VertexStatus(VertexStatusProtoOrBuilder proxy) { this.proxy = proxy; } + public String getId() { + return proxy.getId(); + } + public State getState() { return getState(proxy.getState()); } diff --git a/tez-api/src/main/proto/DAGApiRecords.proto b/tez-api/src/main/proto/DAGApiRecords.proto index 34c369d430..4c8c7f63a9 100644 --- a/tez-api/src/main/proto/DAGApiRecords.proto +++ b/tez-api/src/main/proto/DAGApiRecords.proto @@ -245,10 +245,11 @@ enum VertexStatusStateProto { } message VertexStatusProto { - optional VertexStatusStateProto state = 1; - repeated string diagnostics = 2; - optional ProgressProto progress = 3; - optional TezCountersProto vertexCounters = 4; + required string id = 1; + optional VertexStatusStateProto state = 2; + repeated string diagnostics = 3; + optional ProgressProto progress = 4; + optional TezCountersProto vertexCounters = 5; } enum DAGStatusStateProto { diff --git a/tez-api/src/test/java/org/apache/tez/dag/api/TestDagTypeConverters.java b/tez-api/src/test/java/org/apache/tez/dag/api/TestDagTypeConverters.java index 265fce9d43..edb7fd8445 100644 --- a/tez-api/src/test/java/org/apache/tez/dag/api/TestDagTypeConverters.java +++ b/tez-api/src/test/java/org/apache/tez/dag/api/TestDagTypeConverters.java @@ -34,8 +34,10 @@ import org.apache.tez.common.TezCommonUtils; import org.apache.tez.common.security.DAGAccessControls; import org.apache.tez.dag.api.Vertex.VertexExecutionContext; +import org.apache.tez.dag.api.client.StatusGetOpts; import org.apache.tez.dag.api.records.DAGProtos.ACLInfo; import org.apache.tez.dag.api.records.DAGProtos.AMPluginDescriptorProto; +import org.apache.tez.dag.api.records.DAGProtos.StatusGetOptsProto; import org.apache.tez.dag.api.records.DAGProtos.TezEntityDescriptorProto; import org.apache.tez.dag.api.records.DAGProtos.TezNamedEntityDescriptorProto; import org.apache.tez.dag.api.records.DAGProtos.VertexExecutionContextProto; @@ -235,6 +237,23 @@ public void testAclConversions() { assertSame(DagTypeConverters.convertDAGAccessControlsFromProto(aclInfo), aclInfo); } + /* + * This unit test can catch if a StatusGetOpts <-> StatusGetOptsProto value is not defined at any + * side. + */ + @Test + public void testConvertStatusGetOptsToProtoCoverage() { + StatusGetOpts[] opts = StatusGetOpts.values(); + for (StatusGetOpts opt : opts) { + DagTypeConverters.convertStatusGetOptsToProto(opt); + } + + StatusGetOptsProto[] optProtos = StatusGetOptsProto.values(); + for (StatusGetOptsProto proto : optProtos) { + DagTypeConverters.convertStatusGetOptsFromProto(proto); + } + } + private void assertSame(DAGAccessControls dagAccessControls, ACLInfo aclInfo) { assertEquals(dagAccessControls.getUsersWithViewACLs(), Sets.newHashSet(aclInfo.getUsersWithViewAccessList())); diff --git a/tez-api/src/test/java/org/apache/tez/dag/api/client/rpc/TestDAGClient.java b/tez-api/src/test/java/org/apache/tez/dag/api/client/rpc/TestDAGClient.java index 6a5e817d44..50c9a6061c 100644 --- a/tez-api/src/test/java/org/apache/tez/dag/api/client/rpc/TestDAGClient.java +++ b/tez-api/src/test/java/org/apache/tez/dag/api/client/rpc/TestDAGClient.java @@ -152,6 +152,7 @@ private void setUpData(){ .build(); vertexStatusProtoWithoutCounters = VertexStatusProto.newBuilder() + .setId("vertex_1") .addDiagnostics("V_Diagnostics_0") .setProgress(vertexProgressProto) .setState(VertexStatusStateProto.VERTEX_SUCCEEDED) // make sure the waitForCompletion be able to finish diff --git a/tez-dag/src/main/java/org/apache/tez/dag/api/client/VertexStatusBuilder.java b/tez-dag/src/main/java/org/apache/tez/dag/api/client/VertexStatusBuilder.java index 4de321cf2c..0304fc9d6f 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/api/client/VertexStatusBuilder.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/api/client/VertexStatusBuilder.java @@ -28,6 +28,7 @@ import org.apache.tez.dag.api.records.DAGProtos.VertexStatusStateProto; import org.apache.tez.dag.api.TezUncheckedException; import org.apache.tez.dag.app.dag.VertexState; +import org.apache.tez.dag.records.TezVertexID; public class VertexStatusBuilder extends VertexStatus { @@ -35,6 +36,10 @@ public VertexStatusBuilder() { super(VertexStatusProto.newBuilder()); } + public void setId(TezVertexID vertexId) { + getBuilder().setId(vertexId.toString()); + } + public void setState(VertexState state) { getBuilder().setState(getProtoState(state)); } 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 e21add0e3b..452dae5366 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 @@ -1536,6 +1536,7 @@ public VertexStatusBuilder getVertexStatus( this.readLock.lock(); try { VertexStatusBuilder status = new VertexStatusBuilder(); + status.setId(getVertexId()); status.setState(getInternalState()); status.setDiagnostics(diagnostics); status.setProgress(getVertexProgress()); From 01847814d32aac6b77a6810ea25d6780fb94b257 Mon Sep 17 00:00:00 2001 From: Bodor Laszlo Date: Fri, 23 Apr 2021 11:43:31 +0200 Subject: [PATCH 048/137] TEZ-4157: ShuffleHandler: upgrade to Netty4 and remove Netty3 dependency from tez (#118) (Laszlo Bodor reviewed by Ashutosh Chauhan, Jonathan Turner Eagles) --- pom.xml | 36 +- tez-ext-service-tests/pom.xml | 2 +- .../tez/shufflehandler/ShuffleHandler.java | 269 ++++++------- tez-plugins/tez-aux-services/pom.xml | 4 +- .../tez/auxservices/FadvisedChunkedFile.java | 16 +- .../tez/auxservices/FadvisedFileRegion.java | 55 ++- .../tez/auxservices/ShuffleHandler.java | 380 +++++++++--------- .../tez/auxservices/TestShuffleHandler.java | 177 ++++---- 8 files changed, 479 insertions(+), 460 deletions(-) diff --git a/pom.xml b/pom.xml index 19fd5167cd..c1f0682df1 100644 --- a/pom.xml +++ b/pom.xml @@ -59,7 +59,7 @@ ${user.home}/clover.license 27.0-jre 3.1.3 - 3.10.5.Final + 4.0.52.Final 0.13.0 1.19 1.7.30 @@ -261,7 +261,7 @@ io.netty - netty + netty-all compile ${netty.version} @@ -340,12 +340,22 @@ commons-el commons-el + + io.netty + netty + org.apache.hadoop hadoop-auth ${hadoop.version} + + + io.netty + netty + + org.apache.hadoop @@ -562,6 +572,10 @@ org.apache.hadoop hadoop-yarn-server-common + + io.netty + netty + @@ -578,6 +592,10 @@ org.apache.hadoop hadoop-mapreduce-client-shuffle + + io.netty + netty + @@ -603,6 +621,10 @@ org.apache.hadoop hadoop-mapreduce-client-common + + io.netty + netty + @@ -611,6 +633,12 @@ test test-jar ${hadoop.version} + + + io.netty + netty + + org.apache.hadoop @@ -649,6 +677,10 @@ tomcat jasper-runtime + + io.netty + netty + diff --git a/tez-ext-service-tests/pom.xml b/tez-ext-service-tests/pom.xml index c806bf7124..8dfad0d74e 100644 --- a/tez-ext-service-tests/pom.xml +++ b/tez-ext-service-tests/pom.xml @@ -32,7 +32,7 @@ io.netty - netty + netty-all org.slf4j 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 1d122be025..43f24ba654 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 @@ -14,17 +14,17 @@ 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 static io.netty.buffer.Unpooled.wrappedBuffer; +import static io.netty.handler.codec.http.HttpHeaders.Names.CONTENT_TYPE; +import static io.netty.handler.codec.http.HttpMethod.GET; +import static io.netty.handler.codec.http.HttpResponseStatus.BAD_REQUEST; +import static io.netty.handler.codec.http.HttpResponseStatus.FORBIDDEN; +import static io.netty.handler.codec.http.HttpResponseStatus.INTERNAL_SERVER_ERROR; +import static io.netty.handler.codec.http.HttpResponseStatus.METHOD_NOT_ALLOWED; +import static io.netty.handler.codec.http.HttpResponseStatus.NOT_FOUND; +import static io.netty.handler.codec.http.HttpResponseStatus.OK; +import static io.netty.handler.codec.http.HttpResponseStatus.UNAUTHORIZED; +import static io.netty.handler.codec.http.HttpVersion.HTTP_1_1; import javax.crypto.SecretKey; import java.io.File; @@ -41,15 +41,15 @@ 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.concurrent.atomic.AtomicInteger; import java.util.regex.Pattern; import com.google.common.base.Charsets; + import org.apache.tez.common.Preconditions; -import com.google.common.util.concurrent.ThreadFactoryBuilder; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.LocalDirAllocator; import org.apache.hadoop.fs.Path; @@ -64,36 +64,38 @@ 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.DefaultFileRegion; -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.stream.ChunkedWriteHandler; -import org.jboss.netty.util.CharsetUtil; +import io.netty.bootstrap.ServerBootstrap; +import io.netty.buffer.Unpooled; +import io.netty.channel.Channel; +import io.netty.channel.ChannelFuture; +import io.netty.channel.ChannelFutureListener; +import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.ChannelInboundHandlerAdapter; +import io.netty.channel.ChannelInitializer; +import io.netty.channel.ChannelPipeline; +import io.netty.channel.DefaultFileRegion; +import io.netty.channel.ChannelHandler.Sharable; +import io.netty.channel.group.ChannelGroup; +import io.netty.channel.group.DefaultChannelGroup; +import io.netty.channel.nio.NioEventLoopGroup; +import io.netty.channel.socket.nio.NioServerSocketChannel; +import io.netty.channel.socket.nio.NioSocketChannel; +import io.netty.handler.codec.TooLongFrameException; +import io.netty.handler.codec.http.DefaultFullHttpResponse; +import io.netty.handler.codec.http.DefaultHttpResponse; +import io.netty.handler.codec.http.FullHttpResponse; +import io.netty.handler.codec.http.HttpHeaders; +import io.netty.handler.codec.http.HttpObjectAggregator; +import io.netty.handler.codec.http.HttpRequest; +import io.netty.handler.codec.http.HttpRequestDecoder; +import io.netty.handler.codec.http.HttpResponse; +import io.netty.handler.codec.http.HttpResponseEncoder; +import io.netty.handler.codec.http.HttpResponseStatus; +import io.netty.handler.codec.http.QueryStringDecoder; +import io.netty.handler.stream.ChunkedWriteHandler; +import io.netty.util.CharsetUtil; +import io.netty.util.concurrent.GlobalEventExecutor; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -110,9 +112,13 @@ public class ShuffleHandler { Pattern.CASE_INSENSITIVE); private int port; - private final ChannelFactory selector; - private final ChannelGroup accepted = new DefaultChannelGroup(); - protected HttpPipelineFactory pipelineFact; + + // pipeline items + private Shuffle SHUFFLE; + + private NioEventLoopGroup bossGroup; + private NioEventLoopGroup workerGroup; + private final ChannelGroup accepted = new DefaultChannelGroup(GlobalEventExecutor.INSTANCE); private final Configuration conf; private final ConcurrentMap registeredApps = new ConcurrentHashMap(); @@ -171,17 +177,23 @@ public ShuffleHandler(Configuration conf) { maxShuffleThreads = 2 * Runtime.getRuntime().availableProcessors(); } - 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); + final String BOSS_THREAD_NAME_PREFIX = "ShuffleHandler Netty Boss #"; + AtomicInteger bossThreadCounter = new AtomicInteger(0); + bossGroup = new NioEventLoopGroup(maxShuffleThreads, new ThreadFactory() { + @Override + public Thread newThread(Runnable r) { + return new Thread(r, BOSS_THREAD_NAME_PREFIX + bossThreadCounter.incrementAndGet()); + } + }); + + final String WORKER_THREAD_NAME_PREFIX = "ShuffleHandler Netty Worker #"; + AtomicInteger workerThreadCounter = new AtomicInteger(0); + workerGroup = new NioEventLoopGroup(maxShuffleThreads, new ThreadFactory() { + @Override + public Thread newThread(Runnable r) { + return new Thread(r, WORKER_THREAD_NAME_PREFIX + workerThreadCounter.incrementAndGet()); + } + }); connectionKeepAliveEnabled = conf.getBoolean(SHUFFLE_CONNECTION_KEEP_ALIVE_ENABLED, @@ -199,22 +211,44 @@ public ShuffleHandler(Configuration conf) { 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); + ServerBootstrap bootstrap = new ServerBootstrap() + .channel(NioServerSocketChannel.class) + .group(bossGroup, workerGroup) + .localAddress(port); + initPipeline(bootstrap, conf); port = conf.getInt(SHUFFLE_PORT_CONFIG_KEY, DEFAULT_SHUFFLE_PORT); - Channel ch = bootstrap.bind(new InetSocketAddress(port)); + Channel ch = bootstrap.bind().sync().channel(); accepted.add(ch); - port = ((InetSocketAddress)ch.getLocalAddress()).getPort(); + port = ((InetSocketAddress)ch.localAddress()).getPort(); conf.set(SHUFFLE_PORT_CONFIG_KEY, Integer.toString(port)); - pipelineFact.SHUFFLE.setPort(port); + SHUFFLE.setPort(port); LOG.info("TezShuffleHandler" + " listening on port " + port); } + private void initPipeline(ServerBootstrap bootstrap, Configuration conf) throws Exception { + SHUFFLE = getShuffle(conf); + + if (conf.getBoolean(MRConfig.SHUFFLE_SSL_ENABLED_KEY, + MRConfig.SHUFFLE_SSL_ENABLED_DEFAULT)) { + throw new UnsupportedOperationException( + "SSL Shuffle is not currently supported for the test shuffle handler"); + } + + ChannelInitializer channelInitializer = + new ChannelInitializer() { + @Override + public void initChannel(NioSocketChannel ch) throws Exception { + ChannelPipeline pipeline = ch.pipeline(); + pipeline.addLast("decoder", new HttpRequestDecoder()); + pipeline.addLast("aggregator", new HttpObjectAggregator(1 << 16)); + pipeline.addLast("encoder", new HttpResponseEncoder()); + pipeline.addLast("chunking", new ChunkedWriteHandler()); + pipeline.addLast("shuffle", SHUFFLE); + } + }; + bootstrap.childHandler(channelInitializer); + } + public static void initializeAndStart(Configuration conf) throws Exception { if (!initing.getAndSet(true)) { INSTANCE = new ShuffleHandler(conf); @@ -245,15 +279,13 @@ 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 (bossGroup != null) { + bossGroup.shutdownGracefully(); } - if (pipelineFact != null) { - pipelineFact.destroy(); + if (workerGroup != null) { + workerGroup.shutdownGracefully(); } } @@ -261,7 +293,6 @@ protected Shuffle getShuffle(Configuration conf) { return new Shuffle(conf); } - private void addJobToken(String appIdString, String user, Token jobToken) { String jobIdString = appIdString.replace("application", "job"); @@ -280,40 +311,8 @@ private void removeJobShuffleInfo(String appIdString) { userRsrc.remove(appIdString); } - class HttpPipelineFactory implements ChannelPipelineFactory { - - final Shuffle SHUFFLE; - - 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)) { - throw new UnsupportedOperationException( - "SSL Shuffle is not currently supported for the test shuffle handler"); - } - } - - public void destroy() { - } - - @Override - public ChannelPipeline getPipeline() throws Exception { - ChannelPipeline pipeline = Channels.pipeline(); - 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 { + @Sharable + class Shuffle extends ChannelInboundHandlerAdapter { private final Configuration conf; private final IndexCache indexCache; @@ -343,24 +342,24 @@ private List splitMaps(List mapq) { } @Override - public void channelOpen(ChannelHandlerContext ctx, ChannelStateEvent evt) + public void channelActive(ChannelHandlerContext ctx) 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)", + 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(); + ctx.channel().close(); return; } - accepted.add(evt.getChannel()); - super.channelOpen(ctx, evt); - + accepted.add(ctx.channel()); + super.channelActive(ctx); } @Override - public void messageReceived(ChannelHandlerContext ctx, MessageEvent evt) + public void channelRead(ChannelHandlerContext ctx, Object message) throws Exception { - HttpRequest request = (HttpRequest) evt.getMessage(); + HttpRequest request = (HttpRequest) message; if (request.getMethod() != GET) { sendError(ctx, METHOD_NOT_ALLOWED); return; @@ -372,8 +371,7 @@ public void messageReceived(ChannelHandlerContext ctx, MessageEvent evt) request.headers().get(ShuffleHeader.HTTP_HEADER_VERSION))) { sendError(ctx, "Incompatible shuffle request version", BAD_REQUEST); } - final Map> q = - new QueryStringDecoder(request.getUri()).getParameters(); + final Map> q = new QueryStringDecoder(request.getUri()).parameters(); final List keepAliveList = q.get("keepAlive"); boolean keepAliveParam = false; if (keepAliveList != null && keepAliveList.size() == 1) { @@ -432,7 +430,7 @@ public void messageReceived(ChannelHandlerContext ctx, MessageEvent evt) Map mapOutputInfoMap = new HashMap(); - Channel ch = evt.getChannel(); + Channel ch = ctx.channel(); String user = userRsrc.get(jobId); // $x/$user/appcache/$appId/output/$mapId @@ -444,13 +442,13 @@ public void messageReceived(ChannelHandlerContext ctx, MessageEvent evt) populateHeaders(mapIds, outputBasePathStr, user, reduceId, request, response, keepAliveParam, mapOutputInfoMap); } catch(IOException e) { - ch.write(response); + ch.writeAndFlush(response); LOG.error("Shuffle error in populating headers :", e); String errorMessage = getErrorMessage(e); sendError(ctx,errorMessage , INTERNAL_SERVER_ERROR); return; } - ch.write(response); + ch.writeAndFlush(response); // TODO refactor the following into the pipeline ChannelFuture lastMap = null; for (String mapId : mapIds) { @@ -621,7 +619,7 @@ protected ChannelFuture sendMapOutput(ChannelHandlerContext ctx, Channel ch, new ShuffleHeader(mapId, info.getPartLength(), info.getRawLength(), reduce); final DataOutputBuffer dob = new DataOutputBuffer(); header.write(dob); - ch.write(wrappedBuffer(dob.getData(), 0, dob.getLength())); + ch.writeAndFlush(wrappedBuffer(dob.getData(), 0, dob.getLength())); final File spillfile = new File(mapOutputInfo.mapOutputFileName.toString()); RandomAccessFile spill; @@ -634,15 +632,7 @@ protected ChannelFuture sendMapOutput(ChannelHandlerContext ctx, Channel ch, ChannelFuture writeFuture; final DefaultFileRegion partition = new DefaultFileRegion(spill.getChannel(), info.getStartOffset(), info.getPartLength()); - 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) { - partition.releaseExternalResources(); - } - }); + writeFuture = ch.writeAndFlush(partition); return writeFuture; } @@ -653,25 +643,22 @@ protected void sendError(ChannelHandlerContext ctx, protected void sendError(ChannelHandlerContext ctx, String message, HttpResponseStatus status) { - HttpResponse response = new DefaultHttpResponse(HTTP_1_1, status); + FullHttpResponse response = new DefaultFullHttpResponse(HTTP_1_1, status); response.headers().set(CONTENT_TYPE, "text/plain; charset=UTF-8"); // Put shuffle version into http header response.headers().set(ShuffleHeader.HTTP_HEADER_NAME, ShuffleHeader.DEFAULT_HTTP_HEADER_NAME); response.headers().set(ShuffleHeader.HTTP_HEADER_VERSION, ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION); - response.setContent( - ChannelBuffers.copiedBuffer(message, CharsetUtil.UTF_8)); + response.content().writeBytes(Unpooled.copiedBuffer(message, CharsetUtil.UTF_8)); // Close the connection as soon as the error message is sent. - ctx.getChannel().write(response).addListener(ChannelFutureListener.CLOSE); + ctx.channel().writeAndFlush(response).addListener(ChannelFutureListener.CLOSE); } @Override - public void exceptionCaught(ChannelHandlerContext ctx, ExceptionEvent e) + public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception { - Channel ch = e.getChannel(); - Throwable cause = e.getCause(); if (cause instanceof TooLongFrameException) { sendError(ctx, BAD_REQUEST); return; @@ -688,8 +675,8 @@ public void exceptionCaught(ChannelHandlerContext ctx, ExceptionEvent e) } LOG.error("Shuffle error: ", cause); - if (ch.isConnected()) { - LOG.error("Shuffle error " + e); + if (ctx.channel().isActive()) { + LOG.error("Shuffle error", cause); sendError(ctx, INTERNAL_SERVER_ERROR); } } diff --git a/tez-plugins/tez-aux-services/pom.xml b/tez-plugins/tez-aux-services/pom.xml index 97096c8332..7279eaf684 100644 --- a/tez-plugins/tez-aux-services/pom.xml +++ b/tez-plugins/tez-aux-services/pom.xml @@ -112,7 +112,7 @@ io.netty - netty + netty-all com.google.guava @@ -237,7 +237,7 @@ org.apache.tez.shaded.$0 - org.jboss.netty + io.netty org.apache.tez.shaded.$0 diff --git a/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/FadvisedChunkedFile.java b/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/FadvisedChunkedFile.java index cc3f762f9d..162feb9801 100644 --- a/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/FadvisedChunkedFile.java +++ b/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/FadvisedChunkedFile.java @@ -31,7 +31,9 @@ import static org.apache.hadoop.io.nativeio.NativeIO.POSIX.POSIX_FADV_DONTNEED; -import org.jboss.netty.handler.stream.ChunkedFile; +import io.netty.buffer.ByteBuf; +import io.netty.channel.ChannelHandlerContext; +import io.netty.handler.stream.ChunkedFile; public class FadvisedChunkedFile extends ChunkedFile { @@ -57,13 +59,13 @@ public FadvisedChunkedFile(RandomAccessFile file, long position, long count, } @Override - public Object nextChunk() throws Exception { + public ByteBuf readChunk(ChannelHandlerContext ctx) throws Exception { if (manageOsCache && readaheadPool != null) { readaheadRequest = readaheadPool - .readaheadStream(identifier, fd, getCurrentOffset(), readaheadLength, - getEndOffset(), readaheadRequest); + .readaheadStream(identifier, fd, currentOffset(), readaheadLength, + endOffset(), readaheadRequest); } - return super.nextChunk(); + return super.readChunk(ctx); } @Override @@ -71,11 +73,11 @@ public void close() throws Exception { if (readaheadRequest != null) { readaheadRequest.cancel(); } - if (manageOsCache && getEndOffset() - getStartOffset() > 0) { + if (manageOsCache && endOffset() - startOffset() > 0) { try { NativeIO.POSIX.getCacheManipulator().posixFadviseIfPossible(identifier, fd, - getStartOffset(), getEndOffset() - getStartOffset(), + startOffset(), endOffset() - startOffset(), POSIX_FADV_DONTNEED); } catch (Throwable t) { LOG.warn("Failed to manage OS cache for " + identifier, t); diff --git a/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/FadvisedFileRegion.java b/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/FadvisedFileRegion.java index 40789d806d..23663636e5 100644 --- a/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/FadvisedFileRegion.java +++ b/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/FadvisedFileRegion.java @@ -34,7 +34,7 @@ import static org.apache.hadoop.io.nativeio.NativeIO.POSIX.POSIX_FADV_DONTNEED; -import org.jboss.netty.channel.DefaultFileRegion; +import io.netty.channel.DefaultFileRegion; import com.google.common.annotations.VisibleForTesting; @@ -54,6 +54,7 @@ public class FadvisedFileRegion extends DefaultFileRegion { private final FileChannel fileChannel; private ReadaheadRequest readaheadRequest; + private boolean transferred = false; public FadvisedFileRegion(RandomAccessFile file, long position, long count, boolean manageOsCache, int readaheadLength, ReadaheadPool readaheadPool, @@ -77,15 +78,40 @@ public long transferTo(WritableByteChannel target, long position) throws IOException { if (readaheadPool != null && readaheadLength > 0) { readaheadRequest = readaheadPool.readaheadStream(identifier, fd, - getPosition() + position, readaheadLength, - getPosition() + getCount(), readaheadRequest); + position() + position, readaheadLength, + position() + count(), readaheadRequest); } + long written = 0; if(this.shuffleTransferToAllowed) { - return super.transferTo(target, position); + written = super.transferTo(target, position); } else { - return customShuffleTransfer(target, position); + written = customShuffleTransfer(target, position); } + /* + * At this point, we can assume that the transfer was successful. + */ + transferred = true; + return written; + } + + /** + * Since Netty4, deallocate() is called automatically during cleanup, but before the + * ChannelFutureListeners. Deallocate calls FileChannel.close() and makes the file descriptor + * invalid, so every OS cache operation (e.g. posix_fadvice) with the original file descriptor + * will fail after this operation, so we need to take care of cleanup operations here (before + * deallocating) instead of listeners outside. + */ + @Override + protected void deallocate() { + if (readaheadRequest != null) { + readaheadRequest.cancel(); + } + + if (transferred) { + transferSuccessful(); + } + super.deallocate(); } /** @@ -142,24 +168,19 @@ long customShuffleTransfer(WritableByteChannel target, long position) 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) { + if (manageOsCache && count() > 0) { try { - NativeIO.POSIX.getCacheManipulator().posixFadviseIfPossible(identifier, - fd, getPosition(), getCount(), POSIX_FADV_DONTNEED); + if (fd.valid()) { + NativeIO.POSIX.getCacheManipulator().posixFadviseIfPossible(identifier, fd, position(), + count(), POSIX_FADV_DONTNEED); + } else { + LOG.debug("File descriptor is not valid anymore, skipping posix_fadvise: " + identifier); + } } catch (Throwable t) { LOG.warn("Failed to manage OS cache for " + identifier, t); } diff --git a/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java b/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java index 55389ea78e..e73805a1f0 100644 --- a/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java +++ b/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java @@ -21,17 +21,17 @@ import org.apache.hadoop.util.DiskChecker; import static org.fusesource.leveldbjni.JniDBFactory.asString; import static org.fusesource.leveldbjni.JniDBFactory.bytes; -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 static io.netty.buffer.Unpooled.wrappedBuffer; +import static io.netty.handler.codec.http.HttpHeaders.Names.CONTENT_TYPE; +import static io.netty.handler.codec.http.HttpMethod.GET; +import static io.netty.handler.codec.http.HttpResponseStatus.BAD_REQUEST; +import static io.netty.handler.codec.http.HttpResponseStatus.FORBIDDEN; +import static io.netty.handler.codec.http.HttpResponseStatus.INTERNAL_SERVER_ERROR; +import static io.netty.handler.codec.http.HttpResponseStatus.METHOD_NOT_ALLOWED; +import static io.netty.handler.codec.http.HttpResponseStatus.NOT_FOUND; +import static io.netty.handler.codec.http.HttpResponseStatus.OK; +import static io.netty.handler.codec.http.HttpResponseStatus.UNAUTHORIZED; +import static io.netty.handler.codec.http.HttpVersion.HTTP_1_1; import java.io.File; import java.io.FileNotFoundException; @@ -48,7 +48,7 @@ import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutionException; -import java.util.concurrent.Executors; +import java.util.concurrent.ThreadFactory; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import java.util.regex.Pattern; @@ -102,47 +102,47 @@ import org.iq80.leveldb.DBException; import org.iq80.leveldb.Logger; import org.iq80.leveldb.Options; -import org.jboss.netty.bootstrap.ServerBootstrap; -import org.jboss.netty.buffer.ChannelBuffer; -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.ChannelHandler; -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.NioServerBossPool; -import org.jboss.netty.channel.socket.nio.NioServerSocketChannelFactory; -import org.jboss.netty.channel.socket.nio.NioWorkerPool; -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.handler.timeout.IdleState; -import org.jboss.netty.handler.timeout.IdleStateAwareChannelHandler; -import org.jboss.netty.handler.timeout.IdleStateEvent; -import org.jboss.netty.handler.timeout.IdleStateHandler; -import org.jboss.netty.util.CharsetUtil; -import org.jboss.netty.util.HashedWheelTimer; -import org.jboss.netty.util.ThreadNameDeterminer; -import org.jboss.netty.util.Timer; + +import io.netty.bootstrap.ServerBootstrap; +import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; +import io.netty.channel.Channel; +import io.netty.channel.ChannelHandler.Sharable; +import io.netty.channel.ChannelDuplexHandler; +import io.netty.channel.ChannelFuture; +import io.netty.channel.ChannelFutureListener; +import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.ChannelInboundHandlerAdapter; +import io.netty.channel.ChannelInitializer; +import io.netty.channel.ChannelOption; +import io.netty.channel.ChannelPipeline; +import io.netty.channel.group.ChannelGroup; +import io.netty.channel.group.DefaultChannelGroup; +import io.netty.channel.nio.NioEventLoopGroup; +import io.netty.channel.socket.nio.NioServerSocketChannel; +import io.netty.channel.socket.nio.NioSocketChannel; +import io.netty.handler.codec.TooLongFrameException; +import io.netty.handler.codec.http.DefaultFullHttpResponse; +import io.netty.handler.codec.http.DefaultHttpResponse; +import io.netty.handler.codec.http.FullHttpRequest; +import io.netty.handler.codec.http.FullHttpResponse; +import io.netty.handler.codec.http.HttpHeaders; +import io.netty.handler.codec.http.HttpObjectAggregator; +import io.netty.handler.codec.http.HttpRequest; +import io.netty.handler.codec.http.HttpRequestDecoder; +import io.netty.handler.codec.http.HttpResponse; +import io.netty.handler.codec.http.HttpResponseEncoder; +import io.netty.handler.codec.http.HttpResponseStatus; +import io.netty.handler.codec.http.LastHttpContent; +import io.netty.handler.codec.http.QueryStringDecoder; +import io.netty.handler.ssl.SslHandler; +import io.netty.handler.stream.ChunkedWriteHandler; +import io.netty.handler.timeout.IdleState; +import io.netty.handler.timeout.IdleStateEvent; +import io.netty.handler.timeout.IdleStateHandler; +import io.netty.util.CharsetUtil; +import io.netty.util.concurrent.GlobalEventExecutor; + import org.slf4j.LoggerFactory; import com.google.common.annotations.VisibleForTesting; @@ -183,11 +183,15 @@ public class ShuffleHandler extends AuxiliaryService { private static final String INDEX_FILE_NAME = "file.out.index"; private int port; - private ChannelFactory selector; - private final ChannelGroup accepted = new DefaultChannelGroup(); - protected HttpPipelineFactory pipelineFact; + private NioEventLoopGroup bossGroup; + private NioEventLoopGroup workerGroup; + private final ChannelGroup accepted = new DefaultChannelGroup(GlobalEventExecutor.INSTANCE); private int sslFileBufferSize; + // pipeline items + private Shuffle SHUFFLE; + private SSLFactory sslFactory; + /** * Should the shuffle use posix_fadvise calls to manage the OS cache during * sendfile @@ -263,7 +267,6 @@ public class ShuffleHandler extends AuxiliaryService { boolean connectionKeepAliveEnabled = false; private int connectionKeepAliveTimeOut; private int mapOutputMetaInfoCacheSize; - private Timer timer; @Metrics(about="Shuffle output metrics", context="mapred", name="tez") static class ShuffleMetrics implements ChannelFutureListener { @@ -300,7 +303,7 @@ public ReduceMapFileCount(ReduceContext rc) { @Override public void operationComplete(ChannelFuture future) throws Exception { if (!future.isSuccess()) { - future.getChannel().close(); + future.channel().close(); return; } int waitCount = this.reduceContext.getMapsToWait().decrementAndGet(); @@ -308,21 +311,21 @@ public void operationComplete(ChannelFuture future) throws Exception { metrics.operationComplete(future); // Let the idle timer handler close keep-alive connections if (reduceContext.getKeepAlive()) { - ChannelPipeline pipeline = future.getChannel().getPipeline(); + ChannelPipeline pipeline = future.channel().pipeline(); TimeoutHandler timeoutHandler = (TimeoutHandler) pipeline.get(TIMEOUT_HANDLER); timeoutHandler.setEnabledTimeout(true); } else { - future.getChannel().close(); + future.channel().close(); } } else { - pipelineFact.getSHUFFLE().sendMap(reduceContext); + SHUFFLE.sendMap(reduceContext); } } } /** - * Maintain parameters per messageReceived() Netty context. + * Maintain parameters per channelRead() Netty context. * Allows sendMapOutput calls from operationComplete() */ private static class ReduceContext { @@ -419,9 +422,11 @@ public ShuffleHandler() { */ 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()); + DataOutputBuffer portDob = new DataOutputBuffer(); + portDob.writeInt(port); + ByteBuffer buf = ByteBuffer.wrap(portDob.getData(), 0, portDob.getLength()); + portDob.close(); + return buf; } /** @@ -434,6 +439,7 @@ public static int deserializeMetaData(ByteBuffer meta) throws IOException { DataInputByteBuffer in = new DataInputByteBuffer(); in.reset(meta); int port = in.readInt(); + in.close(); return port; } @@ -516,22 +522,23 @@ protected void serviceInit(Configuration conf) throws Exception { DEFAULT_SHUFFLE_MAX_SESSION_OPEN_FILES); final String BOSS_THREAD_NAME_PREFIX = "Tez Shuffle Handler Boss #"; - NioServerBossPool bossPool = new NioServerBossPool(Executors.newCachedThreadPool(), 1, new ThreadNameDeterminer() { + AtomicInteger bossThreadCounter = new AtomicInteger(0); + bossGroup = new NioEventLoopGroup(maxShuffleThreads, new ThreadFactory() { @Override - public String determineThreadName(String currentThreadName, String proposedThreadName) throws Exception { - return BOSS_THREAD_NAME_PREFIX + currentThreadName.substring(currentThreadName.lastIndexOf('-') + 1); + public Thread newThread(Runnable r) { + return new Thread(r, BOSS_THREAD_NAME_PREFIX + bossThreadCounter.incrementAndGet()); } }); final String WORKER_THREAD_NAME_PREFIX = "Tez Shuffle Handler Worker #"; - NioWorkerPool workerPool = new NioWorkerPool(Executors.newCachedThreadPool(), maxShuffleThreads, new ThreadNameDeterminer() { + AtomicInteger workerThreadCounter = new AtomicInteger(0); + workerGroup = new NioEventLoopGroup(maxShuffleThreads, new ThreadFactory() { @Override - public String determineThreadName(String currentThreadName, String proposedThreadName) throws Exception { - return WORKER_THREAD_NAME_PREFIX + currentThreadName.substring(currentThreadName.lastIndexOf('-') + 1); + public Thread newThread(Runnable r) { + return new Thread(r, WORKER_THREAD_NAME_PREFIX + workerThreadCounter.incrementAndGet()); } }); - selector = new NioServerSocketChannelFactory(bossPool, workerPool); super.serviceInit(new YarnConfiguration(conf)); } @@ -542,25 +549,24 @@ protected void serviceStart() throws Exception { userRsrc = new ConcurrentHashMap(); secretManager = new JobTokenSecretManager(); recoverState(conf); - ServerBootstrap bootstrap = new ServerBootstrap(selector); - // Timer is shared across entire factory and must be released separately - timer = new HashedWheelTimer(); - try { - pipelineFact = new HttpPipelineFactory(conf, timer); - } catch (Exception ex) { - throw new RuntimeException(ex); - } - bootstrap.setOption("backlog", conf.getInt(SHUFFLE_LISTEN_QUEUE_SIZE, - DEFAULT_SHUFFLE_LISTEN_QUEUE_SIZE)); - bootstrap.setOption("child.keepAlive", true); - bootstrap.setPipelineFactory(pipelineFact); + ServerBootstrap bootstrap = new ServerBootstrap() + .channel(NioServerSocketChannel.class) + .group(bossGroup, workerGroup) + .localAddress(port) + .option(ChannelOption.SO_BACKLOG, + conf.getInt(SHUFFLE_LISTEN_QUEUE_SIZE, DEFAULT_SHUFFLE_LISTEN_QUEUE_SIZE)) + .childOption(ChannelOption.SO_KEEPALIVE, true); + initPipeline(bootstrap, conf); port = conf.getInt(SHUFFLE_PORT_CONFIG_KEY, DEFAULT_SHUFFLE_PORT); - Channel ch = bootstrap.bind(new InetSocketAddress(port)); + Channel ch = bootstrap.bind().sync().channel(); accepted.add(ch); - port = ((InetSocketAddress)ch.getLocalAddress()).getPort(); + + // setup port + port = ((InetSocketAddress)ch.localAddress()).getPort(); conf.set(SHUFFLE_PORT_CONFIG_KEY, Integer.toString(port)); - pipelineFact.SHUFFLE.setPort(port); + SHUFFLE.setPort(port); LOG.info(getName() + " listening on port " + port); + super.serviceStart(); sslFileBufferSize = conf.getInt(SUFFLE_SSL_FILE_BUFFER_SIZE_KEY, @@ -576,20 +582,50 @@ protected void serviceStart() throws Exception { DEFAULT_SHUFFLE_MAPOUTPUT_META_INFO_CACHE_SIZE)); } + private void initPipeline(ServerBootstrap bootstrap, Configuration conf) throws Exception { + SHUFFLE = getShuffle(conf); + if (conf.getBoolean(SHUFFLE_SSL_ENABLED_KEY, SHUFFLE_SSL_ENABLED_DEFAULT)) { + LOG.info("Encrypted shuffle is enabled."); + sslFactory = new SSLFactory(SSLFactory.Mode.SERVER, conf); + sslFactory.init(); + } + + ChannelInitializer channelInitializer = + new ChannelInitializer() { + @Override + public void initChannel(NioSocketChannel ch) throws Exception { + ChannelPipeline pipeline = ch.pipeline(); + if (sslFactory != null) { + pipeline.addLast("ssl", new SslHandler(sslFactory.createSSLEngine())); + } + pipeline.addLast("decoder", new HttpRequestDecoder()); + pipeline.addLast("aggregator", new HttpObjectAggregator(1 << 16)); + pipeline.addLast("encoder", new HttpResponseEncoder()); + pipeline.addLast("chunking", new ChunkedWriteHandler()); + pipeline.addLast("shuffle", SHUFFLE); + pipeline.addLast("idle", new IdleStateHandler(0, connectionKeepAliveTimeOut, 0)); + pipeline.addLast(TIMEOUT_HANDLER, new TimeoutHandler()); + } + }; + bootstrap.childHandler(channelInitializer); + } + + private void destroyPipeline() { + if (sslFactory != null) { + sslFactory.destroy(); + } + } + @Override protected void serviceStop() throws Exception { accepted.close().awaitUninterruptibly(10, TimeUnit.SECONDS); - if (selector != null) { - ServerBootstrap bootstrap = new ServerBootstrap(selector); - bootstrap.releaseExternalResources(); - } - if (pipelineFact != null) { - pipelineFact.destroy(); + if (bossGroup != null) { + bossGroup.shutdownGracefully(); } - if (timer != null) { - // Release this shared timer resource - timer.stop(); + if (workerGroup != null) { + workerGroup.shutdownGracefully(); } + destroyPipeline(); if (stateDb != null) { stateDb.close(); } @@ -800,7 +836,7 @@ public void log(String message) { } } - static class TimeoutHandler extends IdleStateAwareChannelHandler { + static class TimeoutHandler extends ChannelDuplexHandler { private boolean enabledTimeout; @@ -809,59 +845,14 @@ void setEnabledTimeout(boolean enabledTimeout) { } @Override - public void channelIdle(ChannelHandlerContext ctx, IdleStateEvent e) { - if (e.getState() == IdleState.WRITER_IDLE && enabledTimeout) { - e.getChannel().close(); - } - } - } - - class HttpPipelineFactory implements ChannelPipelineFactory { - - final Shuffle SHUFFLE; - private SSLFactory sslFactory; - private final ChannelHandler idleStateHandler; - - public HttpPipelineFactory(Configuration conf, Timer timer) throws Exception { - SHUFFLE = getShuffle(conf); - if (conf.getBoolean(SHUFFLE_SSL_ENABLED_KEY, - SHUFFLE_SSL_ENABLED_DEFAULT)) { - LOG.info("Encrypted shuffle is enabled."); - sslFactory = new SSLFactory(SSLFactory.Mode.SERVER, conf); - sslFactory.init(); - } - this.idleStateHandler = new IdleStateHandler(timer, 0, connectionKeepAliveTimeOut, 0); - } - - public Shuffle getSHUFFLE() { - return SHUFFLE; - } - - 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())); + public void userEventTriggered(ChannelHandlerContext ctx, Object evt) throws Exception { + if (evt instanceof IdleStateEvent) { + IdleStateEvent e = (IdleStateEvent) evt; + if (e.state() == IdleState.WRITER_IDLE && enabledTimeout) { + ctx.channel().close(); + } } - 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); - pipeline.addLast("idle", idleStateHandler); - pipeline.addLast(TIMEOUT_HANDLER, new TimeoutHandler()); - 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 } - } protected static class Range { @@ -887,7 +878,8 @@ public String toString() { } } - class Shuffle extends SimpleChannelUpstreamHandler { + @Sharable + class Shuffle extends ChannelInboundHandlerAdapter { private static final int MAX_WEIGHT = 10 * 1024 * 1024; private static final int EXPIRE_AFTER_ACCESS_MINUTES = 5; @@ -973,24 +965,30 @@ private Range splitReduces(List reduceq) { } @Override - public void channelOpen(ChannelHandlerContext ctx, ChannelStateEvent evt) + public void channelActive(ChannelHandlerContext ctx) 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(); + ctx.channel().close(); return; } - accepted.add(evt.getChannel()); - super.channelOpen(ctx, evt); + accepted.add(ctx.channel()); + super.channelActive(ctx); } @Override - public void messageReceived(ChannelHandlerContext ctx, MessageEvent evt) + public void channelRead(ChannelHandlerContext ctx, Object message) throws Exception { - HttpRequest request = (HttpRequest) evt.getMessage(); + FullHttpRequest request = (FullHttpRequest) message; + handleRequest(ctx, request); + request.release(); + } + + private void handleRequest(ChannelHandlerContext ctx, FullHttpRequest request) + throws IOException, Exception { if (request.getMethod() != GET) { sendError(ctx, METHOD_NOT_ALLOWED); return; @@ -1001,9 +999,9 @@ public void messageReceived(ChannelHandlerContext ctx, MessageEvent evt) || !ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION.equals( request.headers().get(ShuffleHeader.HTTP_HEADER_VERSION))) { sendError(ctx, "Incompatible shuffle request version", BAD_REQUEST); + return; } - final Map> q = - new QueryStringDecoder(request.getUri()).getParameters(); + final Map> q = new QueryStringDecoder(request.getUri()).parameters(); final List keepAliveList = q.get("keepAlive"); final List dagCompletedQ = q.get("dagAction"); boolean keepAliveParam = false; @@ -1024,7 +1022,7 @@ public void messageReceived(ChannelHandlerContext ctx, MessageEvent evt) "\n keepAlive: " + keepAliveParam); } // If the request is for Dag Deletion, process the request and send OK. - if (deleteDagDirectories(evt, dagCompletedQ, jobQ, dagIdQ)) { + if (deleteDagDirectories(ctx.channel(), dagCompletedQ, jobQ, dagIdQ)) { return; } if (mapIds == null || reduceRange == null || jobQ == null || dagIdQ == null) { @@ -1073,8 +1071,8 @@ public void messageReceived(ChannelHandlerContext ctx, MessageEvent evt) Map mapOutputInfoMap = new HashMap(); - Channel ch = evt.getChannel(); - ChannelPipeline pipeline = ch.getPipeline(); + Channel ch = ctx.channel(); + ChannelPipeline pipeline = ch.pipeline(); TimeoutHandler timeoutHandler = (TimeoutHandler)pipeline.get(TIMEOUT_HANDLER); timeoutHandler.setEnabledTimeout(false); String user = userRsrc.get(jobId); @@ -1098,19 +1096,23 @@ public void messageReceived(ChannelHandlerContext ctx, MessageEvent evt) return; } ch.write(response); - //Initialize one ReduceContext object per messageReceived call + //Initialize one ReduceContext object per channelRead call boolean keepAlive = keepAliveParam || connectionKeepAliveEnabled; ReduceContext reduceContext = new ReduceContext(mapIds, reduceRange, ctx, user, mapOutputInfoMap, jobId, dagId, keepAlive); for (int i = 0; i < Math.min(maxSessionOpenFiles, mapIds.size()); i++) { ChannelFuture nextMap = sendMap(reduceContext); if(nextMap == null) { + // by this special message flushed, we can make sure the whole response is finished + ch.writeAndFlush(LastHttpContent.EMPTY_LAST_CONTENT); return; } } + // by this special message flushed, we can make sure the whole response is finished + ch.writeAndFlush(LastHttpContent.EMPTY_LAST_CONTENT); } - private boolean deleteDagDirectories(MessageEvent evt, + private boolean deleteDagDirectories(Channel channel, List dagCompletedQ, List jobQ, List dagIdQ) { if (jobQ == null || jobQ.isEmpty()) { @@ -1127,8 +1129,8 @@ private boolean deleteDagDirectories(MessageEvent evt, } catch (IOException e) { LOG.warn("Encountered exception during dag delete "+ e); } - evt.getChannel().write(new DefaultHttpResponse(HTTP_1_1, OK)); - evt.getChannel().close(); + channel.writeAndFlush(new DefaultHttpResponse(HTTP_1_1, OK)) + .addListener(ChannelFutureListener.CLOSE); return true; } return false; @@ -1136,7 +1138,7 @@ private boolean deleteDagDirectories(MessageEvent evt, /** * Calls sendMapOutput for the mapId pointed by ReduceContext.mapsToSend - * and increments it. This method is first called by messageReceived() + * and increments it. This method is first called by channelRead() * maxSessionOpenFiles times and then on the completion of every * sendMapOutput operation. This limits the number of open files on a node, * which can get really large(exhausting file descriptors on the NM) if all @@ -1146,7 +1148,6 @@ private boolean deleteDagDirectories(MessageEvent evt, */ public ChannelFuture sendMap(ReduceContext reduceContext) throws Exception { - ChannelFuture nextMap = null; if (reduceContext.getMapsToSend().get() < reduceContext.getMapIds().size()) { @@ -1155,14 +1156,16 @@ public ChannelFuture sendMap(ReduceContext reduceContext) try { MapOutputInfo info = reduceContext.getInfoMap().get(mapId); + if (info == null) { info = getMapOutputInfo(reduceContext.dagId, mapId, reduceContext.getReduceRange(), reduceContext.getJobId(), reduceContext.getUser()); } + nextMap = sendMapOutput( reduceContext.getCtx(), - reduceContext.getCtx().getChannel(), + reduceContext.getCtx().channel(), reduceContext.getUser(), mapId, reduceContext.getReduceRange(), info); if (null == nextMap) { @@ -1417,23 +1420,12 @@ protected ChannelFuture sendMapOutput(ChannelHandlerContext ctx, Channel ch, return null; } ChannelFuture writeFuture; - if (ch.getPipeline().get(SslHandler.class) == null) { + if (ch.pipeline().get(SslHandler.class) == null) { final FadvisedFileRegion partition = new FadvisedFileRegion(spill, rangeOffset, rangePartLength, 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, @@ -1452,42 +1444,46 @@ protected void sendError(ChannelHandlerContext ctx, HttpResponseStatus status) { } protected void sendError(ChannelHandlerContext ctx, String message, HttpResponseStatus status) { - HttpResponse response = new DefaultHttpResponse(HTTP_1_1, status); + FullHttpResponse response = new DefaultFullHttpResponse(HTTP_1_1, status); sendError(ctx, message, response); + response.release(); } - protected void sendError(ChannelHandlerContext ctx, String message, HttpResponse response) { - sendError(ctx, ChannelBuffers.copiedBuffer(message, CharsetUtil.UTF_8), response); + protected void sendError(ChannelHandlerContext ctx, String message, FullHttpResponse response) { + sendError(ctx, Unpooled.copiedBuffer(message, CharsetUtil.UTF_8), response); } private void sendFakeShuffleHeaderWithError(ChannelHandlerContext ctx, String message, HttpResponse response) throws IOException { + FullHttpResponse fullResponse = + new DefaultFullHttpResponse(response.getProtocolVersion(), response.getStatus()); + fullResponse.headers().set(response.headers()); + ShuffleHeader header = new ShuffleHeader(message, -1, -1, -1); DataOutputBuffer out = new DataOutputBuffer(); header.write(out); - sendError(ctx, wrappedBuffer(out.getData(), 0, out.getLength()), response); + sendError(ctx, wrappedBuffer(out.getData(), 0, out.getLength()), fullResponse); + fullResponse.release(); } - protected void sendError(ChannelHandlerContext ctx, ChannelBuffer content, - HttpResponse response) { + protected void sendError(ChannelHandlerContext ctx, ByteBuf content, + FullHttpResponse response) { response.headers().set(CONTENT_TYPE, "text/plain; charset=UTF-8"); // Put shuffle version into http header response.headers().set(ShuffleHeader.HTTP_HEADER_NAME, ShuffleHeader.DEFAULT_HTTP_HEADER_NAME); response.headers().set(ShuffleHeader.HTTP_HEADER_VERSION, ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION); - response.setContent(content); + response.content().writeBytes(content); // Close the connection as soon as the error message is sent. - ctx.getChannel().write(response).addListener(ChannelFutureListener.CLOSE); + ctx.channel().writeAndFlush(response).addListener(ChannelFutureListener.CLOSE); } @Override - public void exceptionCaught(ChannelHandlerContext ctx, ExceptionEvent e) + public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception { - Channel ch = e.getChannel(); - Throwable cause = e.getCause(); if (cause instanceof TooLongFrameException) { sendError(ctx, BAD_REQUEST); return; @@ -1504,8 +1500,8 @@ public void exceptionCaught(ChannelHandlerContext ctx, ExceptionEvent e) } LOG.error("Shuffle error: ", cause); - if (ch.isConnected()) { - LOG.error("Shuffle error " + e); + if (ctx.channel().isActive()) { + LOG.error("Shuffle error", cause); sendError(ctx, INTERNAL_SERVER_ERROR); } } diff --git a/tez-plugins/tez-aux-services/src/test/java/org/apache/tez/auxservices/TestShuffleHandler.java b/tez-plugins/tez-aux-services/src/test/java/org/apache/tez/auxservices/TestShuffleHandler.java index 5ca4ed8e26..21addd393e 100644 --- a/tez-plugins/tez-aux-services/src/test/java/org/apache/tez/auxservices/TestShuffleHandler.java +++ b/tez-plugins/tez-aux-services/src/test/java/org/apache/tez/auxservices/TestShuffleHandler.java @@ -22,9 +22,7 @@ //import static org.apache.hadoop.test.MetricsAsserts.getMetrics; import org.apache.hadoop.util.DiskChecker.DiskErrorException; import static org.junit.Assert.assertTrue; -import static org.jboss.netty.buffer.ChannelBuffers.wrappedBuffer; -import static org.jboss.netty.handler.codec.http.HttpResponseStatus.OK; -import static org.jboss.netty.handler.codec.http.HttpVersion.HTTP_1_1; +import static io.netty.buffer.Unpooled.wrappedBuffer; import static org.junit.Assert.assertEquals; import static org.junit.Assume.assumeTrue; import static org.mockito.Mockito.mock; @@ -44,6 +42,7 @@ import java.util.ArrayList; import java.util.List; import java.util.Map; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.zip.Checksum; import org.apache.hadoop.conf.Configuration; @@ -55,8 +54,6 @@ import org.apache.hadoop.io.Text; import org.apache.hadoop.io.WritableUtils; import org.apache.hadoop.io.nativeio.NativeIO; -import org.apache.hadoop.mapred.JobID; -import org.apache.hadoop.mapreduce.TypeConverter; import org.apache.tez.runtime.library.common.security.SecureShuffleUtils; import org.apache.tez.common.security.JobTokenIdentifier; import org.apache.tez.common.security.JobTokenSecretManager; @@ -79,20 +76,21 @@ import org.apache.hadoop.yarn.server.records.Version; import org.apache.tez.runtime.library.common.sort.impl.TezIndexRecord; import org.apache.tez.runtime.library.common.sort.impl.TezSpillRecord; -import org.jboss.netty.channel.Channel; -import org.jboss.netty.channel.ChannelFuture; -import org.jboss.netty.channel.ChannelHandlerContext; -import org.jboss.netty.channel.ChannelPipeline; -import org.jboss.netty.channel.socket.SocketChannel; -import org.jboss.netty.channel.MessageEvent; -import org.jboss.netty.channel.AbstractChannel; -import org.jboss.netty.handler.codec.http.DefaultHttpHeaders; -import org.jboss.netty.handler.codec.http.DefaultHttpResponse; -import org.jboss.netty.handler.codec.http.HttpHeaders; -import org.jboss.netty.handler.codec.http.HttpRequest; -import org.jboss.netty.handler.codec.http.HttpResponse; -import org.jboss.netty.handler.codec.http.HttpResponseStatus; -import org.jboss.netty.handler.codec.http.HttpMethod; + +import io.netty.channel.Channel; +import io.netty.channel.ChannelFuture; +import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.ChannelPipeline; +import io.netty.channel.socket.SocketChannel; +import io.netty.channel.AbstractChannel; +import io.netty.handler.codec.http.DefaultFullHttpRequest; +import io.netty.handler.codec.http.FullHttpRequest; +import io.netty.handler.codec.http.HttpHeaders; +import io.netty.handler.codec.http.HttpRequest; +import io.netty.handler.codec.http.HttpResponse; +import io.netty.handler.codec.http.HttpResponseStatus; +import io.netty.handler.codec.http.HttpVersion; +import io.netty.handler.codec.http.HttpMethod; import org.junit.Assert; import org.junit.Test; import org.mockito.invocation.InvocationOnMock; @@ -141,12 +139,12 @@ protected ChannelFuture sendMapOutput(ChannelHandlerContext ctx, new ShuffleHeader("attempt_12345_1_m_1_0", 5678, 5678, 1); DataOutputBuffer dob = new DataOutputBuffer(); header.write(dob); - ch.write(wrappedBuffer(dob.getData(), 0, dob.getLength())); + ch.writeAndFlush(wrappedBuffer(dob.getData(), 0, dob.getLength())); dob = new DataOutputBuffer(); for (int i = 0; i < 100; ++i) { header.write(dob); } - return ch.write(wrappedBuffer(dob.getData(), 0, dob.getLength())); + return ch.writeAndFlush(wrappedBuffer(dob.getData(), 0, dob.getLength())); } }; } @@ -162,8 +160,8 @@ protected Shuffle getShuffle(final Configuration conf) { protected void verifyRequest(String appid, ChannelHandlerContext ctx, HttpRequest request, HttpResponse response, URL requestUri) throws IOException { - SocketChannel channel = (SocketChannel)(ctx.getChannel()); - socketKeepAlive = channel.getConfig().isKeepAlive(); + SocketChannel channel = (SocketChannel)(ctx.channel()); + socketKeepAlive = channel.config().isKeepAlive(); } }; } @@ -258,6 +256,7 @@ public void testShuffleMetrics() throws Exception { sh.metrics.operationComplete(cf); checkShuffleMetrics(ms, 3*MiB, 1, 1, 0); + sh.close(); } static void checkShuffleMetrics(MetricsSystem ms, long bytes, int failed, @@ -279,7 +278,7 @@ static void checkShuffleMetrics(MetricsSystem ms, long bytes, int failed, */ @Test (timeout = 10000) public void testClientClosesConnection() throws Exception { - final ArrayList failures = new ArrayList(1); + final AtomicBoolean failureEncountered = new AtomicBoolean(false); Configuration conf = new Configuration(); conf.set(HADOOP_TMP_DIR, TEST_DIR.getAbsolutePath()); conf.setInt(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY, 0); @@ -321,27 +320,25 @@ protected ChannelFuture sendMapOutput(ChannelHandlerContext ctx, new ShuffleHeader("attempt_12345_1_m_1_0", 5678, 5678, 1); DataOutputBuffer dob = new DataOutputBuffer(); header.write(dob); - ch.write(wrappedBuffer(dob.getData(), 0, dob.getLength())); + ch.writeAndFlush(wrappedBuffer(dob.getData(), 0, dob.getLength())); dob = new DataOutputBuffer(); for (int i = 0; i < 100000; ++i) { header.write(dob); } - return ch.write(wrappedBuffer(dob.getData(), 0, dob.getLength())); + return ch.writeAndFlush(wrappedBuffer(dob.getData(), 0, dob.getLength())); } @Override protected void sendError(ChannelHandlerContext ctx, HttpResponseStatus status) { - if (failures.size() == 0) { - failures.add(new Error()); - ctx.getChannel().close(); + if (failureEncountered.compareAndSet(false, true)) { + ctx.channel().close(); } } @Override protected void sendError(ChannelHandlerContext ctx, String message, HttpResponseStatus status) { - if (failures.size() == 0) { - failures.add(new Error()); - ctx.getChannel().close(); + if (failureEncountered.compareAndSet(false, true)) { + ctx.channel().close(); } } }; @@ -368,9 +365,9 @@ protected void sendError(ChannelHandlerContext ctx, String message, header.readFields(input); input.close(); - shuffleHandler.stop(); + shuffleHandler.close(); Assert.assertTrue("sendError called when client closed connection", - failures.size() == 0); + !failureEncountered.get()); } static class LastSocketAddress { @@ -378,14 +375,14 @@ static class LastSocketAddress { void setAddress(SocketAddress lastAddress) { this.lastAddress = lastAddress; } - SocketAddress getSocketAddres() { + SocketAddress getSocketAddress() { return lastAddress; } } @Test(timeout = 10000) public void testKeepAlive() throws Exception { - final ArrayList failures = new ArrayList(1); + final AtomicBoolean failureEncountered = new AtomicBoolean(false); Configuration conf = new Configuration(); conf.set(HADOOP_TMP_DIR, TEST_DIR.getAbsolutePath()); conf.setInt(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY, 0); @@ -443,8 +440,7 @@ protected void populateHeaders(List mapIds, String jobId, protected ChannelFuture sendMapOutput(ChannelHandlerContext ctx, Channel ch, String user, String mapId, Range reduceRange, MapOutputInfo info) throws IOException { - lastSocketAddress.setAddress(ch.getRemoteAddress()); - HttpResponse response = new DefaultHttpResponse(HTTP_1_1, OK); + lastSocketAddress.setAddress(ch.remoteAddress()); // send a shuffle header and a lot of data down the channel // to trigger a broken pipe @@ -452,29 +448,27 @@ protected ChannelFuture sendMapOutput(ChannelHandlerContext ctx, new ShuffleHeader("attempt_12345_1_m_1_0", 5678, 5678, 1); DataOutputBuffer dob = new DataOutputBuffer(); header.write(dob); - ch.write(wrappedBuffer(dob.getData(), 0, dob.getLength())); + ch.writeAndFlush(wrappedBuffer(dob.getData(), 0, dob.getLength())); dob = new DataOutputBuffer(); for (int i = 0; i < 100000; ++i) { header.write(dob); } - return ch.write(wrappedBuffer(dob.getData(), 0, dob.getLength())); + return ch.writeAndFlush(wrappedBuffer(dob.getData(), 0, dob.getLength())); } @Override protected void sendError(ChannelHandlerContext ctx, HttpResponseStatus status) { - if (failures.size() == 0) { - failures.add(new Error()); - ctx.getChannel().close(); + if (failureEncountered.compareAndSet(false, true)) { + ctx.channel().close(); } } @Override protected void sendError(ChannelHandlerContext ctx, String message, HttpResponseStatus status) { - if (failures.size() == 0) { - failures.add(new Error()); - ctx.getChannel().close(); + if (failureEncountered.compareAndSet(false, true)) { + ctx.channel().close(); } } }; @@ -505,7 +499,7 @@ protected void sendError(ChannelHandlerContext ctx, String message, header.readFields(input); byte[] buffer = new byte[1024]; while (input.read(buffer) != -1) {} - SocketAddress firstAddress = lastSocketAddress.getSocketAddres(); + SocketAddress firstAddress = lastSocketAddress.getSocketAddress(); input.close(); // For keepAlive via URL @@ -527,11 +521,12 @@ protected void sendError(ChannelHandlerContext ctx, String message, header = new ShuffleHeader(); header.readFields(input); input.close(); - SocketAddress secondAddress = lastSocketAddress.getSocketAddres(); + SocketAddress secondAddress = lastSocketAddress.getSocketAddress(); Assert.assertNotNull("Initial shuffle address should not be null", firstAddress); Assert.assertNotNull("Keep-Alive shuffle address should not be null", secondAddress); Assert.assertEquals("Initial shuffle address and keep-alive shuffle " + "address should be the same", firstAddress, secondAddress); + shuffleHandler.close(); } @Test @@ -567,7 +562,7 @@ public void testSocketKeepAlive() throws Exception { if (conn != null) { conn.disconnect(); } - shuffleHandler.stop(); + shuffleHandler.close(); } } @@ -603,7 +598,6 @@ public void testIncompatibleShuffleVersion() throws Exception { HttpURLConnection.HTTP_BAD_REQUEST, conn.getResponseCode()); } - shuffleHandler.stop(); shuffleHandler.close(); } @@ -657,12 +651,12 @@ protected ChannelFuture sendMapOutput(ChannelHandlerContext ctx, new ShuffleHeader("dummy_header", 5678, 5678, 1); DataOutputBuffer dob = new DataOutputBuffer(); header.write(dob); - ch.write(wrappedBuffer(dob.getData(), 0, dob.getLength())); + ch.writeAndFlush(wrappedBuffer(dob.getData(), 0, dob.getLength())); dob = new DataOutputBuffer(); for (int i=0; i<100000; ++i) { header.write(dob); } - return ch.write(wrappedBuffer(dob.getData(), 0, dob.getLength())); + return ch.writeAndFlush(wrappedBuffer(dob.getData(), 0, dob.getLength())); } }; } @@ -689,6 +683,10 @@ protected ChannelFuture sendMapOutput(ChannelHandlerContext ctx, // Try to open numerous connections for (int i = 0; i < connAttempts; i++) { + // connections should be made in a bit relaxed way, otherwise + // non-synced channelActive method will mess them up + Thread.sleep(200); + conns[i].connect(); } @@ -712,7 +710,7 @@ protected ChannelFuture sendMapOutput(ChannelHandlerContext ctx, Assert.fail("Expected a SocketException"); } - shuffleHandler.stop(); + shuffleHandler.close(); } /** @@ -807,7 +805,7 @@ protected void verifyRequest(String appid, ChannelHandlerContext ctx, } } finally { - shuffleHandler.stop(); + shuffleHandler.close(); FileUtil.fullyDelete(absLogDir); } } @@ -900,7 +898,7 @@ protected void verifyRequest(String appid, ChannelHandlerContext ctx, + " did not match expected owner '" + user + "'"; Assert.assertTrue((new String(byteArr)).contains(message)); } finally { - shuffleHandler.stop(); + shuffleHandler.close(); FileUtil.fullyDelete(absLogDir); } } @@ -953,7 +951,6 @@ private static void createIndexFile(File indexFile, Configuration conf) public void testRecovery() throws IOException { final String user = "someuser"; final ApplicationId appId = ApplicationId.newInstance(12345, 1); - final JobID jobId = JobID.downgrade(TypeConverter.fromYarn(appId)); final File tmpDir = new File(System.getProperty("test.build.data", System.getProperty("java.io.tmpdir")), TestShuffleHandler.class.getName()); @@ -1131,7 +1128,7 @@ private static int getShuffleResponseCode(ShuffleHandler shuffle, @Test(timeout = 100000) public void testGetMapOutputInfo() throws Exception { - final ArrayList failures = new ArrayList(1); + final AtomicBoolean failureEncountered = new AtomicBoolean(false); Configuration conf = new Configuration(); conf.set(HADOOP_TMP_DIR, TEST_DIR.getAbsolutePath()); conf.setInt(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY, 0); @@ -1173,9 +1170,8 @@ protected void verifyRequest(String appid, @Override protected void sendError(ChannelHandlerContext ctx, String message, HttpResponseStatus status) { - if (failures.size() == 0) { - failures.add(new Error(message)); - ctx.getChannel().close(); + if (failureEncountered.compareAndSet(false, true)) { + ctx.channel().close(); } } @Override @@ -1187,7 +1183,7 @@ protected ChannelFuture sendMapOutput(ChannelHandlerContext ctx, new ShuffleHeader("attempt_12345_1_m_1_0", 5678, 5678, 1); DataOutputBuffer dob = new DataOutputBuffer(); header.write(dob); - return ch.write(wrappedBuffer(dob.getData(), 0, dob.getLength())); + return ch.writeAndFlush(wrappedBuffer(dob.getData(), 0, dob.getLength())); } }; } @@ -1227,16 +1223,16 @@ protected ChannelFuture sendMapOutput(ChannelHandlerContext ctx, // ignore } Assert.assertEquals("sendError called due to shuffle error", - 0, failures.size()); + false, failureEncountered.get()); } finally { - shuffleHandler.stop(); + shuffleHandler.close(); FileUtil.fullyDelete(absLogDir); } } @Test(timeout = 5000) public void testDagDelete() throws Exception { - final ArrayList failures = new ArrayList(1); + final AtomicBoolean failureEncountered = new AtomicBoolean(false); Configuration conf = new Configuration(); conf.set(HADOOP_TMP_DIR, TEST_DIR.getAbsolutePath()); conf.setInt(ShuffleHandler.MAX_SHUFFLE_CONNECTIONS, 3); @@ -1261,9 +1257,8 @@ protected Shuffle getShuffle(Configuration conf) { @Override protected void sendError(ChannelHandlerContext ctx, String message, HttpResponseStatus status) { - if (failures.size() == 0) { - failures.add(new Error(message)); - ctx.getChannel().close(); + if (failureEncountered.compareAndSet(false, true)) { + ctx.channel().close(); } } }; @@ -1309,9 +1304,9 @@ protected void sendError(ChannelHandlerContext ctx, String message, // ignore } Assert.assertEquals("sendError called due to shuffle error", - 0, failures.size()); + false, failureEncountered.get()); } finally { - shuffleHandler.stop(); + shuffleHandler.close(); FileUtil.fullyDelete(absLogDir); } } @@ -1323,29 +1318,23 @@ public void testSendMapCount() throws Exception { final ChannelHandlerContext mockCtx = mock(ChannelHandlerContext.class); - final MessageEvent mockEvt = mock(MessageEvent.class); final Channel mockCh = mock(AbstractChannel.class); final ChannelPipeline mockPipeline = Mockito.mock(ChannelPipeline.class); // Mock HttpRequest and ChannelFuture - final HttpRequest mockHttpRequest = createMockHttpRequest(); + final FullHttpRequest httpRequest = createHttpRequest(); final ChannelFuture mockFuture = createMockChannelFuture(mockCh, listenerList); final ShuffleHandler.TimeoutHandler timerHandler = new ShuffleHandler.TimeoutHandler(); // Mock Netty Channel Context and Channel behavior - Mockito.doReturn(mockCh).when(mockCtx).getChannel(); - Mockito.when(mockCh.getPipeline()).thenReturn(mockPipeline); + Mockito.doReturn(mockCh).when(mockCtx).channel(); + Mockito.when(mockCh.pipeline()).thenReturn(mockPipeline); Mockito.when(mockPipeline.get(Mockito.any(String.class))).thenReturn(timerHandler); - when(mockCtx.getChannel()).thenReturn(mockCh); - Mockito.doReturn(mockFuture).when(mockCh).write(Mockito.any(Object.class)); - when(mockCh.write(Object.class)).thenReturn(mockFuture); - - //Mock MessageEvent behavior - Mockito.doReturn(mockCh).when(mockEvt).getChannel(); - when(mockEvt.getChannel()).thenReturn(mockCh); - Mockito.doReturn(mockHttpRequest).when(mockEvt).getMessage(); + when(mockCtx.channel()).thenReturn(mockCh); + Mockito.doReturn(mockFuture).when(mockCh).writeAndFlush(Mockito.any(Object.class)); + when(mockCh.writeAndFlush(Object.class)).thenReturn(mockFuture); final ShuffleHandler sh = new MockShuffleHandler(); Configuration conf = new Configuration(); @@ -1356,7 +1345,7 @@ public void testSendMapCount() throws Exception { sh.start(); int maxOpenFiles =conf.getInt(ShuffleHandler.SHUFFLE_MAX_SESSION_OPEN_FILES, ShuffleHandler.DEFAULT_SHUFFLE_MAX_SESSION_OPEN_FILES); - sh.getShuffle(conf).messageReceived(mockCtx, mockEvt); + sh.getShuffle(conf).channelRead(mockCtx, httpRequest); assertTrue("Number of Open files should not exceed the configured " + "value!-Not Expected", listenerList.size() <= maxOpenFiles); @@ -1419,9 +1408,9 @@ public void testShuffleHandlerSendsDiskError() throws Exception { public ChannelFuture createMockChannelFuture(Channel mockCh, final List listenerList) { final ChannelFuture mockFuture = mock(ChannelFuture.class); - when(mockFuture.getChannel()).thenReturn(mockCh); + when(mockFuture.channel()).thenReturn(mockCh); Mockito.doReturn(true).when(mockFuture).isSuccess(); - Mockito.doAnswer(new Answer() { + Mockito.doAnswer(new Answer() { @Override public Object answer(InvocationOnMock invocation) throws Throwable { //Add ReduceMapFileCount listener to a list @@ -1436,19 +1425,11 @@ public Object answer(InvocationOnMock invocation) throws Throwable { return mockFuture; } - public HttpRequest createMockHttpRequest() { - HttpRequest mockHttpRequest = mock(HttpRequest.class); - Mockito.doReturn(HttpMethod.GET).when(mockHttpRequest).getMethod(); - Mockito.doReturn(new DefaultHttpHeaders()).when(mockHttpRequest).headers(); - Mockito.doAnswer(new Answer() { - @Override - public Object answer(InvocationOnMock invocation) throws Throwable { - String uri = "/mapOutput?job=job_12345_1&dag=1&reduce=1"; - for (int i = 0; i < 100; i++) - uri = uri.concat("&map=attempt_12345_1_m_" + i + "_0"); - return uri; - } - }).when(mockHttpRequest).getUri(); - return mockHttpRequest; + public FullHttpRequest createHttpRequest() { + String uri = "/mapOutput?job=job_12345_1&dag=1&reduce=1"; + for (int i = 0; i < 100; i++) { + uri = uri.concat("&map=attempt_12345_1_m_" + i + "_0"); + } + return new DefaultFullHttpRequest(HttpVersion.HTTP_1_1, HttpMethod.GET, uri); } } From 867e4d205f129e198668daf94d0eb000e04c0e8c Mon Sep 17 00:00:00 2001 From: Harish Jaiprakash Date: Wed, 19 May 2021 01:49:07 +0530 Subject: [PATCH 049/137] TEZ-4296 - Use listStatusIterator instead of listStatus in DatePartitionedLogger (#124) Co-authored-by: Harish JP --- .../dag/history/logging/proto/DatePartitionedLogger.java | 9 +++++++-- .../logging/proto/TestProtoHistoryLoggingService.java | 1 + 2 files changed, 8 insertions(+), 2 deletions(-) diff --git a/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/DatePartitionedLogger.java b/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/DatePartitionedLogger.java index 4ac64c6955..a569567d1e 100644 --- a/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/DatePartitionedLogger.java +++ b/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/DatePartitionedLogger.java @@ -32,6 +32,7 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.fs.RemoteIterator; import org.apache.hadoop.yarn.util.Clock; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -138,7 +139,9 @@ public String getNextDirectory(String currentDir) throws IOException { } // Have to scan the directory to find min date greater than currentDir. String dirName = null; - for (FileStatus status : fileSystem.listStatus(basePath)) { + RemoteIterator iter = fileSystem.listStatusIterator(basePath); + while (iter.hasNext()) { + FileStatus status = iter.next(); String name = status.getPath().getName(); // String comparison is good enough, since its of form date=yyyy-MM-dd if (name.compareTo(currentDir) > 0 && (dirName == null || name.compareTo(dirName) < 0)) { @@ -160,7 +163,9 @@ public List scanForChangedFiles(String subDir, Map cur if (!fileSystem.exists(dirPath)) { return newFiles; } - for (FileStatus status : fileSystem.listStatus(dirPath)) { + RemoteIterator iter = fileSystem.listStatusIterator(dirPath); + while (iter.hasNext()) { + FileStatus status = iter.next(); String fileName = status.getPath().getName(); Long offset = currentOffsets.get(fileName); // If the offset was never added or offset < fileSize. diff --git a/tez-plugins/tez-protobuf-history-plugin/src/test/java/org/apache/tez/dag/history/logging/proto/TestProtoHistoryLoggingService.java b/tez-plugins/tez-protobuf-history-plugin/src/test/java/org/apache/tez/dag/history/logging/proto/TestProtoHistoryLoggingService.java index 143faed516..fd3154d904 100644 --- a/tez-plugins/tez-protobuf-history-plugin/src/test/java/org/apache/tez/dag/history/logging/proto/TestProtoHistoryLoggingService.java +++ b/tez-plugins/tez-protobuf-history-plugin/src/test/java/org/apache/tez/dag/history/logging/proto/TestProtoHistoryLoggingService.java @@ -277,6 +277,7 @@ private ProtoHistoryLoggingService createService(boolean splitEvents) throws IOE service.setAppContext(appContext); Configuration conf = new Configuration(false); String basePath = tempFolder.newFolder().getAbsolutePath(); + conf.set("fs.file.impl", "org.apache.hadoop.fs.RawLocalFileSystem"); conf.set(TezConfiguration.TEZ_HISTORY_LOGGING_PROTO_BASE_DIR, basePath); conf.setBoolean(TezConfiguration.TEZ_HISTORY_LOGGING_PROTO_SPLIT_DAG_START, splitEvents); service.init(conf); From 4d379149bcdd0c0097f366d9a01e6a9907410ac6 Mon Sep 17 00:00:00 2001 From: Kishen Das Date: Thu, 20 May 2021 07:36:20 -0700 Subject: [PATCH 050/137] TEZ-4305: Check StreamCapabilities before using HFLUSH from ProtoMessageWriter (#120) (Kishen Das reviewed by Laszlo Bodor and Harish JP) Co-authored-by: Kishen Das --- .../org/apache/tez/common/StreamHelper.java | 49 +++++++++++++++++++ .../impl/SimpleHistoryLoggingService.java | 3 +- .../logging/proto/ProtoMessageWriter.java | 3 +- 3 files changed, 53 insertions(+), 2 deletions(-) create mode 100644 tez-common/src/main/java/org/apache/tez/common/StreamHelper.java diff --git a/tez-common/src/main/java/org/apache/tez/common/StreamHelper.java b/tez-common/src/main/java/org/apache/tez/common/StreamHelper.java new file mode 100644 index 0000000000..789d9b4268 --- /dev/null +++ b/tez-common/src/main/java/org/apache/tez/common/StreamHelper.java @@ -0,0 +1,49 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tez.common; + +import org.apache.hadoop.fs.StreamCapabilities; +import org.apache.hadoop.fs.Syncable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; + +public final class StreamHelper { + + private static final Logger LOG = LoggerFactory.getLogger(StreamHelper.class); + + private StreamHelper() { + } + + public static void hflushIfSupported(Syncable syncable) throws IOException { + if (syncable instanceof StreamCapabilities) { + if (((StreamCapabilities) syncable).hasCapability(StreamCapabilities.HFLUSH)) { + syncable.hflush(); + } else { + // it would be no-op, if hflush is not supported by a given writer. + LOG.debug("skipping hflush, since the writer doesn't support it"); + } + } else { + // this is done for backward compatibility in order to make it work with + // older versions of Hadoop. + syncable.hflush(); + } + } +} diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/logging/impl/SimpleHistoryLoggingService.java b/tez-dag/src/main/java/org/apache/tez/dag/history/logging/impl/SimpleHistoryLoggingService.java index 4372d8ec22..418bc3c5ea 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/logging/impl/SimpleHistoryLoggingService.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/logging/impl/SimpleHistoryLoggingService.java @@ -23,6 +23,7 @@ import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.tez.common.StreamHelper; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.conf.Configuration; @@ -130,7 +131,7 @@ protected void serviceStop() throws Exception { } try { if (outputStream != null) { - outputStream.hflush(); + StreamHelper.hflushIfSupported(outputStream); outputStream.close(); } } catch (IOException ioe) { diff --git a/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/ProtoMessageWriter.java b/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/ProtoMessageWriter.java index 869b603c3e..5b7591b496 100644 --- a/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/ProtoMessageWriter.java +++ b/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/ProtoMessageWriter.java @@ -30,6 +30,7 @@ import com.google.protobuf.MessageLite; import com.google.protobuf.Parser; +import org.apache.tez.common.StreamHelper; public class ProtoMessageWriter implements Closeable { private final Path filePath; @@ -61,7 +62,7 @@ public void writeProto(T message) throws IOException { } public void hflush() throws IOException { - writer.hflush(); + StreamHelper.hflushIfSupported(writer); } @Override From 0af54dfa5fb114712efc1bdbcca32bb673bc53ef Mon Sep 17 00:00:00 2001 From: Ramesh Kumar Thangarajan Date: Thu, 20 May 2021 16:41:32 +0200 Subject: [PATCH 051/137] =?UTF-8?q?TEZ-4309:=20TezUtils.addToConfFromByteS?= =?UTF-8?q?tring=20throws=20com.google.protobuf.CodedInputStream=20excepti?= =?UTF-8?q?on=20(Ramesh=20Kumar=20Thangarajan=20via=20L=C3=A1szl=C3=B3=20B?= =?UTF-8?q?odor)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Laszlo Bodor --- .../java/org/apache/tez/common/TezUtils.java | 14 ++++++--- .../org/apache/tez/common/TestTezUtils.java | 31 ++++++++++++++++--- 2 files changed, 35 insertions(+), 10 deletions(-) diff --git a/tez-api/src/main/java/org/apache/tez/common/TezUtils.java b/tez-api/src/main/java/org/apache/tez/common/TezUtils.java index 3f0241834a..1c0be98dcc 100644 --- a/tez-api/src/main/java/org/apache/tez/common/TezUtils.java +++ b/tez-api/src/main/java/org/apache/tez/common/TezUtils.java @@ -101,6 +101,12 @@ public static UserPayload createUserPayloadFromConf(Configuration conf) throws I return UserPayload.create(ByteBuffer.wrap(createByteStringFromConf(conf).toByteArray())); } + private static DAGProtos.ConfigurationProto createConfProto(SnappyInputStream uncompressIs) throws IOException { + CodedInputStream in = CodedInputStream.newInstance(uncompressIs); + in.setSizeLimit(Integer.MAX_VALUE); + return DAGProtos.ConfigurationProto.parseFrom(in); + } + /** * Convert a byte string to a Configuration object * @@ -112,9 +118,7 @@ public static UserPayload createUserPayloadFromConf(Configuration conf) throws I public static Configuration createConfFromByteString(ByteString byteString) throws IOException { Objects.requireNonNull(byteString, "ByteString must be specified"); try(SnappyInputStream uncompressIs = new SnappyInputStream(byteString.newInput());) { - CodedInputStream in = CodedInputStream.newInstance(uncompressIs); - in.setSizeLimit(Integer.MAX_VALUE); - DAGProtos.ConfigurationProto confProto = DAGProtos.ConfigurationProto.parseFrom(in); + DAGProtos.ConfigurationProto confProto = createConfProto(uncompressIs); Configuration conf = new Configuration(false); readConfFromPB(confProto, conf); TezClassLoader.setupForConfiguration(conf); @@ -129,7 +133,7 @@ public static Configuration createConfFromBaseConfAndPayload(TaskContext context UserPayload payload = context.getUserPayload(); ByteString byteString = ByteString.copyFrom(payload.getPayload()); try(SnappyInputStream uncompressIs = new SnappyInputStream(byteString.newInput())) { - DAGProtos.ConfigurationProto confProto = DAGProtos.ConfigurationProto.parseFrom(uncompressIs); + DAGProtos.ConfigurationProto confProto = createConfProto(uncompressIs); readConfFromPB(confProto, configuration); TezClassLoader.setupForConfiguration(configuration); return configuration; @@ -139,7 +143,7 @@ public static Configuration createConfFromBaseConfAndPayload(TaskContext context public static void addToConfFromByteString(Configuration configuration, ByteString byteString) throws IOException { try(SnappyInputStream uncompressIs = new SnappyInputStream(byteString.newInput())) { - DAGProtos.ConfigurationProto confProto = DAGProtos.ConfigurationProto.parseFrom(uncompressIs); + DAGProtos.ConfigurationProto confProto = createConfProto(uncompressIs); readConfFromPB(confProto, configuration); TezClassLoader.setupForConfiguration(configuration); } diff --git a/tez-common/src/test/java/org/apache/tez/common/TestTezUtils.java b/tez-common/src/test/java/org/apache/tez/common/TestTezUtils.java index f9008b9947..d599cafd76 100644 --- a/tez-common/src/test/java/org/apache/tez/common/TestTezUtils.java +++ b/tez-common/src/test/java/org/apache/tez/common/TestTezUtils.java @@ -54,13 +54,10 @@ public void testByteStringToAndFromConf() throws IOException { checkConf(conf); } - @Test (timeout=20000) - public void testByteStringToAndFromLargeConf() throws IOException { - Configuration conf = getConf(); + private String constructLargeValue() { int largeSizeMinimum = 64 * 1024 * 1024; final String alphaString = "ABCDEFGHIJKLMNOPQRSTUVWXYZ"; int largeSize = (largeSizeMinimum + alphaString.length() - 1) / alphaString.length(); - largeSize *= alphaString.length(); assertTrue(largeSize >= alphaString.length()); StringBuilder sb = new StringBuilder(largeSize); @@ -71,9 +68,20 @@ public void testByteStringToAndFromLargeConf() throws IOException { String largeValue = sb.toString(); Assert.assertEquals(largeSize, largeValue.length()); + return largeValue; + } + + private ByteString createByteString(Configuration conf, String largeValue) throws IOException { conf.set("testLargeValue", largeValue); Assert.assertEquals(conf.size(), 7); - ByteString bsConf = TezUtils.createByteStringFromConf(conf); + return TezUtils.createByteStringFromConf(conf); + } + + @Test (timeout=20000) + public void testByteStringToAndFromLargeConf() throws IOException { + Configuration conf = getConf(); + String largeValue = constructLargeValue(); + ByteString bsConf = createByteString(conf, largeValue); conf.clear(); Assert.assertEquals(conf.size(), 0); conf = TezUtils.createConfFromByteString(bsConf); @@ -82,6 +90,19 @@ public void testByteStringToAndFromLargeConf() throws IOException { Assert.assertEquals(conf.get("testLargeValue"), largeValue); } + @Test (timeout=20000) + public void testByteStringAddToLargeConf() throws IOException { + Configuration conf = getConf(); + String largeValue = constructLargeValue(); + ByteString bsConf = createByteString(conf, largeValue); + conf.clear(); + Assert.assertEquals(conf.size(), 0); + TezUtils.addToConfFromByteString(conf, bsConf); + Assert.assertEquals(conf.size(), 7); + checkConf(conf); + Assert.assertEquals(conf.get("testLargeValue"), largeValue); + } + @Test (timeout=2000) public void testPayloadToAndFromConf() throws IOException { Configuration conf = getConf(); From b6c7fedfb74e8222f5b12bf2f53967a04f99860c Mon Sep 17 00:00:00 2001 From: Bodor Laszlo Date: Wed, 16 Jun 2021 10:37:09 +0200 Subject: [PATCH 052/137] TEZ-4295: Could not decompress data. Buffer length is too small. (#130) (Laszlo Bodor reviewed by Ashutosh Chauhan) --- .../library/common/TezRuntimeUtils.java | 28 --- .../library/common/sort/impl/IFile.java | 10 +- .../tez/runtime/library/utils/CodecUtils.java | 109 +++++++-- .../orderedgrouped/DummyCompressionCodec.java | 20 +- .../orderedgrouped/TestMergeManager.java | 7 +- .../library/common/sort/impl/TestIFile.java | 10 +- .../runtime/library/utils/TestCodecUtils.java | 224 ++++++++++++++++++ 7 files changed, 351 insertions(+), 57 deletions(-) create mode 100644 tez-runtime-library/src/test/java/org/apache/tez/runtime/library/utils/TestCodecUtils.java diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/TezRuntimeUtils.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/TezRuntimeUtils.java index a1df131121..9d9b8c16c0 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/TezRuntimeUtils.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/TezRuntimeUtils.java @@ -26,7 +26,6 @@ import java.nio.ByteBuffer; import org.apache.hadoop.io.DataInputByteBuffer; -import org.apache.hadoop.io.compress.CompressionCodec; import org.apache.tez.common.security.JobTokenSecretManager; import org.apache.tez.http.BaseHttpConnection; import org.apache.tez.http.HttpConnection; @@ -37,8 +36,6 @@ import org.slf4j.LoggerFactory; import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.CommonConfigurationKeys; -import org.apache.hadoop.fs.CommonConfigurationKeysPublic; import org.apache.tez.dag.api.TezUncheckedException; import org.apache.tez.runtime.api.OutputContext; import org.apache.tez.runtime.api.TaskContext; @@ -262,29 +259,4 @@ public static int deserializeShuffleProviderMetaData(ByteBuffer meta) in.close(); } } - - public static String getBufferSizeProperty(CompressionCodec codec) { - return getBufferSizeProperty(codec.getClass().getName()); - } - - public static String getBufferSizeProperty(String className) { - switch (className) { - case "org.apache.hadoop.io.compress.DefaultCodec": - case "org.apache.hadoop.io.compress.BZip2Codec": - case "org.apache.hadoop.io.compress.GzipCodec": - return CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY; - case "org.apache.hadoop.io.compress.SnappyCodec": - return CommonConfigurationKeys.IO_COMPRESSION_CODEC_SNAPPY_BUFFERSIZE_KEY; - case "org.apache.hadoop.io.compress.ZStandardCodec": - return CommonConfigurationKeys.IO_COMPRESSION_CODEC_ZSTD_BUFFER_SIZE_KEY; - case "org.apache.hadoop.io.compress.LzoCodec": - return CommonConfigurationKeys.IO_COMPRESSION_CODEC_LZO_BUFFERSIZE_KEY; - case "com.hadoop.compression.lzo.LzoCodec": - return CommonConfigurationKeys.IO_COMPRESSION_CODEC_LZO_BUFFERSIZE_KEY; - case "org.apache.hadoop.io.compress.Lz4Codec": - return CommonConfigurationKeys.IO_COMPRESSION_CODEC_LZ4_BUFFERSIZE_KEY; - default: - return null; - } - } } diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFile.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFile.java index a4bbf5aabf..8f673185b0 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFile.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFile.java @@ -363,10 +363,10 @@ public Writer(Serialization keySerialization, Serialization valSerialization, FS void setupOutputStream(CompressionCodec codec) throws IOException { this.checksumOut = new IFileOutputStream(this.rawOut); if (codec != null) { - this.compressor = CodecPool.getCompressor(codec); + this.compressor = CodecUtils.getCompressor(codec); if (this.compressor != null) { this.compressor.reset(); - this.compressedOut = codec.createOutputStream(checksumOut, compressor); + this.compressedOut = CodecUtils.createOutputStream(codec, checksumOut, compressor); this.out = new FSDataOutputStream(this.compressedOut, null); this.compressOutput = true; } else { @@ -773,9 +773,9 @@ public Reader(InputStream in, long length, checksumIn = new IFileInputStream(in, length, readAhead, readAheadLength/* , isCompressed */); if (isCompressed && codec != null) { - decompressor = CodecPool.getDecompressor(codec); + decompressor = CodecUtils.getDecompressor(codec); if (decompressor != null) { - this.in = codec.createInputStream(checksumIn, decompressor); + this.in = CodecUtils.createInputStream(codec, checksumIn, decompressor); } else { LOG.warn("Could not obtain decompressor from CodecPool"); this.in = checksumIn; @@ -818,7 +818,7 @@ public static void readToMemory(byte[] buffer, InputStream in, int compressedLen in = checksumIn; Decompressor decompressor = null; if (isCompressed && codec != null) { - decompressor = CodecPool.getDecompressor(codec); + decompressor = CodecUtils.getDecompressor(codec); if (decompressor != null) { decompressor.reset(); in = CodecUtils.getDecompressedInputStreamWithBufferSize(codec, checksumIn, decompressor, diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/utils/CodecUtils.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/utils/CodecUtils.java index 8e5154f3b0..340ecceea8 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/utils/CodecUtils.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/utils/CodecUtils.java @@ -20,27 +20,33 @@ import java.io.IOException; import java.io.InputStream; +import java.io.OutputStream; import org.apache.hadoop.conf.Configurable; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.CommonConfigurationKeys; +import org.apache.hadoop.fs.CommonConfigurationKeysPublic; +import org.apache.hadoop.io.compress.CodecPool; import org.apache.hadoop.io.compress.CompressionCodec; import org.apache.hadoop.io.compress.CompressionInputStream; +import org.apache.hadoop.io.compress.CompressionOutputStream; import org.apache.hadoop.io.compress.Compressor; import org.apache.hadoop.io.compress.Decompressor; import org.apache.hadoop.io.compress.DefaultCodec; import org.apache.hadoop.util.ReflectionUtils; import org.apache.tez.runtime.library.api.TezRuntimeConfiguration; import org.apache.tez.runtime.library.common.ConfigUtils; -import org.apache.tez.runtime.library.common.TezRuntimeUtils; -import org.apache.tez.runtime.library.common.sort.impl.IFile; import org.apache.tez.runtime.library.common.sort.impl.IFileInputStream; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import com.google.common.annotations.VisibleForTesting; + public final class CodecUtils { - private static final Logger LOG = LoggerFactory.getLogger(IFile.class); - private static final int DEFAULT_BUFFER_SIZE = 128 * 1024; + private static final Logger LOG = LoggerFactory.getLogger(CodecUtils.class); + @VisibleForTesting + static final int DEFAULT_BUFFER_SIZE = 256 * 1024; private CodecUtils() { } @@ -76,20 +82,21 @@ public static CompressionCodec getCodec(Configuration conf) throws IOException { public static InputStream getDecompressedInputStreamWithBufferSize(CompressionCodec codec, IFileInputStream checksumIn, Decompressor decompressor, int compressedLength) throws IOException { - String bufferSizeProp = TezRuntimeUtils.getBufferSizeProperty(codec); - Configurable configurableCodec = (Configurable) codec; - int originalSize = bufferSizeProp == null ? DEFAULT_BUFFER_SIZE : - configurableCodec.getConf().getInt(bufferSizeProp, DEFAULT_BUFFER_SIZE); - + String bufferSizeProp = getBufferSizeProperty(codec); CompressionInputStream in = null; if (bufferSizeProp != null) { + Configurable configurableCodec = (Configurable) codec; Configuration conf = configurableCodec.getConf(); - int newBufSize = Math.min(compressedLength, DEFAULT_BUFFER_SIZE); - LOG.trace("buffer size was set according to min(compressedLength, {}): {}={}", - DEFAULT_BUFFER_SIZE, bufferSizeProp, newBufSize); - synchronized (codec) { + synchronized (conf) { + int defaultBufferSize = getDefaultBufferSize(conf, codec); + int originalSize = conf.getInt(bufferSizeProp, defaultBufferSize); + + int newBufSize = Math.min(compressedLength, defaultBufferSize); + LOG.debug("buffer size was set according to min({}, {}) => {}={}", compressedLength, + defaultBufferSize, bufferSizeProp, newBufSize); + conf.setInt(bufferSizeProp, newBufSize); in = codec.createInputStream(checksumIn, decompressor); @@ -117,7 +124,7 @@ public static InputStream getDecompressedInputStreamWithBufferSize(CompressionCo * issues above for Compressor instances as well, even when we tried to leverage from * smaller buffer size only on decompression paths. */ - configurableCodec.getConf().setInt(bufferSizeProp, originalSize); + conf.setInt(bufferSizeProp, originalSize); } } else { in = codec.createInputStream(checksumIn, decompressor); @@ -125,4 +132,78 @@ public static InputStream getDecompressedInputStreamWithBufferSize(CompressionCo return in; } + + public static Compressor getCompressor(CompressionCodec codec) { + synchronized (((Configurable) codec).getConf()) { + return CodecPool.getCompressor(codec); + } + } + + public static Decompressor getDecompressor(CompressionCodec codec) { + synchronized (((Configurable) codec).getConf()) { + return CodecPool.getDecompressor(codec); + } + } + + public static CompressionInputStream createInputStream(CompressionCodec codec, + InputStream checksumIn, Decompressor decompressor) throws IOException { + synchronized (((Configurable) codec).getConf()) { + return codec.createInputStream(checksumIn, decompressor); + } + } + + public static CompressionOutputStream createOutputStream(CompressionCodec codec, + OutputStream checksumOut, Compressor compressor) throws IOException { + synchronized (((Configurable) codec).getConf()) { + return codec.createOutputStream(checksumOut, compressor); + } + } + + public static String getBufferSizeProperty(CompressionCodec codec) { + return getBufferSizeProperty(codec.getClass().getName()); + } + + public static String getBufferSizeProperty(String codecClassName) { + switch (codecClassName) { + case "org.apache.hadoop.io.compress.DefaultCodec": + case "org.apache.hadoop.io.compress.BZip2Codec": + case "org.apache.hadoop.io.compress.GzipCodec": + return CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY; + case "org.apache.hadoop.io.compress.SnappyCodec": + return CommonConfigurationKeys.IO_COMPRESSION_CODEC_SNAPPY_BUFFERSIZE_KEY; + case "org.apache.hadoop.io.compress.ZStandardCodec": + return CommonConfigurationKeys.IO_COMPRESSION_CODEC_ZSTD_BUFFER_SIZE_KEY; + case "org.apache.hadoop.io.compress.LzoCodec": + case "com.hadoop.compression.lzo.LzoCodec": + return CommonConfigurationKeys.IO_COMPRESSION_CODEC_LZO_BUFFERSIZE_KEY; + case "org.apache.hadoop.io.compress.Lz4Codec": + return CommonConfigurationKeys.IO_COMPRESSION_CODEC_LZ4_BUFFERSIZE_KEY; + default: + return null; + } + } + + public static int getDefaultBufferSize(Configuration conf, CompressionCodec codec) { + return getDefaultBufferSize(conf, codec.getClass().getName()); + } + + public static int getDefaultBufferSize(Configuration conf, String codecClassName) { + switch (codecClassName) { + case "org.apache.hadoop.io.compress.DefaultCodec": + case "org.apache.hadoop.io.compress.BZip2Codec": + case "org.apache.hadoop.io.compress.GzipCodec": + return CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_DEFAULT; + case "org.apache.hadoop.io.compress.SnappyCodec": + return CommonConfigurationKeys.IO_COMPRESSION_CODEC_SNAPPY_BUFFERSIZE_DEFAULT; + case "org.apache.hadoop.io.compress.ZStandardCodec": + return CommonConfigurationKeys.IO_COMPRESSION_CODEC_ZSTD_BUFFER_SIZE_DEFAULT; + case "org.apache.hadoop.io.compress.LzoCodec": + case "com.hadoop.compression.lzo.LzoCodec": + return CommonConfigurationKeys.IO_COMPRESSION_CODEC_LZO_BUFFERSIZE_DEFAULT; + case "org.apache.hadoop.io.compress.Lz4Codec": + return CommonConfigurationKeys.IO_COMPRESSION_CODEC_LZ4_BUFFERSIZE_DEFAULT; + default: + return DEFAULT_BUFFER_SIZE; + } + } } \ No newline at end of file diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/DummyCompressionCodec.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/DummyCompressionCodec.java index 962a9e0207..530b9a3732 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/DummyCompressionCodec.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/DummyCompressionCodec.java @@ -18,12 +18,16 @@ package org.apache.tez.runtime.library.common.shuffle.orderedgrouped; +import org.apache.hadoop.conf.Configurable; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.io.compress.CompressionCodec; import org.apache.hadoop.io.compress.CompressionInputStream; import org.apache.hadoop.io.compress.CompressionOutputStream; import org.apache.hadoop.io.compress.Compressor; import org.apache.hadoop.io.compress.Decompressor; +import com.google.common.annotations.VisibleForTesting; + import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; @@ -33,7 +37,10 @@ /** * A dummy codec. It passes everything to underlying stream */ -public class DummyCompressionCodec implements CompressionCodec { +public class DummyCompressionCodec implements CompressionCodec, Configurable { + @VisibleForTesting + int createInputStreamCalled = 0; + private Configuration conf; @Override public CompressionOutputStream createOutputStream(OutputStream out) throws IOException { @@ -62,6 +69,7 @@ public CompressionInputStream createInputStream(InputStream in) throws IOExcepti @Override public CompressionInputStream createInputStream(InputStream in, Decompressor decompressor) throws IOException { + createInputStreamCalled += 1; return new DummyCompressionInputStream(in); } @@ -128,4 +136,14 @@ public void resetState() throws IOException { //no-op } } + + @Override + public void setConf(Configuration conf) { + this.conf = conf; + } + + @Override + public Configuration getConf() { + return conf; + } } diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestMergeManager.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestMergeManager.java index 13f090cffb..dde067beb8 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestMergeManager.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestMergeManager.java @@ -21,7 +21,6 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertTrue; -import static org.mockito.Matchers.any; import static org.mockito.Mockito.atLeastOnce; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; @@ -37,7 +36,6 @@ import com.google.common.collect.Sets; -import org.apache.hadoop.io.compress.CompressionCodec; import org.apache.hadoop.io.serializer.WritableSerialization; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -273,7 +271,8 @@ public void testDiskMergeWithCodec() throws Throwable { InputContext inputContext = createMockInputContext(UUID.randomUUID().toString()); // Create a mock compressor. We will check if it is used. - CompressionCodec dummyCodec = spy(new DummyCompressionCodec()); + DummyCompressionCodec dummyCodec = new DummyCompressionCodec(); + dummyCodec.setConf(conf); MergeManager mergeManager = new MergeManager(conf, localFs, localDirAllocator, inputContext, null, null, null, null, @@ -312,7 +311,7 @@ public void testDiskMergeWithCodec() throws Throwable { mo4.commit(); mergeManager.close(true); - verify(dummyCodec, atLeastOnce()).createOutputStream(any(), any()); + Assert.assertTrue(dummyCodec.createInputStreamCalled > 0); } @Test(timeout = 60000l) diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestIFile.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestIFile.java index bf35955625..960aee345a 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestIFile.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestIFile.java @@ -57,7 +57,6 @@ import org.apache.hadoop.util.NativeCodeLoader; import org.apache.tez.common.TezRuntimeFrameworkConfigs; import org.apache.tez.runtime.library.common.InputAttemptIdentifier; -import org.apache.tez.runtime.library.common.TezRuntimeUtils; import org.apache.tez.runtime.library.common.shuffle.orderedgrouped.InMemoryReader; import org.apache.tez.runtime.library.common.shuffle.orderedgrouped.InMemoryWriter; import org.apache.tez.runtime.library.common.sort.impl.IFile.Reader; @@ -66,6 +65,7 @@ import org.apache.tez.runtime.library.testutils.KVDataGen; import org.apache.tez.runtime.library.testutils.KVDataGen.KVPair; import org.apache.tez.runtime.library.utils.BufferUtils; +import org.apache.tez.runtime.library.utils.CodecUtils; import org.junit.After; import org.junit.Assert; import org.junit.Assume; @@ -734,7 +734,7 @@ public void testReadToDisk() throws IOException { public void testInMemoryBufferSize() throws IOException { Configurable configurableCodec = (Configurable) codec; int originalCodecBufferSize = - configurableCodec.getConf().getInt(TezRuntimeUtils.getBufferSizeProperty(codec), -1); + configurableCodec.getConf().getInt(CodecUtils.getBufferSizeProperty(codec), -1); // for smaller amount of data, codec buffer should be sized according to compressed data length List data = KVDataGen.generateTestData(false, rnd.nextInt(100)); @@ -742,7 +742,7 @@ public void testInMemoryBufferSize() throws IOException { readAndVerifyData(writer.getRawLength(), writer.getCompressedLength(), data, codec); Assert.assertEquals(originalCodecBufferSize, // original size is repaired - configurableCodec.getConf().getInt(TezRuntimeUtils.getBufferSizeProperty(codec), 0)); + configurableCodec.getConf().getInt(CodecUtils.getBufferSizeProperty(codec), 0)); // buffer size cannot grow infinitely with compressed data size data = KVDataGen.generateTestDataOfKeySize(false, 20000, rnd.nextInt(100)); @@ -750,7 +750,7 @@ public void testInMemoryBufferSize() throws IOException { readAndVerifyData(writer.getRawLength(), writer.getCompressedLength(), data, codec); Assert.assertEquals(originalCodecBufferSize, // original size is repaired - configurableCodec.getConf().getInt(TezRuntimeUtils.getBufferSizeProperty(codec), 0)); + configurableCodec.getConf().getInt(CodecUtils.getBufferSizeProperty(codec), 0)); } @Test(expected = IllegalArgumentException.class) @@ -766,7 +766,7 @@ private void tryWriteFileWithBufferSize(int bufferSize, String codecClassName) Configuration conf = new Configuration(); System.out.println("trying with buffer size: " + bufferSize); - conf.set(TezRuntimeUtils.getBufferSizeProperty(codecClassName), Integer.toString(bufferSize)); + conf.set(CodecUtils.getBufferSizeProperty(codecClassName), Integer.toString(bufferSize)); CompressionCodecFactory codecFactory = new CompressionCodecFactory(conf); CompressionCodec codecToTest = codecFactory.getCodecByClassName(codecClassName); diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/utils/TestCodecUtils.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/utils/TestCodecUtils.java new file mode 100644 index 0000000000..afa645974f --- /dev/null +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/utils/TestCodecUtils.java @@ -0,0 +1,224 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.utils; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.lang.reflect.Field; +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 org.apache.hadoop.conf.Configurable; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.CommonConfigurationKeys; +import org.apache.hadoop.fs.CommonConfigurationKeysPublic; +import org.apache.hadoop.io.compress.BZip2Codec; +import org.apache.hadoop.io.compress.CodecPool; +import org.apache.hadoop.io.compress.CompressionCodec; +import org.apache.hadoop.io.compress.CompressionInputStream; +import org.apache.hadoop.io.compress.CompressionOutputStream; +import org.apache.hadoop.io.compress.Compressor; +import org.apache.hadoop.io.compress.Decompressor; +import org.apache.hadoop.io.compress.DecompressorStream; +import org.apache.hadoop.io.compress.DefaultCodec; +import org.apache.hadoop.io.compress.GzipCodec; +import org.apache.hadoop.io.compress.Lz4Codec; +import org.apache.hadoop.io.compress.SnappyCodec; +import org.apache.hadoop.io.compress.ZStandardCodec; +import org.apache.tez.runtime.library.api.TezRuntimeConfiguration; +import org.apache.tez.runtime.library.common.shuffle.orderedgrouped.DummyCompressionCodec; +import org.apache.tez.runtime.library.common.sort.impl.IFileInputStream; +import org.junit.Assert; +import org.junit.Test; +import org.mockito.Mockito; + +public class TestCodecUtils { + + @Test + public void testConcurrentDecompressorCreationWithModifiedBuffersize() throws Exception { + testConcurrentDecompressorCreationWithModifiedBuffersizeOnCodec(new DefaultCodec()); + } + + private void testConcurrentDecompressorCreationWithModifiedBuffersizeOnCodec( + CompressionCodec codec) throws InterruptedException, ExecutionException { + int modifiedBufferSize = 1000; + int numberOfThreads = 1000; + + ExecutorService service = Executors.newFixedThreadPool(numberOfThreads); + + Configuration conf = new Configuration(); + conf.setBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_COMPRESS, true); + ((Configurable) codec).setConf(conf); + + Future[] futures = new Future[numberOfThreads]; + final CountDownLatch latch = new CountDownLatch(1); + + for (int i = 0; i < numberOfThreads; i++) { + futures[i] = service.submit(() -> { + try { + waitForLatch(latch); + + Decompressor decompressor = CodecUtils.getDecompressor(codec); + DecompressorStream stream = + (DecompressorStream) CodecUtils.getDecompressedInputStreamWithBufferSize(codec, + Mockito.mock(IFileInputStream.class), decompressor, modifiedBufferSize); + + Assert.assertEquals("stream buffer size is incorrect", modifiedBufferSize, + getBufferSize(stream)); + + CodecPool.returnDecompressor(decompressor); + } catch (IOException e) { + throw new RuntimeException(e); + } + }); + } + latch.countDown(); + + for (Future f : futures) { + f.get(); + } + } + + @Test + public void testConcurrentCompressorDecompressorCreation() throws Exception { + testConcurrentCompressorDecompressorCreationOnCodec(new DefaultCodec()); + } + + private void testConcurrentCompressorDecompressorCreationOnCodec(CompressionCodec codec) + throws IOException, InterruptedException, ExecutionException { + int modifiedBufferSize = 1000; + int numberOfThreads = 1000; + + ExecutorService service = Executors.newFixedThreadPool(numberOfThreads); + + Configuration conf = new Configuration(); + conf.setBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_COMPRESS, true); + ((Configurable) codec).setConf(conf); + + Future[] futures = new Future[numberOfThreads]; + final CountDownLatch latch = new CountDownLatch(1); + + for (int i = 0; i < numberOfThreads; i++) { + // let's "randomly" choose from scenarios and test them concurrently + // 1. getDecompressedInputStreamWithBufferSize + if (i % 3 == 0) { + futures[i] = service.submit(() -> { + try { + waitForLatch(latch); + + Decompressor decompressor = CodecUtils.getDecompressor(codec); + CompressionInputStream stream = + (CompressionInputStream) CodecUtils.getDecompressedInputStreamWithBufferSize(codec, + Mockito.mock(IFileInputStream.class), decompressor, modifiedBufferSize); + + Assert.assertEquals("stream buffer size is incorrect", modifiedBufferSize, + getBufferSize(stream)); + + CodecPool.returnDecompressor(decompressor); + } catch (IOException e) { + throw new RuntimeException(e); + } + }); + // 2. getCompressor + } else if (i % 3 == 1) { + futures[i] = service.submit(() -> { + try { + waitForLatch(latch); + + Compressor compressor = CodecUtils.getCompressor(codec); + CompressionOutputStream stream = + CodecUtils.createOutputStream(codec, Mockito.mock(OutputStream.class), compressor); + + Assert.assertEquals("stream buffer size is incorrect", + CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_DEFAULT, getBufferSize(stream)); + + CodecPool.returnCompressor(compressor); + } catch (Exception e) { + throw new RuntimeException(e); + } + }); + // 3. getDecompressor + } else if (i % 3 == 2) { + futures[i] = service.submit(() -> { + try { + waitForLatch(latch); + + Decompressor decompressor = CodecUtils.getDecompressor(codec); + CompressionInputStream stream = + CodecUtils.createInputStream(codec, Mockito.mock(InputStream.class), decompressor); + + Assert.assertEquals("stream buffer size is incorrect", + CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_DEFAULT, getBufferSize(stream)); + + CodecPool.returnDecompressor(decompressor); + } catch (Exception e) { + throw new RuntimeException(e); + } + }); + } + } + latch.countDown(); + + for (Future f : futures) { + f.get(); + } + } + + @Test + public void testDefaultBufferSize() { + Configuration conf = new Configuration(); // config with no buffersize set + + Assert.assertEquals(CodecUtils.DEFAULT_BUFFER_SIZE, + CodecUtils.getDefaultBufferSize(conf, new DummyCompressionCodec())); + Assert.assertEquals(CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_DEFAULT, + CodecUtils.getDefaultBufferSize(conf, new DefaultCodec())); + Assert.assertEquals(CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_DEFAULT, + CodecUtils.getDefaultBufferSize(conf, new BZip2Codec())); + Assert.assertEquals(CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_DEFAULT, + CodecUtils.getDefaultBufferSize(conf, new GzipCodec())); + Assert.assertEquals(CommonConfigurationKeys.IO_COMPRESSION_CODEC_SNAPPY_BUFFERSIZE_DEFAULT, + CodecUtils.getDefaultBufferSize(conf, new SnappyCodec())); + Assert.assertEquals(CommonConfigurationKeys.IO_COMPRESSION_CODEC_ZSTD_BUFFER_SIZE_DEFAULT, + CodecUtils.getDefaultBufferSize(conf, new ZStandardCodec())); + Assert.assertEquals(CommonConfigurationKeys.IO_COMPRESSION_CODEC_LZ4_BUFFERSIZE_DEFAULT, + CodecUtils.getDefaultBufferSize(conf, new Lz4Codec())); + } + + private void waitForLatch(CountDownLatch latch) { + try { + latch.await(); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + } + + private int getBufferSize(Object stream) { + try { + Field field = stream.getClass().getDeclaredField("buffer"); + field.setAccessible(true); + byte[] buffer = (byte[]) field.get(stream); + return buffer.length; + } catch (Exception e) { + throw new RuntimeException(e); + } + } +} From d443ae2b30fae8a80c98e9e16ec98313ae14dca1 Mon Sep 17 00:00:00 2001 From: Bodor Laszlo Date: Tue, 22 Jun 2021 22:43:08 +0200 Subject: [PATCH 053/137] TEZ-4298: ShuffleHandler is not source compatible with Hadoop 3.3.0 (#129) --- tez-plugins/tez-aux-services/findbugs-exclude.xml | 7 ++++++- .../java/org/apache/tez/auxservices/ShuffleHandler.java | 6 +++--- 2 files changed, 9 insertions(+), 4 deletions(-) diff --git a/tez-plugins/tez-aux-services/findbugs-exclude.xml b/tez-plugins/tez-aux-services/findbugs-exclude.xml index 5b11308f6d..adfd7041a5 100644 --- a/tez-plugins/tez-aux-services/findbugs-exclude.xml +++ b/tez-plugins/tez-aux-services/findbugs-exclude.xml @@ -12,5 +12,10 @@ limitations under the License. See accompanying LICENSE file. --> - + + + + + + diff --git a/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java b/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java index e73805a1f0..fb28a0f4b1 100644 --- a/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java +++ b/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java @@ -153,7 +153,6 @@ import com.google.common.cache.RemovalListener; import com.google.common.cache.RemovalNotification; import com.google.common.cache.Weigher; -import com.google.protobuf.ByteString; public class ShuffleHandler extends AuxiliaryService { @@ -796,9 +795,10 @@ private void recoverJobShuffleInfo(String jobIdStr, byte[] data) private void recordJobShuffleInfo(JobID jobId, String user, Token jobToken) throws IOException { if (stateDb != null) { + // Discover type instead of assuming ByteString to allow for shading. TokenProto tokenProto = TokenProto.newBuilder() - .setIdentifier(ByteString.copyFrom(jobToken.getIdentifier())) - .setPassword(ByteString.copyFrom(jobToken.getPassword())) + .setIdentifier(TokenProto.getDefaultInstance().getIdentifier().copyFrom(jobToken.getIdentifier())) + .setPassword(TokenProto.getDefaultInstance().getPassword().copyFrom(jobToken.getPassword())) .setKind(jobToken.getKind().toString()) .setService(jobToken.getService().toString()) .build(); From 26b86e7646021c3bdf1e54a7ee225f46397eba42 Mon Sep 17 00:00:00 2001 From: Kuhu Shukla Date: Mon, 28 Jun 2021 10:15:42 -0500 Subject: [PATCH 054/137] TEZ-3918. Setting tez.task.log.level does not work Signed-off-by: Jonathan Eagles --- .../org/apache/tez/client/TezClientUtils.java | 22 +++++++++++-------- .../apache/tez/client/TestTezClientUtils.java | 10 +++++++++ 2 files changed, 23 insertions(+), 9 deletions(-) diff --git a/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java b/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java index 389b58400b..d0fc374a81 100644 --- a/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java +++ b/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java @@ -760,13 +760,6 @@ static DAGPlan prepareAndCreateDAGPlan(DAG dag, AMConfiguration amConfig, static void maybeAddDefaultLoggingJavaOpts(String logLevel, List vargs) { Objects.requireNonNull(vargs); - if (!vargs.isEmpty()) { - for (String arg : vargs) { - if (arg.contains(TezConstants.TEZ_ROOT_LOGGER_NAME)) { - return; - } - } - } TezClientUtils.addLog4jSystemProperties(logLevel, vargs); } @@ -829,8 +822,19 @@ public static void addLog4jSystemProperties(String logLevel, + TezConstants.TEZ_CONTAINER_LOG4J_PROPERTIES_FILE); vargs.add("-D" + YarnConfiguration.YARN_APP_CONTAINER_LOG_DIR + "=" + ApplicationConstants.LOG_DIR_EXPANSION_VAR); - vargs.add("-D" + TezConstants.TEZ_ROOT_LOGGER_NAME + "=" + logLevel - + "," + TezConstants.TEZ_CONTAINER_LOGGER_NAME); + boolean isRootLoggerPresent = false; + String rootLoggerArg = "-D" + TezConstants.TEZ_ROOT_LOGGER_NAME + "=" + logLevel + + "," + TezConstants.TEZ_CONTAINER_LOGGER_NAME; + for (int i = 0; i < vargs.size(); i++) { + String arg = vargs.get(i); + if (arg.contains(TezConstants.TEZ_ROOT_LOGGER_NAME)) { + vargs.set(i, rootLoggerArg); + isRootLoggerPresent = true; + } + } + if (!isRootLoggerPresent) { + vargs.add(rootLoggerArg); + } } static ConfigurationProto createFinalConfProtoForApp(Configuration amConf, diff --git a/tez-api/src/test/java/org/apache/tez/client/TestTezClientUtils.java b/tez-api/src/test/java/org/apache/tez/client/TestTezClientUtils.java index 29e9210538..29a6769716 100644 --- a/tez-api/src/test/java/org/apache/tez/client/TestTezClientUtils.java +++ b/tez-api/src/test/java/org/apache/tez/client/TestTezClientUtils.java @@ -682,6 +682,16 @@ public void testDefaultLoggingJavaOpts() { javaOpts.contains("-Dlog4j.configuratorClass=org.apache.tez.common.TezLog4jConfigurator")); } + @Test + public void testDefaultLoggingJavaOptsWithRootLogger() { + String origJavaOpts = "-D" + TezConstants.TEZ_ROOT_LOGGER_NAME + "=INFO"; + String javaOpts = TezClientUtils.maybeAddDefaultLoggingJavaOpts("FOOBAR", origJavaOpts); + Assert.assertNotNull(javaOpts); + Assert.assertTrue(javaOpts.contains("-D" + TezConstants.TEZ_ROOT_LOGGER_NAME + "=FOOBAR")); + Assert.assertTrue(javaOpts.contains(TezConstants.TEZ_CONTAINER_LOG4J_PROPERTIES_FILE) + && javaOpts.contains("-Dlog4j.configuratorClass=org.apache.tez.common.TezLog4jConfigurator")); + } + @Test (timeout = 5000) public void testConfSerializationForAm() { Configuration conf =new Configuration(false); From 984d09c4d37fe7b73d63bf0ce9d2de7c34bdd735 Mon Sep 17 00:00:00 2001 From: belugabehr <12578579+belugabehr@users.noreply.github.com> Date: Wed, 30 Jun 2021 10:48:30 -0400 Subject: [PATCH 055/137] TEZ-4275: Use Google Guava Intern Facility (#95) * TEZ-4275: Use Google Guava Intern Facility * Also add hint for running finalization * Use Guava String Intern implementation * Fix checkstyle, white-space issues * Remove GC hint Co-authored-by: David Mollitor --- .../org/apache/tez/dag/records/TezDAGID.java | 15 ++++---- .../org/apache/tez/dag/records/TezID.java | 21 ----------- .../tez/dag/records/TezTaskAttemptID.java | 12 +++---- .../org/apache/tez/dag/records/TezTaskID.java | 12 +++---- .../apache/tez/dag/records/TezVertexID.java | 12 +++---- .../org/apache/tez/util/StringInterner.java | 36 +++++++++++++++++++ .../org/apache/tez/dag/app/DAGAppMaster.java | 6 ---- .../tez/dag/app/dag/impl/TaskAttemptImpl.java | 12 +++---- .../tez/dag/app/dag/impl/VertexImpl.java | 6 ++-- .../tez/history/parser/datamodel/DagInfo.java | 8 ++--- .../parser/datamodel/TaskAttemptInfo.java | 16 ++++----- .../history/parser/datamodel/TaskInfo.java | 10 +++--- .../history/parser/datamodel/VertexInfo.java | 10 +++--- .../tez/history/parser/utils/Utils.java | 4 +-- .../tez/runtime/api/impl/EventMetaData.java | 10 +++--- .../tez/runtime/api/impl/GroupInputSpec.java | 8 ++--- .../tez/runtime/api/impl/InputSpec.java | 6 ++-- .../tez/runtime/api/impl/OutputSpec.java | 6 ++-- .../apache/tez/runtime/api/impl/TaskSpec.java | 14 ++++---- .../tez/runtime/api/impl/TaskStatistics.java | 5 +-- ...huffleInputEventHandlerOrderedGrouped.java | 10 +++--- .../sort/impl/dflt/TestDefaultSorter.java | 4 +-- .../plugins/CriticalPathAnalyzer.java | 9 +++-- 23 files changed, 127 insertions(+), 125 deletions(-) create mode 100644 tez-common/src/main/java/org/apache/tez/util/StringInterner.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 24365c95e2..68184fc8c0 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 @@ -22,12 +22,14 @@ import java.io.DataOutput; import java.io.IOException; -import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.tez.common.Preconditions; import org.apache.tez.util.FastNumberFormat; +import com.google.common.collect.Interner; +import com.google.common.collect.Interners; + /** * TezDAGID represents the immutable and unique identifier for * a Tez DAG. @@ -40,7 +42,7 @@ */ public class TezDAGID extends TezID { - private static TezIDCache tezDAGIDCache = new TezIDCache<>(); + private static Interner tezDAGIDCache = Interners.newWeakInterner(); private ApplicationId applicationId; /** @@ -53,12 +55,7 @@ public static TezDAGID getInstance(ApplicationId applicationId, int id) { // will be short-lived. // Alternately the cache can be keyed by the hash of the incoming paramters. Preconditions.checkArgument(applicationId != null, "ApplicationID cannot be null"); - return tezDAGIDCache.getInstance(new TezDAGID(applicationId, id)); - } - - @InterfaceAudience.Private - public static void clearCache() { - tezDAGIDCache.clear(); + return tezDAGIDCache.intern(new TezDAGID(applicationId, id)); } /** @@ -72,7 +69,7 @@ public static TezDAGID getInstance(String yarnRMIdentifier, int appId, int id) { // will be short-lived. // Alternately the cache can be keyed by the hash of the incoming paramters. Preconditions.checkArgument(yarnRMIdentifier != null, "yarnRMIdentifier cannot be null"); - return tezDAGIDCache.getInstance(new TezDAGID(yarnRMIdentifier, appId, id)); + return tezDAGIDCache.intern(new TezDAGID(yarnRMIdentifier, appId, id)); } // Public for Writable serialization. Verify if this is actually required. diff --git a/tez-common/src/main/java/org/apache/tez/dag/records/TezID.java b/tez-common/src/main/java/org/apache/tez/dag/records/TezID.java index cd7b27de45..7efbd9a889 100644 --- a/tez-common/src/main/java/org/apache/tez/dag/records/TezID.java +++ b/tez-common/src/main/java/org/apache/tez/dag/records/TezID.java @@ -21,8 +21,6 @@ import java.io.DataInput; import java.io.DataOutput; import java.io.IOException; -import java.lang.ref.WeakReference; -import java.util.WeakHashMap; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; @@ -42,25 +40,6 @@ public abstract class TezID implements WritableComparable { public static final char SEPARATOR = '_'; protected int id; - public static class TezIDCache { - private final WeakHashMap> cache = new WeakHashMap<>(); - - synchronized T getInstance(final T id) { - final WeakReference cached = cache.get(id); - if (cached != null) { - final T value = cached.get(); - if (value != null) - return value; - } - cache.put(id, new WeakReference(id)); - return id; - } - - synchronized void clear() { - cache.clear(); - } - } - /** constructs an ID object from the given int */ public TezID(int id) { this.id = id; diff --git a/tez-common/src/main/java/org/apache/tez/dag/records/TezTaskAttemptID.java b/tez-common/src/main/java/org/apache/tez/dag/records/TezTaskAttemptID.java index 7aee80f4d6..9ce1b10aa0 100644 --- a/tez-common/src/main/java/org/apache/tez/dag/records/TezTaskAttemptID.java +++ b/tez-common/src/main/java/org/apache/tez/dag/records/TezTaskAttemptID.java @@ -25,6 +25,9 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; +import com.google.common.collect.Interner; +import com.google.common.collect.Interners; + /** * TezTaskAttemptID represents the immutable and unique identifier for * a task attempt. Each task attempt is one particular instance of a Tez Task @@ -46,7 +49,7 @@ public class TezTaskAttemptID extends TezID { public static final String ATTEMPT = "attempt"; private TezTaskID taskId; - private static TezIDCache tezTaskAttemptIDCache = new TezIDCache<>(); + private static Interner tezTaskAttemptIDCache = Interners.newWeakInterner(); // Public for Writable serialization. Verify if this is actually required. public TezTaskAttemptID() { @@ -58,12 +61,7 @@ public TezTaskAttemptID() { * @param id the task attempt number */ public static TezTaskAttemptID getInstance(TezTaskID taskID, int id) { - return tezTaskAttemptIDCache.getInstance(new TezTaskAttemptID(taskID, id)); - } - - @InterfaceAudience.Private - public static void clearCache() { - tezTaskAttemptIDCache.clear(); + return tezTaskAttemptIDCache.intern(new TezTaskAttemptID(taskID, id)); } private TezTaskAttemptID(TezTaskID taskId, int id) { diff --git a/tez-common/src/main/java/org/apache/tez/dag/records/TezTaskID.java b/tez-common/src/main/java/org/apache/tez/dag/records/TezTaskID.java index 15b695c7a5..993df7c87a 100644 --- a/tez-common/src/main/java/org/apache/tez/dag/records/TezTaskID.java +++ b/tez-common/src/main/java/org/apache/tez/dag/records/TezTaskID.java @@ -28,6 +28,9 @@ import org.apache.tez.common.Preconditions; import org.apache.tez.util.FastNumberFormat; +import com.google.common.collect.Interner; +import com.google.common.collect.Interners; + /** * TaskID represents the immutable and unique identifier for * a Tez Task. Each TaskID encompasses multiple attempts made to @@ -51,7 +54,7 @@ public FastNumberFormat initialValue() { } }; - private static TezIDCache tezTaskIDCache = new TezIDCache<>(); + private static Interner tezTaskIDCache = Interners.newWeakInterner(); private TezVertexID vertexId; /** @@ -61,12 +64,7 @@ public FastNumberFormat initialValue() { */ public static TezTaskID getInstance(TezVertexID vertexID, int id) { Preconditions.checkArgument(vertexID != null, "vertexID cannot be null"); - return tezTaskIDCache.getInstance(new TezTaskID(vertexID, id)); - } - - @InterfaceAudience.Private - public static void clearCache() { - tezTaskIDCache.clear(); + return tezTaskIDCache.intern(new TezTaskID(vertexID, id)); } private TezTaskID(TezVertexID vertexID, int id) { diff --git a/tez-common/src/main/java/org/apache/tez/dag/records/TezVertexID.java b/tez-common/src/main/java/org/apache/tez/dag/records/TezVertexID.java index b5a36abe33..5ecfca6a49 100644 --- a/tez-common/src/main/java/org/apache/tez/dag/records/TezVertexID.java +++ b/tez-common/src/main/java/org/apache/tez/dag/records/TezVertexID.java @@ -28,6 +28,9 @@ import org.apache.tez.common.Preconditions; import org.apache.tez.util.FastNumberFormat; +import com.google.common.collect.Interner; +import com.google.common.collect.Interners; + /** * TezVertexID represents the immutable and unique identifier for * a Vertex in a Tez DAG. Each TezVertexID encompasses multiple Tez Tasks. @@ -53,7 +56,7 @@ public FastNumberFormat initialValue() { } }; - private static TezIDCache tezVertexIDCache = new TezIDCache<>(); + private static Interner tezVertexIDCache = Interners.newWeakInterner(); private TezDAGID dagId; // Public for Writable serialization. Verify if this is actually required. @@ -67,12 +70,7 @@ public TezVertexID() { */ public static TezVertexID getInstance(TezDAGID dagId, int id) { Preconditions.checkArgument(dagId != null, "DagID cannot be null"); - return tezVertexIDCache.getInstance(new TezVertexID(dagId, id)); - } - - @InterfaceAudience.Private - public static void clearCache() { - tezVertexIDCache.clear(); + return tezVertexIDCache.intern(new TezVertexID(dagId, id)); } private TezVertexID(TezDAGID dagId, int id) { diff --git a/tez-common/src/main/java/org/apache/tez/util/StringInterner.java b/tez-common/src/main/java/org/apache/tez/util/StringInterner.java new file mode 100644 index 0000000000..b8c911307c --- /dev/null +++ b/tez-common/src/main/java/org/apache/tez/util/StringInterner.java @@ -0,0 +1,36 @@ +/* + * 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 com.google.common.collect.Interner; +import com.google.common.collect.Interners; + +/** + * A class to replace the {@code String.intern()}. The {@code String.intern()} + * has some well-known performance limitations, and should generally be avoided. + * Prefer Google's interner over the JDK's implementation. + */ +public final class StringInterner { + + private static final Interner STRING_INTERNER = + Interners.newWeakInterner(); + + private StringInterner() { + } + + public static String intern(final String str) { + return (str == null) ? null : STRING_INTERNER.intern(str); + } +} 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 1352b68f26..61e1cb684a 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 @@ -84,8 +84,6 @@ import org.apache.tez.dag.app.dag.event.DAGEventInternalError; import org.apache.tez.dag.app.dag.event.DAGEventTerminateDag; import org.apache.tez.dag.history.events.DAGRecoveredEvent; -import org.apache.tez.dag.records.TezTaskAttemptID; -import org.apache.tez.dag.records.TezTaskID; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -869,10 +867,6 @@ protected synchronized void handle(DAGAppMasterEvent event) { taskCommunicatorManager.dagComplete(cleanupEvent.getDag()); nodes.dagComplete(cleanupEvent.getDag()); containers.dagComplete(cleanupEvent.getDag()); - TezTaskAttemptID.clearCache(); - TezTaskID.clearCache(); - TezVertexID.clearCache(); - TezDAGID.clearCache(); LOG.info("Completed cleanup for DAG: name=" + cleanupEvent.getDag().getName() + ", with id=" + cleanupEvent.getDag().getID()); synchronized (idleStateLock) { 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 8b16b2e8b7..593ea6cabf 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java @@ -41,11 +41,11 @@ import org.apache.tez.dag.app.rm.AMSchedulerEventTAStateUpdated; import org.apache.tez.runtime.api.TaskFailureType; import org.apache.tez.serviceplugins.api.TaskScheduler; +import org.apache.tez.util.StringInterner; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.net.NetUtils; -import org.apache.hadoop.util.StringInterner; import org.apache.hadoop.yarn.api.records.Container; import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.NodeId; @@ -1399,7 +1399,7 @@ public void transition(TaskAttemptImpl ta, TaskAttemptEvent event) { ta.container = container; ta.containerId = tEvent.getContainerId(); ta.containerNodeId = container.getNodeId(); - ta.nodeHttpAddress = StringInterner.weakIntern(container.getNodeHttpAddress()); + ta.nodeHttpAddress = StringInterner.intern(container.getNodeHttpAddress()); } if (event instanceof TaskAttemptEventContainerTerminatedBySystem) { @@ -1411,7 +1411,7 @@ public void transition(TaskAttemptImpl ta, TaskAttemptEvent event) { ta.container = container; ta.containerId = tEvent.getContainerId(); ta.containerNodeId = container.getNodeId(); - ta.nodeHttpAddress = StringInterner.weakIntern(container.getNodeHttpAddress()); + ta.nodeHttpAddress = StringInterner.intern(container.getNodeHttpAddress()); } if (ta.recoveryData == null || @@ -1453,8 +1453,8 @@ public void transition(TaskAttemptImpl ta, TaskAttemptEvent origEvent) { ta.container = container; ta.containerId = event.getContainerId(); ta.containerNodeId = container.getNodeId(); - ta.nodeHttpAddress = StringInterner.weakIntern(container.getNodeHttpAddress()); - ta.nodeRackName = StringInterner.weakIntern(RackResolver.resolve(ta.containerNodeId.getHost()) + ta.nodeHttpAddress = StringInterner.intern(container.getNodeHttpAddress()); + ta.nodeRackName = StringInterner.intern(RackResolver.resolve(ta.containerNodeId.getHost()) .getNetworkLocation()); ta.lastNotifyProgressTimestamp = ta.clock.getTime(); @@ -1463,7 +1463,7 @@ public void transition(TaskAttemptImpl ta, TaskAttemptEvent origEvent) { // TODO Resolve to host / IP in case of a local address. InetSocketAddress nodeHttpInetAddr = NetUtils .createSocketAddr(ta.nodeHttpAddress); // TODO: Costly? - ta.trackerName = StringInterner.weakIntern(nodeHttpInetAddr.getHostName()); + ta.trackerName = StringInterner.intern(nodeHttpInetAddr.getHostName()); ta.httpPort = nodeHttpInetAddr.getPort(); ta.sendEvent(createDAGCounterUpdateEventTALaunched(ta)); 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 452dae5366..912339e15e 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 @@ -52,7 +52,6 @@ import org.apache.hadoop.service.AbstractService; import org.apache.hadoop.service.ServiceOperations; import org.apache.hadoop.service.ServiceStateException; -import org.apache.hadoop.util.StringInterner; import org.apache.hadoop.yarn.api.records.LocalResource; import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.event.EventHandler; @@ -190,6 +189,7 @@ import org.apache.tez.runtime.api.impl.TezEvent; import org.apache.tez.state.OnStateChangedCallback; import org.apache.tez.state.StateMachineTez; +import org.apache.tez.util.StringInterner; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -965,7 +965,7 @@ public VertexImpl(TezVertexID vertexId, VertexPlan vertexPlan, StateChangeNotifier entityStatusTracker, Configuration dagOnlyConf) { this.vertexId = vertexId; this.vertexPlan = vertexPlan; - this.vertexName = StringInterner.weakIntern(vertexName); + this.vertexName = StringInterner.intern(vertexName); this.vertexConf = new Configuration(dagConf); this.vertexOnlyConf = new Configuration(dagOnlyConf); if (vertexPlan.hasVertexConf()) { @@ -4514,7 +4514,7 @@ public Resource getTaskResource() { } void addIO(String name) { - ioIndices.put(StringInterner.weakIntern(name), ioIndices.size()); + ioIndices.put(StringInterner.intern(name), ioIndices.size()); } @VisibleForTesting diff --git a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/DagInfo.java b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/DagInfo.java index 85fcfcfb19..5067ec7aa8 100644 --- a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/DagInfo.java +++ b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/DagInfo.java @@ -32,10 +32,10 @@ import org.apache.commons.collections4.bidimap.DualHashBidiMap; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.util.StringInterner; import org.apache.tez.client.CallerContext; import org.apache.tez.dag.api.event.VertexState; import org.apache.tez.dag.history.HistoryEventType; +import org.apache.tez.util.StringInterner; import org.codehaus.jettison.json.JSONArray; import org.codehaus.jettison.json.JSONException; import org.codehaus.jettison.json.JSONObject; @@ -100,7 +100,7 @@ public class DagInfo extends BaseInfo { Preconditions.checkArgument(jsonObject.getString(Constants.ENTITY_TYPE).equalsIgnoreCase (Constants.TEZ_DAG_ID)); - dagId = StringInterner.weakIntern(jsonObject.getString(Constants.ENTITY)); + dagId = StringInterner.intern(jsonObject.getString(Constants.ENTITY)); //Parse additional Info JSONObject otherInfoNode = jsonObject.getJSONObject(Constants.OTHER_INFO); @@ -140,7 +140,7 @@ public class DagInfo extends BaseInfo { diagnostics = otherInfoNode.optString(Constants.DIAGNOSTICS); failedTasks = otherInfoNode.optInt(Constants.NUM_FAILED_TASKS); JSONObject dagPlan = otherInfoNode.optJSONObject(Constants.DAG_PLAN); - name = StringInterner.weakIntern((dagPlan != null) ? (dagPlan.optString(Constants.DAG_NAME)) : null); + name = StringInterner.intern((dagPlan != null) ? (dagPlan.optString(Constants.DAG_NAME)) : null); if (dagPlan != null) { JSONArray vertices = dagPlan.optJSONArray(Constants.VERTICES); if (vertices != null) { @@ -152,7 +152,7 @@ public class DagInfo extends BaseInfo { } else { numVertices = 0; } - status = StringInterner.weakIntern(otherInfoNode.optString(Constants.STATUS)); + status = StringInterner.intern(otherInfoNode.optString(Constants.STATUS)); //parse name id mapping JSONObject vertexIDMappingJson = otherInfoNode.optJSONObject(Constants.VERTEX_NAME_ID_MAPPING); diff --git a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/TaskAttemptInfo.java b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/TaskAttemptInfo.java index 3ce39bdccf..c8d400cc35 100644 --- a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/TaskAttemptInfo.java +++ b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/TaskAttemptInfo.java @@ -26,7 +26,6 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.util.StringInterner; import org.apache.tez.common.ATSConstants; import org.apache.tez.common.counters.DAGCounter; import org.apache.tez.common.counters.TaskCounter; @@ -34,6 +33,7 @@ import org.apache.tez.dag.api.oldrecords.TaskAttemptState; import org.apache.tez.dag.history.HistoryEventType; import org.apache.tez.history.parser.utils.Utils; +import org.apache.tez.util.StringInterner; import org.codehaus.jettison.json.JSONException; import org.codehaus.jettison.json.JSONObject; @@ -51,7 +51,7 @@ public class TaskAttemptInfo extends BaseInfo { private static final Log LOG = LogFactory.getLog(TaskAttemptInfo.class); - private static final String SUCCEEDED = StringInterner.weakIntern(TaskAttemptState.SUCCEEDED.name()); + private static final String SUCCEEDED = TaskAttemptState.SUCCEEDED.name(); private final String taskAttemptId; private final long startTime; @@ -96,7 +96,7 @@ public String getTaskAttemptId() { jsonObject.getString(Constants.ENTITY_TYPE).equalsIgnoreCase (Constants.TEZ_TASK_ATTEMPT_ID)); - taskAttemptId = StringInterner.weakIntern(jsonObject.optString(Constants.ENTITY)); + taskAttemptId = StringInterner.intern(jsonObject.optString(Constants.ENTITY)); //Parse additional Info final JSONObject otherInfoNode = jsonObject.getJSONObject(Constants.OTHER_INFO); @@ -132,15 +132,15 @@ public String getTaskAttemptId() { diagnostics = otherInfoNode.optString(Constants.DIAGNOSTICS); creationTime = otherInfoNode.optLong(Constants.CREATION_TIME); - creationCausalTA = StringInterner.weakIntern( + creationCausalTA = StringInterner.intern( otherInfoNode.optString(Constants.CREATION_CAUSAL_ATTEMPT)); allocationTime = otherInfoNode.optLong(Constants.ALLOCATION_TIME); - containerId = StringInterner.weakIntern(otherInfoNode.optString(Constants.CONTAINER_ID)); + containerId = StringInterner.intern(otherInfoNode.optString(Constants.CONTAINER_ID)); String id = otherInfoNode.optString(Constants.NODE_ID); - nodeId = StringInterner.weakIntern((id != null) ? (id.split(":")[0]) : ""); + nodeId = StringInterner.intern((id != null) ? (id.split(":")[0]) : ""); logUrl = otherInfoNode.optString(Constants.COMPLETED_LOGS_URL); - status = StringInterner.weakIntern(otherInfoNode.optString(Constants.STATUS)); + status = StringInterner.intern(otherInfoNode.optString(Constants.STATUS)); container = new Container(containerId, nodeId); if (otherInfoNode.has(Constants.LAST_DATA_EVENTS)) { List eventInfo = Utils.parseDataEventDependencyFromJSON( @@ -154,7 +154,7 @@ public String getTaskAttemptId() { } } terminationCause = StringInterner - .weakIntern(otherInfoNode.optString(ATSConstants.TASK_ATTEMPT_ERROR_ENUM)); + .intern(otherInfoNode.optString(ATSConstants.TASK_ATTEMPT_ERROR_ENUM)); executionTimeInterval = (endTime > startTime) ? (endTime - startTime) : 0; } diff --git a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/TaskInfo.java b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/TaskInfo.java index de74ad29ad..43886fa5c6 100644 --- a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/TaskInfo.java +++ b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/TaskInfo.java @@ -31,9 +31,9 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.util.StringInterner; import org.apache.tez.dag.api.oldrecords.TaskAttemptState; import org.apache.tez.dag.history.HistoryEventType; +import org.apache.tez.util.StringInterner; import org.codehaus.jettison.json.JSONException; import org.codehaus.jettison.json.JSONObject; @@ -71,7 +71,7 @@ public class TaskInfo extends BaseInfo { jsonObject.getString(Constants.ENTITY_TYPE).equalsIgnoreCase (Constants.TEZ_TASK_ID)); - taskId = StringInterner.weakIntern(jsonObject.optString(Constants.ENTITY)); + taskId = StringInterner.intern(jsonObject.optString(Constants.ENTITY)); //Parse additional Info final JSONObject otherInfoNode = jsonObject.getJSONObject(Constants.OTHER_INFO); @@ -106,10 +106,10 @@ public class TaskInfo extends BaseInfo { endTime = eTime; diagnostics = otherInfoNode.optString(Constants.DIAGNOSTICS); - successfulAttemptId = StringInterner.weakIntern( - otherInfoNode.optString(Constants.SUCCESSFUL_ATTEMPT_ID)); + successfulAttemptId = StringInterner + .intern(otherInfoNode.optString(Constants.SUCCESSFUL_ATTEMPT_ID)); scheduledTime = otherInfoNode.optLong(Constants.SCHEDULED_TIME); - status = StringInterner.weakIntern(otherInfoNode.optString(Constants.STATUS)); + status = StringInterner.intern(otherInfoNode.optString(Constants.STATUS)); } @Override diff --git a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/VertexInfo.java b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/VertexInfo.java index efcce3bf76..c196c48cd4 100644 --- a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/VertexInfo.java +++ b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/VertexInfo.java @@ -30,9 +30,9 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.util.StringInterner; import org.apache.tez.dag.api.oldrecords.TaskState; import org.apache.tez.dag.history.HistoryEventType; +import org.apache.tez.util.StringInterner; import org.codehaus.jettison.json.JSONException; import org.codehaus.jettison.json.JSONObject; @@ -91,7 +91,7 @@ public class VertexInfo extends BaseInfo { jsonObject.getString(Constants.ENTITY_TYPE).equalsIgnoreCase (Constants.TEZ_VERTEX_ID)); - vertexId = StringInterner.weakIntern(jsonObject.optString(Constants.ENTITY)); + vertexId = StringInterner.intern(jsonObject.optString(Constants.ENTITY)); taskInfoMap = Maps.newHashMap(); inEdgeList = Lists.newLinkedList(); @@ -149,9 +149,9 @@ public class VertexInfo extends BaseInfo { killedTasks = otherInfoNode.optInt(Constants.NUM_KILLED_TASKS); numFailedTaskAttempts = otherInfoNode.optInt(Constants.NUM_FAILED_TASKS_ATTEMPTS); - vertexName = StringInterner.weakIntern(otherInfoNode.optString(Constants.VERTEX_NAME)); - processorClass = StringInterner.weakIntern(otherInfoNode.optString(Constants.PROCESSOR_CLASS_NAME)); - status = StringInterner.weakIntern(otherInfoNode.optString(Constants.STATUS)); + vertexName = StringInterner.intern(otherInfoNode.optString(Constants.VERTEX_NAME)); + processorClass = StringInterner.intern(otherInfoNode.optString(Constants.PROCESSOR_CLASS_NAME)); + status = StringInterner.intern(otherInfoNode.optString(Constants.STATUS)); } public static VertexInfo create(JSONObject vertexInfoObject) throws diff --git a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/utils/Utils.java b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/utils/Utils.java index aacec8ee12..08eb92b967 100644 --- a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/utils/Utils.java +++ b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/utils/Utils.java @@ -22,7 +22,6 @@ import com.google.common.collect.Lists; import org.apache.hadoop.classification.InterfaceAudience; -import org.apache.hadoop.util.StringInterner; import org.apache.log4j.ConsoleAppender; import org.apache.log4j.Level; import org.apache.log4j.Logger; @@ -34,6 +33,7 @@ import org.apache.tez.history.parser.datamodel.Constants; import org.apache.tez.history.parser.datamodel.Event; import org.apache.tez.history.parser.datamodel.TaskAttemptInfo.DataDependencyEvent; +import org.apache.tez.util.StringInterner; import org.codehaus.jettison.json.JSONArray; import org.codehaus.jettison.json.JSONException; import org.codehaus.jettison.json.JSONObject; @@ -106,7 +106,7 @@ public static List parseDataEventDependencyFromJSON(JSONObj for (int i=0; i groupVertices, InputDescriptor inputDescriptor) { - this.groupName = StringInterner.weakIntern(groupName); + this.groupName = StringInterner.intern(groupName); this.groupVertices = groupVertices; this.mergedInputDescriptor = inputDescriptor; } @@ -70,11 +70,11 @@ public void write(DataOutput out) throws IOException { @Override public void readFields(DataInput in) throws IOException { - groupName = StringInterner.weakIntern(Text.readString(in)); + groupName = StringInterner.intern(Text.readString(in)); int numMembers = in.readInt(); groupVertices = Lists.newArrayListWithCapacity(numMembers); for (int i=0; i getIOStatistics() { 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 0c55a3a388..c97cfdf3ce 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 @@ -30,7 +30,6 @@ import org.apache.tez.runtime.library.common.shuffle.ShuffleEventHandler; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hadoop.util.StringInterner; import org.apache.hadoop.conf.Configuration; import org.apache.tez.common.TezCommonUtils; import org.apache.tez.common.TezUtilsInternal; @@ -42,6 +41,7 @@ import org.apache.tez.runtime.library.common.InputAttemptIdentifier; import org.apache.tez.runtime.library.common.shuffle.ShuffleUtils; import org.apache.tez.runtime.library.shuffle.impl.ShuffleUserPayloads.DataMovementEventPayloadProto; +import org.apache.tez.util.StringInterner; import com.google.protobuf.InvalidProtocolBufferException; @@ -167,7 +167,7 @@ private void processDataMovementEvent(DataMovementEvent dmEvent, DataMovementEve } } - scheduler.addKnownMapOutput(StringInterner.weakIntern(shufflePayload.getHost()), shufflePayload.getPort(), + scheduler.addKnownMapOutput(StringInterner.intern(shufflePayload.getHost()), shufflePayload.getPort(), partitionId, srcAttemptIdentifier); } @@ -200,7 +200,7 @@ private void processCompositeRoutedDataMovementEvent(CompositeRoutedDataMovement } } - scheduler.addKnownMapOutput(StringInterner.weakIntern(shufflePayload.getHost()), shufflePayload.getPort(), + scheduler.addKnownMapOutput(StringInterner.intern(shufflePayload.getHost()), shufflePayload.getPort(), partitionId, compositeInputAttemptIdentifier); } @@ -221,7 +221,9 @@ private void processTaskFailedEvent(InputFailedEvent ifEvent) { */ private CompositeInputAttemptIdentifier constructInputAttemptIdentifier(int targetIndex, int targetIndexCount, int version, DataMovementEventPayloadProto shufflePayload) { - String pathComponent = (shufflePayload.hasPathComponent()) ? StringInterner.weakIntern(shufflePayload.getPathComponent()) : null; + String pathComponent = (shufflePayload.hasPathComponent()) + ? StringInterner.intern(shufflePayload.getPathComponent()) + : null; int spillEventId = shufflePayload.getSpillId(); CompositeInputAttemptIdentifier srcAttemptIdentifier = null; if (shufflePayload.hasSpillId()) { 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 e0fb15320c..754fbfa497 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 @@ -52,7 +52,6 @@ import org.apache.hadoop.io.DataOutputBuffer; import org.apache.hadoop.io.NullWritable; import org.apache.hadoop.io.Text; -import org.apache.hadoop.util.StringInterner; import org.apache.tez.common.TezRuntimeFrameworkConfigs; import org.apache.tez.common.TezUtils; import org.apache.tez.common.counters.TaskCounter; @@ -78,6 +77,7 @@ import org.apache.tez.runtime.library.conf.OrderedPartitionedKVOutputConfig.SorterImpl; import org.apache.tez.runtime.library.partitioner.HashPartitioner; import org.apache.tez.runtime.library.shuffle.impl.ShuffleUserPayloads; +import org.apache.tez.util.StringInterner; import org.junit.After; import org.junit.AfterClass; import org.junit.Before; @@ -222,7 +222,7 @@ public void testSortLimitsWithLargeRecords() throws IOException { Text key = new Text(i + ""); //Generate random size between 1 MB to 100 MB. int valSize = ThreadLocalRandom.current().nextInt(1 * 1024 * 1024, 100 * 1024 * 1024); - String val = StringInterner.weakIntern(StringUtils.repeat("v", valSize)); + String val = StringInterner.intern(StringUtils.repeat("v", valSize)); sorter.write(key, new Text(val)); i = (i + 1) % 10; } diff --git a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/CriticalPathAnalyzer.java b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/CriticalPathAnalyzer.java index 5944870c77..387b0cf99d 100644 --- a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/CriticalPathAnalyzer.java +++ b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/CriticalPathAnalyzer.java @@ -26,7 +26,6 @@ import java.util.Map; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.util.StringInterner; import org.apache.hadoop.util.ToolRunner; import org.apache.tez.analyzer.Analyzer; import org.apache.tez.analyzer.CSVResult; @@ -54,8 +53,8 @@ public class CriticalPathAnalyzer extends TezAnalyzerBase implements Analyzer { private static final Logger LOG = LoggerFactory.getLogger(CriticalPathAnalyzer.class); - String succeededState = StringInterner.weakIntern(TaskAttemptState.SUCCEEDED.name()); - String failedState = StringInterner.weakIntern(TaskAttemptState.FAILED.name()); + private static final String SUCCEEDED_STATE = TaskAttemptState.SUCCEEDED.name(); + private static final String FAILED_STATE = TaskAttemptState.FAILED.name(); public enum CriticalPathDependency { DATA_DEPENDENCY, @@ -130,8 +129,8 @@ public void analyze(DagInfo dagInfo) throws TezException { for (TaskInfo task : vertex.getTasks()) { for (TaskAttemptInfo attempt : task.getTaskAttempts()) { attempts.put(attempt.getTaskAttemptId(), attempt); - if (attempt.getStatus().equals(succeededState) || - attempt.getStatus().equals(failedState)) { + if (attempt.getStatus().equals(SUCCEEDED_STATE) || + attempt.getStatus().equals(FAILED_STATE)) { if (lastAttemptFinishTime < attempt.getFinishTime()) { lastAttempt = attempt; lastAttemptFinishTime = attempt.getFinishTime(); From 5eeccf0e318e22cdcbbe202a9f554f93d138c207 Mon Sep 17 00:00:00 2001 From: belugabehr <12578579+belugabehr@users.noreply.github.com> Date: Wed, 30 Jun 2021 11:11:09 -0400 Subject: [PATCH 056/137] TEZ-4308: Add Whitespace in ShuffleScheduler Error Message (#122) --- .../shuffle/orderedgrouped/ShuffleScheduler.java | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) 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 67681cedc7..540d44f409 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 @@ -1044,12 +1044,13 @@ boolean isShuffleHealthy(InputAttemptIdentifier srcAttempt) { && !fetcherHealthy && (!reducerProgressedEnough || reducerStalled)) { String errorMsg = (srcNameTrimmed + ": " - + "Shuffle failed with too many fetch failures and insufficient progress!" - + "failureCounts=" + failureCounts.size() + + "Shuffle failed with too many fetch failures and insufficient progress: " + + "[failureCounts=" + failureCounts.size() + ", pendingInputs=" + (numInputs - doneMaps) + ", fetcherHealthy=" + fetcherHealthy + ", reducerProgressedEnough=" + reducerProgressedEnough - + ", reducerStalled=" + reducerStalled); + + ", reducerStalled=" + reducerStalled) + + "]"; LOG.error(errorMsg); if (LOG.isDebugEnabled()) { LOG.debug("Host failures=" + hostFailures.keySet()); @@ -1109,7 +1110,7 @@ public void obsoleteInput(InputAttemptIdentifier srcAttempt) { } IOException exception = new IOException(srcAttempt + " is marked as obsoleteInput, but it " + "exists in shuffleInfoEventMap. Some data could have been already merged " - + "to memory/disk outputs. Failing the fetch early. eventInfo:" + eventInfo.toString()); + + "to memory/disk outputs. Failing the fetch early. eventInfo: " + eventInfo); String message = "Got obsolete event. Killing self as attempt's data could have been consumed"; killSelf(exception, message); return; @@ -1379,7 +1380,7 @@ protected Void callInternal() throws InterruptedException { } catch (InterruptedException e) { if (isShutdown.get()) { LOG.info(srcNameTrimmed + ": " + - "Interrupted while waiting for fetchers to complete" + + "Interrupted while waiting for fetchers to complete " + "and hasBeenShutdown. Breaking out of ShuffleSchedulerCallable loop"); Thread.currentThread().interrupt(); break; From 4684d06bef7898c6de4c6286c9f62cf2be3dcf92 Mon Sep 17 00:00:00 2001 From: Bodor Laszlo Date: Wed, 30 Jun 2021 17:21:57 +0200 Subject: [PATCH 057/137] TEZ-4313: Apache Tez Release 0.10.1 (#134) --- Tez_DOAP.rdf | 7 +++++ docs/pom.xml | 2 +- .../markdown/releases/apache-tez-0-10-1.md | 30 +++++++++++++++++++ docs/src/site/markdown/releases/index.md | 1 + docs/src/site/site.xml | 3 +- hadoop-shim-impls/hadoop-shim-2.7/pom.xml | 2 +- hadoop-shim-impls/hadoop-shim-2.8/pom.xml | 2 +- hadoop-shim-impls/pom.xml | 2 +- hadoop-shim/pom.xml | 2 +- pom.xml | 2 +- tez-api/pom.xml | 2 +- tez-build-tools/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-aux-services/pom.xml | 2 +- tez-plugins/tez-history-parser/pom.xml | 2 +- .../tez-protobuf-history-plugin/pom.xml | 2 +- .../tez-yarn-timeline-cache-plugin/pom.xml | 2 +- .../pom.xml | 2 +- .../tez-yarn-timeline-history-with-fs/pom.xml | 2 +- tez-plugins/tez-yarn-timeline-history/pom.xml | 2 +- tez-runtime-internals/pom.xml | 2 +- tez-runtime-library/pom.xml | 2 +- tez-tests/pom.xml | 2 +- tez-tools/analyzers/job-analyzer/pom.xml | 2 +- tez-tools/analyzers/pom.xml | 2 +- tez-tools/pom.xml | 2 +- tez-tools/tez-javadoc-tools/pom.xml | 2 +- tez-tools/tez-tfile-parser/pom.xml | 2 +- tez-ui/pom.xml | 2 +- 35 files changed, 71 insertions(+), 32 deletions(-) create mode 100644 docs/src/site/markdown/releases/apache-tez-0-10-1.md diff --git a/Tez_DOAP.rdf b/Tez_DOAP.rdf index 055a45cb5f..d4c473d493 100644 --- a/Tez_DOAP.rdf +++ b/Tez_DOAP.rdf @@ -34,6 +34,13 @@ Java + + + Version 0.10.1 + 2021-07-01 + 0.10.1 + + Version 0.10.0 diff --git a/docs/pom.xml b/docs/pom.xml index c618cec4ba..d3bd5cf010 100644 --- a/docs/pom.xml +++ b/docs/pom.xml @@ -27,7 +27,7 @@ org.apache.tez tez - 0.10.1-SNAPSHOT + 0.10.2-SNAPSHOT tez-docs pom diff --git a/docs/src/site/markdown/releases/apache-tez-0-10-1.md b/docs/src/site/markdown/releases/apache-tez-0-10-1.md new file mode 100644 index 0000000000..efe346aae0 --- /dev/null +++ b/docs/src/site/markdown/releases/apache-tez-0-10-1.md @@ -0,0 +1,30 @@ + + +Apache TEZ® 0.10.1 + +Apache TEZ® 0.10.1 +---------------------- + +- [Download Release Artifacts](http://www.apache.org/dyn/closer.lua/tez/0.10.1/) +- [Release Notes](0.10.1/release-notes.txt) +- Documentation + - [API Javadocs](0.10.1/tez-api-javadocs/index.html) : Documentation for the Tez APIs + - [Runtime Library Javadocs](0.10.1/tez-runtime-library-javadocs/index.html) : Documentation for built-in implementations of useful Inputs, Outputs, Processors etc. written based on the Tez APIs + - [Tez Mapreduce Javadocs](0.10.1/tez-mapreduce-javadocs/index.html) : Documentation for built-in implementations of Mapreduce compatible Inputs, Outputs, Processors etc. written based on the Tez APIs + - [Tez Configuration](0.10.1/tez-api-javadocs/configs/TezConfiguration.html) : Documentation for configurations of Tez. These configurations are typically specified in tez-site.xml. + - [Tez Runtime Configuration](0.10.1/tez-runtime-library-javadocs/configs/TezRuntimeConfiguration.html) : Documentation for runtime configurations of Tez. These configurations are typically specified by job submitters. diff --git a/docs/src/site/markdown/releases/index.md b/docs/src/site/markdown/releases/index.md index 26c18b0017..7d5c94871c 100644 --- a/docs/src/site/markdown/releases/index.md +++ b/docs/src/site/markdown/releases/index.md @@ -19,6 +19,7 @@ Releases ------------ +- [Apache TEZ® 0.10.1](./apache-tez-0-10-1.html) (Jul 01, 2021) - [Apache TEZ® 0.10.0](./apache-tez-0-10-0.html) (Oct 15, 2020) - [Apache TEZ® 0.9.2](./apache-tez-0-9-2.html) (Mar 29, 2019) - [Apache TEZ® 0.9.1](./apache-tez-0-9-1.html) (Jan 04, 2018) diff --git a/docs/src/site/site.xml b/docs/src/site/site.xml index 04187dd3f7..c8e6479a8f 100644 --- a/docs/src/site/site.xml +++ b/docs/src/site/site.xml @@ -129,7 +129,8 @@ - + + diff --git a/hadoop-shim-impls/hadoop-shim-2.7/pom.xml b/hadoop-shim-impls/hadoop-shim-2.7/pom.xml index d75ed460c1..2557e036c3 100644 --- a/hadoop-shim-impls/hadoop-shim-2.7/pom.xml +++ b/hadoop-shim-impls/hadoop-shim-2.7/pom.xml @@ -19,7 +19,7 @@ hadoop-shim-impls org.apache.tez - 0.10.1-SNAPSHOT + 0.10.2-SNAPSHOT 4.0.0 hadoop-shim-2.7 diff --git a/hadoop-shim-impls/hadoop-shim-2.8/pom.xml b/hadoop-shim-impls/hadoop-shim-2.8/pom.xml index 9450c75c60..0a149cca04 100644 --- a/hadoop-shim-impls/hadoop-shim-2.8/pom.xml +++ b/hadoop-shim-impls/hadoop-shim-2.8/pom.xml @@ -19,7 +19,7 @@ hadoop-shim-impls org.apache.tez - 0.10.1-SNAPSHOT + 0.10.2-SNAPSHOT 4.0.0 hadoop-shim-2.8 diff --git a/hadoop-shim-impls/pom.xml b/hadoop-shim-impls/pom.xml index 77192a016d..25cf3fd25a 100644 --- a/hadoop-shim-impls/pom.xml +++ b/hadoop-shim-impls/pom.xml @@ -20,7 +20,7 @@ tez org.apache.tez - 0.10.1-SNAPSHOT + 0.10.2-SNAPSHOT hadoop-shim-impls pom diff --git a/hadoop-shim/pom.xml b/hadoop-shim/pom.xml index 383c5cf2b6..18ddaca3df 100644 --- a/hadoop-shim/pom.xml +++ b/hadoop-shim/pom.xml @@ -20,7 +20,7 @@ tez org.apache.tez - 0.10.1-SNAPSHOT + 0.10.2-SNAPSHOT hadoop-shim diff --git a/pom.xml b/pom.xml index c1f0682df1..1cf2bcac93 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ org.apache.tez tez pom - 0.10.1-SNAPSHOT + 0.10.2-SNAPSHOT tez diff --git a/tez-api/pom.xml b/tez-api/pom.xml index 95a7610ee4..8cab40c172 100644 --- a/tez-api/pom.xml +++ b/tez-api/pom.xml @@ -20,7 +20,7 @@ org.apache.tez tez - 0.10.1-SNAPSHOT + 0.10.2-SNAPSHOT tez-api diff --git a/tez-build-tools/pom.xml b/tez-build-tools/pom.xml index baf75b18ec..232a20be89 100644 --- a/tez-build-tools/pom.xml +++ b/tez-build-tools/pom.xml @@ -20,7 +20,7 @@ org.apache.tez tez - 0.10.1-SNAPSHOT + 0.10.2-SNAPSHOT tez-build-tools diff --git a/tez-common/pom.xml b/tez-common/pom.xml index 6b82cdc7f8..5fd6910b18 100644 --- a/tez-common/pom.xml +++ b/tez-common/pom.xml @@ -20,7 +20,7 @@ org.apache.tez tez - 0.10.1-SNAPSHOT + 0.10.2-SNAPSHOT tez-common diff --git a/tez-dag/pom.xml b/tez-dag/pom.xml index c0dccc8642..723082c54e 100644 --- a/tez-dag/pom.xml +++ b/tez-dag/pom.xml @@ -19,7 +19,7 @@ org.apache.tez tez - 0.10.1-SNAPSHOT + 0.10.2-SNAPSHOT tez-dag diff --git a/tez-dist/pom.xml b/tez-dist/pom.xml index ea6041d76a..741ed9d1e9 100644 --- a/tez-dist/pom.xml +++ b/tez-dist/pom.xml @@ -21,7 +21,7 @@ org.apache.tez tez - 0.10.1-SNAPSHOT + 0.10.2-SNAPSHOT tez-dist diff --git a/tez-examples/pom.xml b/tez-examples/pom.xml index 3ac8534b6d..f6ab04c020 100644 --- a/tez-examples/pom.xml +++ b/tez-examples/pom.xml @@ -20,7 +20,7 @@ org.apache.tez tez - 0.10.1-SNAPSHOT + 0.10.2-SNAPSHOT tez-examples diff --git a/tez-ext-service-tests/pom.xml b/tez-ext-service-tests/pom.xml index 8dfad0d74e..327d457187 100644 --- a/tez-ext-service-tests/pom.xml +++ b/tez-ext-service-tests/pom.xml @@ -20,7 +20,7 @@ tez org.apache.tez - 0.10.1-SNAPSHOT + 0.10.2-SNAPSHOT tez-ext-service-tests diff --git a/tez-mapreduce/pom.xml b/tez-mapreduce/pom.xml index a01d028039..ee7c318ff0 100644 --- a/tez-mapreduce/pom.xml +++ b/tez-mapreduce/pom.xml @@ -20,7 +20,7 @@ org.apache.tez tez - 0.10.1-SNAPSHOT + 0.10.2-SNAPSHOT tez-mapreduce diff --git a/tez-plugins/pom.xml b/tez-plugins/pom.xml index 6ce710ea27..27f19289ce 100644 --- a/tez-plugins/pom.xml +++ b/tez-plugins/pom.xml @@ -21,7 +21,7 @@ org.apache.tez tez - 0.10.1-SNAPSHOT + 0.10.2-SNAPSHOT tez-plugins pom diff --git a/tez-plugins/tez-aux-services/pom.xml b/tez-plugins/tez-aux-services/pom.xml index 7279eaf684..965663ce4a 100644 --- a/tez-plugins/tez-aux-services/pom.xml +++ b/tez-plugins/tez-aux-services/pom.xml @@ -20,7 +20,7 @@ tez-plugins org.apache.tez - 0.10.1-SNAPSHOT + 0.10.2-SNAPSHOT tez-aux-services diff --git a/tez-plugins/tez-history-parser/pom.xml b/tez-plugins/tez-history-parser/pom.xml index 1b1d4e8285..5240bba1d8 100644 --- a/tez-plugins/tez-history-parser/pom.xml +++ b/tez-plugins/tez-history-parser/pom.xml @@ -21,7 +21,7 @@ org.apache.tez tez-plugins - 0.10.1-SNAPSHOT + 0.10.2-SNAPSHOT tez-history-parser diff --git a/tez-plugins/tez-protobuf-history-plugin/pom.xml b/tez-plugins/tez-protobuf-history-plugin/pom.xml index f84bb23dad..ab06981f3d 100644 --- a/tez-plugins/tez-protobuf-history-plugin/pom.xml +++ b/tez-plugins/tez-protobuf-history-plugin/pom.xml @@ -19,7 +19,7 @@ org.apache.tez tez-plugins - 0.10.1-SNAPSHOT + 0.10.2-SNAPSHOT tez-protobuf-history-plugin diff --git a/tez-plugins/tez-yarn-timeline-cache-plugin/pom.xml b/tez-plugins/tez-yarn-timeline-cache-plugin/pom.xml index c817dcfbd1..942d43658a 100644 --- a/tez-plugins/tez-yarn-timeline-cache-plugin/pom.xml +++ b/tez-plugins/tez-yarn-timeline-cache-plugin/pom.xml @@ -19,7 +19,7 @@ org.apache.tez tez-plugins - 0.10.1-SNAPSHOT + 0.10.2-SNAPSHOT tez-yarn-timeline-cache-plugin 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 27288953fa..50ed0bd77d 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 @@ org.apache.tez tez-plugins - 0.10.1-SNAPSHOT + 0.10.2-SNAPSHOT tez-yarn-timeline-history-with-acls diff --git a/tez-plugins/tez-yarn-timeline-history-with-fs/pom.xml b/tez-plugins/tez-yarn-timeline-history-with-fs/pom.xml index 8939263799..4ba760f1aa 100644 --- a/tez-plugins/tez-yarn-timeline-history-with-fs/pom.xml +++ b/tez-plugins/tez-yarn-timeline-history-with-fs/pom.xml @@ -19,7 +19,7 @@ org.apache.tez tez-plugins - 0.10.1-SNAPSHOT + 0.10.2-SNAPSHOT tez-yarn-timeline-history-with-fs diff --git a/tez-plugins/tez-yarn-timeline-history/pom.xml b/tez-plugins/tez-yarn-timeline-history/pom.xml index 99406650e6..78b83dd877 100644 --- a/tez-plugins/tez-yarn-timeline-history/pom.xml +++ b/tez-plugins/tez-yarn-timeline-history/pom.xml @@ -19,7 +19,7 @@ org.apache.tez tez-plugins - 0.10.1-SNAPSHOT + 0.10.2-SNAPSHOT tez-yarn-timeline-history diff --git a/tez-runtime-internals/pom.xml b/tez-runtime-internals/pom.xml index 8fe7d79e40..f7e14dae59 100644 --- a/tez-runtime-internals/pom.xml +++ b/tez-runtime-internals/pom.xml @@ -20,7 +20,7 @@ org.apache.tez tez - 0.10.1-SNAPSHOT + 0.10.2-SNAPSHOT tez-runtime-internals diff --git a/tez-runtime-library/pom.xml b/tez-runtime-library/pom.xml index ecabf8976a..2caff344ec 100644 --- a/tez-runtime-library/pom.xml +++ b/tez-runtime-library/pom.xml @@ -20,7 +20,7 @@ org.apache.tez tez - 0.10.1-SNAPSHOT + 0.10.2-SNAPSHOT tez-runtime-library diff --git a/tez-tests/pom.xml b/tez-tests/pom.xml index d76f27a1a9..72f95d56ce 100644 --- a/tez-tests/pom.xml +++ b/tez-tests/pom.xml @@ -20,7 +20,7 @@ org.apache.tez tez - 0.10.1-SNAPSHOT + 0.10.2-SNAPSHOT tez-tests diff --git a/tez-tools/analyzers/job-analyzer/pom.xml b/tez-tools/analyzers/job-analyzer/pom.xml index a857bb3a53..89ae9088ff 100644 --- a/tez-tools/analyzers/job-analyzer/pom.xml +++ b/tez-tools/analyzers/job-analyzer/pom.xml @@ -20,7 +20,7 @@ org.apache.tez tez-perf-analyzer - 0.10.1-SNAPSHOT + 0.10.2-SNAPSHOT tez-job-analyzer diff --git a/tez-tools/analyzers/pom.xml b/tez-tools/analyzers/pom.xml index 64484d3915..3c989e68ec 100644 --- a/tez-tools/analyzers/pom.xml +++ b/tez-tools/analyzers/pom.xml @@ -20,7 +20,7 @@ org.apache.tez tez-tools - 0.10.1-SNAPSHOT + 0.10.2-SNAPSHOT tez-perf-analyzer pom diff --git a/tez-tools/pom.xml b/tez-tools/pom.xml index 34f9f992e0..bffe9abc5d 100644 --- a/tez-tools/pom.xml +++ b/tez-tools/pom.xml @@ -21,7 +21,7 @@ org.apache.tez tez - 0.10.1-SNAPSHOT + 0.10.2-SNAPSHOT tez-tools pom diff --git a/tez-tools/tez-javadoc-tools/pom.xml b/tez-tools/tez-javadoc-tools/pom.xml index e0405b685b..6b0546674b 100644 --- a/tez-tools/tez-javadoc-tools/pom.xml +++ b/tez-tools/tez-javadoc-tools/pom.xml @@ -20,7 +20,7 @@ org.apache.tez tez-tools - 0.10.1-SNAPSHOT + 0.10.2-SNAPSHOT tez-javadoc-tools diff --git a/tez-tools/tez-tfile-parser/pom.xml b/tez-tools/tez-tfile-parser/pom.xml index c2f40201a3..a258a5e92b 100644 --- a/tez-tools/tez-tfile-parser/pom.xml +++ b/tez-tools/tez-tfile-parser/pom.xml @@ -20,7 +20,7 @@ org.apache.tez tez-tools - 0.10.1-SNAPSHOT + 0.10.2-SNAPSHOT tez-tfile-parser diff --git a/tez-ui/pom.xml b/tez-ui/pom.xml index e52eb52dfa..80cb05944b 100644 --- a/tez-ui/pom.xml +++ b/tez-ui/pom.xml @@ -21,7 +21,7 @@ org.apache.tez tez - 0.10.1-SNAPSHOT + 0.10.2-SNAPSHOT tez-ui war From 47a59abba5a44570eff61639d53689518569cab2 Mon Sep 17 00:00:00 2001 From: belugabehr <12578579+belugabehr@users.noreply.github.com> Date: Wed, 30 Jun 2021 12:01:06 -0400 Subject: [PATCH 058/137] TEZ-4267: Remove Superfluous Code from DAGAppMaster (#90) * TEZ-4267: Remove Superfluous Code from DAGAppMaster * Remove superfluous variable * Remove superfluous comment Co-authored-by: David Mollitor --- .../org/apache/tez/dag/app/DAGAppMaster.java | 52 ++++++------------- 1 file changed, 17 insertions(+), 35 deletions(-) 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 61e1cb684a..abc10bd86c 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 @@ -190,7 +190,6 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Function; -import com.google.common.base.Joiner; import org.apache.tez.common.Preconditions; import com.google.common.collect.Maps; import com.google.common.util.concurrent.ListeningExecutorService; @@ -224,7 +223,6 @@ public class DAGAppMaster extends AbstractService { * Priority of the DAGAppMaster shutdown hook. */ public static final int SHUTDOWN_HOOK_PRIORITY = 30; - private static final Joiner PATH_JOINER = Joiner.on('/'); @VisibleForTesting static final String INVALID_SESSION_ERR_MSG = "Initial application attempt in session mode failed. " @@ -311,7 +309,6 @@ public class DAGAppMaster extends AbstractService { /** * set of already executed dag names. */ - Set dagNames = new HashSet(); Set dagIDs = new HashSet(); protected boolean isLastAMRetry = false; @@ -371,19 +368,17 @@ public DAGAppMaster(ApplicationAttemptId applicationAttemptId, this.containerID.toString(), this.appMasterUgi.getShortUserName()); LOG.info("Created DAGAppMaster for application " + applicationAttemptId - + ", versionInfo=" + dagVersionInfo.toString()); + + ", versionInfo=" + dagVersionInfo); TezCommonUtils.logCredentials(LOG, this.appMasterUgi.getCredentials(), "am"); } // Pull this WebAppUtils function into Tez until YARN-4186 - public static String getRunningLogURL(String nodeHttpAddress, + private static String getRunningLogURL(String nodeHttpAddress, String containerId, String user) { - if (nodeHttpAddress == null || nodeHttpAddress.isEmpty() - || containerId == null || containerId.isEmpty() || user == null - || user.isEmpty()) { + if (containerId.isEmpty() || user == null | user.isEmpty()) { return null; } - return PATH_JOINER.join(nodeHttpAddress, "node", "containerlogs", + return String.format("%s/node/containerlogs/%s/%s", nodeHttpAddress, containerId, user); } @@ -695,8 +690,7 @@ private void handleInternalError(String errDiagnosticsPrefix, String errDiagDagE state = DAGAppMasterState.ERROR; if (currentDAG != null) { _updateLoggers(currentDAG, "_post"); - String errDiagnostics = errDiagnosticsPrefix + ". Aborting dag: " + currentDAG.getID(); - LOG.info(errDiagnostics); + LOG.info(errDiagnosticsPrefix + ". Aborting dag: " + currentDAG.getID()); // Inform the current DAG about the error sendEvent(new DAGEventInternalError(currentDAG.getID(), errDiagDagEvent)); } else { @@ -758,8 +752,8 @@ protected synchronized void handle(DAGAppMasterEvent event) { DAGAppMasterEventDAGFinished finishEvt = (DAGAppMasterEventDAGFinished) event; String timeStamp = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss").format(Calendar.getInstance().getTime()); - System.err.println(timeStamp + " Completed Dag: " + finishEvt.getDAGId().toString()); - System.out.println(timeStamp + " Completed Dag: " + finishEvt.getDAGId().toString()); + System.err.println(timeStamp + " Completed Dag: " + finishEvt.getDAGId()); + System.out.println(timeStamp + " Completed Dag: " + finishEvt.getDAGId()); // Stop vertex services if any stopVertexServices(currentDAG); if (!isSession) { @@ -767,13 +761,11 @@ protected synchronized void handle(DAGAppMasterEvent event) { this.taskSchedulerManager.setShouldUnregisterFlag(); _updateLoggers(currentDAG, "_post"); setStateOnDAGCompletion(); - LOG.info("Shutting down on completion of dag:" + - finishEvt.getDAGId().toString()); + LOG.info("Shutting down on completion of dag:" + finishEvt.getDAGId()); shutdownHandler.shutdown(); } else { - LOG.info("DAG completed, dagId=" - + finishEvt.getDAGId().toString() - + ", dagState=" + finishEvt.getDAGState()); + LOG.info("DAG completed, dagId=" + finishEvt.getDAGId() + ", dagState=" + + finishEvt.getDAGState()); lastDAGCompletionTime = clock.getTime(); _updateLoggers(currentDAG, "_post"); if (this.historyEventHandler.hasRecoveryFailed()) { @@ -1028,9 +1020,8 @@ DAGImpl createDAG(DAGPlan dagPB, TezDAGID dagId) { try { if (LOG.isDebugEnabled()) { - LOG.debug("JSON dump for submitted DAG, dagId=" + dagId.toString() - + ", json=" - + DAGUtils.generateSimpleJSONPlan(dagPB).toString()); + LOG.debug("JSON dump for submitted DAG, dagId=" + dagId + ", json=" + + DAGUtils.generateSimpleJSONPlan(dagPB)); } } catch (JSONException e) { LOG.warn("Failed to generate json for DAG", e); @@ -1038,7 +1029,7 @@ DAGImpl createDAG(DAGPlan dagPB, TezDAGID dagId) { writeDebugArtifacts(dagPB, newDag); return newDag; - } // end createDag() + } private void writeDebugArtifacts(DAGPlan dagPB, DAGImpl newDag) { boolean debugArtifacts = @@ -1052,7 +1043,7 @@ private void writeDebugArtifacts(DAGPlan dagPB, DAGImpl newDag) { private void writePBTextFile(DAG dag) { String logFile = logDirs[new Random().nextInt(logDirs.length)] + File.separatorChar - + dag.getID().toString() + "-" + TezConstants.TEZ_PB_PLAN_TEXT_NAME; + + dag.getID() + "-" + TezConstants.TEZ_PB_PLAN_TEXT_NAME; LOG.info("Writing DAG plan to: " + logFile); File outFile = new File(logFile); @@ -1061,7 +1052,7 @@ private void writePBTextFile(DAG dag) { printWriter.println(TezUtilsInternal.convertDagPlanToString(dag.getJobPlan())); printWriter.close(); } catch (IOException e) { - LOG.warn("Failed to write TEZ_PLAN to " + outFile.toString(), e); + LOG.warn("Failed to write TEZ_PLAN to " + outFile, e); } } @@ -2256,15 +2247,6 @@ public void handle(VertexEvent event) { } } - private static void validateInputParam(String value, String param) - throws IOException { - if (value == null) { - String msg = param + " is null"; - LOG.error(msg); - throw new IOException(msg); - } - } - private long checkAndHandleDAGClientTimeout() throws TezException { if (EnumSet.of(DAGAppMasterState.NEW, DAGAppMasterState.RECOVERING).contains(this.state) || sessionStopped.get()) { @@ -2333,8 +2315,8 @@ public static void main(String[] args) { clientVersion = VersionInfo.UNKNOWN; } - validateInputParam(appSubmitTimeStr, - ApplicationConstants.APP_SUBMIT_TIME_ENV); + Objects.requireNonNull(appSubmitTimeStr, + ApplicationConstants.APP_SUBMIT_TIME_ENV + " is null"); ContainerId containerId = ConverterUtils.toContainerId(containerIdStr); ApplicationAttemptId applicationAttemptId = From 8f3044c9dc0f46c113c4d404c1b5fd4e317dc91b Mon Sep 17 00:00:00 2001 From: Bodor Laszlo Date: Wed, 30 Jun 2021 21:32:53 +0200 Subject: [PATCH 059/137] TEZ-4314: Fix some plugin versions in site generation (#131) --- docs/pom.xml | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/docs/pom.xml b/docs/pom.xml index d3bd5cf010..0d55177641 100644 --- a/docs/pom.xml +++ b/docs/pom.xml @@ -467,6 +467,7 @@ org.apache.maven.plugins maven-site-plugin + 2.4 ./target @@ -487,6 +488,7 @@ org.apache.maven.plugins maven-project-info-reports-plugin + 3.1.2 ${project.basedir}/src/site/custom/project-info-report.properties false @@ -494,10 +496,10 @@ - project-team - mailing-list - issue-tracking - license + team + mailing-lists + issue-management + licenses From b31501d9b7f2cc8f4cc4283bd285dc48609dc77b Mon Sep 17 00:00:00 2001 From: Bodor Laszlo Date: Thu, 1 Jul 2021 00:38:57 +0200 Subject: [PATCH 060/137] TEZ-4125: Upgrade to Hadoop 3.2.x (#126) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 1cf2bcac93..1d8194ad58 100644 --- a/pom.xml +++ b/pom.xml @@ -58,7 +58,7 @@ ${user.home}/clover.license 27.0-jre - 3.1.3 + 3.2.2 4.0.52.Final 0.13.0 1.19 From 464d86d8ffd5ee8e3bd7a4d48ab64b10e3db36b0 Mon Sep 17 00:00:00 2001 From: Igor Dvorzhak Date: Fri, 9 Jul 2021 09:03:24 -0700 Subject: [PATCH 061/137] TEZ-4214. Upgrade RoaringBitmap version to 0.7.45 (#74) --- pom.xml | 5 +++-- tez-plugins/tez-aux-services/pom.xml | 1 + 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/pom.xml b/pom.xml index 1d8194ad58..ba65cc6ad6 100644 --- a/pom.xml +++ b/pom.xml @@ -64,11 +64,12 @@ 1.19 1.7.30 2.5.0 + 0.7.45 ${env.PROTOC_PATH} scm:git:https://gitbox.apache.org/repos/asf/tez.git 1.4 3.0.5 - 3.1.1 + 3.1.1 8.35 1.3.6 ${project.build.directory}/tmp @@ -232,7 +233,7 @@ org.roaringbitmap RoaringBitmap - 0.5.21 + ${roaringbitmap.version} org.slf4j diff --git a/tez-plugins/tez-aux-services/pom.xml b/tez-plugins/tez-aux-services/pom.xml index 965663ce4a..2a936d6fa5 100644 --- a/tez-plugins/tez-aux-services/pom.xml +++ b/tez-plugins/tez-aux-services/pom.xml @@ -195,6 +195,7 @@ org.apache.maven.plugins maven-shade-plugin + 3.2.4 package From 3f541d06f4aad0e1ad051bec273a05a678fd7776 Mon Sep 17 00:00:00 2001 From: Bodor Laszlo Date: Mon, 9 Aug 2021 08:24:34 +0200 Subject: [PATCH 062/137] TEZ-4231: Fix multiple history parser and event converter issues (#123) (Laszlo Bodor reviewed by Rajesh Balamohan) --- .../history/parser/ProtoHistoryParser.java | 1 - .../history/parser/SimpleHistoryParser.java | 31 +++- .../history/parser/datamodel/BaseInfo.java | 16 +- .../tez/history/parser/utils/Utils.java | 4 +- .../HistoryEventProtoJsonConversion.java | 12 +- .../org/apache/tez/analyzer/Analyzer.java | 8 - .../tez/analyzer/plugins/AnalyzerDriver.java | 6 + .../plugins/ContainerReuseAnalyzer.java | 9 +- .../plugins/CriticalPathAnalyzer.java | 11 +- .../analyzer/plugins/DagOverviewAnalyzer.java | 132 ++++++++++++++++ .../analyzer/plugins/HungTaskAnalyzer.java | 143 ++++++++++++++++++ .../plugins/InputReadErrorAnalyzer.java | 94 ++++++++++++ .../analyzer/plugins/LocalityAnalyzer.java | 10 +- .../plugins/OneOnOneEdgeAnalyzer.java | 9 +- .../analyzer/plugins/ShuffleTimeAnalyzer.java | 9 +- .../tez/analyzer/plugins/SkewAnalyzer.java | 13 +- .../analyzer/plugins/SlowNodeAnalyzer.java | 9 +- .../analyzer/plugins/SlowTaskIdentifier.java | 11 +- .../plugins/SlowestVertexAnalyzer.java | 8 +- .../analyzer/plugins/SpillAnalyzerImpl.java | 9 +- .../plugins/TaskAssignmentAnalyzer.java | 27 ++-- .../TaskAttemptResultStatisticsAnalyzer.java | 11 +- .../plugins/TaskConcurrencyAnalyzer.java | 8 +- .../tez/analyzer/plugins/TezAnalyzerBase.java | 7 +- .../VertexLevelCriticalPathAnalyzer.java | 9 +- 25 files changed, 462 insertions(+), 145 deletions(-) create mode 100644 tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/DagOverviewAnalyzer.java create mode 100644 tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/HungTaskAnalyzer.java create mode 100644 tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/InputReadErrorAnalyzer.java diff --git a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/ProtoHistoryParser.java b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/ProtoHistoryParser.java index d28fd67d4f..397a46fde9 100644 --- a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/ProtoHistoryParser.java +++ b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/ProtoHistoryParser.java @@ -98,7 +98,6 @@ public boolean hasNext() throws IOException { return message != null; } catch (java.io.EOFException e) { reader.close(); - if (!fileIt.hasNext()) { return false; } else { diff --git a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/SimpleHistoryParser.java b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/SimpleHistoryParser.java index db3f6488e0..c1711ce2cb 100644 --- a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/SimpleHistoryParser.java +++ b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/SimpleHistoryParser.java @@ -236,11 +236,13 @@ protected void readEventsFromSource(String dagId, JSONObjectSource source, // time etc). if (dagJson == null) { dagJson = jsonObject; - } else if (dagJson.optJSONObject(ATSConstants.OTHER_INFO) - .optJSONObject(ATSConstants.DAG_PLAN) == null) { - // if DAG_PLAN is not filled already, let's try to fetch it from other - dagJson.getJSONObject(ATSConstants.OTHER_INFO).put(ATSConstants.DAG_PLAN, jsonObject - .getJSONObject(ATSConstants.OTHER_INFO).getJSONObject(ATSConstants.DAG_PLAN)); + } else { + if (dagJson.optJSONObject(ATSConstants.OTHER_INFO).optJSONObject(ATSConstants.DAG_PLAN) == null) { + // if DAG_PLAN is not filled already, let's try to fetch it from other + dagJson.getJSONObject(ATSConstants.OTHER_INFO).put(ATSConstants.DAG_PLAN, + jsonObject.getJSONObject(ATSConstants.OTHER_INFO).getJSONObject(ATSConstants.DAG_PLAN)); + } + mergeSubJSONArray(jsonObject, dagJson, Constants.EVENTS); } JSONArray relatedEntities = dagJson.optJSONArray(Constants .RELATED_ENTITIES); @@ -268,6 +270,8 @@ protected void readEventsFromSource(String dagId, JSONObjectSource source, } if (!vertexJsonMap.containsKey(vertexName)) { vertexJsonMap.put(vertexName, jsonObject); + } else { + mergeSubJSONArray(jsonObject, vertexJsonMap.get(vertexName), Constants.EVENTS); } populateOtherInfo(jsonObject.optJSONObject(Constants.OTHER_INFO), vertexName, vertexJsonMap); break; @@ -281,6 +285,8 @@ protected void readEventsFromSource(String dagId, JSONObjectSource source, } if (!taskJsonMap.containsKey(taskName)) { taskJsonMap.put(taskName, jsonObject); + } else { + mergeSubJSONArray(jsonObject, taskJsonMap.get(taskName), Constants.EVENTS); } populateOtherInfo(jsonObject.optJSONObject(Constants.OTHER_INFO), taskName, taskJsonMap); break; @@ -294,6 +300,8 @@ protected void readEventsFromSource(String dagId, JSONObjectSource source, } if (!attemptJsonMap.containsKey(taskAttemptName)) { attemptJsonMap.put(taskAttemptName, jsonObject); + } else { + mergeSubJSONArray(jsonObject, attemptJsonMap.get(taskAttemptName), Constants.EVENTS); } populateOtherInfo(jsonObject.optJSONObject(Constants.OTHER_INFO), taskAttemptName, attemptJsonMap); break; @@ -311,4 +319,17 @@ protected void readEventsFromSource(String dagId, JSONObjectSource source, "Please provide a valid/complete history log file containing " + dagId); } } + + private void mergeSubJSONArray(JSONObject source, JSONObject destination, String key) + throws JSONException { + if (source.optJSONArray(key) == null) { + source.put(key, new JSONArray()); + } + if (destination.optJSONArray(key) == null) { + destination.put(key, new JSONArray()); + } + for (int i = 0; i < source.getJSONArray(key).length(); i++) { + destination.getJSONArray(key).put(source.getJSONArray(key).get(i)); + } + } } \ No newline at end of file diff --git a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/BaseInfo.java b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/BaseInfo.java index 3f9666a950..783f486a15 100644 --- a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/BaseInfo.java +++ b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/BaseInfo.java @@ -44,8 +44,20 @@ public abstract class BaseInfo { BaseInfo(JSONObject jsonObject) throws JSONException { final JSONObject otherInfoNode = jsonObject.getJSONObject(Constants.OTHER_INFO); //parse tez counters - tezCounters = Utils.parseTezCountersFromJSON( - otherInfoNode.optJSONObject(Constants.COUNTERS)); + JSONObject countersObj = otherInfoNode.optJSONObject(Constants.COUNTERS); + if (countersObj == null) { + /* + * This is a workaround for formatting differences, where a TaskFinishedEvent's + * counter is a correct json object shown as string, but VertexFinishedEvent's + * counter is an encoded json string, so the latter is interpreted as a String + * while parsing. The issue might be somewhere while converting these event objects + * to proto (HistoryEventProtoConverter). Even if should be fixed there, + * already generated events should be parsed correctly, hence this workaround. + * Will be investigated in the scope of TEZ-4324. + */ + countersObj = new JSONObject(otherInfoNode.optString(Constants.COUNTERS)); + } + tezCounters = Utils.parseTezCountersFromJSON(countersObj); //parse events eventList = Lists.newArrayList(); diff --git a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/utils/Utils.java b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/utils/Utils.java index 08eb92b967..94b50a6c5c 100644 --- a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/utils/Utils.java +++ b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/utils/Utils.java @@ -128,12 +128,12 @@ public static void parseEvents(JSONArray eventNodes, List eventList) thro JSONObject eventNode = eventNodes.optJSONObject(i); final String eventInfo = eventNode.optString(Constants.EVENT_INFO); final String eventType = eventNode.optString(Constants.EVENT_TYPE); - final long time = eventNode.optLong(Constants.EVENT_TIME_STAMP); + final long time = eventNode.optLong(Constants.EVENT_TIME_STAMP) == 0 + ? eventNode.optLong(Constants.TIMESTAMP) : eventNode.optLong(Constants.EVENT_TIME_STAMP); Event event = new Event(eventInfo, eventType, time); eventList.add(event); - } } diff --git a/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/HistoryEventProtoJsonConversion.java b/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/HistoryEventProtoJsonConversion.java index 26e20abeb3..ef84b2ec61 100644 --- a/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/HistoryEventProtoJsonConversion.java +++ b/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/HistoryEventProtoJsonConversion.java @@ -556,12 +556,12 @@ private static JSONObject convertTaskFinishedEvent(HistoryEventProto event) thro events.put(finishEvent); jsonObject.put(ATSConstants.EVENTS, events); - long startTime = getLongDataValueByKey(event, ATSConstants.START_TIME); + long timeTaken = getLongDataValueByKey(event, ATSConstants.TIME_TAKEN); JSONObject otherInfo = new JSONObject(); - otherInfo.put(ATSConstants.START_TIME, startTime); + otherInfo.put(ATSConstants.START_TIME, event.getEventTime() - timeTaken); otherInfo.put(ATSConstants.FINISH_TIME, event.getEventTime()); - otherInfo.put(ATSConstants.TIME_TAKEN, event.getEventTime() - startTime); + otherInfo.put(ATSConstants.TIME_TAKEN, timeTaken); otherInfo.put(ATSConstants.STATUS, getDataValueByKey(event, ATSConstants.STATUS)); otherInfo.put(ATSConstants.DIAGNOSTICS, getDataValueByKey(event, ATSConstants.DIAGNOSTICS)); @@ -620,11 +620,13 @@ private static JSONObject convertVertexFinishedEvent(HistoryEventProto event) events.put(finishEvent); jsonObject.put(ATSConstants.EVENTS, events); - long startTime = getLongDataValueByKey(event, ATSConstants.START_TIME); + long timeTaken = getLongDataValueByKey(event, ATSConstants.TIME_TAKEN); JSONObject otherInfo = new JSONObject(); + otherInfo.put(ATSConstants.START_TIME, event.getEventTime() - timeTaken); otherInfo.put(ATSConstants.FINISH_TIME, event.getEventTime()); - otherInfo.put(ATSConstants.TIME_TAKEN, (event.getEventTime() - startTime)); + otherInfo.put(ATSConstants.TIME_TAKEN, timeTaken); + otherInfo.put(ATSConstants.STATUS, getDataValueByKey(event, ATSConstants.STATUS)); otherInfo.put(ATSConstants.DIAGNOSTICS, getDataValueByKey(event, ATSConstants.DIAGNOSTICS)); otherInfo.put(ATSConstants.COUNTERS, getJSONDataValueByKey(event, ATSConstants.COUNTERS)); diff --git a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/Analyzer.java b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/Analyzer.java index 6021c5897e..1f0a7ad62c 100644 --- a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/Analyzer.java +++ b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/Analyzer.java @@ -18,7 +18,6 @@ package org.apache.tez.analyzer; -import org.apache.hadoop.conf.Configuration; import org.apache.tez.dag.api.TezException; import org.apache.tez.history.parser.datamodel.DagInfo; @@ -54,11 +53,4 @@ public interface Analyzer { * @return description of analyzer */ public String getDescription(); - - /** - * Get config properties related to this analyzer - * - * @return config related to analyzer - */ - public Configuration getConfiguration(); } diff --git a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/AnalyzerDriver.java b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/AnalyzerDriver.java index cad0d98d75..294527cd3e 100644 --- a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/AnalyzerDriver.java +++ b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/AnalyzerDriver.java @@ -48,12 +48,18 @@ public static void main(String argv[]){ "Print task-to-node assignment details of a DAG"); pgd.addClass("TaskAttemptResultStatisticsAnalyzer", TaskAttemptResultStatisticsAnalyzer.class, "Print vertex:node:status level details of task attempt results"); + pgd.addClass("InputReadErrorAnalyzer", InputReadErrorAnalyzer.class, + "Print INPUT_READ_ERROR sources"); pgd.addClass("TaskConcurrencyAnalyzer", TaskConcurrencyAnalyzer.class, "Print the task concurrency details in a DAG"); pgd.addClass("VertexLevelCriticalPathAnalyzer", VertexLevelCriticalPathAnalyzer.class, "Find critical path at vertex level in a DAG"); pgd.addClass("OneOnOneEdgeAnalyzer", OneOnOneEdgeAnalyzer.class, "Find out schedule misses in 1:1 edges in a DAG"); + pgd.addClass("DagOverviewAnalyzer", DagOverviewAnalyzer.class, + "Print basic dag information (dag/vertex events)"); + pgd.addClass("TaskHangAnalyzer", HungTaskAnalyzer.class, + "Print all vertices/tasks and their last attempts with status/duration/node"); exitCode = pgd.run(argv); } catch(Throwable e){ e.printStackTrace(); diff --git a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/ContainerReuseAnalyzer.java b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/ContainerReuseAnalyzer.java index 5b862f87f7..553ff0e2cc 100644 --- a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/ContainerReuseAnalyzer.java +++ b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/ContainerReuseAnalyzer.java @@ -39,15 +39,13 @@ */ public class ContainerReuseAnalyzer extends TezAnalyzerBase implements Analyzer { - private final Configuration config; - private static final String[] headers = { "vertexName", "taskAttempts", "node", "containerId", "reuseCount" }; private final CSVResult csvResult; public ContainerReuseAnalyzer(Configuration config) { - this.config = config; + super(config); this.csvResult = new CSVResult(headers); } @@ -82,11 +80,6 @@ public String getDescription() { return "Get details on container reuse analysis"; } - @Override - public Configuration getConfiguration() { - return config; - } - public static void main(String[] args) throws Exception { Configuration config = new Configuration(); ContainerReuseAnalyzer analyzer = new ContainerReuseAnalyzer(config); diff --git a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/CriticalPathAnalyzer.java b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/CriticalPathAnalyzer.java index 387b0cf99d..3f5e3004b8 100644 --- a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/CriticalPathAnalyzer.java +++ b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/CriticalPathAnalyzer.java @@ -113,10 +113,11 @@ public List getNotes() { ArrayList concurrencyByTime = Lists.newArrayList(); public CriticalPathAnalyzer() { + super(new Configuration()); } public CriticalPathAnalyzer(Configuration conf) { - setConf(conf); + super(conf); } @Override @@ -643,13 +644,9 @@ public String getDescription() { return "Analyze critical path of the DAG"; } - @Override - public Configuration getConfiguration() { - return getConf(); - } - public static void main(String[] args) throws Exception { - int res = ToolRunner.run(new Configuration(), new CriticalPathAnalyzer(), args); + Configuration config = new Configuration(); + int res = ToolRunner.run(config, new CriticalPathAnalyzer(config), args); System.exit(res); } diff --git a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/DagOverviewAnalyzer.java b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/DagOverviewAnalyzer.java new file mode 100644 index 0000000000..b193c30a90 --- /dev/null +++ b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/DagOverviewAnalyzer.java @@ -0,0 +1,132 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.tez.analyzer.plugins; + +import java.text.SimpleDateFormat; +import java.util.Comparator; +import java.util.Date; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.util.ToolRunner; +import org.apache.tez.analyzer.Analyzer; +import org.apache.tez.analyzer.CSVResult; +import org.apache.tez.analyzer.Result; +import org.apache.tez.dag.api.TezException; +import org.apache.tez.history.parser.datamodel.DagInfo; +import org.apache.tez.history.parser.datamodel.Event; +import org.apache.tez.history.parser.datamodel.TaskAttemptInfo; +import org.apache.tez.history.parser.datamodel.TaskInfo; +import org.apache.tez.history.parser.datamodel.VertexInfo; + +public class DagOverviewAnalyzer extends TezAnalyzerBase implements Analyzer { + private final String[] headers = + { "name", "id", "event_type", "status", "event_time", "event_time_str", "vertex_task_stats", "diagnostics" }; + private final CSVResult csvResult; + private static final SimpleDateFormat FORMAT = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSS"); + + public DagOverviewAnalyzer(Configuration config) { + super(config); + csvResult = new CSVResult(headers); + } + + @Override + public void analyze(DagInfo dagInfo) throws TezException { + for (Event event : dagInfo.getEvents()) { + csvResult.addRecord(new String[] { dagInfo.getDagId(), dagInfo.getDagId(), event.getType(), + dagInfo.getStatus(), Long.toString(event.getTime()), toDateStr(event.getTime()), "", "" }); + } + for (VertexInfo vertex : dagInfo.getVertices()) { + for (Event event : vertex.getEvents()) { + String vertexFailureInfoIfAny = ""; + for (TaskAttemptInfo attempt : vertex.getTaskAttempts()) { + if (attempt.getStatus().contains("FAILED")) { + vertexFailureInfoIfAny = attempt.getTaskAttemptId() + ": " + + attempt.getDiagnostics().replaceAll(",", " ").replaceAll("\n", " "); + break; + } + } + csvResult.addRecord(new String[] { vertex.getVertexName(), vertex.getVertexId(), + event.getType(), vertex.getStatus(), Long.toString(event.getTime()), + toDateStr(event.getTime()), getTaskStats(vertex), vertexFailureInfoIfAny }); + } + + // a failed task can lead to dag failure, so hopefully holds valuable information + for (TaskInfo failedTask : vertex.getFailedTasks()) { + for (Event failedTaskEvent : failedTask.getEvents()) { + if (failedTaskEvent.getType().equalsIgnoreCase("TASK_FINISHED")) { + csvResult.addRecord(new String[] { vertex.getVertexName(), failedTask.getTaskId(), + failedTaskEvent.getType(), failedTask.getStatus(), Long.toString(failedTaskEvent.getTime()), + toDateStr(failedTaskEvent.getTime()), getTaskStats(vertex), + failedTask.getDiagnostics().replaceAll(",", " ").replaceAll("\n", " ") }); + } + } + // if we already found a failing task, let's scan the failing attempts as well + for (TaskAttemptInfo failedAttempt : failedTask.getFailedTaskAttempts()) { + for (Event failedTaskAttemptEvent : failedAttempt.getEvents()) { + if (failedTaskAttemptEvent.getType().equalsIgnoreCase("TASK_ATTEMPT_FINISHED")) { + csvResult.addRecord(new String[] { vertex.getVertexName(), + failedAttempt.getTaskAttemptId(), failedTaskAttemptEvent.getType(), + failedAttempt.getStatus(), Long.toString(failedTaskAttemptEvent.getTime()), + toDateStr(failedTaskAttemptEvent.getTime()), getTaskStats(vertex), + failedAttempt.getDiagnostics().replaceAll(",", " ").replaceAll("\n", " ") }); + } + } + } + } + } + + csvResult.sort(new Comparator() { + public int compare(String[] first, String[] second) { + return (int) (Long.parseLong(first[4]) - Long.parseLong(second[4])); + } + }); + } + + private String getTaskStats(VertexInfo vertex) { + return String.format("numTasks: %d failedTasks: %d completedTasks: %d", vertex.getNumTasks(), + vertex.getFailedTasksCount(), vertex.getCompletedTasksCount()); + } + + private static synchronized String toDateStr(long time) { + return FORMAT.format(new Date(time)); + } + + @Override + public Result getResult() throws TezException { + return csvResult; + } + + @Override + public String getName() { + return "Dag overview analyzer"; + } + + @Override + public String getDescription() { + return "High level dag events overview (dag, vertex event summary)." + + " Helps understand the overall progress of a dag by simply listing the dag/vertex related events"; + } + + public static void main(String[] args) throws Exception { + Configuration config = new Configuration(); + DagOverviewAnalyzer analyzer = new DagOverviewAnalyzer(config); + int res = ToolRunner.run(config, analyzer, args); + analyzer.printResults(); + System.exit(res); + } +} diff --git a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/HungTaskAnalyzer.java b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/HungTaskAnalyzer.java new file mode 100644 index 0000000000..9a38e28ba2 --- /dev/null +++ b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/HungTaskAnalyzer.java @@ -0,0 +1,143 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tez.analyzer.plugins; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.util.ToolRunner; +import org.apache.tez.analyzer.Analyzer; +import org.apache.tez.analyzer.CSVResult; +import org.apache.tez.analyzer.Result; +import org.apache.tez.dag.api.TezException; +import org.apache.tez.dag.records.TezTaskAttemptID; +import org.apache.tez.history.parser.datamodel.DagInfo; +import org.apache.tez.history.parser.datamodel.TaskAttemptInfo; +import org.apache.tez.history.parser.datamodel.VertexInfo; + +import java.util.Comparator; +import java.util.HashMap; +import java.util.Map; + +/** + * Gives insights about hanging task attempts by providing details about last attempts of all tasks. + */ +public class HungTaskAnalyzer extends TezAnalyzerBase implements Analyzer { + private final String[] headers = { "vertex", "task", " number_of_attempts", "last_attempt_id", + "last_attempt_status", "last_attempt_duration_ms", "last_attempt_node" }; + private final CSVResult csvResult; + + private static final String HEADER_NUM_ATTEMPTS = "num_attempts"; + private static final String HEADER_LAST_ATTEMPT_ID_AND_STATUS = "last_attempt_id_and_status"; + private static final String HEADER_LAST_ATTEMPT_STATUS = "last_attempt_status"; + private static final String HEADER_LAST_ATTEMPT_NODE = "last_attempt_node"; + private static final String HEADER_LAST_ATTEMPT_DURATION_MS = "last_attempt_duration_ms"; + + public HungTaskAnalyzer(Configuration config) { + super(config); + csvResult = new CSVResult(headers); + } + + @Override + public void analyze(DagInfo dagInfo) throws TezException { + Map> taskData = new HashMap<>(); // task attempt count per task + for (VertexInfo vertex : dagInfo.getVertices()) { + taskData.clear(); + for (TaskAttemptInfo attempt : vertex.getTaskAttempts()) { + String taskId = attempt.getTaskInfo().getTaskId(); + + int numAttemptsForTask = attempt.getTaskInfo().getNumberOfTaskAttempts(); + Map thisTaskData = taskData.get(taskId); + + if (thisTaskData == null) { + thisTaskData = new HashMap<>(); + thisTaskData.put(HEADER_NUM_ATTEMPTS, Integer.toString(numAttemptsForTask)); + taskData.put(taskId, thisTaskData); + } + + int attemptNumber = TezTaskAttemptID.fromString(attempt.getTaskAttemptId()).getId(); + if (attemptNumber == numAttemptsForTask - 1) { + thisTaskData.put(HEADER_LAST_ATTEMPT_ID_AND_STATUS, String.format("%s/%s", attempt.getTaskAttemptId(), attempt.getStatus())); + thisTaskData.put(HEADER_LAST_ATTEMPT_STATUS, attempt.getDetailedStatus()); + thisTaskData.put(HEADER_LAST_ATTEMPT_NODE, attempt.getNodeId()); + + thisTaskData.put(HEADER_LAST_ATTEMPT_DURATION_MS, + (attempt.getFinishTime() == 0 || attempt.getStartTime() == 0) ? "-1" + : Long.toString(attempt.getFinishTime() - attempt.getStartTime())); + } + } + for (Map.Entry> task : taskData.entrySet()) { + addARecord(vertex.getVertexName(), task.getKey(), task.getValue().get(HEADER_NUM_ATTEMPTS), + task.getValue().get(HEADER_LAST_ATTEMPT_ID_AND_STATUS), task.getValue().get(HEADER_LAST_ATTEMPT_STATUS), + task.getValue().get(HEADER_LAST_ATTEMPT_DURATION_MS), + task.getValue().get(HEADER_LAST_ATTEMPT_NODE)); + } + } + + csvResult.sort(new Comparator() { + public int compare(String[] first, String[] second) { + int vertexOrder = first[0].compareTo(second[0]); + int lastAttemptStatusOrder = + (first[4] == null || second[4] == null) ? 0 : first[4].compareTo(second[4]); + int attemptNumberOrder = Integer.valueOf(second[2]).compareTo(Integer.valueOf(first[2])); + + return vertexOrder == 0 + ? (lastAttemptStatusOrder == 0 ? attemptNumberOrder : lastAttemptStatusOrder) + : vertexOrder; + } + }); + } + + private void addARecord(String vertexName, String taskId, String numAttempts, + String lastAttemptId, String lastAttemptStatus, String lastAttemptDuration, + String lastAttemptNode) { + String[] record = new String[7]; + record[0] = vertexName; + record[1] = taskId; + record[2] = numAttempts; + record[3] = lastAttemptId; + record[4] = lastAttemptStatus; + record[5] = lastAttemptDuration; + record[6] = lastAttemptNode; + + csvResult.addRecord(record); + } + + @Override + public Result getResult() throws TezException { + return csvResult; + } + + @Override + public String getName() { + return "Hung Task Analyzer"; + } + + @Override + public String getDescription() { + return "TaskHandAnalyzer can give quick insights about hanging task attempts" + + " by giving an overview of all tasks and their last attempts' status, duration, etc."; + } + + public static void main(String[] args) throws Exception { + Configuration config = new Configuration(); + HungTaskAnalyzer analyzer = new HungTaskAnalyzer(config); + int res = ToolRunner.run(config, analyzer, args); + analyzer.printResults(); + System.exit(res); + } +} diff --git a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/InputReadErrorAnalyzer.java b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/InputReadErrorAnalyzer.java new file mode 100644 index 0000000000..3cb523ff9a --- /dev/null +++ b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/InputReadErrorAnalyzer.java @@ -0,0 +1,94 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.tez.analyzer.plugins; + +import java.util.Comparator; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.util.ToolRunner; +import org.apache.tez.analyzer.Analyzer; +import org.apache.tez.analyzer.CSVResult; +import org.apache.tez.analyzer.Result; +import org.apache.tez.dag.api.TezException; +import org.apache.tez.history.parser.datamodel.DagInfo; +import org.apache.tez.history.parser.datamodel.Event; +import org.apache.tez.history.parser.datamodel.TaskAttemptInfo; +import org.apache.tez.history.parser.datamodel.VertexInfo; + +/** + * Helps finding the root cause of shuffle errors, e.g. which node(s) can be blamed for them. + */ +public class InputReadErrorAnalyzer extends TezAnalyzerBase implements Analyzer { + private final String[] headers = { "vertex:attempt", "status", "time", "node", "diagnostics" }; + private final CSVResult csvResult; + + public InputReadErrorAnalyzer(Configuration config) { + super(config); + csvResult = new CSVResult(headers); + } + + @Override + public void analyze(DagInfo dagInfo) throws TezException { + for (VertexInfo vertex : dagInfo.getVertices()) { + for (TaskAttemptInfo attempt : vertex.getTaskAttempts()) { + String terminationCause = attempt.getTerminationCause(); + if ("INPUT_READ_ERROR".equalsIgnoreCase(terminationCause) + || "OUTPUT_LOST".equalsIgnoreCase(terminationCause) + || "NODE_FAILED".equalsIgnoreCase(terminationCause)) { + for (Event event : attempt.getEvents()) { + if (event.getType().equalsIgnoreCase("TASK_ATTEMPT_FINISHED")) { + csvResult.addRecord(new String[] { + vertex.getVertexName() + ":" + attempt.getTaskAttemptId(), + attempt.getDetailedStatus(), String.valueOf(event.getTime()), attempt.getNodeId(), + attempt.getDiagnostics().replaceAll(",", " ").replaceAll("\n", " ") }); + } + } + } + } + } + + csvResult.sort(new Comparator() { + public int compare(String[] first, String[] second) { + return (int) (Long.parseLong(second[2]) - Long.parseLong(first[2])); + } + }); + } + + @Override + public Result getResult() throws TezException { + return csvResult; + } + + @Override + public String getName() { + return "Input read error analyzer"; + } + + @Override + public String getDescription() { + return "Prints every task attempt (with node) which are related to input read errors"; + } + + public static void main(String[] args) throws Exception { + Configuration config = new Configuration(); + InputReadErrorAnalyzer analyzer = new InputReadErrorAnalyzer(config); + int res = ToolRunner.run(config, analyzer, args); + analyzer.printResults(); + System.exit(res); + } +} diff --git a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/LocalityAnalyzer.java b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/LocalityAnalyzer.java index ec72df17ac..d640704f93 100644 --- a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/LocalityAnalyzer.java +++ b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/LocalityAnalyzer.java @@ -52,12 +52,10 @@ public class LocalityAnalyzer extends TezAnalyzerBase implements Analyzer { private static final String DATA_LOCAL_RATIO = "tez.locality-analyzer.data.local.ratio"; private static final float DATA_LOCAL_RATIO_DEFAULT = 0.5f; - private final Configuration config; - private final CSVResult csvResult; public LocalityAnalyzer(Configuration config) { - this.config = config; + super(config); csvResult = new CSVResult(headers); } @@ -119,7 +117,7 @@ public void analyze(DagInfo dagInfo) throws TezException { record.add(otherTaskResult.avgHDFSBytesRead + ""); String recommendation = ""; - if (dataLocalRatio < config.getFloat(DATA_LOCAL_RATIO, DATA_LOCAL_RATIO_DEFAULT)) { + if (dataLocalRatio < getConf().getFloat(DATA_LOCAL_RATIO, DATA_LOCAL_RATIO_DEFAULT)) { recommendation = "Data locality is poor for this vertex. Try tuning " + TezConfiguration.TEZ_AM_CONTAINER_REUSE_LOCALITY_DELAY_ALLOCATION_MILLIS + ", " + TezConfiguration.TEZ_AM_CONTAINER_REUSE_RACK_FALLBACK_ENABLED + ", " @@ -182,10 +180,6 @@ private TaskAttemptDetails computeAverages(VertexInfo vertexInfo, DAGCounter cou return "Analyze for locality information (data local, rack local, off-rack)"; } - @Override public Configuration getConfiguration() { - return config; - } - /** * Placeholder for task attempt details */ diff --git a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/OneOnOneEdgeAnalyzer.java b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/OneOnOneEdgeAnalyzer.java index 2ba715ed43..a6cb3f1511 100644 --- a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/OneOnOneEdgeAnalyzer.java +++ b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/OneOnOneEdgeAnalyzer.java @@ -55,12 +55,10 @@ public class OneOnOneEdgeAnalyzer extends TezAnalyzerBase implements Analyzer { // DataMovementType::ONE_TO_ONE private static final String ONE_TO_ONE = "ONE_TO_ONE"; - private final Configuration config; - private final CSVResult csvResult; public OneOnOneEdgeAnalyzer(Configuration config) { - this.config = config; + super(config); csvResult = new CSVResult(headers); } @@ -140,11 +138,6 @@ public String getDescription() { return "To understand the locality miss in 1:1 edge"; } - @Override - public Configuration getConfiguration() { - return config; - } - public static void main(String[] args) throws Exception { Configuration conf = new Configuration(); OneOnOneEdgeAnalyzer analyzer = new OneOnOneEdgeAnalyzer(conf); diff --git a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/ShuffleTimeAnalyzer.java b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/ShuffleTimeAnalyzer.java index 57e91c62d3..f8f9112bb7 100644 --- a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/ShuffleTimeAnalyzer.java +++ b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/ShuffleTimeAnalyzer.java @@ -66,14 +66,12 @@ public class ShuffleTimeAnalyzer extends TezAnalyzerBase implements Analyzer { private final CSVResult csvResult = new CSVResult(headers); - private final Configuration config; - private final float realWorkDoneRatio; private final long minShuffleRecords; public ShuffleTimeAnalyzer(Configuration config) { - this.config = config; + super(config); realWorkDoneRatio = config.getFloat (REAL_WORK_DONE_RATIO, REAL_WORK_DONE_RATIO_DEFAULT); @@ -208,11 +206,6 @@ public String getDescription() { + "and the real work done in the task"; } - @Override - public Configuration getConfiguration() { - return config; - } - public static void main(String[] args) throws Exception { Configuration config = new Configuration(); ShuffleTimeAnalyzer analyzer = new ShuffleTimeAnalyzer(config); diff --git a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/SkewAnalyzer.java b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/SkewAnalyzer.java index 6025541fc9..a7d14fae25 100644 --- a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/SkewAnalyzer.java +++ b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/SkewAnalyzer.java @@ -85,14 +85,12 @@ public class SkewAnalyzer extends TezAnalyzerBase implements Analyzer { private final CSVResult csvResult = new CSVResult(headers); - private final Configuration config; - private final float minRatio; private final float maxRatio; private final long maxShuffleBytesPerSource; public SkewAnalyzer(Configuration config) { - this.config = config; + super(config); maxRatio = config.getFloat(ATTEMPT_SHUFFLE_KEY_GROUP_MAX_RATIO, ATTEMPT_SHUFFLE_KEY_GROUP_MAX_RATIO_DEFAULT); minRatio = config.getFloat(ATTEMPT_SHUFFLE_KEY_GROUP_MIN_RATIO, @@ -214,7 +212,7 @@ private void analyzeRecordSkewPerSource(TaskAttemptInfo attemptInfo) { if (vertexNumTasks > 1) { if (ratio > maxRatio) { //input records > 60% of vertex level record count - if (inputRecordsCount > (vertexLevelInputRecordsCount * 0.60)) { + if (inputRecordsCount > (vertexLevelInputRecordsCount * 0.6f)) { List result = Lists.newLinkedList(); result.add(attemptInfo.getTaskInfo().getVertexInfo().getVertexName()); result.add(attemptInfo.getTaskAttemptId()); @@ -305,12 +303,7 @@ public String getName() { @Override public String getDescription() { - return "Analyzer reducer skews by mining reducer task counters"; - } - - @Override - public Configuration getConfiguration() { - return null; + return "Analyze reducer skews by mining reducer task counters"; } public static void main(String[] args) throws Exception { diff --git a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/SlowNodeAnalyzer.java b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/SlowNodeAnalyzer.java index a810a8a645..9e573c2033 100644 --- a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/SlowNodeAnalyzer.java +++ b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/SlowNodeAnalyzer.java @@ -59,10 +59,8 @@ public class SlowNodeAnalyzer extends TezAnalyzerBase implements Analyzer { private final CSVResult csvResult = new CSVResult(headers); - private final Configuration config; - public SlowNodeAnalyzer(Configuration config) { - this.config = config; + super(config); } @Override @@ -182,11 +180,6 @@ public String getDescription() { return sb.toString(); } - @Override - public Configuration getConfiguration() { - return config; - } - public static void main(String[] args) throws Exception { Configuration config = new Configuration(); SlowNodeAnalyzer analyzer = new SlowNodeAnalyzer(config); diff --git a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/SlowTaskIdentifier.java b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/SlowTaskIdentifier.java index d2474ad0f6..7c9958b250 100644 --- a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/SlowTaskIdentifier.java +++ b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/SlowTaskIdentifier.java @@ -51,10 +51,8 @@ public class SlowTaskIdentifier extends TezAnalyzerBase implements Analyzer { private static final String NO_OF_TASKS = "tez.slow-task-analyzer.task.count"; private static final int NO_OF_TASKS_DEFAULT = 100; - private final Configuration config; - public SlowTaskIdentifier(Configuration config) { - this.config = config; + super(config); this.csvResult = new CSVResult(headers); } @@ -75,7 +73,7 @@ public void analyze(DagInfo dagInfo) throws TezException { }); int limit = Math.min(taskAttempts.size(), - Math.max(0, config.getInt(NO_OF_TASKS, NO_OF_TASKS_DEFAULT))); + Math.max(0, getConf().getInt(NO_OF_TASKS, NO_OF_TASKS_DEFAULT))); if (limit == 0) { return; @@ -111,11 +109,6 @@ public String getDescription() { return "Identifies slow tasks in the DAG"; } - @Override - public Configuration getConfiguration() { - return config; - } - public static void main(String[] args) throws Exception { Configuration config = new Configuration(); SlowTaskIdentifier analyzer = new SlowTaskIdentifier(config); diff --git a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/SlowestVertexAnalyzer.java b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/SlowestVertexAnalyzer.java index 33f2421699..efa39a3223 100644 --- a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/SlowestVertexAnalyzer.java +++ b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/SlowestVertexAnalyzer.java @@ -49,7 +49,6 @@ public class SlowestVertexAnalyzer extends TezAnalyzerBase implements Analyzer { private final CSVResult csvResult = new CSVResult(headers); - private final Configuration config; private final MetricRegistry metrics = new MetricRegistry(); private Histogram taskAttemptRuntimeHistorgram; @@ -59,7 +58,7 @@ public class SlowestVertexAnalyzer extends TezAnalyzerBase implements Analyzer { private final long vertexRuntimeThreshold; public SlowestVertexAnalyzer(Configuration config) { - this.config = config; + super(config); this.vertexRuntimeThreshold = Math.max(1, config.getLong(MAX_VERTEX_RUNTIME, MAX_VERTEX_RUNTIME_DEFAULT)); @@ -204,11 +203,6 @@ public String getDescription() { return "Identify the slowest vertex in the DAG, which needs to be looked into first"; } - @Override - public Configuration getConfiguration() { - return config; - } - public static void main(String[] args) throws Exception { Configuration config = new Configuration(); SlowestVertexAnalyzer analyzer = new SlowestVertexAnalyzer(config); diff --git a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/SpillAnalyzerImpl.java b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/SpillAnalyzerImpl.java index d69ca23b5a..026dd1593f 100644 --- a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/SpillAnalyzerImpl.java +++ b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/SpillAnalyzerImpl.java @@ -60,10 +60,8 @@ public class SpillAnalyzerImpl extends TezAnalyzerBase implements Analyzer { private final long minOutputBytesPerTask; - private final Configuration config; - public SpillAnalyzerImpl(Configuration config) { - this.config = config; + super(config); minOutputBytesPerTask = Math.max(0, config.getLong(OUTPUT_BYTES_THRESHOLD, OUTPUT_BYTES_THRESHOLD_DEFAULT)); this.csvResult = new CSVResult(headers); @@ -130,11 +128,6 @@ public String getDescription() { return "Analyze spill details in the task"; } - @Override - public Configuration getConfiguration() { - return config; - } - public static void main(String[] args) throws Exception { Configuration config = new Configuration(); SpillAnalyzerImpl analyzer = new SpillAnalyzerImpl(config); diff --git a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/TaskAssignmentAnalyzer.java b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/TaskAssignmentAnalyzer.java index ce6fa417c1..02b821f367 100644 --- a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/TaskAssignmentAnalyzer.java +++ b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/TaskAssignmentAnalyzer.java @@ -36,29 +36,27 @@ */ public class TaskAssignmentAnalyzer extends TezAnalyzerBase implements Analyzer { - private final String[] headers = { "vertex", "node", "numTasks", "load" }; - private final Configuration config; + private final String[] headers = { "vertex", "node", "numTaskAttempts", "load" }; private final CSVResult csvResult; public TaskAssignmentAnalyzer(Configuration config) { - this.config = config; + super(config); csvResult = new CSVResult(headers); } @Override public void analyze(DagInfo dagInfo) throws TezException { - Map map = new HashMap<>(); + Map taskAttemptsPerNode = new HashMap<>(); for (VertexInfo vertex : dagInfo.getVertices()) { - map.clear(); + taskAttemptsPerNode.clear(); for (TaskAttemptInfo attempt : vertex.getTaskAttempts()) { - Integer previousValue = map.get(attempt.getNodeId()); - map.put(attempt.getNodeId(), - previousValue == null ? 1 : previousValue + 1); + Integer previousValue = taskAttemptsPerNode.get(attempt.getNodeId()); + taskAttemptsPerNode.put(attempt.getNodeId(), previousValue == null ? 1 : previousValue + 1); } - double mean = vertex.getTaskAttempts().size() / Math.max(1.0, map.size()); - for (Map.Entry assignment : map.entrySet()) { - addARecord(vertex.getVertexName(), assignment.getKey(), - assignment.getValue(), assignment.getValue() * 100 / mean); + double mean = vertex.getTaskAttempts().size() / Math.max(1.0, taskAttemptsPerNode.size()); + for (Map.Entry assignment : taskAttemptsPerNode.entrySet()) { + addARecord(vertex.getVertexName(), assignment.getKey(), assignment.getValue(), + assignment.getValue() * 100 / mean); } } } @@ -88,11 +86,6 @@ public String getDescription() { return "Get the Task assignments on different nodes of the cluster"; } - @Override - public Configuration getConfiguration() { - return config; - } - public static void main(String[] args) throws Exception { Configuration config = new Configuration(); TaskAssignmentAnalyzer analyzer = new TaskAssignmentAnalyzer(config); diff --git a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/TaskAttemptResultStatisticsAnalyzer.java b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/TaskAttemptResultStatisticsAnalyzer.java index df2f95cdce..cf6b2f0d8e 100644 --- a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/TaskAttemptResultStatisticsAnalyzer.java +++ b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/TaskAttemptResultStatisticsAnalyzer.java @@ -44,11 +44,10 @@ public class TaskAttemptResultStatisticsAnalyzer extends TezAnalyzerBase implements Analyzer { private final String[] headers = { "vertex (+task stats: all/succeeded/failed/killed)", "node", "status", "numAttempts" }; - private final Configuration config; private final CSVResult csvResult; public TaskAttemptResultStatisticsAnalyzer(Configuration config) { - this.config = config; + super(config); csvResult = new CSVResult(headers); } @@ -71,7 +70,8 @@ public void analyze(DagInfo dagInfo) throws TezException { } map.forEach((key, value) -> { - addARecord(key.split("#")[0], key.split("#")[1], key.split("#")[2], value); + String[] keys = key.split("#"); + addARecord(keys[0], keys[1], keys.length > 2 ? keys[2] : "", value); }); csvResult.sort(new Comparator() { @@ -110,11 +110,6 @@ public String getDescription() { return "Get statistics about task attempts states in vertex:node:status level"; } - @Override - public Configuration getConfiguration() { - return config; - } - public static void main(String[] args) throws Exception { Configuration config = new Configuration(); TaskAttemptResultStatisticsAnalyzer analyzer = new TaskAttemptResultStatisticsAnalyzer(config); diff --git a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/TaskConcurrencyAnalyzer.java b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/TaskConcurrencyAnalyzer.java index 72f3b36a5b..91f51b4c21 100644 --- a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/TaskConcurrencyAnalyzer.java +++ b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/TaskConcurrencyAnalyzer.java @@ -41,11 +41,10 @@ public class TaskConcurrencyAnalyzer extends TezAnalyzerBase implements Analyzer private static final String[] headers = { "time", "vertexName", "concurrentTasksRunning" }; private final CSVResult csvResult; - private final Configuration config; public TaskConcurrencyAnalyzer(Configuration conf) { + super(conf); this.csvResult = new CSVResult(headers); - this.config = conf; } private enum EventType {START, FINISH} @@ -153,11 +152,6 @@ public String getDescription() { + "would be helpful in understanding whether any starvation was there or not."; } - @Override - public Configuration getConfiguration() { - return config; - } - public static void main(String[] args) throws Exception { Configuration config = new Configuration(); TaskConcurrencyAnalyzer analyzer = new TaskConcurrencyAnalyzer(config); diff --git a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/TezAnalyzerBase.java b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/TezAnalyzerBase.java index 75a55a754d..705c6e9cfb 100644 --- a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/TezAnalyzerBase.java +++ b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/TezAnalyzerBase.java @@ -33,6 +33,7 @@ import org.apache.commons.cli.Options; import org.apache.commons.cli.ParseException; import org.apache.commons.io.FileUtils; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configured; import org.apache.hadoop.fs.Path; import org.apache.hadoop.util.Tool; @@ -67,7 +68,11 @@ public abstract class TezAnalyzerBase extends Configured implements Tool, Analyz private String outputDir; private boolean saveResults = false; - + + public TezAnalyzerBase(Configuration config) { + setConf(config); + } + @SuppressWarnings("static-access") private static Options buildOptions() { Option dagIdOption = OptionBuilder.withArgName(DAG_ID).withLongOpt(DAG_ID) diff --git a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/VertexLevelCriticalPathAnalyzer.java b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/VertexLevelCriticalPathAnalyzer.java index 06b8983e9d..78a4d41f38 100644 --- a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/VertexLevelCriticalPathAnalyzer.java +++ b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/VertexLevelCriticalPathAnalyzer.java @@ -44,8 +44,6 @@ * Identify a set of vertices which fall in the critical path in a DAG. */ public class VertexLevelCriticalPathAnalyzer extends TezAnalyzerBase implements Analyzer { - private final Configuration config; - private static final String[] headers = { "CriticalPath", "Score" }; private final CSVResult csvResult; @@ -58,7 +56,7 @@ public class VertexLevelCriticalPathAnalyzer extends TezAnalyzerBase implements private static final String CONNECTOR = "-->"; public VertexLevelCriticalPathAnalyzer(Configuration config) { - this.config = config; + super(config); this.csvResult = new CSVResult(headers); this.dotFileLocation = config.get(DOT_FILE_DIR, DOT_FILE_DIR_DEFAULT); } @@ -105,11 +103,6 @@ public String getDescription() { return "Analyze vertex level critical path of the DAG"; } - @Override - public Configuration getConfiguration() { - return config; - } - private static Map sortByValues(Map result) { //Sort result by time in reverse order final Ordering reversValueOrdering = From f56075c552dd3c41f8d812d595882eed22f98537 Mon Sep 17 00:00:00 2001 From: belugabehr <12578579+belugabehr@users.noreply.github.com> Date: Mon, 9 Aug 2021 16:05:13 -0400 Subject: [PATCH 063/137] TEZ-4321: Do Not Wrap Singleton Collections (#138) --- .../apache/tez/dag/app/dag/impl/BroadcastEdgeManager.java | 6 ++---- .../apache/tez/dag/app/dag/impl/OneToOneEdgeManager.java | 3 +-- .../tez/dag/app/dag/impl/OneToOneEdgeManagerOnDemand.java | 3 +-- .../java/org/apache/tez/dag/app/TestMockDAGAppMaster.java | 3 +-- 4 files changed, 5 insertions(+), 10 deletions(-) diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/BroadcastEdgeManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/BroadcastEdgeManager.java index ca510f7fc4..867403fcd5 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/BroadcastEdgeManager.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/BroadcastEdgeManager.java @@ -53,8 +53,7 @@ public int getNumSourceTaskPhysicalOutputs(int sourceTaskIndex) { public void routeDataMovementEventToDestination(DataMovementEvent event, int sourceTaskIndex, int sourceOutputIndex, Map> destinationTaskAndInputIndices) { - List inputIndices = - Collections.unmodifiableList(Collections.singletonList(sourceTaskIndex)); + List inputIndices = Collections.singletonList(sourceTaskIndex); // for each task make the i-th source task as the i-th physical input for (int i=0; i> destinationTaskAndInputIndices) { - List inputIndices = - Collections.unmodifiableList(Collections.singletonList(sourceTaskIndex)); + List inputIndices = Collections.singletonList(sourceTaskIndex); // for each task make the i-th source task as the i-th physical input for (int i=0; i destinationInputIndices = - Collections.unmodifiableList(Collections.singletonList(0)); + final List destinationInputIndices = Collections.singletonList(0); final AtomicBoolean stateChecked = new AtomicBoolean(false); public OneToOneEdgeManager(EdgeManagerPluginContext context) { 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 index e2dbf457cf..464d87df59 100644 --- 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 @@ -34,8 +34,7 @@ public class OneToOneEdgeManagerOnDemand extends EdgeManagerPluginOnDemand { - final List destinationInputIndices = - Collections.unmodifiableList(Collections.singletonList(0)); + final List destinationInputIndices = Collections.singletonList(0); final AtomicBoolean stateChecked = new AtomicBoolean(false); final EventRouteMetadata commonRouteMeta = diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/TestMockDAGAppMaster.java b/tez-dag/src/test/java/org/apache/tez/dag/app/TestMockDAGAppMaster.java index 5dca6940ea..aa5d3b9535 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/TestMockDAGAppMaster.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/TestMockDAGAppMaster.java @@ -300,8 +300,7 @@ public void testBasicEvents() throws Exception { } public static class LegacyEdgeTestEdgeManager extends EdgeManagerPlugin { - List destinationInputIndices = - Collections.unmodifiableList(Collections.singletonList(0)); + List destinationInputIndices = Collections.singletonList(0); public LegacyEdgeTestEdgeManager(EdgeManagerPluginContext context) { super(context); } From fc63c04039c17ca05107f515023d6d627fc302d8 Mon Sep 17 00:00:00 2001 From: Bodor Laszlo Date: Tue, 10 Aug 2021 23:02:40 +0200 Subject: [PATCH 064/137] TEZ-4293: Use Tez credentials for CI (#141) --- Jenkinsfile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/Jenkinsfile b/Jenkinsfile index 835a0523a4..aadfaa953e 100644 --- a/Jenkinsfile +++ b/Jenkinsfile @@ -61,7 +61,7 @@ pipeline { stage ('precommit-run') { steps { withCredentials( - [usernamePassword(credentialsId: 'apache-hadoop-at-github.com', + [usernamePassword(credentialsId: 'apache-tez-at-github.com', passwordVariable: 'GITHUB_TOKEN', usernameVariable: 'GITHUB_USER'), usernamePassword(credentialsId: 'tez-ci', From 8c97365325b9667178537a454bc150cc5ce9b981 Mon Sep 17 00:00:00 2001 From: Takanobu Asanuma Date: Thu, 19 Aug 2021 19:42:24 +0900 Subject: [PATCH 065/137] TEZ-4323: Jetty jars were removed from dist package with TEZ-4114 (#140) (Takanobu Asanuma reviewed by Jonathan Turner Eagles) --- pom.xml | 4 ---- 1 file changed, 4 deletions(-) diff --git a/pom.xml b/pom.xml index ba65cc6ad6..3785352bf5 100644 --- a/pom.xml +++ b/pom.xml @@ -289,10 +289,6 @@ commons-logging commons-logging-api - - org.eclipse.jetty - * - com.sun.jersey jersey-core From e2c4ee0842e43e67e4d1dadc8308b427b24aadee Mon Sep 17 00:00:00 2001 From: jteagles Date: Tue, 24 Aug 2021 10:13:15 -0500 Subject: [PATCH 066/137] TEZ-4328. Import external tez component em-helpers (#145) Co-authored-by: Sreenath Somarajapuram --- .../src/main/resources/META-INF/LICENSE.txt | 1 - .../webapp/app/components/em-breadcrumbs.js | 69 +++++++++ .../main/webapp/app/components/em-progress.js | 102 ++++++++++++ tez-ui/src/main/webapp/app/helpers/txt.js | 62 ++++++++ tez-ui/src/main/webapp/app/styles/app.less | 5 + .../webapp/app/styles/em-breadcrumbs.less | 29 ++++ .../main/webapp/app/styles/em-progress.less | 52 +++++++ tez-ui/src/main/webapp/app/styles/shared.less | 4 + tez-ui/src/main/webapp/app/styles/txt.less | 24 +++ .../templates/components/em-breadcrumbs.hbs | 45 ++++++ .../app/templates/components/em-progress.hbs | 25 +++ .../src/main/webapp/app/utils/formatters.js | 146 ++++++++++++++++++ tez-ui/src/main/webapp/config/environment.js | 4 + tez-ui/src/main/webapp/package.json | 1 - .../components/em-breadcrumbs-test.js | 107 +++++++++++++ .../components/em-progress-test.js | 73 +++++++++ .../webapp/tests/unit/helpers/txt-test.js | 59 +++++++ .../tests/unit/utils/formatters-test.js | 99 ++++++++++++ tez-ui/src/main/webapp/yarn.lock | 10 -- 19 files changed, 905 insertions(+), 12 deletions(-) create mode 100644 tez-ui/src/main/webapp/app/components/em-breadcrumbs.js create mode 100644 tez-ui/src/main/webapp/app/components/em-progress.js create mode 100644 tez-ui/src/main/webapp/app/helpers/txt.js create mode 100644 tez-ui/src/main/webapp/app/styles/em-breadcrumbs.less create mode 100644 tez-ui/src/main/webapp/app/styles/em-progress.less create mode 100644 tez-ui/src/main/webapp/app/styles/txt.less create mode 100644 tez-ui/src/main/webapp/app/templates/components/em-breadcrumbs.hbs create mode 100644 tez-ui/src/main/webapp/app/templates/components/em-progress.hbs create mode 100644 tez-ui/src/main/webapp/app/utils/formatters.js create mode 100644 tez-ui/src/main/webapp/tests/integration/components/em-breadcrumbs-test.js create mode 100644 tez-ui/src/main/webapp/tests/integration/components/em-progress-test.js create mode 100644 tez-ui/src/main/webapp/tests/unit/helpers/txt-test.js create mode 100644 tez-ui/src/main/webapp/tests/unit/utils/formatters-test.js diff --git a/tez-ui/src/main/resources/META-INF/LICENSE.txt b/tez-ui/src/main/resources/META-INF/LICENSE.txt index ba6f29b344..354d7457a9 100644 --- a/tez-ui/src/main/resources/META-INF/LICENSE.txt +++ b/tez-ui/src/main/resources/META-INF/LICENSE.txt @@ -233,7 +233,6 @@ The Apache TEZ tez-ui bundles the following files under the MIT License: - snippet-ss v1.11.0 (https://github.com/sreenaths/snippet-ss) - em-tgraph v0.0.4 (https://github.com/sreenaths/em-tgraph) - em-table v0.3.12 (https://github.com/sreenaths/em-table) - - em-helpers v0.5.8 (https://github.com/sreenaths/em-helpers) - ember-cli-app-version v1.0.0 (https://github.com/EmberSherpa/ember-cli-app-version) - Authored by Taras Mankovski - ember-cli-auto-register v1.1.0 (https://github.com/williamsbdev/ember-cli-auto-register) - Copyright © 2015 Brandon Williams http://williamsbdev.com - ember-cli-content-security-policy v0.4.0 (https://github.com/rwjblue/ember-cli-content-security-policy) diff --git a/tez-ui/src/main/webapp/app/components/em-breadcrumbs.js b/tez-ui/src/main/webapp/app/components/em-breadcrumbs.js new file mode 100644 index 0000000000..fcf90cfc5c --- /dev/null +++ b/tez-ui/src/main/webapp/app/components/em-breadcrumbs.js @@ -0,0 +1,69 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import Ember from 'ember'; +import layout from '../templates/components/em-breadcrumbs'; + +export default Ember.Component.extend({ + layout: layout, + + itemStyle: Ember.computed("items", function () { + var itemCount = this.get("items.length"); + + if(itemCount) { + let widthPercent = 100 / itemCount; + return new Ember.Handlebars.SafeString(`max-width: ${widthPercent}%`); + } + }), + + normalizedItems: Ember.computed("items", function () { + var items = this.get("items"); + + if(items) { + let lastIndex = items.length - 1; + items = items.map(function (item, index) { + var itemDef = { + text: item.text || "", + classNames: item.classNames || [], + }; + + Ember.assert("classNames must be an array", Array.isArray(itemDef.classNames)); + + if(index === lastIndex) { + itemDef.classNames.push("active"); + } + else { + itemDef.routeName = item.routeName; + itemDef.model = item.model; + itemDef.href = item.href; + if(item.queryParams) { + itemDef.queryParams = { + isQueryParams: true, + values: item.queryParams + }; + } + } + + itemDef.classNames = itemDef.classNames.join(" "); + return itemDef; + }); + } + + return items; + }) +}); diff --git a/tez-ui/src/main/webapp/app/components/em-progress.js b/tez-ui/src/main/webapp/app/components/em-progress.js new file mode 100644 index 0000000000..926764b300 --- /dev/null +++ b/tez-ui/src/main/webapp/app/components/em-progress.js @@ -0,0 +1,102 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import Ember from 'ember'; +import layout from '../templates/components/em-progress'; + +export default Ember.Component.extend({ + layout: layout, + + value: 0, + valueMin: 0, + valueMax: 1, + + classNames: ["em-progress-container"], + classNameBindings: ["animated", "striped"], + + striped: false, + style: null, + + progressBar: null, + + widthPercent: Ember.computed("value", "valueMin", "valueMax", function () { + var value = this.get("value"), + valueMin = this.get("valueMin"), + valueMax = this.get("valueMax"); + + if(value < valueMin) { + value = valueMin; + } + else if(value > valueMax) { + value = valueMax; + } + + value -= valueMin; + valueMax -= valueMin; + + return (value / valueMax) * 100; + }), + + progressText: Ember.computed("widthPercent", function () { + var percent = parseInt(this.get("widthPercent")); + if(isNaN(percent)) { + percent = 0; + } + return percent + "%"; + }), + + animated: Ember.computed("widthPercent", "striped", function () { + return this.get('striped') && this.get('widthPercent') > 0 && this.get('widthPercent') < 100; + }), + + progressBarClasses: Ember.computed("style", "striped", "animated", function () { + var classes = [], + style = this.get("style"); + + if(style) { + classes.push(`progress-bar-${style}`); + } + if(this.get("striped")) { + classes.push("progress-bar-striped"); + } + if(this.get("animated")) { + classes.push("active"); + } + + return classes.join(" "); + }), + + renderProgress: Ember.observer("progressBar", "widthPercent", function () { + var widthPercent = this.get('widthPercent'); + this.get("progressBar").width(widthPercent + "%"); + }), + + didInsertElement: function () { + Ember.run.scheduleOnce('afterRender', this, function() { + this.setProperties({ + progressBar: this.$(".progress-bar") + }); + }); + }, + + willDestroy: function () { + this.setProperties({ + progressBar: null, + }); + } +}); diff --git a/tez-ui/src/main/webapp/app/helpers/txt.js b/tez-ui/src/main/webapp/app/helpers/txt.js new file mode 100644 index 0000000000..81fdbdaee5 --- /dev/null +++ b/tez-ui/src/main/webapp/app/helpers/txt.js @@ -0,0 +1,62 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import Ember from 'ember'; +import formatters from '../utils/formatters'; + +export function txt(value, hash) { + var message, + dataType = hash.type, + formatter = hash.formatter, + titleAttr = ""; + + if(value) { + value = value[0]; + } + + if(value instanceof Error) { + message = value.message; + titleAttr = `title="${value.message}" `; + } + else { + try { + if(value !== undefined && !formatter && dataType) { + formatter = formatters[dataType]; + } + + if(formatter && value !== undefined && value !== null) { + value = formatter(value, hash); + } + + if(value === undefined || value === null) { + message = 'Not Available!'; + } + else { + return Ember.String.htmlSafe(Ember.Handlebars.Utils.escapeExpression(value.toString())); + } + } + catch(error) { + message = "Invalid Data!"; + Ember.Logger.error(error); + } + } + + return Ember.String.htmlSafe(` ${message} `); +} + +export default Ember.Helper.helper(txt); diff --git a/tez-ui/src/main/webapp/app/styles/app.less b/tez-ui/src/main/webapp/app/styles/app.less index 7c662298c9..f8a66e376d 100644 --- a/tez-ui/src/main/webapp/app/styles/app.less +++ b/tez-ui/src/main/webapp/app/styles/app.less @@ -44,6 +44,8 @@ @import "em-table-status-cell"; @import "query-timeline"; @import "home-table-controls"; +@import "em-progress"; +@import "em-breadcrumbs"; // Modals @import "column-selector"; @@ -54,3 +56,6 @@ @import "details-page"; @import "swimlane-page"; @import "vertex-configs-page"; + +// Helpers +@import "txt"; diff --git a/tez-ui/src/main/webapp/app/styles/em-breadcrumbs.less b/tez-ui/src/main/webapp/app/styles/em-breadcrumbs.less new file mode 100644 index 0000000000..05ed77ed27 --- /dev/null +++ b/tez-ui/src/main/webapp/app/styles/em-breadcrumbs.less @@ -0,0 +1,29 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + + +@import "bower_components/snippet-ss/less/no"; + +.breadcrumb { + .no-wrap; + li { + .no-wrap; + overflow: hidden; + text-overflow: ellipsis; + } +} diff --git a/tez-ui/src/main/webapp/app/styles/em-progress.less b/tez-ui/src/main/webapp/app/styles/em-progress.less new file mode 100644 index 0000000000..416b6966d9 --- /dev/null +++ b/tez-ui/src/main/webapp/app/styles/em-progress.less @@ -0,0 +1,52 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + + +@import "bower_components/bootstrap/less/variables"; +@import "bower_components/snippet-ss/less/effects"; + +.progress { + .progress-bar { + .progress-text { + padding-left: 10px; + .text-outer-glow(@progress-bar-bg); + } + } + + .progress-bar-success { + .progress-text { + .text-outer-glow(@progress-bar-success-bg); + } + } + .progress-bar-info { + .progress-text { + .text-outer-glow(@progress-bar-info-bg); + } + } + .progress-bar-warning { + .progress-text { + .text-outer-glow(@progress-bar-warning-bg); + } + } + .progress-bar-danger { + .progress-text { + .text-outer-glow(@progress-bar-danger-bg); + } + } +} + diff --git a/tez-ui/src/main/webapp/app/styles/shared.less b/tez-ui/src/main/webapp/app/styles/shared.less index dedac8c91b..b34cfa6045 100644 --- a/tez-ui/src/main/webapp/app/styles/shared.less +++ b/tez-ui/src/main/webapp/app/styles/shared.less @@ -86,3 +86,7 @@ b { } } } + +.em-message { + opacity: .5; +} diff --git a/tez-ui/src/main/webapp/app/styles/txt.less b/tez-ui/src/main/webapp/app/styles/txt.less new file mode 100644 index 0000000000..171c250f73 --- /dev/null +++ b/tez-ui/src/main/webapp/app/styles/txt.less @@ -0,0 +1,24 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + + +@import "./shared"; + +.txt-message { + .em-message; +} diff --git a/tez-ui/src/main/webapp/app/templates/components/em-breadcrumbs.hbs b/tez-ui/src/main/webapp/app/templates/components/em-breadcrumbs.hbs new file mode 100644 index 0000000000..f50972a01a --- /dev/null +++ b/tez-ui/src/main/webapp/app/templates/components/em-breadcrumbs.hbs @@ -0,0 +1,45 @@ +{{! + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +}} + +

diff --git a/tez-ui/src/main/webapp/app/templates/components/em-progress.hbs b/tez-ui/src/main/webapp/app/templates/components/em-progress.hbs new file mode 100644 index 0000000000..bd7fafcda5 --- /dev/null +++ b/tez-ui/src/main/webapp/app/templates/components/em-progress.hbs @@ -0,0 +1,25 @@ +{{! + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +}} + +
+
+ + {{progressText}} + +
+
diff --git a/tez-ui/src/main/webapp/app/utils/formatters.js b/tez-ui/src/main/webapp/app/utils/formatters.js new file mode 100644 index 0000000000..d724eb85ba --- /dev/null +++ b/tez-ui/src/main/webapp/app/utils/formatters.js @@ -0,0 +1,146 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import Ember from 'ember'; + +import moment from 'moment'; +import numeral from 'numeral'; + +const DEFAULT_DATE_TIMEZONE = "UTC", + DEFAULT_DATE_FORMAT = "DD MMM YYYY HH:mm:ss", + DEFAULT_NUM_FORMAT = '0,0', + DEFAULT_MEM_FORMAT = '0 b'; + +function durationFormatter(arr, value, unit) { + if(value > 0) { + if(value > 1) { + unit += 's'; + } + arr.push(value + unit); + } +} + +const DURATION_FORMATS = { + long: { + collateFunction: durationFormatter, + + year: " year", + month: " month", + day: " day", + hour: " hour", + minute: " minute", + second: " second", + millisecond: " millisecond" + }, + short: { + collateFunction: durationFormatter, + + year: " yr", + month: " mo", + day: " day", + hour: " hr", + minute: " min", + second: " sec", + millisecond: " msec" + }, + xshort: { + collateFunction: function (arr, value, unit) { + if(value > 0) { + arr.push(value + unit); + } + }, + + year: "Y", + month: "M", + day: "D", + hour: "h", + minute: "m", + second: "s", + millisecond: "ms" + } +}; + +function validateNumber(value, message) { + value = parseFloat(value); + + if(isNaN(value)) { + throw new Error(message || "Invalid number!"); + } + + return value; +} + +export default Ember.Controller.create({ + date: function (value, options) { + var date = moment.tz(value, options.valueFormat, options.valueTimeZone || DEFAULT_DATE_TIMEZONE); + + date = options.timeZone ? date.tz(options.timeZone) : date.local(); + date = date.format(options.format || DEFAULT_DATE_FORMAT); + + if(date === "Invalid date") { + throw new Error(date); + } + + return date; + }, + duration: function (value, options) { + var format = DURATION_FORMATS[options.format || "xshort"], + duration, + ret = []; + + value = validateNumber(value, "Invalid duration"); + + if(value === 0) { + return `0${format.millisecond}`; + } + + duration = moment.duration(value, options.valueUnit); + + format.collateFunction(ret, duration.years(), format.year); + format.collateFunction(ret, duration.months(), format.month); + format.collateFunction(ret, duration.days(), format.day); + format.collateFunction(ret, duration.hours(), format.hour); + format.collateFunction(ret, duration.minutes(), format.minute); + format.collateFunction(ret, duration.seconds(), format.second); + format.collateFunction(ret, Math.round(duration.milliseconds()), format.millisecond); + + return ret.join(" "); + }, + number: function (value, options) { + value = validateNumber(value); + return numeral(value).format(options.format || DEFAULT_NUM_FORMAT); + }, + memory: function (value) { + value = validateNumber(value, "Invalid memory"); + if(value === 0) { + return "0 B"; + } + return numeral(value).format(DEFAULT_MEM_FORMAT); + }, + json: function (value, options) { + if(value && typeof value === "object" && value.constructor === Object) { + try { + value = JSON.stringify(value, options.replacer, options.space || 4); + } + catch(err){ + Ember.Logger.error(err); + } + } + return value; + } +}); diff --git a/tez-ui/src/main/webapp/config/environment.js b/tez-ui/src/main/webapp/config/environment.js index 0c755ac700..68a16cf803 100644 --- a/tez-ui/src/main/webapp/config/environment.js +++ b/tez-ui/src/main/webapp/config/environment.js @@ -39,6 +39,10 @@ module.exports = function(environment) { 'child-src': "'self' 'unsafe-inline'", 'style-src': "'self' 'unsafe-inline'", 'script-src': "'self' 'unsafe-inline'" + }, + + moment: { + includeTimezone: '2010-2020' } }; diff --git a/tez-ui/src/main/webapp/package.json b/tez-ui/src/main/webapp/package.json index 92f54d266d..fa80389b94 100644 --- a/tez-ui/src/main/webapp/package.json +++ b/tez-ui/src/main/webapp/package.json @@ -61,7 +61,6 @@ "phantomjs-prebuilt": "2.1.13" }, "dependencies": { - "em-helpers": "0.8.0", "em-table": "0.11.3", "em-tgraph": "0.0.14" } diff --git a/tez-ui/src/main/webapp/tests/integration/components/em-breadcrumbs-test.js b/tez-ui/src/main/webapp/tests/integration/components/em-breadcrumbs-test.js new file mode 100644 index 0000000000..63edbc491a --- /dev/null +++ b/tez-ui/src/main/webapp/tests/integration/components/em-breadcrumbs-test.js @@ -0,0 +1,107 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import Ember from 'ember'; + +import { moduleForComponent, test } from 'ember-qunit'; +import hbs from 'htmlbars-inline-precompile'; + +moduleForComponent('em-breadcrumbs', 'Integration | Component | em breadcrumbs', { + integration: true +}); + +test('Basic creation test', function(assert) { + + // Set any properties with this.set('myProperty', 'value'); + // Handle any actions with this.on('myAction', function(val) { ... });" + EOL + EOL + + + this.render(hbs`{{em-breadcrumbs}}`); + + assert.equal(this.$().text().trim(), ''); + + // Template block usage:" + EOL + + this.render(hbs` + {{#em-breadcrumbs}} + template block text + {{/em-breadcrumbs}} + `); + + assert.equal(this.$().text().trim(), ''); +}); + +test('Test with one link-to item', function(assert) { + var testItems = [{ + routeName: "foo", + text: "fooText" + }], + elements; + + this.set("items", testItems); + this.render(hbs`{{em-breadcrumbs items=items}}`); + + elements = this.$("li"); + + assert.equal(elements.length, 1); + assert.equal(Ember.$(elements[0]).text().trim(), testItems[0].text); + assert.equal(elements[0].title, testItems[0].text); + assert.equal(elements[0].style.maxWidth, "100%"); +}); + +test('Test with two link-to item', function(assert) { + var testItems = [{ + routeName: "foo", + text: "fooText" + },{ + routeName: "bar", + text: "barText" + }], + elements; + + this.set("items", testItems); + this.render(hbs`{{em-breadcrumbs items=items}}`); + + elements = this.$("li"); + + assert.equal(elements.length, 2); + + assert.equal(Ember.$(elements[0]).text().trim(), testItems[0].text); + assert.equal(elements[0].title, testItems[0].text); + assert.equal(elements[0].style.maxWidth, "50%"); + + assert.equal(Ember.$(elements[1]).text().trim(), testItems[1].text); + assert.equal(elements[1].title, testItems[1].text); + assert.equal(elements[1].style.maxWidth, "50%"); +}); + +test('Test with one anchor tag item', function(assert) { + var testItems = [{ + href: "foo.bar", + text: "fooText" + }], + elements; + + this.set("items", testItems); + this.render(hbs`{{em-breadcrumbs items=items}}`); + + elements = this.$("li"); + + assert.equal(elements.length, 1); + assert.equal(Ember.$(elements[0]).text().trim(), testItems[0].text); + assert.equal(elements[0].title, testItems[0].text); + assert.equal(elements[0].style.maxWidth, "100%"); +}); diff --git a/tez-ui/src/main/webapp/tests/integration/components/em-progress-test.js b/tez-ui/src/main/webapp/tests/integration/components/em-progress-test.js new file mode 100644 index 0000000000..8fcdfaf1d0 --- /dev/null +++ b/tez-ui/src/main/webapp/tests/integration/components/em-progress-test.js @@ -0,0 +1,73 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import { moduleForComponent, test } from 'ember-qunit'; +import hbs from 'htmlbars-inline-precompile'; + +moduleForComponent('em-progress', 'Integration | Component | em progress', { + integration: true +}); + +test('It renders', function(assert) { + + // Set any properties with this.set('myProperty', 'value'); + // Handle any actions with this.on('myAction', function(val) { ... });" + EOL + EOL + + + this.render(hbs`{{em-progress}}`); + + assert.equal(this.$().text().trim(), '0%'); + + this.render(hbs`{{#em-progress}}{{/em-progress}}`); + assert.equal(this.$().text().trim(), '0%'); +}); + +test('With a specific value', function(assert) { + this.render(hbs`{{em-progress value=0.5}}`); + assert.equal(this.$().text().trim(), '50%'); +}); + +test('Custom valueMin & valueMax', function(assert) { + this.render(hbs`{{em-progress value=15 valueMin=10 valueMax=20}}`); + assert.equal(this.$().text().trim(), '50%'); + + assert.notOk(this.$('.striped')[0], "Striped class added"); +}); + +test('Check for stripes & animation while in progress', function(assert) { + this.render(hbs`{{em-progress value=0.5 striped=true}}`); + + assert.equal(this.$().text().trim(), '50%'); + assert.ok(this.$('.striped')[0], "Striped class added"); + assert.ok(this.$('.animated')[0], "Animated class should be added!"); +}); + +test('Check for stripes & animation while starting', function(assert) { + this.render(hbs`{{em-progress value=0 striped=true}}`); + + assert.equal(this.$().text().trim(), '0%'); + assert.ok(this.$('.striped')[0], "Striped class added"); + assert.ok(!this.$('.animated')[0], "Animated class shouldn't be added!"); +}); + +test('Check for stripes & animation on completion', function(assert) { + this.render(hbs`{{em-progress value=1 striped=true}}`); + + assert.equal(this.$().text().trim(), '100%'); + assert.ok(this.$('.striped')[0], "Striped class added"); + assert.ok(!this.$('.animated')[0], "Animated class shouldn't be added!"); +}); diff --git a/tez-ui/src/main/webapp/tests/unit/helpers/txt-test.js b/tez-ui/src/main/webapp/tests/unit/helpers/txt-test.js new file mode 100644 index 0000000000..18d3de7ace --- /dev/null +++ b/tez-ui/src/main/webapp/tests/unit/helpers/txt-test.js @@ -0,0 +1,59 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import { txt } from '../../../helpers/txt'; +import { module, test } from 'qunit'; + +module('Unit | Helper | txt'); + +test('txt: created', function(assert) { + assert.ok(txt); +}); + +test('txt: String', function(assert) { + assert.equal(txt(["Abc"], {}), "Abc"); + assert.equal(txt(null, {}).string, ' Not Available! '); +}); + +test('txt: String - success', function(assert) { + assert.equal(txt(["Abc"], {}), "Abc"); + assert.equal(txt(null, {}).string, ' Not Available! '); + assert.equal(txt([null], {}).string, ' Not Available! '); +}); + +test('txt: String - error', function(assert) { + var obj = {}; + + obj.toString = null; + assert.equal(txt([obj], {}).string, ' Invalid Data! '); +}); + +test('txt: json', function(assert) { + var obj = { + x: 1, + y: 2 + }; + assert.equal(txt([obj], { + type: "json", + }).string, '{\n "x": 1,\n "y": 2\n}'); +}); + +test('txt: error', function(assert) { + var err = new Error("testError"); + assert.equal(txt([err], {}).string, ' testError '); +}); diff --git a/tez-ui/src/main/webapp/tests/unit/utils/formatters-test.js b/tez-ui/src/main/webapp/tests/unit/utils/formatters-test.js new file mode 100644 index 0000000000..4ecc14309c --- /dev/null +++ b/tez-ui/src/main/webapp/tests/unit/utils/formatters-test.js @@ -0,0 +1,99 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import fmts from '../../../utils/formatters'; +import { module, test } from 'qunit'; + +import Ember from 'ember'; + +module('Unit | Utility | formatters'); + +test('Formatter functions created', function(assert) { + assert.ok(fmts); + + assert.ok(fmts.date); + assert.ok(fmts.duration); + assert.ok(fmts.number); + assert.ok(fmts.memory); +}); + +test('duration', function(assert) { + var options = { + format: "long" + }; + assert.equal(fmts.duration(0, options), "0 millisecond"); + assert.equal(fmts.duration(1, options), "1 millisecond"); + assert.equal(fmts.duration(60, options), "60 milliseconds"); + assert.equal(fmts.duration(6000, options), "6 seconds"); + assert.equal(fmts.duration(66000, options), "1 minute 6 seconds"); + assert.equal(fmts.duration(666000, options), "11 minutes 6 seconds"); + assert.equal(fmts.duration(6666000, options), "1 hour 51 minutes 6 seconds"); + assert.equal(fmts.duration(66666000, options), "18 hours 31 minutes 6 seconds"); + + options = { + format: "short" + }; // By default format = short + assert.equal(fmts.duration(0, options), "0 msec"); + assert.equal(fmts.duration(60, options), "60 msecs"); + assert.equal(fmts.duration(6000, options), "6 secs"); + assert.equal(fmts.duration(66000, options), "1 min 6 secs"); + assert.equal(fmts.duration(666000, options), "11 mins 6 secs"); + assert.equal(fmts.duration(6666000, options), "1 hr 51 mins 6 secs"); + assert.equal(fmts.duration(66666000, options), "18 hrs 31 mins 6 secs"); + + assert.equal(fmts.duration(60.4, options), "60 msecs"); + assert.equal(fmts.duration(60.6, options), "61 msecs"); + + options = {}; // By default format = xshort + assert.equal(fmts.duration(0, options), "0ms"); + assert.equal(fmts.duration(60, options), "60ms"); + assert.equal(fmts.duration(6000, options), "6s"); + assert.equal(fmts.duration(66000, options), "1m 6s"); + assert.equal(fmts.duration(666000, options), "11m 6s"); + assert.equal(fmts.duration(6666000, options), "1h 51m 6s"); + assert.equal(fmts.duration(66666000, options), "18h 31m 6s"); +}); + +test('number', function(assert) { + assert.equal(fmts.number(6000, {}), "6,000"); + assert.equal(fmts.number(6000000, {}), "6,000,000"); +}); + +test('memory', function(assert) { + assert.equal(fmts.memory(0, {}), "0 B"); + assert.equal(fmts.memory(600, {}), "600 B"); + assert.equal(fmts.memory(1024, {}), "1 KB"); + assert.equal(fmts.memory(1024 * 1024, {}), "1 MB"); + assert.equal(fmts.memory(1024 * 1024 * 1024, {}), "1 GB"); + assert.equal(fmts.memory(1024 * 1024 * 1024 * 1024, {}), "1 TB"); +}); + +test('json', function(assert) { + var str = "testString", + complexObj = Ember.Object.create(); + + assert.equal(fmts.json(str, {}), str); + assert.equal(fmts.json(complexObj, {}), complexObj); + + assert.equal(fmts.json(null, {}), null); + assert.equal(fmts.json(undefined, {}), undefined); + + assert.equal(fmts.json({x: 1}, {}), '{\n "x": 1\n}'); + assert.equal(fmts.json({x: 1, y: 2}, {space: 1}), '{\n "x": 1,\n "y": 2\n}'); + assert.equal(fmts.json({x: 1, y: {z: 3}}, {space: 1}), '{\n "x": 1,\n "y": {\n "z": 3\n }\n}'); +}); diff --git a/tez-ui/src/main/webapp/yarn.lock b/tez-ui/src/main/webapp/yarn.lock index fbcdd21369..00250e82cf 100644 --- a/tez-ui/src/main/webapp/yarn.lock +++ b/tez-ui/src/main/webapp/yarn.lock @@ -1391,16 +1391,6 @@ ee-first@1.1.1: version "1.1.1" resolved "https://registry.yarnpkg.com/ee-first/-/ee-first-1.1.1.tgz#590c61156b0ae2f4f0255732a158b266bc56b21d" -em-helpers@0.8.0: - version "0.8.0" - resolved "https://registry.yarnpkg.com/em-helpers/-/em-helpers-0.8.0.tgz#01678f3692a61d563cce68e49459e206d14db095" - dependencies: - ember-cli-htmlbars "^1.0.1" - ember-cli-less "^1.4.0" - source-map "^0.5.6" - optionalDependencies: - phantomjs-prebuilt "2.1.13" - em-table@0.11.3: version "0.11.3" resolved "https://registry.yarnpkg.com/em-table/-/em-table-0.11.3.tgz#20e605cc3814214e644199399a2383cee8d23eeb" From 0ccf440e64e73646a14ee7366833eb274a3533c9 Mon Sep 17 00:00:00 2001 From: jteagles Date: Thu, 26 Aug 2021 09:14:29 -0500 Subject: [PATCH 067/137] TEZ-4329. Import external tez component em-table (#146) Co-authored-by: Sreenath Somarajapuram --- .../src/main/resources/META-INF/LICENSE.txt | 1 - .../webapp/app/components/em-table-cell.js | 124 +++++++ .../webapp/app/components/em-table-column.js | 109 ++++++ .../components/em-table-facet-panel-values.js | 199 ++++++++++ .../app/components/em-table-facet-panel.js | 86 +++++ .../app/components/em-table-header-cell.js | 64 ++++ .../app/components/em-table-linked-cell.js | 64 ++++ .../app/components/em-table-pagination-ui.js | 98 +++++ .../app/components/em-table-progress-cell.js | 46 +++ .../app/components/em-table-search-ui.js | 95 +++++ .../app/components/em-table-status-cell.js | 2 + .../main/webapp/app/components/em-table.js | 281 ++++++++++++++ .../webapp/app/controllers/app/configs.js | 2 +- .../main/webapp/app/controllers/app/dags.js | 2 +- .../webapp/app/controllers/counters-table.js | 2 +- .../webapp/app/controllers/dag/attempts.js | 2 +- .../webapp/app/controllers/dag/graphical.js | 2 +- .../webapp/app/controllers/dag/index/index.js | 2 +- .../webapp/app/controllers/dag/swimlane.js | 2 +- .../main/webapp/app/controllers/dag/tasks.js | 2 +- .../webapp/app/controllers/dag/vertices.js | 2 +- .../main/webapp/app/controllers/home/index.js | 4 +- .../webapp/app/controllers/home/queries.js | 4 +- .../webapp/app/controllers/query/configs.js | 2 +- .../webapp/app/controllers/query/timeline.js | 2 +- .../src/main/webapp/app/controllers/table.js | 2 +- .../webapp/app/controllers/task/attempts.js | 2 +- .../webapp/app/controllers/vertex/attempts.js | 2 +- .../webapp/app/controllers/vertex/configs.js | 2 +- .../webapp/app/controllers/vertex/tasks.js | 2 +- tez-ui/src/main/webapp/app/styles/app.less | 1 + .../app/styles/em-table-facet-panel.less | 218 +++++++++++ .../src/main/webapp/app/styles/em-table.less | 344 ++++++++++++++++++ .../main/webapp/app/styles/pagination-ui.less | 88 +++++ .../main/webapp/app/styles/progress-cell.less | 26 ++ .../src/main/webapp/app/styles/search-ui.less | 37 ++ tez-ui/src/main/webapp/app/styles/shared.less | 115 ++++++ .../src/main/webapp/app/styles/variables.less | 28 ++ .../templates/components/em-table-cell.hbs | 41 +++ .../templates/components/em-table-column.hbs | 22 ++ .../em-table-facet-panel-values.hbs | 50 +++ .../components/em-table-facet-panel.hbs | 33 ++ .../components/em-table-header-cell.hbs | 30 ++ .../components/em-table-linked-cell.hbs | 41 +++ .../components/em-table-pagination-ui.hbs | 45 +++ .../components/em-table-progress-cell.hbs | 29 ++ .../components/em-table-search-ui.hbs | 52 +++ .../app/templates/components/em-table.hbs | 106 ++++++ .../webapp/app/utils/column-definition.js | 125 +++++++ .../app/utils/counter-column-definition.js | 2 +- .../main/webapp/app/utils/data-processor.js | 275 ++++++++++++++ .../src/main/webapp/app/utils/facet-types.js | 85 +++++ tez-ui/src/main/webapp/app/utils/sql.js | 94 +++++ .../main/webapp/app/utils/table-definition.js | 61 ++++ tez-ui/src/main/webapp/bower.json | 1 + tez-ui/src/main/webapp/ember-cli-build.js | 1 + tez-ui/src/main/webapp/package.json | 1 - .../components/em-table-cell-test.js | 45 +++ .../components/em-table-column-test.js | 30 ++ .../components/em-table-facet-panel-test.js | 43 +++ .../em-table-facet-panel-values-test.js | 44 +++ .../components/em-table-header-cell-test.js | 30 ++ .../components/em-table-linked-cell-test.js | 30 ++ .../components/em-table-pagination-ui-test.js | 204 +++++++++++ .../components/em-table-progress-cell-test.js | 43 +++ .../components/em-table-search-ui-test.js | 30 ++ .../integration/components/em-table-test.js | 48 +++ .../integration/em-table-status-cell-test.js | 40 ++ .../unit/utils/column-definition-test.js | 104 ++++++ .../tests/unit/utils/data-processor-test.js | 137 +++++++ .../tests/unit/utils/facet-types-test.js | 28 ++ .../main/webapp/tests/unit/utils/sql-test.js | 90 +++++ .../tests/unit/utils/table-definition-test.js | 52 +++ tez-ui/src/main/webapp/yarn.lock | 10 - 74 files changed, 4135 insertions(+), 33 deletions(-) create mode 100644 tez-ui/src/main/webapp/app/components/em-table-cell.js create mode 100644 tez-ui/src/main/webapp/app/components/em-table-column.js create mode 100644 tez-ui/src/main/webapp/app/components/em-table-facet-panel-values.js create mode 100644 tez-ui/src/main/webapp/app/components/em-table-facet-panel.js create mode 100644 tez-ui/src/main/webapp/app/components/em-table-header-cell.js create mode 100644 tez-ui/src/main/webapp/app/components/em-table-linked-cell.js create mode 100644 tez-ui/src/main/webapp/app/components/em-table-pagination-ui.js create mode 100644 tez-ui/src/main/webapp/app/components/em-table-progress-cell.js create mode 100644 tez-ui/src/main/webapp/app/components/em-table-search-ui.js create mode 100644 tez-ui/src/main/webapp/app/components/em-table.js create mode 100644 tez-ui/src/main/webapp/app/styles/em-table-facet-panel.less create mode 100644 tez-ui/src/main/webapp/app/styles/em-table.less create mode 100644 tez-ui/src/main/webapp/app/styles/pagination-ui.less create mode 100644 tez-ui/src/main/webapp/app/styles/progress-cell.less create mode 100644 tez-ui/src/main/webapp/app/styles/search-ui.less create mode 100644 tez-ui/src/main/webapp/app/styles/variables.less create mode 100644 tez-ui/src/main/webapp/app/templates/components/em-table-cell.hbs create mode 100644 tez-ui/src/main/webapp/app/templates/components/em-table-column.hbs create mode 100644 tez-ui/src/main/webapp/app/templates/components/em-table-facet-panel-values.hbs create mode 100644 tez-ui/src/main/webapp/app/templates/components/em-table-facet-panel.hbs create mode 100644 tez-ui/src/main/webapp/app/templates/components/em-table-header-cell.hbs create mode 100644 tez-ui/src/main/webapp/app/templates/components/em-table-linked-cell.hbs create mode 100644 tez-ui/src/main/webapp/app/templates/components/em-table-pagination-ui.hbs create mode 100644 tez-ui/src/main/webapp/app/templates/components/em-table-progress-cell.hbs create mode 100644 tez-ui/src/main/webapp/app/templates/components/em-table-search-ui.hbs create mode 100644 tez-ui/src/main/webapp/app/templates/components/em-table.hbs create mode 100644 tez-ui/src/main/webapp/app/utils/column-definition.js create mode 100644 tez-ui/src/main/webapp/app/utils/data-processor.js create mode 100644 tez-ui/src/main/webapp/app/utils/facet-types.js create mode 100644 tez-ui/src/main/webapp/app/utils/sql.js create mode 100644 tez-ui/src/main/webapp/app/utils/table-definition.js create mode 100644 tez-ui/src/main/webapp/tests/integration/components/em-table-cell-test.js create mode 100644 tez-ui/src/main/webapp/tests/integration/components/em-table-column-test.js create mode 100644 tez-ui/src/main/webapp/tests/integration/components/em-table-facet-panel-test.js create mode 100644 tez-ui/src/main/webapp/tests/integration/components/em-table-facet-panel-values-test.js create mode 100644 tez-ui/src/main/webapp/tests/integration/components/em-table-header-cell-test.js create mode 100644 tez-ui/src/main/webapp/tests/integration/components/em-table-linked-cell-test.js create mode 100644 tez-ui/src/main/webapp/tests/integration/components/em-table-pagination-ui-test.js create mode 100644 tez-ui/src/main/webapp/tests/integration/components/em-table-progress-cell-test.js create mode 100644 tez-ui/src/main/webapp/tests/integration/components/em-table-search-ui-test.js create mode 100644 tez-ui/src/main/webapp/tests/integration/components/em-table-test.js create mode 100644 tez-ui/src/main/webapp/tests/integration/em-table-status-cell-test.js create mode 100644 tez-ui/src/main/webapp/tests/unit/utils/column-definition-test.js create mode 100644 tez-ui/src/main/webapp/tests/unit/utils/data-processor-test.js create mode 100644 tez-ui/src/main/webapp/tests/unit/utils/facet-types-test.js create mode 100644 tez-ui/src/main/webapp/tests/unit/utils/sql-test.js create mode 100644 tez-ui/src/main/webapp/tests/unit/utils/table-definition-test.js diff --git a/tez-ui/src/main/resources/META-INF/LICENSE.txt b/tez-ui/src/main/resources/META-INF/LICENSE.txt index 354d7457a9..608dc614c8 100644 --- a/tez-ui/src/main/resources/META-INF/LICENSE.txt +++ b/tez-ui/src/main/resources/META-INF/LICENSE.txt @@ -232,7 +232,6 @@ The Apache TEZ tez-ui bundles the following files under the MIT License: - more-js v0.8.2 (https://github.com/sreenaths/snippet-ss) - snippet-ss v1.11.0 (https://github.com/sreenaths/snippet-ss) - em-tgraph v0.0.4 (https://github.com/sreenaths/em-tgraph) - - em-table v0.3.12 (https://github.com/sreenaths/em-table) - ember-cli-app-version v1.0.0 (https://github.com/EmberSherpa/ember-cli-app-version) - Authored by Taras Mankovski - ember-cli-auto-register v1.1.0 (https://github.com/williamsbdev/ember-cli-auto-register) - Copyright © 2015 Brandon Williams http://williamsbdev.com - ember-cli-content-security-policy v0.4.0 (https://github.com/rwjblue/ember-cli-content-security-policy) diff --git a/tez-ui/src/main/webapp/app/components/em-table-cell.js b/tez-ui/src/main/webapp/app/components/em-table-cell.js new file mode 100644 index 0000000000..d4e6a54f53 --- /dev/null +++ b/tez-ui/src/main/webapp/app/components/em-table-cell.js @@ -0,0 +1,124 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import Ember from 'ember'; +import layout from '../templates/components/em-table-cell'; + +export default Ember.Component.extend({ + layout: layout, + + classNames: ['table-cell'], + classNameBindings: ['innerCell', 'isWaiting'], + + innerCell: Ember.computed('index', function () { + if(this.get('index')) { + return 'inner'; + } + }), + + row: null, + columnDefinition: null, + + isWaiting: false, + + _value: null, + _observedPath: null, + _comment: null, + _cellContent: Ember.computed({ + set: function (key, value, prevValue) { + if(value !== prevValue) { + this.highlightCell(); + } + return value; + } + }), + + _addObserver: function (path) { + this._removeObserver(); + this.get('row').addObserver(path, this, this._onValueChange); + this.set('_observedPath', path); + }, + + _removeObserver: function () { + var path = this.get('_observedPath'); + if(path) { + this.get('row').removeObserver(path, this, this._onValueChange); + this.set('_observedPath', null); + } + }, + + _pathObserver: Ember.on('init', Ember.observer('row', 'columnDefinition.contentPath', 'columnDefinition.observePath', function () { + var path = this.get('columnDefinition.contentPath'); + if(path && this.get('columnDefinition.observePath')) { + this._addObserver(path); + } + })), + + _onValueChange: function (row, path) { + this.set('_value', row.get(path)); + }, + + setContent: function (content) { + var comment; + + if(content && content.hasOwnProperty("content")) { + comment = content.comment; + content = content.content; + } + + this.setProperties({ + _comment: comment, + _cellContent: content, + isWaiting: false + }); + }, + + _cellContentObserver: Ember.on('init', Ember.observer('row', 'columnDefinition', '_value', function () { + var cellContent = this.get('columnDefinition').getCellContent(this.get('row'), this.get("_value")), + that = this; + + if(cellContent && cellContent.then) { + cellContent.then(function (content) { + that.setContent(content); + }); + this.set('isWaiting', true); + } + else if(cellContent === undefined && this.get('columnDefinition.observePath')) { + this.set('isWaiting', true); + } + else { + this.setContent(cellContent); + } + })), + + highlightCell: function () { + var element = this.$(); + if(element) { + element.removeClass("bg-transition"); + element.addClass("highlight"); + Ember.run.later(function () { + element.addClass("bg-transition"); + element.removeClass("highlight"); + }, 100); + } + }, + + willDestroy: function () { + this._removeObserver(); + } +}); diff --git a/tez-ui/src/main/webapp/app/components/em-table-column.js b/tez-ui/src/main/webapp/app/components/em-table-column.js new file mode 100644 index 0000000000..a3cb5a9072 --- /dev/null +++ b/tez-ui/src/main/webapp/app/components/em-table-column.js @@ -0,0 +1,109 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import Ember from 'ember'; + +import layout from '../templates/components/em-table-column'; + +export default Ember.Component.extend({ + layout: layout, + + definition: null, + rows: null, + index: 0, + + tableDefinition: null, + dataProcessor: null, + adjustedWidth: null, + defaultWidth: "", + + classNames: ['table-column'], + classNameBindings: ['inner', 'extraClassNames'], + + inner: Ember.computed('index', function () { + return !!this.get('index'); + }), + + extraClassNames: Ember.computed("definition.classNames", function () { + var classNames = this.get("definition.classNames"); + if(classNames) { + return classNames.join(" "); + } + }), + + didInsertElement: function () { + Ember.run.scheduleOnce('afterRender', this, function() { + this.setWidth(); + this.setMinWidth(); + }); + }, + + setMinWidth: Ember.observer("definition.minWidth", function () { + this.$().css("minWidth", this.get('definition.minWidth')); + }), + + setWidth: Ember.observer("adjustedWidth", "defaultWidth", function () { + var thisElement = this.$(); + thisElement.css("width", this.get('adjustedWidth') || this.get('defaultWidth')); + Ember.run.scheduleOnce('afterRender', this, function() { + this.get('parentView').send('columnWidthChanged', thisElement.width(), this.get("definition"), this.get("index")); + }); + }), + + _onColResize: function (event) { + var data = event.data, + width; + + if(!data.startEvent) { + data.startEvent = event; + } + + width = data.startWidth + event.clientX - data.startEvent.clientX; + data.thisObj.set('adjustedWidth', width); + }, + + _endColResize: function (event) { + var thisObj = event.data.thisObj; + Ember.$(document).off('mousemove', thisObj._onColResize); + Ember.$(document).off('mouseup', thisObj._endColResize); + }, + + actions: { + sort: function () { + var definition = this.get('definition'), + beforeSort = definition.get('beforeSort'); + + if(!beforeSort || beforeSort.call(definition, definition)) { + let columnId = this.get('definition.id'), + sortOrder = this.get('tableDefinition.sortOrder') === 'desc' ? 'asc' : 'desc'; + + this.get('parentView').send('sort', columnId, sortOrder); + } + }, + startColResize: function () { + var mouseTracker = { + thisObj: this, + startWidth: this.$().width(), + startEvent: null + }; + + Ember.$(document).on('mousemove', mouseTracker, this._onColResize); + Ember.$(document).on('mouseup', mouseTracker, this._endColResize); + } + } +}); diff --git a/tez-ui/src/main/webapp/app/components/em-table-facet-panel-values.js b/tez-ui/src/main/webapp/app/components/em-table-facet-panel-values.js new file mode 100644 index 0000000000..ec88181f4d --- /dev/null +++ b/tez-ui/src/main/webapp/app/components/em-table-facet-panel-values.js @@ -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. + */ + +import Ember from 'ember'; +import layout from '../templates/components/em-table-facet-panel-values'; + +const LIST_LIMIT = 7; + +export default Ember.Component.extend({ + layout: layout, + + data: null, + checkedCount: null, + + tableDefinition: null, + dataProcessor: null, + + tmpFacetConditions: null, + + hideValues: true, + + currentPage: 1, + + classNames: ['em-table-facet-panel-values'], + classNameBindings: ['hideValues', 'hideFilter', 'hideSelectAll'], + + filterText: null, + allButtonTitle: Ember.computed("filterText", function () { + let filterText = this.get("filterText"); + return filterText ? `Select all with substring '${filterText}'` : "Select all"; + }), + isVisible: Ember.computed("data.facets.length", "tableDefinition.minValuesToDisplay", function () { + return this.get("data.facets.length") >= this.get("tableDefinition.minValuesToDisplay"); + }), + hideFilter: Ember.computed("allFacets.length", function () { + return this.get("allFacets.length") < LIST_LIMIT; + }), + hideSelectAll: Ember.computed("fieldFacetConditions", "checkedCount", "data.facets", function () { + return this.get("fieldFacetConditions.in.length") === this.get("data.facets.length"); + }), + + fieldFacetConditions: Ember.computed("tmpFacetConditions", "data.column.id", function () { + var columnID = this.get("data.column.id"), + conditions = this.get(`tmpFacetConditions.${columnID}`), + facets = this.get("data.facets") || []; + + if(!conditions) { + conditions = { + in: facets.map(facet => facet.value) + }; + this.set(`tmpFacetConditions.${columnID}`, conditions); + } + + return conditions; + }), + + allFacets: Ember.computed("data.facets", "fieldFacetConditions", function () { + var facets = this.get("data.facets") || [], + + checkedValues = this.get("fieldFacetConditions.in"), + selectionHash = {}; + + if(checkedValues) { + checkedValues.forEach(function (valueText) { + selectionHash[valueText] = 1; + }); + } + + return Ember.A(facets.map(function (facet) { + facet = Ember.Object.create(facet); + facet.set("checked", selectionHash[facet.value]); + + if(!facet.get("displayText")) { + facet.set("displayText", facet.get("value")); + } + + return facet; + })); + }), + + filteredFacets: Ember.computed("allFacets", "filterText", function () { + var allFacets = this.get("allFacets"), + filterText = this.get("filterText"), + filteredFacets; + + if(filterText) { + filteredFacets = allFacets.filter(function (facet) { + return facet.get("value").match(filterText); + }); + } + else { + filteredFacets = allFacets; + } + + return filteredFacets; + }), + + _filterObserver: Ember.observer("filterText", function () { + this.set("currentPage", 1); + }), + + totalPages: Ember.computed("filteredFacets.length", "tableDefinition.facetValuesPageSize", function () { + return Math.ceil(this.get("filteredFacets.length") / this.get("tableDefinition.facetValuesPageSize")); + }), + showPagination: Ember.computed("totalPages", function () { + return this.get("totalPages") > 1; + }), + showPrevious: Ember.computed("currentPage", function () { + return this.get("currentPage") > 1; + }), + showNext: Ember.computed("currentPage", "totalPages", function () { + return this.get("currentPage") < this.get("totalPages"); + }), + + paginatedFacets: Ember.computed("filteredFacets", "currentPage", "tableDefinition.facetValuesPageSize", function () { + let currentPage = this.get("currentPage"), + pageSize = this.get("tableDefinition.facetValuesPageSize"); + return this.get("filteredFacets").slice( + (currentPage - 1) * pageSize, + currentPage * pageSize); + }), + + actions: { + changePage: function (factor) { + var newPage = this.get("currentPage") + factor; + if(newPage > 0 && newPage <= this.get("totalPages")) { + this.set("currentPage", newPage); + } + }, + toggleValueDisplay: function () { + this.toggleProperty("hideValues"); + this.get("parentView").sendAction("toggleValuesDisplayAction", !this.get("hideValues"), this.get("data")); + }, + clickedCheckbox: function (facet) { + var checkedValues = this.get("fieldFacetConditions.in"), + value = facet.get("value"), + valueIndex = checkedValues.indexOf(value); + + facet.toggleProperty("checked"); + + if(facet.get("checked")) { + if(valueIndex === -1) { + checkedValues.push(value); + } + } + else if(valueIndex !== -1) { + checkedValues.splice(valueIndex, 1); + } + + this.set("checkedCount", checkedValues.length); + }, + + selectAll: function () { + var filteredFacets = this.get("filteredFacets"), + checkedValues = this.get("fieldFacetConditions.in"); + + filteredFacets.forEach(function (facet) { + if(!facet.get("checked")) { + checkedValues.push(facet.get("value")); + } + + facet.set("checked", true); + }); + + this.set("fieldFacetConditions.in", checkedValues); + this.set("checkedCount", checkedValues.length); + }, + clickedOnly: function (facet) { + var allFacets = this.get("allFacets"), + checkedValues = []; + + allFacets.forEach(function (facet) { + facet.set("checked", false); + }); + + facet.set("checked", true); + checkedValues.push(facet.get("value")); + + this.set("fieldFacetConditions.in", checkedValues); + this.set("checkedCount", checkedValues.length); + } + } + +}); diff --git a/tez-ui/src/main/webapp/app/components/em-table-facet-panel.js b/tez-ui/src/main/webapp/app/components/em-table-facet-panel.js new file mode 100644 index 0000000000..fdbd8f5b89 --- /dev/null +++ b/tez-ui/src/main/webapp/app/components/em-table-facet-panel.js @@ -0,0 +1,86 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import Ember from 'ember'; +import layout from '../templates/components/em-table-facet-panel'; + +export default Ember.Component.extend({ + layout: layout, + + classNames: ["em-table-facet-panel"], + classNameBindings: ['isEmpty', 'hideFilter'], + + isVisible: Ember.computed.alias('tableDefinition.enableFaceting'), + + tableDefinition: null, + dataProcessor: null, + tmpFacetConditions: {}, + + filterText: null, + isEmpty: Ember.computed("dataProcessor.facetedFields.length", function () { + return this.get("dataProcessor.facetedFields.length") === 0; + }), + hideFilter: Ember.computed("dataProcessor.facetedFields.length", "tableDefinition.minFieldsForFilter", function () { + return this.get("dataProcessor.facetedFields.length") < this.get("tableDefinition.minFieldsForFilter"); + }), + + didInsertElement: Ember.observer("filterText", "dataProcessor.facetedFields", function () { + var fields = this.get("dataProcessor.facetedFields"), + filterText = this.get("filterText"), + filterRegex = new RegExp(filterText, "i"), + elements = Ember.$(this.get("element")).find(".field-list>li"); + + elements.each(function (index, element) { + var foundMatch = !filterText || Ember.get(fields, `${index}.column.headerTitle`).match(filterRegex); + Ember.$(element)[foundMatch ? "show" : "hide"](); + }); + }), + + _facetConditionsObserver: Ember.observer("tableDefinition.facetConditions", "dataProcessor.processedRows.[]", function () { + var facetConditions = Ember.$.extend({}, this.get("tableDefinition.facetConditions")); + this.set("tmpFacetConditions", facetConditions); + }), + + actions: { + applyFilters: function () { + var tmpFacetConditions = this.get("tmpFacetConditions"), + facetedFields = this.get("dataProcessor.facetedFields"), + normalizedTmpFacetConditions = {}; + + facetedFields.forEach(function (field) { + var column = field.column, + columnId = column.get("id"), + facetType = column.get("facetType"), + normalizedConditions; + + if(facetType) { + normalizedConditions = facetType.normaliseConditions(tmpFacetConditions[columnId], field.facets); + if(normalizedConditions) { + normalizedTmpFacetConditions[columnId] = normalizedConditions; + } + } + }); + + this.set("tableDefinition.facetConditions", normalizedTmpFacetConditions); + }, + clearFilters: function () { + this.set("tmpFacetConditions", {}); + this.set("tableDefinition.facetConditions", {}); + }, + } +}); diff --git a/tez-ui/src/main/webapp/app/components/em-table-header-cell.js b/tez-ui/src/main/webapp/app/components/em-table-header-cell.js new file mode 100644 index 0000000000..c0a8e12379 --- /dev/null +++ b/tez-ui/src/main/webapp/app/components/em-table-header-cell.js @@ -0,0 +1,64 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import Ember from 'ember'; +import layout from '../templates/components/em-table-header-cell'; + +export default Ember.Component.extend({ + layout: layout, + + title: null, // Header cell Name + attributeBindings: ['title'], + + definition: null, + tableDefinition: null, + dataProcessor: null, + + classNames: ['table-header-cell'], + classNameBindings: ['isSorting'], + + isSorting: Ember.computed("dataProcessor.isSorting", function () { + return this.get("dataProcessor.isSorting") && this.get('tableDefinition.sortColumnId') === this.get('definition.id'); + }), + + sortIconCSS: Ember.computed('tableDefinition.sortOrder', 'tableDefinition.sortColumnId', function () { + if(this.get('tableDefinition.sortColumnId') === this.get('definition.id')) { + return this.get('tableDefinition.sortOrder'); + } + }), + + sortToggledTitle: Ember.computed('tableDefinition.sortOrder', 'tableDefinition.sortColumnId', function () { + if(this.get('tableDefinition.sortColumnId') === this.get('definition.id')) { + switch(this.get('tableDefinition.sortOrder')) { + case "asc": + return "descending"; + case "desc": + return "ascending"; + } + } + }), + + actions: { + sort: function () { + this.get('parentView').send('sort'); + }, + startColResize: function () { + this.get('parentView').send('startColResize'); + } + } +}); diff --git a/tez-ui/src/main/webapp/app/components/em-table-linked-cell.js b/tez-ui/src/main/webapp/app/components/em-table-linked-cell.js new file mode 100644 index 0000000000..c42c56613b --- /dev/null +++ b/tez-ui/src/main/webapp/app/components/em-table-linked-cell.js @@ -0,0 +1,64 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import Ember from 'ember'; +import layout from '../templates/components/em-table-linked-cell'; + +export default Ember.Component.extend({ + layout: layout, + + definition: null, + content: null, + + normalizedLinks: Ember.computed("content", function () { + var content = this.get("content"), + links; + + if(content) { + if(!Array.isArray(content)) { + content = [content]; + } + + links = content.map(function (link) { + var model, + text = Ember.get(link, "text") || Ember.get(link, "displayText"); + + if(text) { + link = Ember.Object.create(link, { + text: text + }); + + if(link.get("model") === undefined) { + link.set("model", link.get("id")); + } + + model = link.get("model"); + link.set("withModel", model !== undefined); + + return link; + } + }); + + links = links.filter(function (link) { + return link; + }); + } + + return links; + }) +}); diff --git a/tez-ui/src/main/webapp/app/components/em-table-pagination-ui.js b/tez-ui/src/main/webapp/app/components/em-table-pagination-ui.js new file mode 100644 index 0000000000..858928b60b --- /dev/null +++ b/tez-ui/src/main/webapp/app/components/em-table-pagination-ui.js @@ -0,0 +1,98 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import Ember from 'ember'; +import layout from '../templates/components/em-table-pagination-ui'; + +export default Ember.Component.extend({ + layout: layout, + + tableDefinition: null, + dataProcessor: null, + + classNames: ['pagination-ui'], + isVisible: Ember.computed.alias('tableDefinition.enablePagination'), + + showFirst: Ember.computed('_possiblePages', function () { + return this.get("dataProcessor.totalPages") && this.get('_possiblePages.0.pageNum') !== 1; + }), + + showLast: Ember.computed('_possiblePages', 'dataProcessor.totalPages', function () { + var possiblePages = this.get("_possiblePages"); + if(possiblePages.length) { + return possiblePages[possiblePages.length - 1].pageNum !== this.get("dataProcessor.totalPages"); + } + }), + + rowCountOptions: Ember.computed('tableDefinition.rowCountOptions', 'tableDefinition.rowCount', function () { + var options = this.get('tableDefinition.rowCountOptions'), + rowCount = this.get('tableDefinition.rowCount'); + + return options.map(function (option) { + return { + value: option, + selected: option === rowCount + }; + }); + }), + + _possiblePages: Ember.computed('tableDefinition.pageNum', 'dataProcessor.totalPages', function () { + var pageNum = this.get('tableDefinition.pageNum'), + totalPages = this.get('dataProcessor.totalPages'), + possiblePages = [], + startPage = 1, + endPage = totalPages, + delta = 0; + + if(totalPages > 5) { + startPage = pageNum - 2; + endPage = pageNum + 2; + + if(startPage < 1) { + delta = 1 - startPage; + } + else if(endPage > totalPages) { + delta = totalPages - endPage; + } + + startPage += delta; + endPage += delta; + } + + while(startPage <= endPage) { + possiblePages.push({ + isCurrent: startPage === pageNum, + pageNum: startPage++ + }); + } + + return possiblePages; + }), + + actions: { + rowSelected: function (value) { + value = parseInt(value); + if(this.get('tableDefinition.rowCount') !== value) { + this.get('parentView').send('rowChanged', value); + } + }, + changePage: function (value) { + this.get('parentView').send('pageChanged', value); + } + } +}); diff --git a/tez-ui/src/main/webapp/app/components/em-table-progress-cell.js b/tez-ui/src/main/webapp/app/components/em-table-progress-cell.js new file mode 100644 index 0000000000..32f75c473a --- /dev/null +++ b/tez-ui/src/main/webapp/app/components/em-table-progress-cell.js @@ -0,0 +1,46 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import Ember from 'ember'; +import layout from '../templates/components/em-table-progress-cell'; + +export default Ember.Component.extend({ + layout: layout, + + content: null, + + message: Ember.computed("content", function () { + var content = this.get("content"); + + if(content === undefined || content === null) { + return "Not Available!"; + } + else if(isNaN(parseFloat(content))){ + return "Invalid Data!"; + } + }), + + _definition: Ember.computed("definition", function () { + return Ember.Object.extend({ + valueMin: 0, + valueMax: 1, + striped: true, + style: null + }).create(this.get("definition")); + }) +}); diff --git a/tez-ui/src/main/webapp/app/components/em-table-search-ui.js b/tez-ui/src/main/webapp/app/components/em-table-search-ui.js new file mode 100644 index 0000000000..58c4f75cdf --- /dev/null +++ b/tez-ui/src/main/webapp/app/components/em-table-search-ui.js @@ -0,0 +1,95 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import Ember from 'ember'; +import layout from '../templates/components/em-table-search-ui'; + +export default Ember.Component.extend({ + layout: layout, + + tableDefinition: null, + dataProcessor: null, + + classNames: ['search-ui'], + classNameBindings: ['hasError'], + isVisible: Ember.computed.alias('tableDefinition.enableSearch'), + + searchTypes: ["Regex", "SQL"], + actualSearchType: null, + + text: Ember.computed.oneWay('tableDefinition.searchText'), + + _actualSearchTypeDecider: Ember.observer("tableDefinition.searchType", "text", function () { + var searchType = this.get("tableDefinition.searchType"), + actualSearchType = this.get("actualSearchType"); + + switch(searchType) { + case "SQL": + case "Regex": + actualSearchType = searchType; + break; + + case "manual": + if(!actualSearchType) { + actualSearchType = "Regex"; + } + // Will be set from the template + break; + + case "auto": + var text = this.get("text"), + columns = this.get('tableDefinition.columns'); + + if(text) { + actualSearchType = this.get("dataProcessor.sql").validateClause(text, columns) ? "SQL" : "Regex"; + } + else { + actualSearchType = null; + } + break; + } + + this.set("actualSearchType", actualSearchType); + }), + + hasError: Ember.computed("text", "actualSearchType", "tableDefinition.searchType", function () { + var text = this.get("text"), + columns = this.get('tableDefinition.columns'), + actualSearchType = this.get("actualSearchType"); + + if(text) { + switch(actualSearchType) { + case "SQL": + return !this.get("dataProcessor.sql").validateClause(text, columns); + case "Regex": + try { + new RegExp(text); + } + catch(e) { + return true; + } + } + } + }), + + actions: { + search: function () { + this.get('parentView').send('search', this.get('text'), this.get("actualSearchType")); + } + } +}); diff --git a/tez-ui/src/main/webapp/app/components/em-table-status-cell.js b/tez-ui/src/main/webapp/app/components/em-table-status-cell.js index 7751719865..7f1fee8e35 100644 --- a/tez-ui/src/main/webapp/app/components/em-table-status-cell.js +++ b/tez-ui/src/main/webapp/app/components/em-table-status-cell.js @@ -17,8 +17,10 @@ */ import Ember from 'ember'; +import layout from '../templates/components/em-table-status-cell'; export default Ember.Component.extend({ + layout: layout, content: null, diff --git a/tez-ui/src/main/webapp/app/components/em-table.js b/tez-ui/src/main/webapp/app/components/em-table.js new file mode 100644 index 0000000000..79aae3d480 --- /dev/null +++ b/tez-ui/src/main/webapp/app/components/em-table.js @@ -0,0 +1,281 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import Ember from 'ember'; +import Definition from '../utils/table-definition'; +import ColumnDefinition from '../utils/column-definition'; +import DataProcessor from '../utils/data-processor'; + +import layout from '../templates/components/em-table'; + +const DEFAULT_ROW_HIGHLIGHT_COLOR = "#EEE"; + +function createAssigner(targetPath, targetKey, sourcePath) { + return Ember.on("init", Ember.observer(targetPath, sourcePath, function () { + var target = this.get(targetPath), + source = this.get(sourcePath); + if(target && source !== undefined) { + target.set(targetKey, source); + } + })); +} + +const HANDLERS = { + // Mouse handlers + mouseOver: function(event) { + var index = Ember.$(this).index() + 1; + event.data.highlightRow(index); + }, + mouseLeave: function(event) { + event.data.highlightRow(-1); + }, + + // Scroll handler + onScroll: function(event) { + var tableBody = event.currentTarget, + scrollValues = event.data.get("scrollValues"); + + scrollValues.set("left", tableBody.scrollLeft); + scrollValues.set("width", tableBody.scrollWidth); + } +}; + +export default Ember.Component.extend({ + layout: layout, + + classNames: ["em-table"], + classNameBindings: ["showScrollShadow", "showLeftScrollShadow", "showRightScrollShadow"], + + definition: null, + dataProcessor: null, + + highlightRowOnMouse: false, // Could be true or {color: "#XYZ"} + + headerComponentNames: ['em-table-search-ui', 'em-table-pagination-ui'], + footerComponentNames: ['em-table-pagination-ui'], + + leftPanelComponentName: "em-table-facet-panel", + rightPanelComponentName: "", + + columnWidthChangeAction: null, + + scrollChangeAction: null, + scrollValues: null, + _widthTrackerTimer: null, + + init: function() { + this._super(); + this.set("scrollValues", Ember.Object.create({ + left: 0, + width: 0, + viewPortWidth: 0 + })); + }, + + showScrollShadow: false, + showLeftScrollShadow: false, + showRightScrollShadow: false, + + assignDefinitionInProcessor: createAssigner('_dataProcessor', 'tableDefinition', '_definition'), + assignRowsInProcessor: createAssigner('_dataProcessor', 'rows', 'rows'), + assignColumnsInDefinition: createAssigner('_definition', 'columns', 'columns'), + + assignEnableSortInDefinition: createAssigner('_definition', 'enableSort', 'enableSort'), + assignEnableSearchInDefinition: createAssigner('_definition', 'enableSearch', 'enableSearch'), + assignEnablePaginationInDefinition: createAssigner('_definition', 'enablePagination', 'enablePagination'), + assignRowCountInDefinition: createAssigner('_definition', 'rowCount', 'rowCount'), + + _definition: Ember.computed('definition', 'definitionClass', function () { + return this.get('definition') || (this.get('definitionClass') || Definition).create(); + }), + _dataProcessor: Ember.computed('dataProcessor', 'dataProcessorClass', function () { + return this.get('dataProcessor') || (this.get('dataProcessorClass') || DataProcessor).create(); + }), + + displayFooter: Ember.computed("_definition.minRowsForFooter", "_dataProcessor.processedRows.length", function () { + return this.get("_definition.minRowsForFooter") <= this.get("_dataProcessor.processedRows.length"); + }), + + _processedRowsObserver: Ember.observer('_dataProcessor.processedRows', function () { + this.sendAction('rowsChanged', this.get('_dataProcessor.processedRows')); + }), + + _setColumnWidth: function (columns) { + var widthText = (100 / columns.length) + "%"; + columns.forEach(function (column) { + if(!column.width) { + column.width = widthText; + } + }); + }, + + _columns: Ember.computed('_definition.columns', function () { + var rawColumns = this.get('_definition.columns'), + normalisedColumns = { + left: [], + center: [], + right: [], + length: rawColumns.length + }; + + rawColumns.forEach(function (column) { + normalisedColumns[column.get("pin")].push({ + definition: column, + width: column.width + }); + }); + + if(normalisedColumns.center.length === 0) { + normalisedColumns.center = [{ + definition: ColumnDefinition.fillerColumn, + }]; + } + + this._setColumnWidth(normalisedColumns.center); + + return normalisedColumns; + }), + + message: Ember.computed('_dataProcessor.message', '_columns.length', '_dataProcessor.processedRows.length', function () { + var message = this.get("_dataProcessor.message"); + if(message) { + return message; + } + else if(!this.get('_columns.length')) { + return "No columns available!"; + } + else if(!this.get("_dataProcessor.processedRows.length")) { + let identifiers = Ember.String.pluralize(this.get('_definition.recordType') || "record"); + return `No ${identifiers} available!`; + } + }), + + highlightRow: function (index) { + var element = Ember.$(this.get("element")), + sheet = element.find("style")[0].sheet, + elementID = element.attr("id"), + color = this.get("highlightRowOnMouse.color") || DEFAULT_ROW_HIGHLIGHT_COLOR; + + try { + sheet.deleteRule(0); + }catch(e){} + + if(index >= 0) { + sheet.insertRule(`#${elementID} .table-cell:nth-child(${index}){ background-color: ${color}; }`, 0); + } + }, + + didInsertElement: function () { + Ember.run.scheduleOnce('afterRender', this, function() { + this.highlightRowOnMouseObserver(); + this.scrollChangeActionObserver(); + }); + }, + + highlightRowOnMouseObserver: Ember.observer("highlightRowOnMouse", function () { + var highlightRowOnMouse = this.get("highlightRowOnMouse"), + element = this.get("element"); + + if(element) { + element = Ember.$(element).find(".table-mid"); + + if(highlightRowOnMouse) { + element.on('mouseover', '.table-cell', this, HANDLERS.mouseOver); + element.on('mouseleave', this, HANDLERS.mouseLeave); + } + else { + element.off('mouseover', '.table-cell', HANDLERS.mouseOver); + element.off('mouseleave', HANDLERS.mouseLeave); + } + } + }), + + scrollValuesObserver: Ember.observer("scrollValues.left", "scrollValues.width", "scrollValues.viewPortWidth", function () { + var scrollValues = this.get("scrollValues"); + + this.sendAction("scrollChangeAction", scrollValues); + + + this.set("showLeftScrollShadow", scrollValues.left > 1); + this.set("showRightScrollShadow", scrollValues.left < (scrollValues.width - scrollValues.viewPortWidth)); + }), + + scrollChangeActionObserver: Ember.observer("scrollChangeAction", "message", "showScrollShadow", function () { + Ember.run.scheduleOnce('afterRender', this, function() { + var addScrollListener = this.get("scrollChangeAction") || this.get("showScrollShadow"), + element = this.$().find(".table-body"), + scrollValues = this.get("scrollValues"); + + if(addScrollListener && element) { + element = element.get(0); + + clearInterval(this.get("_widthTrackerTimer")); + + if(element) { + if(addScrollListener) { + Ember.$(element).on('scroll', this, HANDLERS.onScroll); + + this.set("_widthTrackerTimer", setInterval(function () { + scrollValues.setProperties({ + width: element.scrollWidth, + viewPortWidth: element.offsetWidth + }); + }, 1000)); + } + else { + element.off('scroll', HANDLERS.onScroll); + } + } + } + }); + }), + + willDestroyElement: function () { + this._super(); + clearInterval(this.get("_widthTrackerTimer")); + Ember.$(this.$().find(".table-body")).off(); + Ember.$(this.$().find(".table-mid")).off(); + Ember.$(this.$()).off(); + }, + + actions: { + search: function (searchText, actualSearchType) { + this.set('_definition.searchText', searchText); + this.set('_definition._actualSearchType', actualSearchType); + this.sendAction("searchAction", searchText); + }, + sort: function (sortColumnId, sortOrder) { + this.get("_definition").setProperties({ + sortColumnId, + sortOrder + }); + this.sendAction("sortAction", sortColumnId, sortOrder); + }, + rowChanged: function (rowCount) { + this.set('_definition.rowCount', rowCount); + this.sendAction("rowAction", rowCount); + }, + pageChanged: function (pageNum) { + this.set('_definition.pageNum', pageNum); + this.sendAction("pageAction", pageNum); + }, + columnWidthChanged: function (width, columnDefinition, index) { + this.sendAction("columnWidthChangeAction", width, columnDefinition, index); + } + } +}); diff --git a/tez-ui/src/main/webapp/app/controllers/app/configs.js b/tez-ui/src/main/webapp/app/controllers/app/configs.js index 838abc1512..e8f13fc0e0 100644 --- a/tez-ui/src/main/webapp/app/controllers/app/configs.js +++ b/tez-ui/src/main/webapp/app/controllers/app/configs.js @@ -20,7 +20,7 @@ import Ember from 'ember'; import TableController from '../table'; -import ColumnDefinition from 'em-table/utils/column-definition'; +import ColumnDefinition from '../../utils/column-definition'; var MoreObject = more.Object; diff --git a/tez-ui/src/main/webapp/app/controllers/app/dags.js b/tez-ui/src/main/webapp/app/controllers/app/dags.js index bb4502a9b1..1febc66f97 100644 --- a/tez-ui/src/main/webapp/app/controllers/app/dags.js +++ b/tez-ui/src/main/webapp/app/controllers/app/dags.js @@ -17,7 +17,7 @@ */ import MultiTableController from '../multi-table'; -import ColumnDefinition from 'em-table/utils/column-definition'; +import ColumnDefinition from '../../utils/column-definition'; export default MultiTableController.extend({ breadcrumbs: [{ diff --git a/tez-ui/src/main/webapp/app/controllers/counters-table.js b/tez-ui/src/main/webapp/app/controllers/counters-table.js index 42361b4bb5..37bae66d06 100644 --- a/tez-ui/src/main/webapp/app/controllers/counters-table.js +++ b/tez-ui/src/main/webapp/app/controllers/counters-table.js @@ -20,7 +20,7 @@ import Ember from 'ember'; import TableController from './table'; -import ColumnDefinition from 'em-table/utils/column-definition'; +import ColumnDefinition from '../utils/column-definition'; var MoreObject = more.Object; diff --git a/tez-ui/src/main/webapp/app/controllers/dag/attempts.js b/tez-ui/src/main/webapp/app/controllers/dag/attempts.js index 47e95d9aaa..4616638cba 100644 --- a/tez-ui/src/main/webapp/app/controllers/dag/attempts.js +++ b/tez-ui/src/main/webapp/app/controllers/dag/attempts.js @@ -17,7 +17,7 @@ */ import MultiTableController from '../multi-table'; -import ColumnDefinition from 'em-table/utils/column-definition'; +import ColumnDefinition from '../../utils/column-definition'; export default MultiTableController.extend({ breadcrumbs: [{ diff --git a/tez-ui/src/main/webapp/app/controllers/dag/graphical.js b/tez-ui/src/main/webapp/app/controllers/dag/graphical.js index c55ab8b3a4..cc4130fd80 100644 --- a/tez-ui/src/main/webapp/app/controllers/dag/graphical.js +++ b/tez-ui/src/main/webapp/app/controllers/dag/graphical.js @@ -19,7 +19,7 @@ import Ember from 'ember'; import MultiTableController from '../multi-table'; -import ColumnDefinition from 'em-table/utils/column-definition'; +import ColumnDefinition from '../../utils/column-definition'; export default MultiTableController.extend({ diff --git a/tez-ui/src/main/webapp/app/controllers/dag/index/index.js b/tez-ui/src/main/webapp/app/controllers/dag/index/index.js index c9adde4c6b..6196b9aa87 100644 --- a/tez-ui/src/main/webapp/app/controllers/dag/index/index.js +++ b/tez-ui/src/main/webapp/app/controllers/dag/index/index.js @@ -19,7 +19,7 @@ import Ember from 'ember'; import MultiTableController from '../../multi-table'; -import ColumnDefinition from 'em-table/utils/column-definition'; +import ColumnDefinition from '../../../utils/column-definition'; export default MultiTableController.extend({ columns: ColumnDefinition.make([{ diff --git a/tez-ui/src/main/webapp/app/controllers/dag/swimlane.js b/tez-ui/src/main/webapp/app/controllers/dag/swimlane.js index bbac40b26d..1fe2988858 100644 --- a/tez-ui/src/main/webapp/app/controllers/dag/swimlane.js +++ b/tez-ui/src/main/webapp/app/controllers/dag/swimlane.js @@ -19,7 +19,7 @@ import Ember from 'ember'; import MultiTableController from '../multi-table'; -import ColumnDefinition from 'em-table/utils/column-definition'; +import ColumnDefinition from '../../utils/column-definition'; import VertexProcess from '../../utils/vertex-process'; import fullscreen from 'em-tgraph/utils/fullscreen'; diff --git a/tez-ui/src/main/webapp/app/controllers/dag/tasks.js b/tez-ui/src/main/webapp/app/controllers/dag/tasks.js index 92f674ad8e..834abf9b64 100644 --- a/tez-ui/src/main/webapp/app/controllers/dag/tasks.js +++ b/tez-ui/src/main/webapp/app/controllers/dag/tasks.js @@ -17,7 +17,7 @@ */ import MultiTableController from '../multi-table'; -import ColumnDefinition from 'em-table/utils/column-definition'; +import ColumnDefinition from '../../utils/column-definition'; export default MultiTableController.extend({ breadcrumbs: [{ diff --git a/tez-ui/src/main/webapp/app/controllers/dag/vertices.js b/tez-ui/src/main/webapp/app/controllers/dag/vertices.js index 313a5a9be5..34295e5fde 100644 --- a/tez-ui/src/main/webapp/app/controllers/dag/vertices.js +++ b/tez-ui/src/main/webapp/app/controllers/dag/vertices.js @@ -17,7 +17,7 @@ */ import MultiTableController from '../multi-table'; -import ColumnDefinition from 'em-table/utils/column-definition'; +import ColumnDefinition from '../../utils/column-definition'; export default MultiTableController.extend({ breadcrumbs: [{ diff --git a/tez-ui/src/main/webapp/app/controllers/home/index.js b/tez-ui/src/main/webapp/app/controllers/home/index.js index 754e5e6b99..53c4e6d4bf 100644 --- a/tez-ui/src/main/webapp/app/controllers/home/index.js +++ b/tez-ui/src/main/webapp/app/controllers/home/index.js @@ -19,8 +19,8 @@ import Ember from 'ember'; import TableController from '../table'; -import ColumnDefinition from 'em-table/utils/column-definition'; -import TableDefinition from 'em-table/utils/table-definition'; +import ColumnDefinition from '../../utils/column-definition'; +import TableDefinition from '../../utils/table-definition'; export default TableController.extend({ diff --git a/tez-ui/src/main/webapp/app/controllers/home/queries.js b/tez-ui/src/main/webapp/app/controllers/home/queries.js index ba7e6e3776..b5e483de6e 100644 --- a/tez-ui/src/main/webapp/app/controllers/home/queries.js +++ b/tez-ui/src/main/webapp/app/controllers/home/queries.js @@ -19,8 +19,8 @@ import Ember from 'ember'; import TableController from '../table'; -import ColumnDefinition from 'em-table/utils/column-definition'; -import TableDefinition from 'em-table/utils/table-definition'; +import ColumnDefinition from '../../utils/column-definition'; +import TableDefinition from '../../utils/table-definition'; export default TableController.extend({ diff --git a/tez-ui/src/main/webapp/app/controllers/query/configs.js b/tez-ui/src/main/webapp/app/controllers/query/configs.js index 8dcc91cd0d..d828088f2e 100644 --- a/tez-ui/src/main/webapp/app/controllers/query/configs.js +++ b/tez-ui/src/main/webapp/app/controllers/query/configs.js @@ -20,7 +20,7 @@ import Ember from 'ember'; import TableController from '../table'; -import ColumnDefinition from 'em-table/utils/column-definition'; +import ColumnDefinition from '../../utils/column-definition'; var MoreObject = more.Object; diff --git a/tez-ui/src/main/webapp/app/controllers/query/timeline.js b/tez-ui/src/main/webapp/app/controllers/query/timeline.js index b52fc26011..a7cd85eca2 100644 --- a/tez-ui/src/main/webapp/app/controllers/query/timeline.js +++ b/tez-ui/src/main/webapp/app/controllers/query/timeline.js @@ -19,7 +19,7 @@ import Ember from 'ember'; import TableController from '../table'; -import ColumnDefinition from 'em-table/utils/column-definition'; +import ColumnDefinition from '../../utils/column-definition'; var MoreObject = more.Object; diff --git a/tez-ui/src/main/webapp/app/controllers/table.js b/tez-ui/src/main/webapp/app/controllers/table.js index 57adf00469..01aec40569 100644 --- a/tez-ui/src/main/webapp/app/controllers/table.js +++ b/tez-ui/src/main/webapp/app/controllers/table.js @@ -20,7 +20,7 @@ import Ember from 'ember'; import AbstractController from './abstract'; -import TableDefinition from 'em-table/utils/table-definition'; +import TableDefinition from '../utils/table-definition'; import isIOCounter from '../utils/misc'; import CounterColumnDefinition from '../utils/counter-column-definition'; diff --git a/tez-ui/src/main/webapp/app/controllers/task/attempts.js b/tez-ui/src/main/webapp/app/controllers/task/attempts.js index a6acaecc84..04eb22ab7c 100644 --- a/tez-ui/src/main/webapp/app/controllers/task/attempts.js +++ b/tez-ui/src/main/webapp/app/controllers/task/attempts.js @@ -17,7 +17,7 @@ */ import MultiTableController from '../multi-table'; -import ColumnDefinition from 'em-table/utils/column-definition'; +import ColumnDefinition from '../../utils/column-definition'; import AutoCounterColumn from '../../mixins/auto-counter-column'; diff --git a/tez-ui/src/main/webapp/app/controllers/vertex/attempts.js b/tez-ui/src/main/webapp/app/controllers/vertex/attempts.js index b07be92e06..107ecdb82b 100644 --- a/tez-ui/src/main/webapp/app/controllers/vertex/attempts.js +++ b/tez-ui/src/main/webapp/app/controllers/vertex/attempts.js @@ -17,7 +17,7 @@ */ import MultiTableController from '../multi-table'; -import ColumnDefinition from 'em-table/utils/column-definition'; +import ColumnDefinition from '../../utils/column-definition'; import AutoCounterColumn from '../../mixins/auto-counter-column'; diff --git a/tez-ui/src/main/webapp/app/controllers/vertex/configs.js b/tez-ui/src/main/webapp/app/controllers/vertex/configs.js index 1cf4a3d31d..2e1d94e647 100644 --- a/tez-ui/src/main/webapp/app/controllers/vertex/configs.js +++ b/tez-ui/src/main/webapp/app/controllers/vertex/configs.js @@ -20,7 +20,7 @@ import Ember from 'ember'; import TableController from '../table'; -import ColumnDefinition from 'em-table/utils/column-definition'; +import ColumnDefinition from '../../utils/column-definition'; var MoreObject = more.Object; diff --git a/tez-ui/src/main/webapp/app/controllers/vertex/tasks.js b/tez-ui/src/main/webapp/app/controllers/vertex/tasks.js index 560c8ba0f9..dac000e247 100644 --- a/tez-ui/src/main/webapp/app/controllers/vertex/tasks.js +++ b/tez-ui/src/main/webapp/app/controllers/vertex/tasks.js @@ -17,7 +17,7 @@ */ import MultiTableController from '../multi-table'; -import ColumnDefinition from 'em-table/utils/column-definition'; +import ColumnDefinition from '../../utils/column-definition'; import AutoCounterColumn from '../../mixins/auto-counter-column'; diff --git a/tez-ui/src/main/webapp/app/styles/app.less b/tez-ui/src/main/webapp/app/styles/app.less index f8a66e376d..44bfb836e0 100644 --- a/tez-ui/src/main/webapp/app/styles/app.less +++ b/tez-ui/src/main/webapp/app/styles/app.less @@ -41,6 +41,7 @@ @import "em-swimlane"; @import "em-tooltip"; @import "em-swimlane-vertex-name"; +@import "em-table.less"; @import "em-table-status-cell"; @import "query-timeline"; @import "home-table-controls"; diff --git a/tez-ui/src/main/webapp/app/styles/em-table-facet-panel.less b/tez-ui/src/main/webapp/app/styles/em-table-facet-panel.less new file mode 100644 index 0000000000..28be8f0ef1 --- /dev/null +++ b/tez-ui/src/main/webapp/app/styles/em-table-facet-panel.less @@ -0,0 +1,218 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +.em-table-facet-panel { + width: 160px; + + margin: 5px 5px 0 0; + + border: 1px solid @border-color; + border-radius: @border-radius; + + padding: 5px 10px 10px 10px; + + background-color: @table-bg; + + overflow: hidden; + + .field-filter-box { + width: 100%; + } + + &.hide-filter { + .field-filter-box { + display: none; + } + } + + .filter-message { + color: #999; + } + + h4 { + text-align: center; + margin-top: 5px; + margin-bottom: 0px; + } + + ul { + list-style-type: none; + } + + li { + margin: 2px 0; + } + + ul.field-list { + padding-top: 5px; + padding-left: 0px; + + .em-table-facet-panel-values { + + position: relative; + + .field-name { + .no-select; + + padding-right: 20px; + + cursor: pointer; + display: flex; + + &::before { + content: "\25bc"; + font-size: .7em; + color: @text-light; + margin-top: 5px; + } + + .field-title { + overflow-x: hidden; + text-overflow: ellipsis; + white-space: nowrap; + margin: 0 3px; + } + + .field-count { + color: @text-light; + white-space: nowrap; + } + + .all-button { + margin-left: 5px; + position: absolute; + right: 0px; + } + } + + &.hide-select-all { + .field-name { + padding-right: 0px; + + .all-button { + display: none; + } + } + } + + .value-list { + overflow: hidden; + + padding-left: 10px; + + .filter-box { + width: 100%; + } + + li { + display: flex; + + .checkbox-container { + order: 0; + flex: 0 1 auto; + align-self: auto; + + padding-right: 5px; + } + + .facet-value { + order: 0; + flex: 1 1 auto; + align-self: auto; + padding-right: 2px; + + overflow-x: hidden; + text-overflow: ellipsis; + white-space: nowrap; + } + + .only-button { + order: 0; + flex: 0 1 auto; + align-self: auto; + + cursor: pointer; + + display: none; + padding: 0 5px; + } + + .facet-count { + order: 0; + flex: 0 1 auto; + align-self: auto; + + &:hover{ + text-decoration: none; + } + } + + &:hover { + .only-button { + display: inline; + } + } + } + + .pagination-controls { + padding-top: 5px; + + position: relative; + + .arrows { + position: absolute; + top: 5px; + right: 0px; + } + + span { + user-select: none; + color: lightgrey; + + &.active { + cursor: pointer; + color: #3B99FC; + } + } + } + + } + + &.hide-values { + .value-list { + display: none; + } + + .field-name::before { + transform: rotate(-90deg) translate(2px, 2px); + } + + .field-name .all-button { + display: none; + } + } + + &.hide-filter { + .filter-box { + display: none; + } + } + + } + } +} diff --git a/tez-ui/src/main/webapp/app/styles/em-table.less b/tez-ui/src/main/webapp/app/styles/em-table.less new file mode 100644 index 0000000000..d859d78475 --- /dev/null +++ b/tez-ui/src/main/webapp/app/styles/em-table.less @@ -0,0 +1,344 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// Imports +@import (once) "bower_components/bootstrap/less/bootstrap"; + +@import (once) "bower_components/snippet-ss/less/use"; +@import (once) "bower_components/snippet-ss/less/background"; +@import (once) "bower_components/snippet-ss/less/effects"; + +@import "./variables"; +@import "./shared"; + +@import "./search-ui"; +@import "./pagination-ui"; + +@import "./progress-cell"; + +@import "./em-table-facet-panel"; + +.em-table { + font-size: @font-size; + color: @text-color; + + margin: 10px 0px; + overflow: hidden; + + .table-header { + .clear-fix; + } + + .table-mid { + display: flex; + flex-direction: row; + flex-wrap: nowrap; + justify-content: flex-start; + align-content: stretch; + align-items: flex-start; + + .table-panel-left, table-panel-right, .table-body-left, .table-body-right { + order: 0; + flex: 0 1 auto; + align-self: auto; + } + + .table-body, .table-message { + order: 0; + flex: 1 1 auto; + align-self: auto; + border: 1px solid @border-color; + + margin-top: 5px; + } + + &>div:nth-child(2) { + border-top-left-radius: @border-radius; + border-bottom-left-radius: @border-radius; + border-left: 1px solid @border-color; + } + + &>div:nth-last-child(2) { + border-top-right-radius: @border-radius; + border-bottom-right-radius: @border-radius; + border-right: 1px solid @border-color; + } + } + + .table-footer { + .clear-fix; + } + + .table-body-left, .table-body-right, .table-body { + border-top: 1px solid @border-color; + border-bottom: 1px solid @border-color; + background-color: @table-bg; + } + + .table-message { + border-radius: @border-radius; + background-color: @table-bg; + + text-align: center; + padding: 10px; + } + + .table-body-left, .table-body-right, .table-body { + margin: 5px 0px; + } + + .table-body-left, .table-body-right { + white-space: nowrap; + font-size: 0; // If not set, each column will have a space in between + } + + .table-body{ + .force-scrollbar; + + .table-scroll-body { + //Adding this here will keep the column, and table background same white + //making the UI look better when scroll bar is shown + .dotted-bg; + + white-space: nowrap; + font-size: 0; // If not set, each column will have a space in between + } + } + + &.show-scroll-shadow { + .left-scroll-shadow, .right-scroll-shadow { + order: 0; + flex: 0 1 auto; + align-self: stretch; + position: relative; + + opacity: 0; + transition: opacity 0.3s; + + width: 0px; + z-index: 99; + + pointer-events: none; + + .shadow-container { + position: absolute; + overflow: hidden; + + top: 0px; + bottom: 0px; + width: 50px; + + &:before { + content: ""; + position: absolute; + + top: 10px; + bottom: 15px; + width: 50px; + } + } + } + .left-scroll-shadow { + .shadow-container { + &:before { + left: -50px; + box-shadow: 12px 0 40px -4px rgba(0, 0, 0, 0.2); + } + } + } + .right-scroll-shadow { + .shadow-container { + right: 0px; + &:before { + left: 50px; + box-shadow: -12px 0 40px -4px rgba(0, 0, 0, 0.2); + } + } + } + + &.show-left-scroll-shadow { + .left-scroll-shadow { + opacity: 1; + } + } + &.show-right-scroll-shadow { + .right-scroll-shadow { + opacity: 1; + } + } + } + + .table-column { + .use-border-padding-in-width-height; + + background-color: @table-bg; + + vertical-align: top; + overflow: hidden; + display: inline-block; + min-width: 150px; + + &.inner { + border-left: 1px solid @border-color; + } + + // Just the shaded header + .table-header-cell { + background-color: @bg-grey; + border-bottom: 1px solid @border-color; + + &.is-sorting { + .animated-stripes; + } + } + + .header-body, .table-cell { + font-size: @font-size; + white-space: nowrap; + + text-overflow: ellipsis; + overflow: hidden; + + height: 2.1em; + padding: 5px; + + .ember-view { + text-overflow: ellipsis; + overflow: hidden; + } + } + + .header-body { + font-weight: bold; + + padding-right: 1.1em; // To compensate space occupied by sort/resize buttons + position: relative; // So that buttons can be positioned + + .sort-bar { + cursor: pointer; + position: absolute; + + left: 0; + right: .5em; + top: 0; + bottom: 0; + } + + .sort-icon { + cursor: pointer; + position: absolute; + right: .5em; + top: .2em; + + &:before, &:after { + font-size: .7em; + opacity: .5; + position: absolute; + } + + &:before { + content: "\25B2"; + + top: 0em; + right: 0px; + } + + &:after { + content: "\25BC"; + + top: 1em; + right: 0px; + } + + &.asc{ + &:before { + opacity: 1; + } + &:after { + opacity: .5; + } + } + + &.desc { + &:before { + opacity: .5; + } + &:after { + opacity: 1; + } + } + } + + .resize-column:after { + content: "\22EE"; + cursor: col-resize; + opacity: .3; + + position: absolute; + right: 2px; + top: 6px; + } + } + + .table-cell { + position: relative; + + .comment-indicator { + position: absolute; + + color: white; + font-size: 10px; + padding-left: 4px; + + top: -4px; + right: -4px; + + width: 10px; + height: 10px; + background-color: orange; + border-radius: 10px; + + opacity: 0.6; + + &:hover { + top: -2px; + right: -2px; + } + } + + &.bg-transition { + -webkit-transition: box-shadow 500ms ease-out 500ms; + -moz-transition: box-shadow 500ms ease-out 500ms; + -o-transition: box-shadow 500ms ease-out 500ms; + transition: box-shadow 500ms ease-out 500ms; + } + + &.highlight { + box-shadow: 0 0 60px lighten(@brand-primary, 10%) inset; + } + &.is-waiting { + .animated-stripes; + } + &.inner { + border-top: 1px dotted @border-color; + margin-top: -1px; + } + } + } + +} diff --git a/tez-ui/src/main/webapp/app/styles/pagination-ui.less b/tez-ui/src/main/webapp/app/styles/pagination-ui.less new file mode 100644 index 0000000000..df3d7c070f --- /dev/null +++ b/tez-ui/src/main/webapp/app/styles/pagination-ui.less @@ -0,0 +1,88 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +.pagination-ui { + .inline-block; + .align-top; + + float: right; + + .page-list { + .inline-block; + .align-top; + + overflow: hidden; + + border: 1px solid @border-color; + border-radius: 5px; + background-color: @table-bg; + + padding: 0px; + + font-size: 0px; + + li { + .inline-block; + + padding: 6px 12px; + height: 32px; + + font-size: @font-size; + color: @text-light; + + border-left: 1px solid @border-color; + + pointer-events: none; + + &.clickable { + pointer-events: auto; + color: @text-color; + + &:hover { + background-color: @bg-grey; + cursor: pointer; + } + } + } + + .total-page-count { + font-size: .8em; + } + + :first-child { + border-left: none; + } + } + + .row-select { + margin-left: 5px; + + display: inline-block; + text-align: center; + + select { + cursor: pointer; + } + } +} + +.table-footer { + .pagination-ui { + position: static; + } +} diff --git a/tez-ui/src/main/webapp/app/styles/progress-cell.less b/tez-ui/src/main/webapp/app/styles/progress-cell.less new file mode 100644 index 0000000000..9030216a53 --- /dev/null +++ b/tez-ui/src/main/webapp/app/styles/progress-cell.less @@ -0,0 +1,26 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + + +.table-cell { + .em-progress-container { + .progress { + margin: -1px 0 0 0; + } + } +} diff --git a/tez-ui/src/main/webapp/app/styles/search-ui.less b/tez-ui/src/main/webapp/app/styles/search-ui.less new file mode 100644 index 0000000000..b850031c19 --- /dev/null +++ b/tez-ui/src/main/webapp/app/styles/search-ui.less @@ -0,0 +1,37 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +.search-ui { + .inline-block; + .align-top; + + max-width: 500px; + + margin-bottom: 2px; + + .type-select { + width: 5em; + margin-right: -3px !important; + } + + .search-syntax { + .label; + .label-default; + .no-select; + } +} diff --git a/tez-ui/src/main/webapp/app/styles/shared.less b/tez-ui/src/main/webapp/app/styles/shared.less index b34cfa6045..b448c30685 100644 --- a/tez-ui/src/main/webapp/app/styles/shared.less +++ b/tez-ui/src/main/webapp/app/styles/shared.less @@ -33,6 +33,99 @@ b { padding-left: 10px; } +.no-select { + -webkit-user-select: none; + -moz-user-select: none; + -ms-user-select: none; + + cursor: default; +} + +.no-display { + display: none !important; +} + +.no-visible { + visibility: hidden !important; +} + +.no-margin { + margin: 0px !important; +} + +.no-pointer { + pointer-events: none; +} + +.inactive { + .no-pointer; + opacity: 0.4; +} + +.no-wrap { + white-space: nowrap; +} + +.no-border { + border: none !important; +} + +.align-top { + vertical-align: top; +} + +.align-super { + vertical-align: super; +} + +.inline-block { + display: inline-block; +} + +.dotted-background { + background: + radial-gradient(#EEE 15%, transparent 17%) 0 0, + radial-gradient(#EEE 15%, transparent 17%) 5px -5px, + radial-gradient(#EEE 15%, transparent 17%) 5px 5px; + background-color: #DDD; + background-size: 10px 10px; +} + +.absolute { + position: absolute; +} + +.use-gpu { + -webkit-transform: translateZ(0); + -moz-transform: translateZ(0); + -ms-transform: translateZ(0); + -o-transform: translateZ(0); + transform: translateZ(0); +} + +.force-scrollbar { + overflow: auto; + + &::-webkit-scrollbar { + -webkit-appearance: none; + } + &::-webkit-scrollbar:vertical { + width: 11px; + } + &::-webkit-scrollbar:horizontal { + height: 11px; + } + &::-webkit-scrollbar-thumb { + border-radius: 8px; + border: 2px solid #EEE; + background-color: #BBB; + } + &::-webkit-scrollbar-track { + background-color: #EEE; + border-radius: 8px; + } +} + .align-checknradio { input[type=checkbox], input[type=radio] { vertical-align: middle; @@ -41,6 +134,28 @@ b { } } +.left-divider{ + padding-left: 5px; + border-left: 1px solid lightgrey; + margin-left: 5px; +} + +.clear-fix { + &:after { + content: " "; + visibility: hidden; + display: block; + height: 0; + clear: both; + } +} + +.animated-stripes { + .diagonal-stripes-background(#FFF, #EEE); + .animate; + .white-inner-glow; +} + .diagnostics { padding: 10px; white-space: pre-line; diff --git a/tez-ui/src/main/webapp/app/styles/variables.less b/tez-ui/src/main/webapp/app/styles/variables.less new file mode 100644 index 0000000000..e6dbeca7b2 --- /dev/null +++ b/tez-ui/src/main/webapp/app/styles/variables.less @@ -0,0 +1,28 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +@text-light: #BBBBBB; +@text-color: #222222; + +@bg-grey: #f0f0f0; +@table-bg: white; + +@border-color: #dcdcdc; +@border-radius: 5px; + +@font-size: 14px; diff --git a/tez-ui/src/main/webapp/app/templates/components/em-table-cell.hbs b/tez-ui/src/main/webapp/app/templates/components/em-table-cell.hbs new file mode 100644 index 0000000000..777be936f4 --- /dev/null +++ b/tez-ui/src/main/webapp/app/templates/components/em-table-cell.hbs @@ -0,0 +1,41 @@ +{{! + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +}} + +{{#if isWaiting}} + Waiting... +{{else}} + {{#if columnDefinition.cellComponentName}} + {{component columnDefinition.cellComponentName content=_cellContent definition=columnDefinition.cellDefinition}} + {{else}} + {{#unless columnDefinition.cellDefinition}} + {{txt _cellContent}} + {{else}} + {{txt _cellContent + type=columnDefinition.cellDefinition.type + format=columnDefinition.cellDefinition.format + timeZone=columnDefinition.cellDefinition.timeZone + valueFormat=columnDefinition.cellDefinition.valueFormat + valueTimeZone=columnDefinition.cellDefinition.valueTimeZone + valueUnit=columnDefinition.cellDefinition.valueUnit + }} + {{/unless}} + {{/if}} + {{#if _comment}} +
+ {{/if}} +{{/if}} diff --git a/tez-ui/src/main/webapp/app/templates/components/em-table-column.hbs b/tez-ui/src/main/webapp/app/templates/components/em-table-column.hbs new file mode 100644 index 0000000000..a3908db67a --- /dev/null +++ b/tez-ui/src/main/webapp/app/templates/components/em-table-column.hbs @@ -0,0 +1,22 @@ +{{! + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +}} + +{{em-table-header-cell title=definition.headerTitle definition=definition tableDefinition=tableDefinition dataProcessor=dataProcessor}} +{{#each rows as |row rowIndex|}} + {{em-table-cell columnDefinition=definition row=row index=rowIndex}} +{{/each}} diff --git a/tez-ui/src/main/webapp/app/templates/components/em-table-facet-panel-values.hbs b/tez-ui/src/main/webapp/app/templates/components/em-table-facet-panel-values.hbs new file mode 100644 index 0000000000..579e9eb2e1 --- /dev/null +++ b/tez-ui/src/main/webapp/app/templates/components/em-table-facet-panel-values.hbs @@ -0,0 +1,50 @@ +{{! + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +}} + +
+
{{data.column.headerTitle}}
+
({{data.facets.length}})
+ All +
+ +
    + {{input type="text" class="filter-box" value=filterText placeholder="Filter"}} + + {{#if showPagination}} +
    + {{currentPage}}/{{totalPages}} +
    + + +
    +
    + {{/if}} + + {{#each paginatedFacets key="value" as |facet index|}} +
  • +
    + +
    +
    {{facet.displayText}}
    + only + {{facet.count}} +
  • + {{else}} + No fields! + {{/each}} +
diff --git a/tez-ui/src/main/webapp/app/templates/components/em-table-facet-panel.hbs b/tez-ui/src/main/webapp/app/templates/components/em-table-facet-panel.hbs new file mode 100644 index 0000000000..d513786207 --- /dev/null +++ b/tez-ui/src/main/webapp/app/templates/components/em-table-facet-panel.hbs @@ -0,0 +1,33 @@ +{{! + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +}} + +{{#if dataProcessor.facetedFields.length}} +
    + {{input type="text" class="field-filter-box" value=filterText placeholder="Filter"}} + {{#each dataProcessor.facetedFields key="column.id" as |field fieldIndex|}} +
  • {{component field.column.facetType.componentName data=field tableDefinition=tableDefinition dataProcessor=dataProcessor tmpFacetConditions=tmpFacetConditions}}
  • + {{/each}} +
+ +
+ + +
+{{else}} +

Not Available!

+{{/if}} diff --git a/tez-ui/src/main/webapp/app/templates/components/em-table-header-cell.hbs b/tez-ui/src/main/webapp/app/templates/components/em-table-header-cell.hbs new file mode 100644 index 0000000000..c48eb7a12a --- /dev/null +++ b/tez-ui/src/main/webapp/app/templates/components/em-table-header-cell.hbs @@ -0,0 +1,30 @@ +{{! + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +}} + +
+ {{title}} + {{#if tableDefinition.enableSort}}{{#if definition.enableSort}} + + {{#if tableDefinition.headerAsSortButton}} + + {{/if}} + {{/if}}{{/if}} + {{#if tableDefinition.enableColumnResize}}{{#if definition.enableColumnResize}} + + {{/if}}{{/if}} +
diff --git a/tez-ui/src/main/webapp/app/templates/components/em-table-linked-cell.hbs b/tez-ui/src/main/webapp/app/templates/components/em-table-linked-cell.hbs new file mode 100644 index 0000000000..8a82631983 --- /dev/null +++ b/tez-ui/src/main/webapp/app/templates/components/em-table-linked-cell.hbs @@ -0,0 +1,41 @@ +{{! + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +}} + +{{#if normalizedLinks.length}} + {{#each normalizedLinks as |link|}} + {{#if link.routeName}} + {{#if link.withModel}} + {{#link-to link.routeName link.model target=definition.target}} + {{link.text}} + {{/link-to}} + {{else}} + {{#link-to link.routeName target=definition.target}} + {{link.text}} + {{/link-to}} + {{/if}} + {{else if link.href}} + + {{link.text}} + + {{else}} + {{link.text}} + {{/if}} + {{/each}} +{{else}} + Not Available! +{{/if}} diff --git a/tez-ui/src/main/webapp/app/templates/components/em-table-pagination-ui.hbs b/tez-ui/src/main/webapp/app/templates/components/em-table-pagination-ui.hbs new file mode 100644 index 0000000000..78edcaece4 --- /dev/null +++ b/tez-ui/src/main/webapp/app/templates/components/em-table-pagination-ui.hbs @@ -0,0 +1,45 @@ +{{! + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +}} + +
    + {{#if showFirst}} +
  • + First +
  • + {{/if}} + {{#each _possiblePages as |page|}} +
  • + {{page.pageNum}} +
  • + {{/each}} + {{#if showLast}} +
  • + Last - {{dataProcessor.totalPages}} +
  • + {{/if}} +
+ +
+ +
diff --git a/tez-ui/src/main/webapp/app/templates/components/em-table-progress-cell.hbs b/tez-ui/src/main/webapp/app/templates/components/em-table-progress-cell.hbs new file mode 100644 index 0000000000..9df8be0360 --- /dev/null +++ b/tez-ui/src/main/webapp/app/templates/components/em-table-progress-cell.hbs @@ -0,0 +1,29 @@ +{{! + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +}} + +{{#if message}} + {{message}} +{{else}} + {{em-progress + value=content + valueMin=_definition.valueMin + valueMax=_definition.valueMax + striped=_definition.striped + style=_definition.style + }} +{{/if}} diff --git a/tez-ui/src/main/webapp/app/templates/components/em-table-search-ui.hbs b/tez-ui/src/main/webapp/app/templates/components/em-table-search-ui.hbs new file mode 100644 index 0000000000..13cb1ddadc --- /dev/null +++ b/tez-ui/src/main/webapp/app/templates/components/em-table-search-ui.hbs @@ -0,0 +1,52 @@ +{{! + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +}} + +
+ {{#if (eq tableDefinition.searchType "manual")}} + + + + {{/if}} + + {{input + type="text" + class="form-control" + placeholder="Search..." + enter="search" + value=text + }} + + + + +
diff --git a/tez-ui/src/main/webapp/app/templates/components/em-table.hbs b/tez-ui/src/main/webapp/app/templates/components/em-table.hbs new file mode 100644 index 0000000000..f7be9d41b3 --- /dev/null +++ b/tez-ui/src/main/webapp/app/templates/components/em-table.hbs @@ -0,0 +1,106 @@ +{{! + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. +}} + +{{!--To add CSS rules at runtime!--}} + + +{{!--Header--}} +
+ {{#each headerComponentNames as |componentName|}} + {{component componentName tableDefinition=_definition dataProcessor=_dataProcessor}} + {{/each}} +
+ +
+
+ {{#if leftPanelComponentName}} + {{component leftPanelComponentName tableDefinition=_definition dataProcessor=_dataProcessor}} + {{/if}} +
+ + {{#if message}} +

{{message}}

+ {{else}} + {{!--Body--}} + {{#if _columns.left.length}} +
+ {{#each _columns.left as |column colIndex|}} + {{em-table-column + rows=_dataProcessor.processedRows + definition=column.definition + defaultWidth=column.width + tableDefinition=_definition + dataProcessor=_dataProcessor + index=colIndex + }} + {{/each}} +
+ {{/if}} + + + + +
+
+ {{#each _columns.center as |column colIndex|}} + {{em-table-column + rows=_dataProcessor.processedRows + definition=column.definition + defaultWidth=column.width + tableDefinition=_definition + dataProcessor=_dataProcessor + index=colIndex + }} + {{/each}} +
+
+ + + + + {{#if _columns.right.length}} +
+ {{#each _columns.right as |column colIndex|}} + {{em-table-column + rows=_dataProcessor.processedRows + definition=column.definition + defaultWidth=column.width + tableDefinition=_definition + dataProcessor=_dataProcessor + index=colIndex + }} + {{/each}} +
+ {{/if}} + {{/if}} + +
+ {{#if rightPanelComponentName}} + {{component rightPanelComponentName tableDefinition=_definition dataProcessor=_dataProcessor}} + {{/if}} +
+
+ +{{!--Footer--}} +{{#if displayFooter}} + +{{/if}} diff --git a/tez-ui/src/main/webapp/app/utils/column-definition.js b/tez-ui/src/main/webapp/app/utils/column-definition.js new file mode 100644 index 0000000000..1316866ab1 --- /dev/null +++ b/tez-ui/src/main/webapp/app/utils/column-definition.js @@ -0,0 +1,125 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import Ember from 'ember'; + +import facetTypes from './facet-types'; + +function getContentAtPath(row) { + var contentPath = this.get('contentPath'); + + if(contentPath) { + return Ember.get(row, contentPath); + } + else { + throw new Error("contentPath not set!"); + } +} + +function returnEmptyString() { + return ""; +} + +var ColumnDefinition = Ember.Object.extend({ + id: "", + headerTitle: "Not Available!", + + classNames: [], + + cellComponentName: null, + + enableSearch: true, + enableSort: true, + enableColumnResize: true, + + width: null, + minWidth: "150px", + + contentPath: null, + observePath: false, + + cellDefinition: null, + + pin: "center", + + facetType: facetTypes.VALUES, + + beforeSort: null, + getCellContent: getContentAtPath, + getSearchValue: getContentAtPath, + getSortValue: getContentAtPath, + + init: function () { + if(!this.get("id")) { + throw new Error("ID is not set."); + } + }, +}); + +ColumnDefinition.make = function (rawDefinition) { + if(Array.isArray(rawDefinition)) { + return rawDefinition.map(function (def) { + return ColumnDefinition.create(def); + }); + } + else if(typeof rawDefinition === 'object') { + return ColumnDefinition.create(rawDefinition); + } + else { + throw new Error("rawDefinition must be an Array or an Object."); + } +}; + +ColumnDefinition.makeFromModel = function (ModelClass, columnOptions) { + var attributes = Ember.get(ModelClass, 'attributes'), + columns = []; + if(attributes) { + attributes.forEach(function (meta, name) { + var column = Ember.Object.create({ + id: name, + headerTitle: name.capitalize(), + contentPath: name, + }); + + if(columnOptions) { + column.setProperties(columnOptions); + } + + columns.push(column); + }); + + return ColumnDefinition.make(columns); + } + else { + throw new Error("Value passed is not a model class"); + } +}; + +ColumnDefinition.fillerColumn = ColumnDefinition.create({ + id: "fillerColumn", + headerTitle: "", + getCellContent: returnEmptyString, + getSearchValue: returnEmptyString, + getSortValue: returnEmptyString, + + enableSearch: false, + enableSort: false, + enableColumnResize: false, +}); + +export default ColumnDefinition; diff --git a/tez-ui/src/main/webapp/app/utils/counter-column-definition.js b/tez-ui/src/main/webapp/app/utils/counter-column-definition.js index d66e551eed..5590e10244 100644 --- a/tez-ui/src/main/webapp/app/utils/counter-column-definition.js +++ b/tez-ui/src/main/webapp/app/utils/counter-column-definition.js @@ -19,7 +19,7 @@ import Ember from 'ember'; import isIOCounter from '../utils/misc'; -import ColumnDefinition from 'em-table/utils/column-definition'; +import ColumnDefinition from './column-definition'; /* * Returns a counter value from for a row diff --git a/tez-ui/src/main/webapp/app/utils/data-processor.js b/tez-ui/src/main/webapp/app/utils/data-processor.js new file mode 100644 index 0000000000..07d31c09bf --- /dev/null +++ b/tez-ui/src/main/webapp/app/utils/data-processor.js @@ -0,0 +1,275 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import Ember from 'ember'; + +import SQL from './sql'; + +/** + * Handles Sorting, Searching & Pagination + */ +export default Ember.Object.extend({ + isSorting: false, + isSearching: false, + + tableDefinition: null, + + sql: SQL.create(), + + rows: [], + _sortedRows: [], + _searchedRows: [], + _facetFilteredRows: [], + + _searchObserver: Ember.on("init", Ember.observer('tableDefinition.searchText', 'tableDefinition._actualSearchType', '_sortedRows.[]', function () { + Ember.run.once(this, "startSearch"); + })), + + _sortObserver: Ember.on("init", Ember.observer( + 'tableDefinition.sortColumnId', + 'tableDefinition.sortOrder', + 'rows.[]', function () { + Ember.run.once(this, "startSort"); + })), + + _facetedFilterObserver: Ember.on("init", Ember.observer('tableDefinition.facetConditions', '_searchedRows.[]', function () { + Ember.run.once(this, "startFacetedFilter"); + })), + + regexSearch: function (clause, rows, columns) { + var regex; + + try { + regex = new RegExp(clause, "i"); + } + catch(e) { + regex = new RegExp("", "i"); + } + + function checkRow(column) { + var value; + if(!column.get('enableSearch')) { + return false; + } + value = column.getSearchValue(this); + + if(typeof value === 'string') { + value = value.toLowerCase(); + return value.match(regex); + } + + return false; + } + + return rows.filter(function (row) { + return columns.some(checkRow, row); + }); + }, + + startSearch: function () { + var searchText = String(this.get('tableDefinition.searchText')), + rows = this.get('_sortedRows') || [], + columns = this.get('tableDefinition.columns'), + actualSearchType = this.get('tableDefinition._actualSearchType'), + that = this; + + if(searchText) { + this.set("isSearching", true); + + Ember.run.later(function () { + var result; + + switch(actualSearchType) { + case "SQL": + result = that.get("sql").search(searchText, rows, columns); + break; + + //case "Regex": Commenting as default will be called anyways + default: + result = that.regexSearch(searchText, rows, columns); + break; + } + + that.setProperties({ + _searchedRows: result, + isSearching: false + }); + }); + } + else { + this.set("_searchedRows", rows); + } + }, + + compareFunction: function (a, b){ + // Checking for undefined and null to handle some special cases in JavaScript comparison + // Eg: 1 > undefined = false & 1 < undefined = false + // "a1" > null = false & "a1" < null = false + if(a === undefined || a === null) { + return -1; + } + else if(b === undefined || b === null) { + return 1; + } + else if(a < b) { + return -1; + } + else if(a > b) { + return 1; + } + else { + return 0; + } + }, + + startSort: function () { + var rows = this.get('rows'), + tableDefinition = this.get('tableDefinition'), + sortColumnId = this.get('tableDefinition.sortColumnId'), + descending = this.get('tableDefinition.sortOrder') === 'desc', + that = this, + column; + + if(tableDefinition) { + column = tableDefinition.get('columns').find(function (element) { + return element.get('id') === sortColumnId; + }); + } + + if(rows && Array.isArray(rows.content)) { + rows = rows.toArray(); + } + + if(rows && rows.get('length') > 0 && column) { + this.set('isSorting', true); + + Ember.run.later(function () { + /* + * Creating sortArray as calling getSortValue form inside the + * sort function every time would be more costly. + */ + var sortArray = rows.map(function (row) { + return { + value: column.getSortValue(row), + row: row + }; + }), + compareFunction = that.get("compareFunction"); + + sortArray.sort(function (a, b) { + var result = compareFunction(a.value, b.value); + if(descending && result) { + result = -result; + } + return result; + }); + + that.setProperties({ + _sortedRows: sortArray.map(function (record) { + return record.row; + }), + isSorting: false + }); + }); + } + else { + this.set('_sortedRows', rows); + } + }, + + startFacetedFilter: function () { + var clause = this.get("sql").createFacetClause(this.get('tableDefinition.facetConditions'), this.get("tableDefinition.columns")), + rows = this.get('_searchedRows') || [], + columns = this.get('tableDefinition.columns'), + that = this; + + if(clause && columns) { + this.set("isSearching", true); + + Ember.run.later(function () { + var result = that.get("sql").search(clause, rows, columns); + + that.setProperties({ + _facetFilteredRows: result, + isSearching: false + }); + }); + } + else { + this.set("_facetFilteredRows", rows); + } + }, + + facetedFields: Ember.computed('_searchedRows.[]', 'tableDefinition.columns', function () { + var searchedRows = this.get("_searchedRows"), + columns = this.get('tableDefinition.columns'), + fields = []; + + if(columns) { + columns.forEach(function (column) { + var facetedData; + if(column.facetType) { + facetedData = column.facetType.facetRows(column, searchedRows); + if(facetedData) { + fields.push({ + column: column, + facets: facetedData + }); + } + } + }); + } + + return fields; + }), + + pageDetails: Ember.computed("tableDefinition.rowCount", "tableDefinition.pageNum", "_facetFilteredRows.length", function () { + var tableDefinition = this.get("tableDefinition"), + + pageNum = tableDefinition.get('pageNum'), + rowCount = tableDefinition.get('rowCount'), + + startIndex = (pageNum - 1) * rowCount, + + totalRecords = this.get('_facetFilteredRows.length'); + + if(startIndex < 0) { + startIndex = 0; + } + + return { + pageNum: pageNum, + totalPages: Math.ceil(totalRecords / rowCount), + rowCount: rowCount, + + startIndex: startIndex, + + fromRecord: totalRecords ? startIndex + 1 : 0, + toRecord: Math.min(startIndex + rowCount, totalRecords), + totalRecords: totalRecords + }; + }), + totalPages: Ember.computed.alias("pageDetails.totalPages"), // Adding an alias for backward compatibility + + // Paginate + processedRows: Ember.computed('_facetFilteredRows.[]', 'tableDefinition.rowCount', 'tableDefinition.pageNum', function () { + var rowCount = this.get('tableDefinition.rowCount'), + startIndex = (this.get('tableDefinition.pageNum') - 1) * rowCount; + return this.get('_facetFilteredRows').slice(startIndex, startIndex + rowCount); + }), +}); diff --git a/tez-ui/src/main/webapp/app/utils/facet-types.js b/tez-ui/src/main/webapp/app/utils/facet-types.js new file mode 100644 index 0000000000..0a340bbf7d --- /dev/null +++ b/tez-ui/src/main/webapp/app/utils/facet-types.js @@ -0,0 +1,85 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import Ember from 'ember'; + +var facetTypes = { + VALUES: { + componentName: "em-table-facet-panel-values", + + toClause: function (column, facetConditions) { + var values, clauses = []; + + if(facetConditions) { + if(Ember.get(facetConditions, "in.length")) { + values = facetConditions.in.map(function (value) { + value = value.replace(/'/g, "''"); + return `'${value}'`; + }); + clauses.push(`${column.id} IN (${values})`); + } + + if(Ember.get(facetConditions, "notIn.length")) { + values = facetConditions.notIn.map(function (value) { + value = value.replace(/'/g, "''"); + return `'${value}'`; + }); + clauses.push(`${column.id} NOT IN (${values})`); + } + + return clauses.join(" AND "); + } + }, + + facetRows: function (column, rows) { + var facetedDataHash = {}, + facetedDataArr = []; + + rows.forEach(function (row) { + var value = column.getSearchValue(row); + + if(typeof value === "string") { + if(!facetedDataHash[value]) { + facetedDataHash[value] = { + count: 0, + value: value + }; + facetedDataArr.push(facetedDataHash[value]); + } + facetedDataHash[value].count++; + } + + }); + + if(facetedDataArr.length) { + facetedDataArr = facetedDataArr.sort(function (a, b) { + return -(a.count - b.count); // Sort in reverse order + }); + return facetedDataArr; + } + }, + + normaliseConditions: function (conditions, data) { + if(Ember.get(conditions, "in.length") < data.length) { + return conditions; + } + } + }, +}; + +export default facetTypes; diff --git a/tez-ui/src/main/webapp/app/utils/sql.js b/tez-ui/src/main/webapp/app/utils/sql.js new file mode 100644 index 0000000000..81db3a07f5 --- /dev/null +++ b/tez-ui/src/main/webapp/app/utils/sql.js @@ -0,0 +1,94 @@ +/*global alasql*/ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + + +import Ember from 'ember'; + +/* + * A wrapper around AlaSQL + */ +export default Ember.Object.extend({ + + constructQuery: function(clause) { + return `SELECT * FROM ? WHERE ${clause}`; + }, + + validateClause: function (clause, columns) { + clause = clause.toString(); + + var query = this.constructQuery(this.normaliseClause(clause, columns || [])), + valid = false; + + if(clause.match(/\W/g)) { // If it contain special characters including space + try { + alasql(query, [[{}]]); + valid = true; + } + catch(e) {} + } + + return valid; + }, + + createFacetClause: function (conditions, columns) { + if(conditions && columns) { + return columns.map(function (column) { + if(column.get("facetType")) { + return column.get("facetType.toClause")(column, conditions[Ember.get(column, "id")]); + } + }).filter(clause => clause).join(" AND "); + } + }, + + normaliseClause: function (clause, columns) { + clause = clause.toString(); + columns.forEach(function (column) { + var headerTitle = column.get("headerTitle"); + clause = clause.replace(new RegExp(`"${headerTitle}"`, "gi"), column.get("id")); + }); + return clause; + }, + + search: function (clause, rows, columns) { + clause = this.normaliseClause(clause, columns); + + // Convert into a form that alasql can digest easily + var dataSet = rows.map(function (row, index) { + var rowObj = { + _index_: index + }; + + columns.forEach(function (column) { + if(column.get("enableSearch") && row) { + rowObj[column.get("id")] = column.getSearchValue(row); + } + }); + + return rowObj; + }); + + // Search + dataSet = alasql(this.constructQuery(clause), [dataSet]); + + return dataSet.map(function (data) { + return rows[data._index_]; + }); + } + +}); diff --git a/tez-ui/src/main/webapp/app/utils/table-definition.js b/tez-ui/src/main/webapp/app/utils/table-definition.js new file mode 100644 index 0000000000..c304ec4e80 --- /dev/null +++ b/tez-ui/src/main/webapp/app/utils/table-definition.js @@ -0,0 +1,61 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import Ember from 'ember'; + +export default Ember.Object.extend({ + + recordType: "", + + // Search + enableSearch: true, + searchText: '', + searchType: 'auto', // Can be either of auto, manual, regex OR sql + _actualSearchType: "Regex", // Set from em-table-search-ui + + // Faceting + enableFaceting: false, + facetConditions: null, + minFieldsForFilter: 15, + minValuesToDisplay: 2, + facetValuesPageSize: 10, + + // Sort + enableSort: true, + sortColumnId: '', + sortOrder: '', + headerAsSortButton: false, + + // Pagination + enablePagination: true, + pageNum: 1, + rowCount: 10, + rowCountOptions: [5, 10, 25, 50, 100], + + enableColumnResize: true, + showScrollShadow: false, + + minRowsForFooter: 25, + + columns: [], + + _pageNumResetObserver: Ember.observer('searchText', 'facetConditions', 'rowCount', function () { + this.set('pageNum', 1); + }), + +}); diff --git a/tez-ui/src/main/webapp/bower.json b/tez-ui/src/main/webapp/bower.json index 56a69f323e..cca56d817a 100644 --- a/tez-ui/src/main/webapp/bower.json +++ b/tez-ui/src/main/webapp/bower.json @@ -1,6 +1,7 @@ { "name": "tez-ui", "dependencies": { + "alasql": "^0.4.0", "ember": "2.2.0", "ember-cli-shims": "0.0.6", "ember-cli-test-loader": "0.2.1", diff --git a/tez-ui/src/main/webapp/ember-cli-build.js b/tez-ui/src/main/webapp/ember-cli-build.js index 7bbc77d334..e4217e9591 100644 --- a/tez-ui/src/main/webapp/ember-cli-build.js +++ b/tez-ui/src/main/webapp/ember-cli-build.js @@ -71,6 +71,7 @@ module.exports = function(defaults) { app.import('bower_components/codemirror/mode/sql/sql.js'); app.import('bower_components/codemirror/mode/pig/pig.js'); app.import('bower_components/codemirror/lib/codemirror.css'); + app.import('bower_components/alasql/dist/alasql.js'); return app.toTree(new MergeTrees([configEnv, zipWorker, copyFonts])); }; diff --git a/tez-ui/src/main/webapp/package.json b/tez-ui/src/main/webapp/package.json index fa80389b94..ad3aa74c5d 100644 --- a/tez-ui/src/main/webapp/package.json +++ b/tez-ui/src/main/webapp/package.json @@ -61,7 +61,6 @@ "phantomjs-prebuilt": "2.1.13" }, "dependencies": { - "em-table": "0.11.3", "em-tgraph": "0.0.14" } } diff --git a/tez-ui/src/main/webapp/tests/integration/components/em-table-cell-test.js b/tez-ui/src/main/webapp/tests/integration/components/em-table-cell-test.js new file mode 100644 index 0000000000..ccf535884e --- /dev/null +++ b/tez-ui/src/main/webapp/tests/integration/components/em-table-cell-test.js @@ -0,0 +1,45 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import Ember from 'ember'; + +import { moduleForComponent, test } from 'ember-qunit'; +import hbs from 'htmlbars-inline-precompile'; + +import ColumnDefinition from '../../../utils/column-definition'; + +moduleForComponent('em-table-cell', 'Integration | Component | em table cell', { + integration: true +}); + +test('Basic rendering test', function(assert) { + var columnDefinition = ColumnDefinition.create({ + id: 'id', + contentPath: 'keyA' + }), + row = Ember.Object.create({ + keyA: 'valueA', + keyB: 'valueB' + }); + + this.set('columnDefinition', columnDefinition); + this.set('row', row); + this.render(hbs`{{em-table-cell columnDefinition=columnDefinition row=row}}`); + + assert.equal(this.$().text().trim(), 'valueA'); +}); diff --git a/tez-ui/src/main/webapp/tests/integration/components/em-table-column-test.js b/tez-ui/src/main/webapp/tests/integration/components/em-table-column-test.js new file mode 100644 index 0000000000..96eff7af2e --- /dev/null +++ b/tez-ui/src/main/webapp/tests/integration/components/em-table-column-test.js @@ -0,0 +1,30 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import { moduleForComponent, test } from 'ember-qunit'; +import hbs from 'htmlbars-inline-precompile'; + +moduleForComponent('em-table-column', 'Integration | Component | em table column', { + integration: true +}); + +test('Basic rendering test', function(assert) { + this.render(hbs`{{em-table-column}}`); + + assert.equal(this.$().text().trim(), ''); +}); diff --git a/tez-ui/src/main/webapp/tests/integration/components/em-table-facet-panel-test.js b/tez-ui/src/main/webapp/tests/integration/components/em-table-facet-panel-test.js new file mode 100644 index 0000000000..cc0f1f0741 --- /dev/null +++ b/tez-ui/src/main/webapp/tests/integration/components/em-table-facet-panel-test.js @@ -0,0 +1,43 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import { moduleForComponent, test } from 'ember-qunit'; +import hbs from 'htmlbars-inline-precompile'; + +moduleForComponent('em-table-facet-panel', 'Integration | Component | em table facet panel', { + integration: true +}); + +test('Basic renders', function(assert) { + + // Set any properties with this.set('myProperty', 'value'); + // Handle any actions with this.on('myAction', function(val) { ... });" + EOL + EOL + + + this.render(hbs`{{em-table-facet-panel}}`); + + assert.equal(this.$().text().replace(/\n|\r\n|\r| /g, '').trim(), 'NotAvailable!'); + + // Template block usage:" + EOL + + this.render(hbs` + {{#em-table-facet-panel}} + template block text + {{/em-table-facet-panel}} + `); + + assert.equal(this.$().text().replace(/\n|\r\n|\r| /g, '').trim(), 'NotAvailable!'); +}); diff --git a/tez-ui/src/main/webapp/tests/integration/components/em-table-facet-panel-values-test.js b/tez-ui/src/main/webapp/tests/integration/components/em-table-facet-panel-values-test.js new file mode 100644 index 0000000000..f401a7da6e --- /dev/null +++ b/tez-ui/src/main/webapp/tests/integration/components/em-table-facet-panel-values-test.js @@ -0,0 +1,44 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import { moduleForComponent, test } from 'ember-qunit'; +import hbs from 'htmlbars-inline-precompile'; + +moduleForComponent('em-table-facet-panel-values', 'Integration | Component | em table facet panel values', { + integration: true +}); + +test('Basic render test', function(assert) { + + // Set any properties with this.set('myProperty', 'value'); + // Handle any actions with this.on('myAction', function(val) { ... });" + EOL + EOL + + + this.set("tmpFacetConditions", {}); + this.render(hbs`{{em-table-facet-panel-values tmpFacetConditions=tmpFacetConditions}}`); + + assert.ok(this.$().text().trim()); + + // Template block usage:" + EOL + + this.render(hbs` + {{#em-table-facet-panel-values tmpFacetConditions=tmpFacetConditions}} + template block text + {{/em-table-facet-panel-values}} + `); + + assert.ok(this.$().text().trim()); +}); diff --git a/tez-ui/src/main/webapp/tests/integration/components/em-table-header-cell-test.js b/tez-ui/src/main/webapp/tests/integration/components/em-table-header-cell-test.js new file mode 100644 index 0000000000..0c502ce9df --- /dev/null +++ b/tez-ui/src/main/webapp/tests/integration/components/em-table-header-cell-test.js @@ -0,0 +1,30 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import { moduleForComponent, test } from 'ember-qunit'; +import hbs from 'htmlbars-inline-precompile'; + +moduleForComponent('em-table-header-cell', 'Integration | Component | em table header cell', { + integration: true +}); + +test('Basic rendering test', function(assert) { + this.render(hbs`{{em-table-header-cell}}`); + + assert.equal(this.$().text().trim(), ''); +}); diff --git a/tez-ui/src/main/webapp/tests/integration/components/em-table-linked-cell-test.js b/tez-ui/src/main/webapp/tests/integration/components/em-table-linked-cell-test.js new file mode 100644 index 0000000000..7553c41014 --- /dev/null +++ b/tez-ui/src/main/webapp/tests/integration/components/em-table-linked-cell-test.js @@ -0,0 +1,30 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import { moduleForComponent, test } from 'ember-qunit'; +import hbs from 'htmlbars-inline-precompile'; + +moduleForComponent('em-table-linked-cell', 'Integration | Component | em table linked cell', { + integration: true +}); + +test('Basic rendering test', function(assert) { + this.render(hbs`{{em-table-linked-cell}}`); + + assert.equal(this.$().text().trim(), 'Not Available!'); +}); diff --git a/tez-ui/src/main/webapp/tests/integration/components/em-table-pagination-ui-test.js b/tez-ui/src/main/webapp/tests/integration/components/em-table-pagination-ui-test.js new file mode 100644 index 0000000000..0333d0cf11 --- /dev/null +++ b/tez-ui/src/main/webapp/tests/integration/components/em-table-pagination-ui-test.js @@ -0,0 +1,204 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import Ember from 'ember'; + +import DataProcessor from '../../../utils/data-processor'; +import TableDefinition from '../../../utils/table-definition'; + +import { moduleForComponent, test } from 'ember-qunit'; +import hbs from 'htmlbars-inline-precompile'; + +moduleForComponent('em-table-pagination-ui', 'Integration | Component | em table pagination ui', { + integration: true +}); + +test('Basic rendering test', function(assert) { + var customRowCount = 25, + definition = TableDefinition.create({ + rowCount: customRowCount + }), + processor; + + Ember.run(function () { + processor = DataProcessor.create({ + tableDefinition: definition, + rows: Ember.A([Ember.Object.create()]) + }); + }); + + this.set('definition', definition); + this.set('processor', processor); + this.render(hbs`{{em-table-pagination-ui tableDefinition=definition dataProcessor=processor}}`); + + var paginationItems = this.$('li'); + assert.equal(paginationItems.length, 1); + assert.equal($(paginationItems[0]).text().trim(), "1"); + + var rowSelection = this.$('select')[0]; + assert.ok(rowSelection); + assert.equal($(rowSelection).val(), customRowCount); +}); + +test('No data test', function(assert) { + var customRowCount = 2, + definition = TableDefinition.create({ + rowCount: customRowCount + }), + processor; + + Ember.run(function () { + processor = DataProcessor.create({ + tableDefinition: definition, + rows: Ember.A() + }); + }); + + this.set('definition', definition); + this.set('processor', processor); + this.render(hbs`{{em-table-pagination-ui tableDefinition=definition dataProcessor=processor}}`); + + var paginationItems = this.$('li'); + assert.equal(paginationItems.length, 0); +}); + +test('Multiple page test; without first & last', function(assert) { + var customRowCount = 2, + definition = TableDefinition.create({ + rowCount: customRowCount + }), + processor; + + Ember.run(function () { + processor = DataProcessor.create({ + tableDefinition: definition, + rows: Ember.A([Ember.Object.create(), Ember.Object.create(), Ember.Object.create()]) + }); + }); + + this.set('definition', definition); + this.set('processor', processor); + this.render(hbs`{{em-table-pagination-ui tableDefinition=definition dataProcessor=processor}}`); + + var paginationItems = this.$('li'); + assert.equal(paginationItems.length, 2); + assert.equal($(paginationItems[0]).text().trim(), "1"); + assert.equal($(paginationItems[1]).text().trim(), "2"); +}); + +test('Display last test', function(assert) { + var customRowCount = 5, + definition = TableDefinition.create({ + rowCount: customRowCount + }), + processor, + rows = []; + + for(var i = 0; i < 100; i++) { + rows.push(Ember.Object.create()); + } + + Ember.run(function () { + processor = DataProcessor.create({ + tableDefinition: definition, + rows: Ember.A(rows) + }); + }); + + this.set('definition', definition); + this.set('processor', processor); + this.render(hbs`{{em-table-pagination-ui tableDefinition=definition dataProcessor=processor}}`); + + var paginationItems = this.$('li'); + assert.equal(paginationItems.length, 6); + assert.equal($(paginationItems[0]).text().trim(), "1"); + assert.equal($(paginationItems[1]).text().trim(), "2"); + assert.equal($(paginationItems[2]).text().trim(), "3"); + assert.equal($(paginationItems[3]).text().trim(), "4"); + assert.equal($(paginationItems[4]).text().trim(), "5"); + assert.equal($(paginationItems[5]).text().trim(), "Last - 20"); +}); + +test('Display first test', function(assert) { + var customRowCount = 5, + definition = TableDefinition.create({ + pageNum: 20, + rowCount: customRowCount + }), + processor, + rows = []; + + for(var i = 0; i < 100; i++) { + rows.push(Ember.Object.create()); + } + + Ember.run(function () { + processor = DataProcessor.create({ + tableDefinition: definition, + rows: Ember.A(rows) + }); + }); + + this.set('definition', definition); + this.set('processor', processor); + this.render(hbs`{{em-table-pagination-ui tableDefinition=definition dataProcessor=processor}}`); + + var paginationItems = this.$('li'); + assert.equal(paginationItems.length, 6); + assert.equal($(paginationItems[0]).text().trim(), "First"); + assert.equal($(paginationItems[1]).text().trim(), "16"); + assert.equal($(paginationItems[2]).text().trim(), "17"); + assert.equal($(paginationItems[3]).text().trim(), "18"); + assert.equal($(paginationItems[4]).text().trim(), "19"); + assert.equal($(paginationItems[5]).text().trim(), "20"); +}); + +test('Display first & last test', function(assert) { + var customRowCount = 5, + definition = TableDefinition.create({ + pageNum: 10, + rowCount: customRowCount + }), + processor, + rows = []; + + for(var i = 0; i < 100; i++) { + rows.push(Ember.Object.create()); + } + + Ember.run(function () { + processor = DataProcessor.create({ + tableDefinition: definition, + rows: Ember.A(rows) + }); + }); + + this.set('definition', definition); + this.set('processor', processor); + this.render(hbs`{{em-table-pagination-ui tableDefinition=definition dataProcessor=processor}}`); + + var paginationItems = this.$('li'); + assert.equal(paginationItems.length, 7); + assert.equal($(paginationItems[0]).text().trim(), "First"); + assert.equal($(paginationItems[1]).text().trim(), "8"); + assert.equal($(paginationItems[2]).text().trim(), "9"); + assert.equal($(paginationItems[3]).text().trim(), "10"); + assert.equal($(paginationItems[4]).text().trim(), "11"); + assert.equal($(paginationItems[5]).text().trim(), "12"); + assert.equal($(paginationItems[6]).text().trim(), "Last - 20"); +}); diff --git a/tez-ui/src/main/webapp/tests/integration/components/em-table-progress-cell-test.js b/tez-ui/src/main/webapp/tests/integration/components/em-table-progress-cell-test.js new file mode 100644 index 0000000000..b7eced31d6 --- /dev/null +++ b/tez-ui/src/main/webapp/tests/integration/components/em-table-progress-cell-test.js @@ -0,0 +1,43 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import { moduleForComponent, test } from 'ember-qunit'; +import hbs from 'htmlbars-inline-precompile'; + +moduleForComponent('em-table-progress-cell', 'Integration | Component | em table progress cell', { + integration: true +}); + +test('Basic creation test', function(assert) { + + // Set any properties with this.set('myProperty', 'value'); + // Handle any actions with this.on('myAction', function(val) { ... });" + EOL + EOL + + + this.render(hbs`{{em-table-progress-cell content=0.5}}`); + + assert.equal(this.$().text().trim(), '50%'); + + // Template block usage:" + EOL + + this.render(hbs` + {{#em-table-progress-cell content=0.5}} + template block text + {{/em-table-progress-cell}} + `); + + assert.equal(this.$().text().trim(), '50%'); +}); diff --git a/tez-ui/src/main/webapp/tests/integration/components/em-table-search-ui-test.js b/tez-ui/src/main/webapp/tests/integration/components/em-table-search-ui-test.js new file mode 100644 index 0000000000..0cd2bbca15 --- /dev/null +++ b/tez-ui/src/main/webapp/tests/integration/components/em-table-search-ui-test.js @@ -0,0 +1,30 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import { moduleForComponent, test } from 'ember-qunit'; +import hbs from 'htmlbars-inline-precompile'; + +moduleForComponent('em-table-search-ui', 'Integration | Component | em table search ui', { + integration: true +}); + +test('Basic rendering test', function(assert) { + this.render(hbs`{{em-table-search-ui}}`); + + assert.equal(this.$().text().trim(), 'Search'); +}); diff --git a/tez-ui/src/main/webapp/tests/integration/components/em-table-test.js b/tez-ui/src/main/webapp/tests/integration/components/em-table-test.js new file mode 100644 index 0000000000..96baf79fa9 --- /dev/null +++ b/tez-ui/src/main/webapp/tests/integration/components/em-table-test.js @@ -0,0 +1,48 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import { moduleForComponent, test } from 'ember-qunit'; +import hbs from 'htmlbars-inline-precompile'; + +import TableDefinition from '../../../utils/table-definition'; +import ColumnDefinition from '../../../utils/column-definition'; + +moduleForComponent('em-table', 'Integration | Component | em table', { + integration: true +}); + +test('Basic rendering test', function(assert) { + this.render(hbs`{{em-table}}`); + + assert.equal(this.$('.table-message').text().trim(), 'No columns available!'); +}); + +test('Records missing test', function(assert) { + var definition = TableDefinition.create({ + recordType: "vertex" + }); + + this.set("columns", [ColumnDefinition.fillerColumn]); + + this.render(hbs`{{em-table columns=columns}}`); + assert.equal(this.$('.table-message').text().trim(), 'No records available!'); + + this.set("definition", definition); + this.render(hbs`{{em-table columns=columns definition=definition}}`); + assert.equal(this.$('.table-message').text().trim(), 'No vertices available!'); +}); diff --git a/tez-ui/src/main/webapp/tests/integration/em-table-status-cell-test.js b/tez-ui/src/main/webapp/tests/integration/em-table-status-cell-test.js new file mode 100644 index 0000000000..31483395e9 --- /dev/null +++ b/tez-ui/src/main/webapp/tests/integration/em-table-status-cell-test.js @@ -0,0 +1,40 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import { moduleForComponent, test } from 'ember-qunit'; +import hbs from 'htmlbars-inline-precompile'; + +moduleForComponent('em-table-status-cell', 'Integration | Component | em table status cell', { + integration: true +}); + +test('Basic creation test', function(assert) { + + this.render(hbs`{{em-table-status-cell}}`); + + assert.equal(this.$().text().trim(), 'Not Available!'); + + // Template block usage:" + EOL + + this.render(hbs` + {{#em-table-status-cell}} + template block text + {{/em-table-status-cell}} + `); + + assert.equal(this.$().text().trim(), 'Not Available!'); +}); diff --git a/tez-ui/src/main/webapp/tests/unit/utils/column-definition-test.js b/tez-ui/src/main/webapp/tests/unit/utils/column-definition-test.js new file mode 100644 index 0000000000..5ee9a49023 --- /dev/null +++ b/tez-ui/src/main/webapp/tests/unit/utils/column-definition-test.js @@ -0,0 +1,104 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import Ember from 'ember'; +import ColumnDefinition from '../../../utils/column-definition'; +import { module, test } from 'qunit'; + +module('Unit | Utility | column definition'); + +test('Class creation test', function(assert) { + assert.ok(ColumnDefinition); + + assert.ok(ColumnDefinition.make); + assert.ok(ColumnDefinition.makeFromModel); +}); + +test('make - Instance creation test', function(assert) { + + var definition = ColumnDefinition.make({ + id: "testId" + }); + var definitions = ColumnDefinition.make([{ + id: "testId 1" + },{ + id: "testId 2" + }]); + + // Single + assert.ok(definition); + + // Multiple + assert.ok(definitions); + assert.ok(Array.isArray(definitions)); + assert.equal(definitions.length, 2); +}); + +test('make - Instance creation failure test', function(assert) { + assert.throws(function () { + ColumnDefinition.make({}); + }); +}); + +test('makeFromModel test', function(assert) { + var attributes = Ember.Map.create(), + DummyModel = Ember.Object.create({ + attributes: attributes + }), + getCellContent = function () {}, + columns; + + attributes.set("attr1", "path1"); + attributes.set("attr2", "path2"); + attributes.set("attr3", "path3"); + + columns = ColumnDefinition.makeFromModel(DummyModel, { + getCellContent: getCellContent + }); + + assert.equal(columns.length, 3); + assert.equal(columns[0].id, "attr1"); + assert.equal(columns[0].headerTitle, "Attr1"); + assert.equal(columns[0].contentPath, "attr1"); + assert.equal(columns[0].getCellContent, getCellContent); +}); + +test('Instance test', function(assert) { + var definition = ColumnDefinition.make({ + id: "testId", + contentPath: "a.b" + }); + var data = Ember.Object.create({ + a: { + b: 42 + } + }); + + assert.ok(definition.getCellContent); + assert.ok(definition.getSearchValue); + assert.ok(definition.getSortValue); + + assert.equal(definition.id, "testId"); + assert.equal(definition.headerTitle, "Not Available!"); + assert.equal(definition.minWidth, "150px"); + assert.equal(definition.contentPath, "a.b"); + + assert.equal(definition.getCellContent(data), 42); + assert.equal(definition.getSearchValue(data), 42); + assert.equal(definition.getSortValue(data), 42); +}); diff --git a/tez-ui/src/main/webapp/tests/unit/utils/data-processor-test.js b/tez-ui/src/main/webapp/tests/unit/utils/data-processor-test.js new file mode 100644 index 0000000000..58f52dd013 --- /dev/null +++ b/tez-ui/src/main/webapp/tests/unit/utils/data-processor-test.js @@ -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. + */ + +import Ember from 'ember'; + +import DataProcessor from '../../../utils/data-processor'; +import ColumnDefinition from '../../../utils/column-definition'; +import { module, test } from 'qunit'; + +module('Unit | Utility | data processor'); + +test('Class creation test', function(assert) { + assert.ok(DataProcessor); +}); + +test('Instance default test', function(assert) { + var processor; + + Ember.run(function () { + processor = DataProcessor.create({ + tableDefinition: Ember.Object.create(), + startSearch: function () { + // Test Search + }, + startSort: function () { + // Test Sort + } + }); + }); + + assert.ok(processor); + assert.equal(processor.get('isSorting'), false); + assert.equal(processor.get('isSearching'), false); + + assert.ok(processor._searchObserver); + assert.ok(processor._sortObserver); + assert.ok(processor.startSearch); + assert.ok(processor.startSort); + assert.ok(processor.compareFunction); + assert.ok(processor.totalPages); + assert.ok(processor.processedRows); +}); + +test('compareFunction test', function(assert) { + var processor; + + Ember.run(function () { + processor = DataProcessor.create({ + tableDefinition: Ember.Object.create(), + startSearch: function () {}, + startSort: function () {} + }); + }); + + assert.equal(processor.compareFunction(1, 1), 0); + assert.equal(processor.compareFunction(1, 2), -1); + assert.equal(processor.compareFunction(2, 1), 1); + + assert.equal(processor.compareFunction("a", "a"), 0); + assert.equal(processor.compareFunction("a", "b"), -1); + assert.equal(processor.compareFunction("b", "a"), 1); + + assert.equal(processor.compareFunction(null, null), -1); + assert.equal(processor.compareFunction(1, null), 1); + assert.equal(processor.compareFunction(null, 2), -1); + assert.equal(processor.compareFunction("a", null), 1); + assert.equal(processor.compareFunction(null, "b"), -1); + + assert.equal(processor.compareFunction(undefined, undefined), -1); + assert.equal(processor.compareFunction(1, undefined), 1); + assert.equal(processor.compareFunction(undefined, 2), -1); + assert.equal(processor.compareFunction("a", undefined), 1); + assert.equal(processor.compareFunction(undefined, "b"), -1); +}); + +test('startSearch test', function(assert) { + var processor, + runLater = Ember.run.later; + + assert.expect(3); + + Ember.run.later = function (callback) { + callback(); + assert.equal(processor.get("_searchedRows.length"), 2); + assert.equal(processor.get("_searchedRows.0.foo"), "Foo1"); + assert.equal(processor.get("_searchedRows.1.foo"), "Foo12"); + + Ember.run.later = runLater; // Reset + }; + + Ember.run(function () { + processor = DataProcessor.create({ + tableDefinition: Ember.Object.create({ + searchText: "foo1", + columns: [ColumnDefinition.create({ + id: "foo", + contentPath: 'foo' + }), ColumnDefinition.create({ + id: "bar", + contentPath: 'bar' + })] + }), + startSort: function () { + // Test Sort + }, + _sortedRows: [Ember.Object.create({ + foo: "Foo1", + bar: "Bar1" + }), Ember.Object.create({ + foo: "Foo12", + bar: "Bar2" + }), Ember.Object.create({ + foo: "Foo3", + bar: "Bar3" + }), Ember.Object.create({ + foo: "Foo4", + bar: "Bar4" + })], + }); + }); + +}); diff --git a/tez-ui/src/main/webapp/tests/unit/utils/facet-types-test.js b/tez-ui/src/main/webapp/tests/unit/utils/facet-types-test.js new file mode 100644 index 0000000000..f3af95249f --- /dev/null +++ b/tez-ui/src/main/webapp/tests/unit/utils/facet-types-test.js @@ -0,0 +1,28 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import facetTypes from '../../../utils/facet-types'; +import { module, test } from 'qunit'; + +module('Unit | Utility | facet types'); + +test('Basic creation test', function(assert) { + assert.ok(facetTypes); + + assert.ok(facetTypes.VALUES); +}); diff --git a/tez-ui/src/main/webapp/tests/unit/utils/sql-test.js b/tez-ui/src/main/webapp/tests/unit/utils/sql-test.js new file mode 100644 index 0000000000..7aed218801 --- /dev/null +++ b/tez-ui/src/main/webapp/tests/unit/utils/sql-test.js @@ -0,0 +1,90 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import SQL from '../../../utils/sql'; +import ColumnDefinition from '../../../utils/column-definition'; +import { module, test } from 'qunit'; + +module('Unit | Utility | sql'); + +test('Class creation test', function(assert) { + var sql = SQL.create(); + + assert.ok(sql.constructQuery); + assert.ok(sql.validateClause); + assert.ok(sql.normaliseClause); + assert.ok(sql.search); +}); + +test('constructQuery test', function(assert) { + var sql = SQL.create(); + + assert.equal(sql.constructQuery("x = y"), "SELECT * FROM ? WHERE x = y"); +}); + +test('validateClause test', function(assert) { + var sql = SQL.create(); + + assert.ok(sql.validateClause("x = y")); + assert.ok(sql.validateClause("x = y AND a = b")); + assert.ok(sql.validateClause("(x = y OR y = z) AND a = b")); + assert.ok(sql.validateClause("x BETWEEN 1 AND 2")); + + assert.notOk(sql.validateClause("foo")); + assert.notOk(sql.validateClause("foo bar")); + assert.notOk(sql.validateClause("^[a-z0-9_-]{3,16}$")); + assert.notOk(sql.validateClause("^[a-z0-9_-]{6,18}$")); + assert.notOk(sql.validateClause("^[a-z0-9-]+$")); +}); + +test('normaliseClause test', function(assert) { + var sql = SQL.create(), + column = ColumnDefinition.create({ + headerTitle: "Column Header", + id: "columnID", + contentPath: "col" + }); + + assert.equal(sql.normaliseClause('"Column Header" = value', [column]), "columnID = value"); + assert.equal(sql.normaliseClause('"Another Column Header" = value', [column]), '"Another Column Header" = value'); +}); + +test('search test', function(assert) { + var sql = SQL.create(), + data = [{ + colA: "x1", + colB: "y1" + }, { + colA: "x2", + colB: "y2" + }, { + colA: "x1", + colB: "y3" + }], + columns = [ColumnDefinition.create({ + headerTitle: "Column A", + id: "colA", + contentPath: "colA" + })]; + + var result = sql.search('"Column A" = "x1"', data, columns); + + assert.equal(result.length, 2); + assert.equal(result[0].colB, "y1"); + assert.equal(result[1].colB, "y3"); +}); diff --git a/tez-ui/src/main/webapp/tests/unit/utils/table-definition-test.js b/tez-ui/src/main/webapp/tests/unit/utils/table-definition-test.js new file mode 100644 index 0000000000..234994b192 --- /dev/null +++ b/tez-ui/src/main/webapp/tests/unit/utils/table-definition-test.js @@ -0,0 +1,52 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import TableDefinition from '../../../utils/table-definition'; +import { module, test } from 'qunit'; + +module('Unit | Utility | table definition'); + +test('Class creation test', function(assert) { + assert.ok(TableDefinition); +}); + +test('Default instance test', function(assert) { + var definition = TableDefinition.create(); + + assert.ok(definition); + + assert.equal(definition.pageNum, 1); + assert.equal(definition.rowCount, 10); + assert.equal(definition.minRowsForFooter, 25); +}); + +test('Page-num reset test', function(assert) { + var definition = TableDefinition.create(); + + assert.equal(definition.pageNum, 1); + + definition.set("pageNum", 5); + assert.equal(definition.pageNum, 5); + + definition.set("searchText", "x"); + assert.equal(definition.pageNum, 1); + + definition.set("pageNum", 5); + definition.set("rowCount", 5); + assert.equal(definition.pageNum, 1); +}); diff --git a/tez-ui/src/main/webapp/yarn.lock b/tez-ui/src/main/webapp/yarn.lock index 00250e82cf..660ac80d87 100644 --- a/tez-ui/src/main/webapp/yarn.lock +++ b/tez-ui/src/main/webapp/yarn.lock @@ -1391,16 +1391,6 @@ ee-first@1.1.1: version "1.1.1" resolved "https://registry.yarnpkg.com/ee-first/-/ee-first-1.1.1.tgz#590c61156b0ae2f4f0255732a158b266bc56b21d" -em-table@0.11.3: - version "0.11.3" - resolved "https://registry.yarnpkg.com/em-table/-/em-table-0.11.3.tgz#20e605cc3814214e644199399a2383cee8d23eeb" - dependencies: - ember-cli-htmlbars "^1.0.1" - ember-cli-less "^1.4.0" - source-map "^0.5.6" - optionalDependencies: - phantomjs-prebuilt "2.1.13" - em-tgraph@0.0.14: version "0.0.14" resolved "https://registry.yarnpkg.com/em-tgraph/-/em-tgraph-0.0.14.tgz#4d48b911760f85dec41904e4056ec52542391cc1" From c875b8216fa6c236230a04c8cdbdc25c38e7ef61 Mon Sep 17 00:00:00 2001 From: Bodor Laszlo Date: Thu, 26 Aug 2021 21:46:23 +0200 Subject: [PATCH 068/137] TEZ-4332: Change ASF logo on Tez webpage according to recent guidelines (#148) --- docs/src/site/site.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/src/site/site.xml b/docs/src/site/site.xml index c8e6479a8f..5964cbc877 100644 --- a/docs/src/site/site.xml +++ b/docs/src/site/site.xml @@ -47,8 +47,8 @@ Apache Software Foundation - http://www.apache.org/images/asf_logo.gif - http://www.apache.org/ + https://apache.org/foundation/press/kit/asf_logo_wide.png + https://www.apache.org/ From 58fca8bb77d63c1ca6e6eb400eb60588159d6ae0 Mon Sep 17 00:00:00 2001 From: Bodor Laszlo Date: Thu, 14 Oct 2021 16:12:57 +0200 Subject: [PATCH 069/137] TEZ-4180: Show convenient input -> output vertex names in output/sort messages (#154) --- .../apache/tez/runtime/api/OutputContext.java | 14 +++-- .../apache/tez/mapreduce/output/MROutput.java | 4 +- .../api/impl/TezOutputContextImpl.java | 6 ++ .../common/sort/impl/ExternalSorter.java | 4 +- .../common/sort/impl/PipelinedSorter.java | 59 ++++++++++--------- .../common/sort/impl/dflt/DefaultSorter.java | 29 ++++----- .../writers/UnorderedPartitionedKVWriter.java | 42 +++++++------ .../output/OrderedPartitionedKVOutput.java | 4 +- .../library/output/UnorderedKVOutput.java | 4 +- .../output/UnorderedPartitionedKVOutput.java | 2 +- .../library/output/OutputTestHelpers.java | 1 + .../output/TestOnFileSortedOutput.java | 1 + 12 files changed, 97 insertions(+), 73 deletions(-) diff --git a/tez-api/src/main/java/org/apache/tez/runtime/api/OutputContext.java b/tez-api/src/main/java/org/apache/tez/runtime/api/OutputContext.java index 33fe772f85..f0de897fda 100644 --- a/tez-api/src/main/java/org/apache/tez/runtime/api/OutputContext.java +++ b/tez-api/src/main/java/org/apache/tez/runtime/api/OutputContext.java @@ -33,21 +33,27 @@ public interface OutputContext extends TaskContext { * Output's data * @return Name of the Destination Vertex */ - public String getDestinationVertexName(); - + String getDestinationVertexName(); + + /** + * Returns a convenient, human-readable string describing the input and output vertices. + * @return the convenient string + */ + String getInputOutputVertexNames(); + /** * Get the index of the output in the set of all outputs for the task. The * index will be consistent and valid only among the tasks of this vertex. * @return index */ - public int getOutputIndex(); + int getOutputIndex(); /** * Get an {@link OutputStatisticsReporter} for this {@link Output} that can * be used to report statistics like data size * @return {@link OutputStatisticsReporter} */ - public OutputStatisticsReporter getStatisticsReporter(); + OutputStatisticsReporter getStatisticsReporter(); /** * Notify the context that at this point no more events should be sent. diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/output/MROutput.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/output/MROutput.java index 19ece5a0f6..9aeae25bd9 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/output/MROutput.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/output/MROutput.java @@ -458,7 +458,7 @@ protected List initializeBase() throws IOException, InterruptedException initCommitter(jobConf, useNewApi); } - LOG.info(getContext().getDestinationVertexName() + ": " + LOG.info(getContext().getInputOutputVertexNames() + ": " + "outputFormat=" + outputFormatClassName + ", using newmapreduce API=" + useNewApi); return null; @@ -576,7 +576,7 @@ public void handleEvents(List outputEvents) { @Override public synchronized List close() throws IOException { flush(); - LOG.info(getContext().getDestinationVertexName() + " closed"); + LOG.info(getContext().getInputOutputVertexNames() + " closed"); long outputRecords = getContext().getCounters() .findCounter(TaskCounter.OUTPUT_RECORDS).getValue(); getContext().getStatisticsReporter().reportItemsProcessed(outputRecords); diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezOutputContextImpl.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezOutputContextImpl.java index 20ec0622c7..a17bc8900d 100644 --- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezOutputContextImpl.java +++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezOutputContextImpl.java @@ -154,6 +154,12 @@ public String getDestinationVertexName() { return destinationVertexName; } + + @Override + public String getInputOutputVertexNames() { + return String.format("%s -> %s", getTaskVertexName(), getDestinationVertexName()); + } + @Override public void fatalError(Throwable exception, String message) { super.signalFatalError(exception, message, sourceInfo); 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 3ff74f72bb..758c069799 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 @@ -183,7 +183,7 @@ public ExternalSorter(OutputContext outputContext, Configuration conf, int numOu rfs = ((LocalFileSystem)FileSystem.getLocal(this.conf)).getRaw(); if (LOG.isDebugEnabled()) { - LOG.debug(outputContext.getDestinationVertexName() + ": Initial Mem bytes : " + + LOG.debug(outputContext.getInputOutputVertexNames() + ": Initial Mem bytes : " + initialMemoryAvailable + ", in MB=" + ((initialMemoryAvailable >> 20))); } int assignedMb = (int) (initialMemoryAvailable >> 20); @@ -201,7 +201,7 @@ public ExternalSorter(OutputContext outputContext, Configuration conf, int numOu this.serializationContext = new SerializationContext(this.conf); keySerializer = serializationContext.getKeySerializer(); valSerializer = serializationContext.getValueSerializer(); - LOG.info(outputContext.getDestinationVertexName() + " using: " + LOG.info(outputContext.getInputOutputVertexNames() + " using: " + "memoryMb=" + assignedMb + ", keySerializerClass=" + serializationContext.getKeyClass() + ", valueSerializerClass=" + valSerializer 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 b70d6c4360..08786c9b2c 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 @@ -161,7 +161,7 @@ public PipelinedSorter(OutputContext outputContext, Configuration conf, int numO } StringBuilder initialSetupLogLine = new StringBuilder("Setting up PipelinedSorter for ") - .append(outputContext.getDestinationVertexName()).append(": "); + .append(outputContext.getInputOutputVertexNames()).append(": "); partitionBits = bitcount(partitions)+1; boolean confPipelinedShuffle = this.conf.getBoolean(TezRuntimeConfiguration @@ -235,10 +235,9 @@ public PipelinedSorter(OutputContext outputContext, Configuration conf, int numO TezRuntimeConfiguration.TEZ_RUNTIME_PIPELINED_SORTER_SORT_THREADS_DEFAULT); sortmaster = Executors.newFixedThreadPool(sortThreads, new ThreadFactoryBuilder().setDaemon(true) - .setNameFormat("Sorter {" + TezUtilsInternal - .cleanVertexName(outputContext.getDestinationVertexName()) + "} #%d") - .build()); - + .setNameFormat("Sorter {" + TezUtilsInternal.cleanVertexName(outputContext.getTaskVertexName()) + " -> " + + TezUtilsInternal.cleanVertexName(outputContext.getDestinationVertexName()) + "} #%d") + .build()); valSerializer.open(span.out); keySerializer.open(span.out); @@ -336,7 +335,8 @@ public void sort() throws IOException { boolean ret = spill(true); stopWatch.stop(); if (LOG.isDebugEnabled()) { - LOG.debug(outputContext.getDestinationVertexName() + ": Time taken for spill " + (stopWatch.now(TimeUnit.MILLISECONDS)) + " ms"); + LOG.debug(outputContext.getInputOutputVertexNames() + ": Time taken for spill " + + (stopWatch.now(TimeUnit.MILLISECONDS)) + " ms"); } if (pipelinedShuffle && ret) { sendPipelinedShuffleEvents(); @@ -380,7 +380,7 @@ private void sendPipelinedShuffleEvents() throws IOException{ partitions, sendEmptyPartitionDetails, pathComponent, partitionStats, reportDetailedPartitionStats(), auxiliaryService, deflater); outputContext.sendEvents(events); - LOG.info(outputContext.getDestinationVertexName() + + LOG.info(outputContext.getInputOutputVertexNames() + ": Added spill event for spill (final update=false), spillId=" + (numSpills - 1)); } @@ -496,7 +496,7 @@ private void spillSingleRecord(final Object key, final Object value, ensureSpillFilePermissions(filename, rfs); try { - LOG.info(outputContext.getDestinationVertexName() + ": Spilling to " + filename.toString() + + LOG.info(outputContext.getInputOutputVertexNames() + ": Spilling to " + filename.toString() + ", indexFilename=" + indexFilename); for (int i = 0; i < partitions; ++i) { if (isThreadInterrupted()) { @@ -568,8 +568,9 @@ public boolean spill(boolean ignoreEmptySpills) throws IOException { } } catch (InterruptedException e) { Thread.currentThread().interrupt(); - LOG.info(outputContext.getDestinationVertexName() + ": Interrupted while waiting for mergers to complete"); - throw new IOInterruptedException(outputContext.getDestinationVertexName() + ": Interrupted while waiting for mergers to complete", e); + LOG.info(outputContext.getInputOutputVertexNames() + ": Interrupted while waiting for mergers to complete"); + throw new IOInterruptedException( + outputContext.getInputOutputVertexNames() + ": Interrupted while waiting for mergers to complete", e); } // create spill file @@ -581,7 +582,7 @@ public boolean spill(boolean ignoreEmptySpills) throws IOException { spillFilePaths.put(numSpills, filename); out = rfs.create(filename, true, 4096); ensureSpillFilePermissions(filename, rfs); - LOG.info(outputContext.getDestinationVertexName() + ": Spilling to " + filename.toString()); + LOG.info(outputContext.getInputOutputVertexNames() + ": Spilling to " + filename.toString()); for (int i = 0; i < partitions; ++i) { if (isThreadInterrupted()) { return false; @@ -652,8 +653,9 @@ private boolean isThreadInterrupted() throws IOException { cleanup(); } sortmaster.shutdownNow(); - LOG.info(outputContext.getDestinationVertexName() + ": Thread interrupted, cleaned up stale data, sorter threads shutdown=" + sortmaster - .isShutdown() + ", terminated=" + sortmaster.isTerminated()); + LOG.info(outputContext.getInputOutputVertexNames() + + ": Thread interrupted, cleaned up stale data, sorter threads shutdown=" + sortmaster.isShutdown() + + ", terminated=" + sortmaster.isTerminated()); return true; } return false; @@ -674,7 +676,7 @@ public void flush() throws IOException { } try { - LOG.info(outputContext.getDestinationVertexName() + ": Starting flush of map output"); + LOG.info(outputContext.getInputOutputVertexNames() + ": Starting flush of map output"); span.end(); merger.add(span.sort(sorter)); // force a spill in flush() @@ -698,7 +700,7 @@ public void flush() throws IOException { * NPE leading to distraction when debugging. */ if (LOG.isDebugEnabled()) { - LOG.debug(outputContext.getDestinationVertexName() + LOG.debug(outputContext.getInputOutputVertexNames() + ": Index list is empty... returning"); } return; @@ -717,7 +719,8 @@ public void flush() throws IOException { outputContext, i, indexCacheList.get(i), partitions, sendEmptyPartitionDetails, pathComponent, partitionStats, reportDetailedPartitionStats(), auxiliaryService, deflater); - LOG.info(outputContext.getDestinationVertexName() + ": Adding spill event for spill (final update=" + isLastEvent + "), spillId=" + i); + LOG.info(outputContext.getInputOutputVertexNames() + ": Adding spill event for spill (final update=" + + isLastEvent + "), spillId=" + i); } return; } @@ -736,7 +739,7 @@ public void flush() throws IOException { sameVolRename(filename, finalOutputFile); sameVolRename(indexFilename, finalIndexFile); if (LOG.isDebugEnabled()) { - LOG.debug(outputContext.getDestinationVertexName() + ": numSpills=" + numSpills + + LOG.debug(outputContext.getInputOutputVertexNames() + ": numSpills=" + numSpills + ", finalOutputFile=" + finalOutputFile + ", " + "finalIndexFile=" + finalIndexFile + ", filename=" + filename + ", indexFilename=" + indexFilename); @@ -759,7 +762,7 @@ public void flush() throws IOException { mapOutputFile.getOutputIndexFileForWrite(0); //TODO if (LOG.isDebugEnabled()) { - LOG.debug(outputContext.getDestinationVertexName() + ": " + + LOG.debug(outputContext.getInputOutputVertexNames() + ": " + "numSpills: " + numSpills + ", finalOutputFile:" + finalOutputFile + ", finalIndexFile:" + finalIndexFile); } @@ -944,7 +947,7 @@ public SortSpan(ByteBuffer source, int maxItems, int perItem, RawComparator comp } ByteBuffer reserved = source.duplicate(); reserved.mark(); - LOG.info(outputContext.getDestinationVertexName() + ": " + "reserved.remaining()=" + + LOG.info(outputContext.getInputOutputVertexNames() + ": " + "reserved.remaining()=" + reserved.remaining() + ", reserved.metasize=" + metasize); reserved.position(metasize); kvbuffer = reserved.slice(); @@ -966,8 +969,8 @@ public SpanIterator sort(IndexedSorter sorter) { if(length() > 1) { sorter.sort(this, 0, length(), progressable); } - LOG.info(outputContext.getDestinationVertexName() + ": " + "done sorting span=" + index + ", length=" + length() + ", " - + "time=" + (System.currentTimeMillis() - start)); + LOG.info(outputContext.getInputOutputVertexNames() + ": " + "done sorting span=" + index + ", length=" + length() + + ", " + "time=" + (System.currentTimeMillis() - start)); return new SpanIterator((SortSpan)this); } @@ -1042,8 +1045,9 @@ public SortSpan next() { } newSpan = new SortSpan(remaining, items, perItem, newComparator); newSpan.index = index+1; - LOG.info(String.format(outputContext.getDestinationVertexName() + ": " + "New Span%d.length = %d, perItem = %d", newSpan.index, newSpan - .length(), perItem) + ", counter:" + mapOutputRecordCounter.getValue()); + LOG.info( + String.format(outputContext.getInputOutputVertexNames() + ": " + "New Span%d.length = %d, perItem = %d", + newSpan.index, newSpan.length(), perItem) + ", counter:" + mapOutputRecordCounter.getValue()); return newSpan; } return null; @@ -1064,13 +1068,14 @@ public ByteBuffer end() { return null; } int perItem = kvbuffer.position()/items; - LOG.info(outputContext.getDestinationVertexName() + ": " + String.format("Span%d.length = %d, perItem = %d", index, length(), perItem)); + LOG.info(outputContext.getInputOutputVertexNames() + ": " + + String.format("Span%d.length = %d, perItem = %d", index, length(), perItem)); if(remaining.remaining() < METASIZE+perItem) { //Check if we can get the next Buffer from the main buffer list ByteBuffer space = allocateSpace(); if (space != null) { - LOG.info(outputContext.getDestinationVertexName() + ": " + "Getting memory from next block in the list, recordsWritten=" + - mapOutputRecordCounter.getValue()); + LOG.info(outputContext.getInputOutputVertexNames() + ": " + + "Getting memory from next block in the list, recordsWritten=" + mapOutputRecordCounter.getValue()); reinit = true; return space; } @@ -1403,7 +1408,7 @@ public final boolean ready() throws IOException, InterruptedException { total += sp.span.length(); eq += sp.span.getEq(); } - LOG.info(outputContext.getDestinationVertexName() + ": " + "Heap = " + sb.toString()); + LOG.info(outputContext.getInputOutputVertexNames() + ": " + "Heap = " + sb.toString()); return true; } catch(ExecutionException e) { LOG.error("Heap size={}, total={}, eq={}, partition={}, gallop={}, totalItr={}," 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 dd6c083109..7c678749b2 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 @@ -153,7 +153,7 @@ public DefaultSorter(OutputContext outputContext, Configuration conf, int numOut .TEZ_RUNTIME_PIPELINED_SHUFFLE_ENABLED_DEFAULT); if (confPipelinedShuffle) { - LOG.warn(outputContext.getDestinationVertexName() + ": " + + LOG.warn(outputContext.getInputOutputVertexNames() + ": " + TezRuntimeConfiguration.TEZ_RUNTIME_PIPELINED_SHUFFLE_ENABLED + " does not work " + "with DefaultSorter. It is supported only with PipelinedSorter."); } @@ -371,7 +371,8 @@ synchronized void collect(Object key, Object value, final int partition kvindex = (int)(((long)kvindex - NMETA + kvmeta.capacity()) % kvmeta.capacity()); totalKeys++; } catch (MapBufferTooSmallException e) { - LOG.info(outputContext.getDestinationVertexName() + ": Record too large for in-memory buffer: " + e.getMessage()); + LOG.info( + outputContext.getInputOutputVertexNames() + ": Record too large for in-memory buffer: " + e.getMessage()); spillSingleRecord(key, value, partition); mapOutputRecordCounter.increment(1); return; @@ -390,7 +391,7 @@ private void setEquator(int pos) { // Cast one of the operands to long to avoid integer overflow kvindex = (int) (((long) aligned - METASIZE + kvbuffer.length) % kvbuffer.length) / 4; if (LOG.isInfoEnabled()) { - LOG.info(outputContext.getDestinationVertexName() + ": " + "(EQUATOR) " + pos + " kvi " + kvindex + + LOG.info(outputContext.getInputOutputVertexNames() + ": " + "(EQUATOR) " + pos + " kvi " + kvindex + "(" + (kvindex * 4) + ")"); } } @@ -408,7 +409,7 @@ private void resetSpill() { // Cast one of the operands to long to avoid integer overflow kvstart = kvend = (int) (((long) aligned - METASIZE + kvbuffer.length) % kvbuffer.length) / 4; if (LOG.isInfoEnabled()) { - LOG.info(outputContext.getDestinationVertexName() + ": " + "(RESET) equator " + e + " kv " + kvstart + "(" + + LOG.info(outputContext.getInputOutputVertexNames() + ": " + "(RESET) equator " + e + " kv " + kvstart + "(" + (kvstart * 4) + ")" + " kvi " + kvindex + "(" + (kvindex * 4) + ")"); } } @@ -664,7 +665,7 @@ void interruptSpillThread() throws IOException { spillThread.interrupt(); spillThread.join(); } catch (InterruptedException e) { - LOG.info(outputContext.getDestinationVertexName() + ": " + "Spill thread interrupted"); + LOG.info(outputContext.getInputOutputVertexNames() + ": " + "Spill thread interrupted"); //Reset status Thread.currentThread().interrupt(); throw new IOInterruptedException("Spill failed", e); @@ -673,7 +674,7 @@ void interruptSpillThread() throws IOException { @Override public void flush() throws IOException { - LOG.info(outputContext.getDestinationVertexName() + ": " + "Starting flush of map output"); + LOG.info(outputContext.getInputOutputVertexNames() + ": " + "Starting flush of map output"); outputContext.notifyProgress(); if (Thread.currentThread().isInterrupted()) { /** @@ -710,7 +711,7 @@ public void flush() throws IOException { bufend = bufmark; if (LOG.isInfoEnabled()) { LOG.info( - outputContext.getDestinationVertexName() + ": " + "Sorting & Spilling map output. " + outputContext.getInputOutputVertexNames() + ": " + "Sorting & Spilling map output. " + "bufstart = " + bufstart + ", bufend = " + bufmark + ", bufvoid = " + bufvoid + "; " + "kvstart=" + kvstart + "(" + (kvstart * 4) + ")" + ", kvend = " + kvend + "(" + (kvend * 4) + ")" @@ -781,7 +782,7 @@ public void run() { spillLock.unlock(); sortAndSpill(sameKeyCount, totalKeysCount); } catch (Throwable t) { - LOG.warn(outputContext.getDestinationVertexName() + ": " + "Got an exception in sortAndSpill", t); + LOG.warn(outputContext.getInputOutputVertexNames() + ": " + "Got an exception in sortAndSpill", t); sortSpillException = t; } finally { spillLock.lock(); @@ -794,7 +795,7 @@ public void run() { } } } catch (InterruptedException e) { - LOG.info(outputContext.getDestinationVertexName() + ": " + "Spill thread interrupted"); + LOG.info(outputContext.getInputOutputVertexNames() + ": " + "Spill thread interrupted"); Thread.currentThread().interrupt(); } finally { spillLock.unlock(); @@ -830,7 +831,7 @@ private void startSpill() { bufend = bufmark; spillInProgress = true; if (LOG.isInfoEnabled()) { - LOG.info(outputContext.getDestinationVertexName() + ": Spilling map output." + LOG.info(outputContext.getInputOutputVertexNames() + ": Spilling map output." + "bufstart=" + bufstart + ", bufend = " + bufmark + ", bufvoid = " + bufvoid +"; kvstart=" + kvstart + "(" + (kvstart * 4) + ")" +", kvend = " + kvend + "(" + (kvend * 4) + ")" @@ -936,7 +937,7 @@ protected void spill(int mstart, int mend, long sameKeyCount, long totalKeysCoun TezRawKeyValueIterator kvIter = new MRResultIterator(spstart, spindex); if (LOG.isDebugEnabled()) { - LOG.debug(outputContext.getDestinationVertexName() + ": " + "Running combine processor"); + LOG.debug(outputContext.getInputOutputVertexNames() + ": " + "Running combine processor"); } runCombineProcessor(kvIter, writer); } @@ -975,7 +976,7 @@ protected void spill(int mstart, int mend, long sameKeyCount, long totalKeysCoun totalIndexCacheMemory += spillRec.size() * MAP_OUTPUT_INDEX_RECORD_LENGTH; } - LOG.info(outputContext.getDestinationVertexName() + ": " + "Finished spill " + numSpills + LOG.info(outputContext.getInputOutputVertexNames() + ": " + "Finished spill " + numSpills + " at " + filename.toString()); ++numSpills; if (!isFinalMergeEnabled()) { @@ -1172,7 +1173,7 @@ private void maybeSendEventForSpill(List events, boolean isLastEvent, outputContext, index, spillRecord, partitions, sendEmptyPartitionDetails, pathComponent, partitionStats, reportDetailedPartitionStats(), auxiliaryService, deflater); - LOG.info(outputContext.getDestinationVertexName() + ": " + + LOG.info(outputContext.getInputOutputVertexNames() + ": " + "Adding spill event for spill (final update=" + isLastEvent + "), spillId=" + index); if (sendEvent) { @@ -1339,7 +1340,7 @@ private void mergeParts() throws IOException, InterruptedException { segmentList.add(s); } if (LOG.isDebugEnabled()) { - LOG.debug(outputContext.getDestinationVertexName() + ": " + LOG.debug(outputContext.getInputOutputVertexNames() + ": " + "TaskIdentifier=" + taskIdentifier + " Partition=" + parts + "Spill =" + i + "(" + indexRecord.getStartOffset() + "," + indexRecord.getRawLength() + ", " + diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/writers/UnorderedPartitionedKVWriter.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/writers/UnorderedPartitionedKVWriter.java index 5ff2944766..faf75866b9 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/writers/UnorderedPartitionedKVWriter.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/writers/UnorderedPartitionedKVWriter.java @@ -104,7 +104,7 @@ public class UnorderedPartitionedKVWriter extends BaseUnorderedPartitionedKVWrit // Maybe setup a separate statistics class which can be shared between the // buffer and the main path instead of having multiple arrays. - private final String destNameTrimmed; + private final String sourceDestNameTrimmed; private final long availableMemory; @VisibleForTesting final WrappedBuffer[] buffers; @@ -206,7 +206,8 @@ public UnorderedPartitionedKVWriter(OutputContext outputContext, Configuration c Preconditions.checkArgument(availableMemoryBytes >= 0, "availableMemory should be >= 0 bytes"); - this.destNameTrimmed = TezUtilsInternal.cleanVertexName(outputContext.getDestinationVertexName()); + this.sourceDestNameTrimmed = TezUtilsInternal.cleanVertexName(outputContext.getTaskVertexName()) + " -> " + + TezUtilsInternal.cleanVertexName(outputContext.getDestinationVertexName()); //Not checking for TEZ_RUNTIME_ENABLE_FINAL_MERGE_IN_OUTPUT as it might not add much value in // this case. Add it later if needed. boolean pipelinedShuffleConf = this.conf.getBoolean(TezRuntimeConfiguration @@ -257,7 +258,7 @@ public UnorderedPartitionedKVWriter(OutputContext outputContext, Configuration c buffers[0] = new WrappedBuffer(numOutputs, sizePerBuffer); numInitializedBuffers = 1; if (LOG.isDebugEnabled()) { - LOG.debug(destNameTrimmed + ": " + "Initializing Buffer #" + + LOG.debug(sourceDestNameTrimmed + ": " + "Initializing Buffer #" + numInitializedBuffers + " with size=" + sizePerBuffer); } currentBuffer = buffers[0]; @@ -313,7 +314,7 @@ public UnorderedPartitionedKVWriter(OutputContext outputContext, Configuration c skipBuffers = false; writer = null; } - LOG.info(destNameTrimmed + ": " + LOG.info(sourceDestNameTrimmed + ": " + "numBuffers=" + numBuffers + ", sizePerBuffer=" + sizePerBuffer + ", skipBuffers=" + skipBuffers @@ -493,7 +494,7 @@ private void setupNextBuffer() throws IOException { // Update overall stats final int filledBufferCount = filledBuffers.size(); if (LOG.isDebugEnabled() || (filledBufferCount % 10) == 0) { - LOG.info(destNameTrimmed + ": " + "Moving to next buffer. Total filled buffers: " + filledBufferCount); + LOG.info(sourceDestNameTrimmed + ": " + "Moving to next buffer. Total filled buffers: " + filledBufferCount); } updateGlobalStats(currentBuffer); @@ -531,7 +532,7 @@ private boolean scheduleSpill(boolean block) throws IOException { final int filledBufferCount = filledBuffers.size(); if (LOG.isDebugEnabled() || (filledBufferCount % 10) == 0) { - LOG.info(destNameTrimmed + ": triggering spill. filledBuffers.size=" + filledBufferCount); + LOG.info(sourceDestNameTrimmed + ": triggering spill. filledBuffers.size=" + filledBufferCount); } pendingSpillCount.incrementAndGet(); int spillNumber = numSpills.getAndIncrement(); @@ -673,10 +674,10 @@ protected SpillResult callInternal() throws IOException { spillResult = new SpillResult(compressedLength, this.filledBuffers); handleSpillIndex(spillPathDetails, spillRecord); - LOG.info(destNameTrimmed + ": " + "Finished spill " + spillIndex); + LOG.info(sourceDestNameTrimmed + ": " + "Finished spill " + spillIndex); if (LOG.isDebugEnabled()) { - LOG.debug(destNameTrimmed + ": " + "Spill=" + spillIndex + ", indexPath=" + LOG.debug(sourceDestNameTrimmed + ": " + "Spill=" + spillIndex + ", indexPath=" + spillPathDetails.indexFilePath + ", outputPath=" + spillPathDetails.outputFilePath); } return spillResult; @@ -754,7 +755,8 @@ public List close() throws IOException, InterruptedException { isShutdown.set(true); spillLock.lock(); try { - LOG.info(destNameTrimmed + ": " + "Waiting for all spills to complete : Pending : " + pendingSpillCount.get()); + LOG.info( + sourceDestNameTrimmed + ": " + "Waiting for all spills to complete : Pending : " + pendingSpillCount.get()); while (pendingSpillCount.get() != 0 && spillException == null) { spillInProgress.await(); } @@ -762,7 +764,7 @@ public List close() throws IOException, InterruptedException { spillLock.unlock(); } if (spillException != null) { - LOG.error(destNameTrimmed + ": " + "Error during spill, throwing"); + LOG.error(sourceDestNameTrimmed + ": " + "Error during spill, throwing"); // Assuming close will be called on the same thread as the write cleanup(); currentBuffer.cleanup(); @@ -773,7 +775,7 @@ public List close() throws IOException, InterruptedException { throw new IOException(spillException); } } else { - LOG.info(destNameTrimmed + ": " + "All spills complete"); + LOG.info(sourceDestNameTrimmed + ": " + "All spills complete"); // Assuming close will be called on the same thread as the write cleanup(); @@ -1082,7 +1084,8 @@ private void mergeAll() throws IOException { for (int i = 0; i < numPartitions; i++) { long segmentStart = out.getPos(); if (numRecordsPerPartition[i] == 0) { - LOG.info(destNameTrimmed + ": " + "Skipping partition: " + i + " in final merge since it has no records"); + LOG.info( + sourceDestNameTrimmed + ": " + "Skipping partition: " + i + " in final merge since it has no records"); continue; } writer = new Writer(keySerialization, valSerialization, out, keyClass, valClass, codec, null, null); @@ -1136,7 +1139,7 @@ private void mergeAll() throws IOException { } finalSpillRecord.writeToFile(finalIndexPath, conf, localFs); fileOutputBytesCounter.increment(indexFileSizeEstimate); - LOG.info(destNameTrimmed + ": " + "Finished final spill after merging : " + numSpills.get() + " spills"); + LOG.info(sourceDestNameTrimmed + ": " + "Finished final spill after merging : " + numSpills.get() + " spills"); } private void deleteIntermediateSpills() { @@ -1208,9 +1211,10 @@ private void writeLargeRecord(final Object key, final Object value, final int pa mayBeSendEventsForSpill(emptyPartitions, sizePerPartition, spillIndex, false); - LOG.info(destNameTrimmed + ": " + "Finished writing large record of size " + outSize + " to spill file " + spillIndex); + LOG.info(sourceDestNameTrimmed + ": " + "Finished writing large record of size " + outSize + " to spill file " + + spillIndex); if (LOG.isDebugEnabled()) { - LOG.debug(destNameTrimmed + ": " + "LargeRecord Spill=" + spillIndex + ", indexPath=" + LOG.debug(sourceDestNameTrimmed + ": " + "LargeRecord Spill=" + spillIndex + ", indexPath=" + spillPathDetails.indexFilePath + ", outputPath=" + spillPathDetails.outputFilePath); } @@ -1346,7 +1350,7 @@ private void mayBeSendEventsForSpill( try { events = generateEventForSpill(emptyPartitions, sizePerPartition, spillNumber, isFinalUpdate); - LOG.info(destNameTrimmed + ": " + "Adding spill event for spill" + LOG.info(sourceDestNameTrimmed + ": " + "Adding spill event for spill" + " (final update=" + isFinalUpdate + "), spillId=" + spillNumber); if (pipelinedShuffle) { //Send out an event for consuming. @@ -1355,7 +1359,7 @@ private void mayBeSendEventsForSpill( this.finalEvents.addAll(events); } } catch (IOException e) { - LOG.error(destNameTrimmed + ": " + "Error in sending pipelined events", e); + LOG.error(sourceDestNameTrimmed + ": " + "Error in sending pipelined events", e); outputContext.reportFailure(TaskFailureType.NON_FATAL, e, "Error in sending events."); } @@ -1414,7 +1418,7 @@ public void onSuccess(SpillResult result) { availableBuffers.add(buffer); } } catch (Throwable e) { - LOG.error(destNameTrimmed + ": Failure while attempting to reset buffer after spill", e); + LOG.error(sourceDestNameTrimmed + ": Failure while attempting to reset buffer after spill", e); outputContext.reportFailure(TaskFailureType.NON_FATAL, e, "Failure while attempting to reset buffer after spill"); } @@ -1444,7 +1448,7 @@ public void onSuccess(SpillResult result) { public void onFailure(Throwable t) { // spillException setup to throw an exception back to the user. Requires synchronization. // Consider removing it in favor of having Tez kill the task - LOG.error(destNameTrimmed + ": " + "Failure while spilling to disk", t); + LOG.error(sourceDestNameTrimmed + ": " + "Failure while spilling to disk", t); spillException = t; outputContext.reportFailure(TaskFailureType.NON_FATAL, t, "Failure while spilling to disk"); spillLock.lock(); diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/OrderedPartitionedKVOutput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/OrderedPartitionedKVOutput.java index 676fe17a5f..44cb9d6aae 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/OrderedPartitionedKVOutput.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/OrderedPartitionedKVOutput.java @@ -135,7 +135,7 @@ public synchronized void start() throws Exception { if (pipelinedShuffle) { if (finalMergeEnabled) { - LOG.info(getContext().getDestinationVertexName() + " disabling final merge as " + LOG.info(getContext().getInputOutputVertexNames() + " disabling final merge as " + TezRuntimeConfiguration.TEZ_RUNTIME_PIPELINED_SHUFFLE_ENABLED + " is enabled."); finalMergeEnabled = false; conf.setBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_ENABLE_FINAL_MERGE_IN_OUTPUT, false); @@ -194,7 +194,7 @@ public synchronized List close() throws IOException { returnEvents.addAll(generateEvents()); sorter = null; } else { - LOG.warn(getContext().getDestinationVertexName() + + LOG.warn(getContext().getInputOutputVertexNames() + ": Attempting to close output {} of type {} before it was started. Generating empty events", getContext().getDestinationVertexName(), this.getClass().getSimpleName()); returnEvents = generateEmptyEvents(); diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/UnorderedKVOutput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/UnorderedKVOutput.java index e7a4429d95..bcacc5238e 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/UnorderedKVOutput.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/UnorderedKVOutput.java @@ -106,7 +106,7 @@ public synchronized void start() throws Exception { this.kvWriter = new UnorderedPartitionedKVWriter(getContext(), conf, 1, memoryUpdateCallbackHandler.getMemoryAssigned()); isStarted.set(true); - LOG.info(getContext().getDestinationVertexName() + " started. MemoryAssigned=" + LOG.info(getContext().getInputOutputVertexNames() + " started. MemoryAssigned=" + memoryUpdateCallbackHandler.getMemoryAssigned()); } } @@ -130,7 +130,7 @@ public synchronized List close() throws Exception { returnEvents = kvWriter.close(); kvWriter = null; } else { - LOG.warn(getContext().getDestinationVertexName() + + LOG.warn(getContext().getInputOutputVertexNames() + ": Attempting to close output {} of type {} before it was started. Generating empty events", getContext().getDestinationVertexName(), this.getClass().getSimpleName()); returnEvents = new LinkedList(); diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/UnorderedPartitionedKVOutput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/UnorderedPartitionedKVOutput.java index 439b732db5..9bc7ea40cd 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/UnorderedPartitionedKVOutput.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/UnorderedPartitionedKVOutput.java @@ -108,7 +108,7 @@ public synchronized List close() throws Exception { returnEvents = kvWriter.close(); kvWriter = null; } else { - LOG.warn(getContext().getDestinationVertexName() + + LOG.warn(getContext().getInputOutputVertexNames() + ": Attempting to close output {} of type {} before it was started. Generating empty events", getContext().getDestinationVertexName(), this.getClass().getSimpleName()); returnEvents = new LinkedList(); diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/OutputTestHelpers.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/OutputTestHelpers.java index b81c2bd036..a7c7ca28cd 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/OutputTestHelpers.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/OutputTestHelpers.java @@ -69,6 +69,7 @@ static OutputContext createOutputContext(Configuration conf, Configuration userP }).when(ctx).requestInitialMemory(anyLong(), any(MemoryUpdateCallback.class)); doReturn(conf).when(ctx).getContainerConfiguration(); doReturn(TezUtils.createUserPayloadFromConf(userPayloadConf)).when(ctx).getUserPayload(); + doReturn("taskVertex").when(ctx).getTaskVertexName(); doReturn("destinationVertex").when(ctx).getDestinationVertexName(); doReturn("UUID").when(ctx).getUniqueIdentifier(); doReturn(new String[] { workingDir.toString() }).when(ctx).getWorkDirs(); diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/TestOnFileSortedOutput.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/TestOnFileSortedOutput.java index 2c9c3b2ace..7999d45fcd 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/TestOnFileSortedOutput.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/TestOnFileSortedOutput.java @@ -406,6 +406,7 @@ private OutputContext createTezOutputContext() throws IOException { doReturn(payLoad).when(context).getUserPayload(); doReturn(5 * 1024 * 1024l).when(context).getTotalMemoryAvailableToTask(); doReturn(UniqueID).when(context).getUniqueIdentifier(); + doReturn("v0").when(context).getTaskVertexName(); doReturn("v1").when(context).getDestinationVertexName(); doReturn(ByteBuffer.wrap(serviceProviderMetaData.getData())).when(context) .getServiceProviderMetaData From 3326978dfbe799f92f262d9a2b16c99a8ef61836 Mon Sep 17 00:00:00 2001 From: Bodor Laszlo Date: Mon, 25 Oct 2021 13:19:48 +0200 Subject: [PATCH 070/137] TEZ-4342: TestSecureShuffle is broken - No subject alternative names present (#158) (Laszlo Bodor reviewed by Rajesh Balamohan) --- .../apache/tez/test/TestSecureShuffle.java | 50 +++++++++++++++++-- 1 file changed, 46 insertions(+), 4 deletions(-) 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 aed240997e..2b2221230f 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 @@ -25,14 +25,19 @@ import java.io.File; import java.io.IOException; import java.io.OutputStreamWriter; +import java.math.BigInteger; import java.net.InetAddress; import java.security.KeyPair; +import java.security.SecureRandom; import java.security.cert.X509Certificate; import java.util.ArrayList; import java.util.Collection; +import java.util.Date; import java.util.HashMap; import java.util.Map; +import javax.security.auth.x500.X500Principal; + import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; @@ -46,6 +51,10 @@ import org.apache.tez.dag.api.TezConfiguration; import org.apache.tez.mapreduce.examples.TestOrderedWordCount; import org.apache.tez.runtime.library.api.TezRuntimeConfiguration; +import org.bouncycastle.asn1.x509.GeneralName; +import org.bouncycastle.asn1.x509.GeneralNames; +import org.bouncycastle.asn1.x509.X509Extensions; +import org.bouncycastle.x509.X509V3CertificateGenerator; import org.junit.After; import org.junit.AfterClass; import org.junit.Before; @@ -222,8 +231,9 @@ private static void setupKeyStores() throws Exception { * (as discussed in https://github.com/AsyncHttpClient/async-http-client/issues/928), that's why * it cannot be set for an async http connection. So instead of hacking an ALLOW_ALL verifier * somehow (which cannot be propagated to netty), a valid certificate with the actual hostname - * should be generated in setupSSLConfig, so the only change is the usage of - * "InetAddress.getLocalHost().getHostName()". + * should be generated in setupSSLConfig. So, one change is the usage of + * InetAddress.getLocalHost().getHostName(), the other is using local generateCertificate, + * which fixes another issue. */ public static void setupSSLConfig(String keystoresDir, String sslConfDir, Configuration config, boolean useClientCert, boolean trustStore, String excludeCiphers) throws Exception { @@ -242,7 +252,7 @@ public static void setupSSLConfig(String keystoresDir, String sslConfDir, Config if (useClientCert) { KeyPair cKP = KeyStoreTestUtil.generateKeyPair("RSA"); X509Certificate cCert = - KeyStoreTestUtil.generateCertificate("CN=localhost, O=client", cKP, 30, "SHA1withRSA"); + generateCertificate("CN=localhost, O=client", cKP, 30, "SHA1withRSA"); KeyStoreTestUtil.createKeyStore(clientKS, clientPassword, "client", cKP.getPrivate(), cCert); certs.put("client", cCert); } @@ -250,7 +260,7 @@ public static void setupSSLConfig(String keystoresDir, String sslConfDir, Config String localhostName = InetAddress.getLocalHost().getHostName(); KeyPair sKP = KeyStoreTestUtil.generateKeyPair("RSA"); X509Certificate sCert = - KeyStoreTestUtil.generateCertificate("CN="+localhostName+", O=server", sKP, 30, "SHA1withRSA"); + generateCertificate("CN="+localhostName+", O=server", sKP, 30, "SHA1withRSA"); KeyStoreTestUtil.createKeyStore(serverKS, serverPassword, "server", sKP.getPrivate(), sCert); certs.put("server", sCert); @@ -274,4 +284,36 @@ public static void setupSSLConfig(String keystoresDir, String sslConfDir, Config config.set(SSLFactory.SSL_SERVER_CONF_KEY, sslServerConfFile.getName()); config.setBoolean(SSLFactory.SSL_REQUIRE_CLIENT_CERT_KEY, useClientCert); } + + /** + * This is a copied version of hadoop's KeyStoreTestUtil.generateCertificate, which takes care of setting + * IP address as a SSL Subject Alternative Name (SAN). Without this, SSL shuffle failed with async http client. + * Introduced by TEZ-4342. + */ + public static X509Certificate generateCertificate(String dn, KeyPair pair, int days, String algorithm) + throws Exception { + + Date from = new Date(); + Date to = new Date(from.getTime() + days * 86400000L); + BigInteger sn = new BigInteger(64, new SecureRandom()); + KeyPair keyPair = pair; + X509V3CertificateGenerator certGen = new X509V3CertificateGenerator(); + + String hostAddress = InetAddress.getLocalHost().getHostAddress(); + certGen.addExtension(X509Extensions.SubjectAlternativeName, false, + new GeneralNames(new GeneralName(GeneralName.iPAddress, hostAddress))); + + X500Principal dnName = new X500Principal(dn); + + certGen.setSerialNumber(sn); + certGen.setIssuerDN(dnName); + certGen.setNotBefore(from); + certGen.setNotAfter(to); + certGen.setSubjectDN(dnName); + certGen.setPublicKey(keyPair.getPublic()); + certGen.setSignatureAlgorithm(algorithm); + + X509Certificate cert = certGen.generate(pair.getPrivate()); + return cert; + } } From 6863a2d9923ed1633d44c708631e454303503d46 Mon Sep 17 00:00:00 2001 From: Bodor Laszlo Date: Tue, 2 Nov 2021 18:26:47 +0100 Subject: [PATCH 071/137] TEZ-4336: ShuffleScheduler should try to report the original exception (when shuffle becomes unhealthy) (#155) (Laszlo Bodor reviewed by Rajesh Balamohan) --- .../common/shuffle/InputAttemptFetchFailure.java | 10 ++++++++++ .../orderedgrouped/FetcherOrderedGrouped.java | 4 ++-- .../shuffle/orderedgrouped/ShuffleScheduler.java | 12 +++++++----- 3 files changed, 19 insertions(+), 7 deletions(-) diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/InputAttemptFetchFailure.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/InputAttemptFetchFailure.java index d94db35c2f..4ce1699cf5 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/InputAttemptFetchFailure.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/InputAttemptFetchFailure.java @@ -33,6 +33,7 @@ public class InputAttemptFetchFailure { private final InputAttemptIdentifier inputAttemptIdentifier; private final boolean isLocalFetch; private final boolean isDiskErrorAtSource; + private Throwable cause = null; public InputAttemptFetchFailure(InputAttemptIdentifier inputAttemptIdentifier) { this(inputAttemptIdentifier, false, false); @@ -112,4 +113,13 @@ public String toString() { return String.format("%s, isLocalFetch: %s, isDiskErrorAtSource: %s", inputAttemptIdentifier.toString(), isLocalFetch, isDiskErrorAtSource); } + + public InputAttemptFetchFailure withCause(Throwable throwable) { + this.cause = throwable; + return this; + } + + public Throwable getCause() { + return cause; + } } 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 c9bd092f05..a4328af44a 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 @@ -378,7 +378,7 @@ boolean setupConnection(MapHost host, Collection attempt for (InputAttemptIdentifier left : remaining.values()) { // Need to be handling temporary glitches .. // Report read error to the AM to trigger source failure heuristics - scheduler.copyFailed(InputAttemptFetchFailure.fromAttempt(left), host, connectSucceeded, + scheduler.copyFailed(InputAttemptFetchFailure.fromAttempt(left).withCause(ie), host, connectSucceeded, !connectSucceeded); } return false; @@ -738,7 +738,7 @@ protected void setupLocalDiskFetch(MapHost host) throws InterruptedException { if (!stopped) { hasFailures = true; ioErrs.increment(1); - scheduler.copyFailed(InputAttemptFetchFailure.fromLocalFetchFailure(srcAttemptId), + scheduler.copyFailed(InputAttemptFetchFailure.fromLocalFetchFailure(srcAttemptId).withCause(e), host, true, false); LOG.warn("Failed to read local disk output of " + srcAttemptId + " from " + host.getHostIdentifier(), e); 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 540d44f409..dd27d45aed 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 @@ -178,6 +178,7 @@ enum ShuffleErrors { private final Referee referee; @VisibleForTesting final Map failureCounts = new HashMap(); + final Set uniqueHosts = Sets.newHashSet(); private final Map hostFailures = new HashMap(); private final InputContext inputContext; @@ -792,7 +793,7 @@ public synchronized void copyFailed(InputAttemptFetchFailure fetchFailure, MapHo } //Restart consumer in case shuffle is not healthy - if (!isShuffleHealthy(fetchFailure.getInputAttemptIdentifier())) { + if (!isShuffleHealthy(fetchFailure)) { return; } @@ -1006,8 +1007,8 @@ private boolean isFetcherHealthy(String logContext) { return fetcherHealthy; } - boolean isShuffleHealthy(InputAttemptIdentifier srcAttempt) { - + boolean isShuffleHealthy(InputAttemptFetchFailure fetchFailure) { + InputAttemptIdentifier srcAttempt = fetchFailure.getInputAttemptIdentifier(); if (isAbortLimitExceeedFor(srcAttempt)) { return false; } @@ -1049,14 +1050,15 @@ boolean isShuffleHealthy(InputAttemptIdentifier srcAttempt) { + ", pendingInputs=" + (numInputs - doneMaps) + ", fetcherHealthy=" + fetcherHealthy + ", reducerProgressedEnough=" + reducerProgressedEnough - + ", reducerStalled=" + reducerStalled) + + ", reducerStalled=" + reducerStalled + + ", hostFailures=" + hostFailures) + "]"; LOG.error(errorMsg); if (LOG.isDebugEnabled()) { LOG.debug("Host failures=" + hostFailures.keySet()); } // Shuffle knows how to deal with failures post shutdown via the onFailure hook - exceptionReporter.reportException(new IOException(errorMsg)); + exceptionReporter.reportException(new IOException(errorMsg, fetchFailure.getCause())); return false; } return true; From f39a51e588fe7f731356cf3aa68755a5d98ff12a Mon Sep 17 00:00:00 2001 From: Bodor Laszlo Date: Tue, 2 Nov 2021 18:32:45 +0100 Subject: [PATCH 072/137] TEZ-4339: Expose real-time memory consumption of AM and task containers via DagClient (#157) (Laszlo Bodor reviewed by Rajesh Balamohan) --- .../apache/tez/dag/api/DagTypeConverters.java | 4 ++++ .../apache/tez/dag/api/client/DAGStatus.java | 20 +++++++++++++------ .../tez/dag/api/client/StatusGetOpts.java | 3 ++- tez-api/src/main/proto/DAGApiRecords.proto | 3 +++ .../tez/dag/api/client/DAGStatusBuilder.java | 7 +++++++ .../tez/dag/app/TaskCommunicatorManager.java | 8 ++++++++ .../app/TaskCommunicatorManagerInterface.java | 1 + .../tez/dag/app/TezTaskCommunicatorImpl.java | 6 ++++++ .../apache/tez/dag/app/dag/impl/DAGImpl.java | 8 ++++++++ .../serviceplugins/api/TaskCommunicator.java | 9 +++++++++ .../apache/tez/dag/app/MockDAGAppMaster.java | 4 ++-- .../runtime/api/impl/TezHeartbeatRequest.java | 11 +++++++++- .../apache/tez/runtime/task/TaskReporter.java | 18 ++++++++++++++++- .../tez/mapreduce/TestMRRJobsDAGApi.java | 8 +++++++- 14 files changed, 98 insertions(+), 12 deletions(-) diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/DagTypeConverters.java b/tez-api/src/main/java/org/apache/tez/dag/api/DagTypeConverters.java index 5a2cb64cc3..c563f1fb47 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/DagTypeConverters.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/DagTypeConverters.java @@ -628,6 +628,8 @@ public static DAGProtos.StatusGetOptsProto convertStatusGetOptsToProto( switch (statusGetOpts) { case GET_COUNTERS: return DAGProtos.StatusGetOptsProto.GET_COUNTERS; + case GET_MEMORY_USAGE: + return DAGProtos.StatusGetOptsProto.GET_MEMORY_USAGE; } throw new TezUncheckedException("Could not convert StatusGetOpts to" + " proto"); } @@ -636,6 +638,8 @@ public static StatusGetOpts convertStatusGetOptsFromProto(DAGProtos.StatusGetOpt switch (proto) { case GET_COUNTERS: return StatusGetOpts.GET_COUNTERS; + case GET_MEMORY_USAGE: + return StatusGetOpts.GET_MEMORY_USAGE; } throw new TezUncheckedException("Could not convert to StatusGetOpts from" + " proto"); } diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGStatus.java b/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGStatus.java index cbf641e00f..1f8db62b80 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGStatus.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGStatus.java @@ -152,6 +152,14 @@ public TezCounters getDAGCounters() { return dagCounters; } + public long getMemoryUsedByAM() { + return proxy.getMemoryUsedByAM(); + } + + public long getMemoryUsedByTasks() { + return proxy.getMemoryUsedByTasks(); + } + @InterfaceAudience.Private DagStatusSource getSource() { return this.source; @@ -201,12 +209,12 @@ public int hashCode() { @Override public String toString() { StringBuilder sb = new StringBuilder(); - sb.append("status=" + getState() - + ", progress=" + getDAGProgress() - + ", diagnostics=" - + StringUtils.join(getDiagnostics(), LINE_SEPARATOR) - + ", counters=" - + (getDAGCounters() == null ? "null" : getDAGCounters().toString())); + sb.append("status=" + getState()); + sb.append(", progress=" + getDAGProgress()); + sb.append(", diagnostics=" + StringUtils.join(getDiagnostics(), LINE_SEPARATOR)); + sb.append(", memoryUsedByAM=").append(proxy.getMemoryUsedByAM()); + sb.append(", memoryUsedByTasks=").append(proxy.getMemoryUsedByTasks()); + sb.append(", counters=" + (getDAGCounters() == null ? "null" : getDAGCounters().toString())); return sb.toString(); } diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/client/StatusGetOpts.java b/tez-api/src/main/java/org/apache/tez/dag/api/client/StatusGetOpts.java index 1a9df7afa1..3518d33c2c 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/client/StatusGetOpts.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/client/StatusGetOpts.java @@ -29,5 +29,6 @@ @Evolving public enum StatusGetOpts { /** Retrieve Counters with Status */ - GET_COUNTERS + GET_COUNTERS, + GET_MEMORY_USAGE } diff --git a/tez-api/src/main/proto/DAGApiRecords.proto b/tez-api/src/main/proto/DAGApiRecords.proto index 4c8c7f63a9..15f681db96 100644 --- a/tez-api/src/main/proto/DAGApiRecords.proto +++ b/tez-api/src/main/proto/DAGApiRecords.proto @@ -275,6 +275,8 @@ message DAGStatusProto { optional ProgressProto DAGProgress = 3; repeated StringProgressPairProto vertexProgress = 4; optional TezCountersProto dagCounters = 5; + optional int64 memoryUsedByAM = 6; + optional int64 memoryUsedByTasks = 7; } message PlanLocalResourcesProto { @@ -299,6 +301,7 @@ message TezCountersProto { enum StatusGetOptsProto { GET_COUNTERS = 0; + GET_MEMORY_USAGE = 1; } message VertexLocationHintProto { diff --git a/tez-dag/src/main/java/org/apache/tez/dag/api/client/DAGStatusBuilder.java b/tez-dag/src/main/java/org/apache/tez/dag/api/client/DAGStatusBuilder.java index 0002d8b604..931c6d05ca 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/api/client/DAGStatusBuilder.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/api/client/DAGStatusBuilder.java @@ -61,6 +61,13 @@ public void addVertexProgress(String name, ProgressBuilder progress) { getBuilder().addVertexProgress(builder.build()); } + //TODO: let this be a map of values in protobuf 3.x + public void setMemoryUsage(long memoryUsedByAM, long memoryUsedByTasks) { + Builder builder = getBuilder(); + builder.setMemoryUsedByAM(memoryUsedByAM); + builder.setMemoryUsedByTasks(memoryUsedByTasks); + } + public DAGStatusProto getProto() { return getBuilder().build(); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorManager.java index 3a99456ed7..ac2f7605ae 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorManager.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorManager.java @@ -678,4 +678,12 @@ public String getCompletedLogsUrl(int taskCommId, TezTaskAttemptID attemptID, No return null; } + @Override + public long getTotalUsedMemory() { + long totalUsedMemory = 0; + for (int i = 0; i < taskCommunicators.length; i++) { + totalUsedMemory += taskCommunicators[i].getTaskCommunicator().getTotalUsedMemory(); + } + return totalUsedMemory; + } } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorManagerInterface.java b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorManagerInterface.java index 254e74c734..150977a94e 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorManagerInterface.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorManagerInterface.java @@ -54,4 +54,5 @@ public interface TaskCommunicatorManagerInterface { String getCompletedLogsUrl(int taskCommId, TezTaskAttemptID attemptID, NodeId containerNodeId); + long getTotalUsedMemory(); } 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 6d69d36014..48aee3103b 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 @@ -104,6 +104,7 @@ public static final class ContainerInfo { Credentials credentials = null; boolean credentialsChanged = false; boolean taskPulled = false; + long usedMemory = 0; void reset() { taskSpec = null; @@ -382,6 +383,7 @@ public TezHeartbeatResponse heartbeat(TezHeartbeatRequest request) throws IOExce response.setLastRequestId(requestId); containerInfo.lastRequestId = requestId; containerInfo.lastResponse = response; + containerInfo.usedMemory = request.getUsedMemory(); return response; } @@ -466,4 +468,8 @@ protected ContainerInfo getContainerInfo(ContainerId containerId) { protected ContainerId getContainerForAttempt(TezTaskAttemptID taskAttemptId) { return attemptToContainerMap.get(taskAttemptId); } + + public long getTotalUsedMemory() { + return registeredContainers.values().stream().mapToLong(c -> c.usedMemory).sum(); + } } \ No newline at end of file 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 026ca29b7a..07715cdfe7 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 @@ -19,6 +19,8 @@ package org.apache.tez.dag.app.dag.impl; import java.io.IOException; +import java.lang.management.ManagementFactory; +import java.lang.management.MemoryMXBean; import java.net.URL; import java.security.PrivilegedExceptionAction; import java.util.ArrayList; @@ -244,6 +246,8 @@ public class DAGImpl implements org.apache.tez.dag.app.dag.DAG, private static final CommitCompletedTransition COMMIT_COMPLETED_TRANSITION = new CommitCompletedTransition(); + private final MemoryMXBean memoryMXBean = ManagementFactory.getMemoryMXBean(); + protected static final StateMachineFactory stateMachineFactory @@ -940,6 +944,10 @@ public DAGStatusBuilder getDAGStatus(Set statusOptions) { if (statusOptions.contains(StatusGetOpts.GET_COUNTERS)) { status.setDAGCounters(getAllCounters()); } + if (statusOptions.contains(StatusGetOpts.GET_MEMORY_USAGE)) { + status.setMemoryUsage(memoryMXBean.getHeapMemoryUsage().getUsed(), + taskCommunicatorManagerInterface.getTotalUsedMemory()); + } return status; } finally { readLock.unlock(); diff --git a/tez-dag/src/main/java/org/apache/tez/serviceplugins/api/TaskCommunicator.java b/tez-dag/src/main/java/org/apache/tez/serviceplugins/api/TaskCommunicator.java index fceddf2522..be6ad68d55 100644 --- a/tez-dag/src/main/java/org/apache/tez/serviceplugins/api/TaskCommunicator.java +++ b/tez-dag/src/main/java/org/apache/tez/serviceplugins/api/TaskCommunicator.java @@ -237,4 +237,13 @@ public String getCompletedLogsUrl(TezTaskAttemptID attemptID, NodeId containerNo return null; } + /** + * Return the amount of memory used by the containers. Each container is supposed to refresh + * its current state via heartbeat requests, and the TaskCommunicator implementation is supposed + * to aggregate this properly. + * @return memory in MB + */ + public long getTotalUsedMemory() { + return 0; + } } 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 9bceaece93..b3ddaa0f86 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 @@ -431,7 +431,7 @@ public Void call() throws Exception { EventProducerConsumerType.SYSTEM, cData.vName, "", cData.taId), MockDAGAppMaster.this.getContext().getClock().getTime())); TezHeartbeatRequest request = new TezHeartbeatRequest(cData.numUpdates, events, - cData.nextPreRoutedFromEventId, cData.cIdStr, cData.taId, cData.nextFromEventId, 50000); + cData.nextPreRoutedFromEventId, cData.cIdStr, cData.taId, cData.nextFromEventId, 50000, 0); doHeartbeat(request, cData); } else if (version != null && cData.taId.getId() <= version.intValue()) { preemptContainer(cData); @@ -443,7 +443,7 @@ public Void call() throws Exception { EventProducerConsumerType.SYSTEM, cData.vName, "", cData.taId), MockDAGAppMaster.this.getContext().getClock().getTime())); TezHeartbeatRequest request = new TezHeartbeatRequest(++cData.numUpdates, events, - cData.nextPreRoutedFromEventId, cData.cIdStr, cData.taId, cData.nextFromEventId, 10000); + cData.nextPreRoutedFromEventId, cData.cIdStr, cData.taId, cData.nextFromEventId, 10000, 0); doHeartbeat(request, cData); cData.clear(); } diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezHeartbeatRequest.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezHeartbeatRequest.java index 7ed89f813b..fd5bc17521 100644 --- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezHeartbeatRequest.java +++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezHeartbeatRequest.java @@ -39,13 +39,14 @@ public class TezHeartbeatRequest implements Writable { private int preRoutedStartIndex; private int maxEvents; private long requestId; + private long usedMemory; public TezHeartbeatRequest() { } public TezHeartbeatRequest(long requestId, List events, int preRoutedStartIndex, String containerIdentifier, - TezTaskAttemptID taskAttemptID, int startIndex, int maxEvents) { + TezTaskAttemptID taskAttemptID, int startIndex, int maxEvents, long usedMemory) { this.containerIdentifier = containerIdentifier; this.requestId = requestId; this.events = Collections.unmodifiableList(events); @@ -53,6 +54,7 @@ public TezHeartbeatRequest(long requestId, List events, this.preRoutedStartIndex = preRoutedStartIndex; this.maxEvents = maxEvents; this.currentTaskAttemptID = taskAttemptID; + this.usedMemory = usedMemory; } public String getContainerIdentifier() { @@ -83,6 +85,10 @@ public TezTaskAttemptID getCurrentTaskAttemptID() { return currentTaskAttemptID; } + public long getUsedMemory() { + return usedMemory; + } + @Override public void write(DataOutput out) throws IOException { if (events != null) { @@ -105,6 +111,7 @@ public void write(DataOutput out) throws IOException { out.writeInt(maxEvents); out.writeLong(requestId); Text.writeString(out, containerIdentifier); + out.writeLong(usedMemory); } @Override @@ -128,6 +135,7 @@ public void readFields(DataInput in) throws IOException { maxEvents = in.readInt(); requestId = in.readLong(); containerIdentifier = Text.readString(in); + usedMemory = in.readLong(); } @Override @@ -140,6 +148,7 @@ public String toString() { + ", maxEventsToGet=" + maxEvents + ", taskAttemptId=" + currentTaskAttemptID + ", eventCount=" + (events != null ? events.size() : 0) + + ", usedMemory=" + usedMemory + " }"; } } 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 978942d4e7..eeb24343be 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 @@ -19,6 +19,8 @@ package org.apache.tez.runtime.task; import java.io.IOException; +import java.lang.management.ManagementFactory; +import java.lang.management.MemoryMXBean; import java.util.ArrayList; import java.util.Collection; import java.util.List; @@ -136,6 +138,7 @@ static class HeartbeatCallable implements Callable { private static final int LOG_COUNTER_START_INTERVAL = 5000; // 5 seconds private static final float LOG_COUNTER_BACKOFF = 1.3f; + private static final int HEAP_MEMORY_USAGE_UPDATE_INTERVAL = 5000; // 5 seconds private final RuntimeTask task; private final EventMetaData updateEventMetadata; @@ -157,6 +160,10 @@ static class HeartbeatCallable implements Callable { private final ReentrantLock lock = new ReentrantLock(); private final Condition condition = lock.newCondition(); + private final MemoryMXBean memoryMXBean = ManagementFactory.getMemoryMXBean(); + private long usedMemory = 0; + private long heapMemoryUsageUpdatedTime = System.currentTimeMillis() - HEAP_MEMORY_USAGE_UPDATE_INTERVAL; + /* * Keeps track of regular timed heartbeats. Is primarily used as a timing mechanism to send / * log counters. @@ -263,7 +270,7 @@ private synchronized ResponseWrapper heartbeat(Collection eventsArg) t int fromPreRoutedEventId = task.getNextPreRoutedEventId(); int maxEvents = Math.min(maxEventsToGet, task.getMaxEventsToHandle()); TezHeartbeatRequest request = new TezHeartbeatRequest(requestId, events, fromPreRoutedEventId, - containerIdStr, task.getTaskAttemptID(), fromEventId, maxEvents); + containerIdStr, task.getTaskAttemptID(), fromEventId, maxEvents, getUsedMemory()); LOG.debug("Sending heartbeat to AM, request={}", request); maybeLogCounters(); @@ -305,6 +312,15 @@ private synchronized ResponseWrapper heartbeat(Collection eventsArg) t return new ResponseWrapper(false, numEventsReceived); } + private long getUsedMemory() { + long now = System.currentTimeMillis(); + if (now - heapMemoryUsageUpdatedTime > HEAP_MEMORY_USAGE_UPDATE_INTERVAL) { + usedMemory = memoryMXBean.getHeapMemoryUsage().getUsed(); + heapMemoryUsageUpdatedTime = now; + } + return usedMemory; + } + public void markComplete() { // Notify to clear pending events, if any. lock.lock(); diff --git a/tez-tests/src/test/java/org/apache/tez/mapreduce/TestMRRJobsDAGApi.java b/tez-tests/src/test/java/org/apache/tez/mapreduce/TestMRRJobsDAGApi.java index 96b7bbf655..95d5bcf30b 100644 --- a/tez-tests/src/test/java/org/apache/tez/mapreduce/TestMRRJobsDAGApi.java +++ b/tez-tests/src/test/java/org/apache/tez/mapreduce/TestMRRJobsDAGApi.java @@ -20,6 +20,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; import java.io.File; import java.io.IOException; @@ -211,13 +212,18 @@ public void testSleepJob() throws TezException, IOException, InterruptedExceptio + dagStatus.getState()); Thread.sleep(500l); dagStatus = dagClient.getDAGStatus(null); + assertTrue("Memory used by AM is supposed to be 0 if not requested", dagStatus.getMemoryUsedByAM() == 0); + assertTrue("Memory used by tasks is supposed to be 0 if not requested", dagStatus.getMemoryUsedByTasks() == 0); } - dagStatus = dagClient.getDAGStatus(Sets.newHashSet(StatusGetOpts.GET_COUNTERS)); + dagStatus = dagClient.getDAGStatus(Sets.newHashSet(StatusGetOpts.GET_COUNTERS, StatusGetOpts.GET_MEMORY_USAGE)); assertEquals(DAGStatus.State.SUCCEEDED, dagStatus.getState()); assertNotNull(dagStatus.getDAGCounters()); assertNotNull(dagStatus.getDAGCounters().getGroup(FileSystemCounter.class.getName())); assertNotNull(dagStatus.getDAGCounters().findCounter(TaskCounter.GC_TIME_MILLIS)); + assertTrue("Memory used by AM is supposed to be >0", dagStatus.getMemoryUsedByAM() > 0); + assertTrue("Memory used by tasks is supposed to be >0", dagStatus.getMemoryUsedByTasks() > 0); + ExampleDriver.printDAGStatus(dagClient, new String[] { "SleepVertex" }, true, true); tezSession.stop(); } From 211b59bf4a9692491de0c8306c94407d6a6536f4 Mon Sep 17 00:00:00 2001 From: Bodor Laszlo Date: Fri, 19 Nov 2021 11:09:47 +0100 Subject: [PATCH 073/137] TEZ-4338: Tez should consider node information to realize OUTPUT_LOST as early as possible - upstream(mapper) problems (#152) (Laszlo Bodor reviewed by Rajesh Balamohan) --- .../apache/tez/dag/api/TezConfiguration.java | 18 +++ .../api/events/InputReadErrorEvent.java | 29 +++-- .../api/ContainerLauncherContext.java | 2 +- tez-api/src/main/proto/Events.proto | 1 + .../org/apache/tez/dag/app/dag/Vertex.java | 5 + .../event/TaskAttemptEventOutputFailed.java | 4 +- .../org/apache/tez/dag/app/dag/impl/Edge.java | 15 ++- .../tez/dag/app/dag/impl/TaskAttemptImpl.java | 101 +++++++++++----- .../tez/dag/app/dag/impl/VertexImpl.java | 27 +++++ .../tez/dag/app/rm/node/AMNodeImpl.java | 6 + .../tez/dag/app/rm/node/AMNodeTracker.java | 13 +- .../dag/app/rm/node/PerSourceNodeTracker.java | 4 + .../tez/dag/app/dag/impl/TestTaskAttempt.java | 77 +++++++++++- .../apache/tez/runtime/api/impl/TezEvent.java | 4 +- .../library/api/TezRuntimeConfiguration.java | 27 ++++- .../library/common/shuffle/Fetcher.java | 33 ++++-- .../shuffle/FetcherErrorTestingConfig.java | 111 ++++++++++++++++++ .../shuffle/FetcherWithInjectableErrors.java | 87 ++++++++++++++ .../common/shuffle/impl/ShuffleManager.java | 14 ++- .../orderedgrouped/FetcherOrderedGrouped.java | 13 +- ...herOrderedGroupedWithInjectableErrors.java | 68 +++++++++++ .../orderedgrouped/ShuffleScheduler.java | 30 +++-- 22 files changed, 608 insertions(+), 81 deletions(-) create mode 100644 tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/FetcherErrorTestingConfig.java create mode 100644 tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/FetcherWithInjectableErrors.java create mode 100644 tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/FetcherOrderedGroupedWithInjectableErrors.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 17a826e7e7..5063b04eac 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 @@ -300,6 +300,24 @@ public TezConfiguration(boolean loadDefaults) { TEZ_AM_PREFIX + "max.allowed.time-sec.for-read-error"; public static final int TEZ_AM_MAX_ALLOWED_TIME_FOR_TASK_READ_ERROR_SEC_DEFAULT = 300; + /** + * Double value. Assuming that a certain number of downstream hosts reported fetch failure for a + * given upstream host, this config drives the max allowed ratio of (downstream hosts) / (all hosts). + * The total number of used hosts are tracked by AMNodeTracker, which divides the distinct number of + * downstream hosts blaming source(upstream) tasks in a given vertex. If the fraction is beyond this + * limit, the upstream task attempt is marked as failed (so blamed for the fetch failure). + * E.g. if this set to 0.2, in case of 3 different hosts reporting fetch failure + * for the same upstream host in a cluster which currently utilizes 10 nodes, the upstream task + * is immediately blamed for the fetch failure. + * + * Expert level setting. + */ + @ConfigurationScope(Scope.AM) + @ConfigurationProperty(type="integer") + public static final String TEZ_AM_MAX_ALLOWED_DOWNSTREAM_HOST_FAILURES_FRACTION = + TEZ_AM_PREFIX + "max.allowed.downstream.host.failures.fraction"; + public static final double TEZ_AM_MAX_ALLOWED_DOWNSTREAM_HOST_FAILURES_FRACTION_DEFAULT = 0.2; + /** * Boolean value. Determines when the final outputs to data sinks are committed. Commit is an * output specific operation and typically involves making the output visible for consumption. diff --git a/tez-api/src/main/java/org/apache/tez/runtime/api/events/InputReadErrorEvent.java b/tez-api/src/main/java/org/apache/tez/runtime/api/events/InputReadErrorEvent.java index 8ef50ebac4..1d0c44d6c1 100644 --- a/tez-api/src/main/java/org/apache/tez/runtime/api/events/InputReadErrorEvent.java +++ b/tez-api/src/main/java/org/apache/tez/runtime/api/events/InputReadErrorEvent.java @@ -62,8 +62,13 @@ public final class InputReadErrorEvent extends Event { */ private final boolean isDiskErrorAtSource; + /** + * The localhostName of the destination task attempt. + */ + private final String destinationLocalhostName; + private InputReadErrorEvent(final String diagnostics, final int index, final int version, - final int numFailures, boolean isLocalFetch, boolean isDiskErrorAtSource) { + final int numFailures, boolean isLocalFetch, boolean isDiskErrorAtSource, String destinationLocalhostName) { super(); this.diagnostics = diagnostics; this.index = index; @@ -71,24 +76,30 @@ private InputReadErrorEvent(final String diagnostics, final int index, final int this.numFailures = numFailures; this.isLocalFetch = isLocalFetch; this.isDiskErrorAtSource = isDiskErrorAtSource; + this.destinationLocalhostName = destinationLocalhostName; } public static InputReadErrorEvent create(String diagnostics, int index, int version, boolean isLocalFetch, boolean isDiskErrorAtSource) { - return create(diagnostics, index, version, 1, isLocalFetch, isDiskErrorAtSource); + return create(diagnostics, index, version, 1, isLocalFetch, isDiskErrorAtSource, null); } public static InputReadErrorEvent create(String diagnostics, int index, int version) { - return create(diagnostics, index, version, 1, false, false); + return create(diagnostics, index, version, 1, false, false, null); + } + + public static InputReadErrorEvent create(String diagnostics, int index, int version, boolean isLocalFetch, + boolean isDiskErrorAtSource, String destinationLocalhostName) { + return create(diagnostics, index, version, 1, isLocalFetch, isDiskErrorAtSource, destinationLocalhostName); } /** * Create an InputReadErrorEvent. */ - public static InputReadErrorEvent create(final String diagnostics, final int index, - final int version, final int numFailures, boolean isLocalFetch, boolean isDiskErrorAtSource) { - return new InputReadErrorEvent(diagnostics, index, version, numFailures, isLocalFetch, - isDiskErrorAtSource); + public static InputReadErrorEvent create(final String diagnostics, final int index, final int version, + final int numFailures, boolean isLocalFetch, boolean isDiskErrorAtSource, String destinationLocalhostName) { + return new InputReadErrorEvent(diagnostics, index, version, numFailures, isLocalFetch, isDiskErrorAtSource, + destinationLocalhostName); } public String getDiagnostics() { @@ -118,6 +129,10 @@ public boolean isDiskErrorAtSource() { return isDiskErrorAtSource; } + public String getDestinationLocalhostName(){ + return destinationLocalhostName; + } + @Override public int hashCode() { return Objects.hash(index, version); diff --git a/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLauncherContext.java b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLauncherContext.java index ed1d58f78a..16d54f05b7 100644 --- a/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLauncherContext.java +++ b/tez-api/src/main/java/org/apache/tez/serviceplugins/api/ContainerLauncherContext.java @@ -80,7 +80,7 @@ void containerCompleted(ContainerId containerId, int exitStatus, String diagnost * Get the number of nodes being handled by the specified source * * @param sourceName the relevant source name - * @return the initial payload + * @return the number of nodes */ int getNumNodes(String sourceName); diff --git a/tez-api/src/main/proto/Events.proto b/tez-api/src/main/proto/Events.proto index e041c33f60..9949b0bc8c 100644 --- a/tez-api/src/main/proto/Events.proto +++ b/tez-api/src/main/proto/Events.proto @@ -41,6 +41,7 @@ message InputReadErrorEventProto { optional int32 version = 3; optional bool is_local_fetch = 4; optional bool is_disk_error_at_source = 5; + optional string destination_localhost_name = 6; } message InputFailedEventProto { 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 ba3079d4e1..ff83e19a85 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 @@ -230,8 +230,13 @@ interface VertexConfig { * @return tez.am.max.allowed.time-sec.for-read-error. */ int getMaxAllowedTimeForTaskReadErrorSec(); + /** + * @return tez.am.max.allowed.downstream.host.failures.fraction. + */ + double getMaxAllowedDownstreamHostFailuresFraction(); } void incrementRejectedTaskAttemptCount(); int getRejectedTaskAttemptCount(); + Map> getDownstreamBlamingHosts(); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventOutputFailed.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventOutputFailed.java index 6bc110a1a0..fbdd2305c7 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventOutputFailed.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventOutputFailed.java @@ -28,9 +28,9 @@ public class TaskAttemptEventOutputFailed extends TaskAttemptEvent private TezEvent inputFailedEvent; private int consumerTaskNumber; - public TaskAttemptEventOutputFailed(TezTaskAttemptID attemptId, + public TaskAttemptEventOutputFailed(TezTaskAttemptID sourceTaskAttemptId, TezEvent tezEvent, int numConsumers) { - super(attemptId, TaskAttemptEventType.TA_OUTPUT_FAILED); + super(sourceTaskAttemptId, TaskAttemptEventType.TA_OUTPUT_FAILED); this.inputFailedEvent = tezEvent; this.consumerTaskNumber = numConsumers; } 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 848b49199d..99b56fbf07 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 @@ -374,7 +374,7 @@ public void sendTezEventToSourceTasks(TezEvent tezEvent) throws AMUserCodeExcept if (!bufferEvents.get()) { switch (tezEvent.getEventType()) { case INPUT_READ_ERROR_EVENT: - InputReadErrorEvent event = (InputReadErrorEvent) tezEvent.getEvent(); + InputReadErrorEvent inputReadErrorEvent = (InputReadErrorEvent) tezEvent.getEvent(); TezTaskAttemptID destAttemptId = tezEvent.getSourceInfo() .getTaskAttemptID(); int destTaskIndex = destAttemptId.getTaskID().getId(); @@ -383,10 +383,10 @@ public void sendTezEventToSourceTasks(TezEvent tezEvent) throws AMUserCodeExcept try { if (onDemandRouting) { srcTaskIndex = ((EdgeManagerPluginOnDemand) edgeManager).routeInputErrorEventToSource( - destTaskIndex, event.getIndex()); + destTaskIndex, inputReadErrorEvent.getIndex()); } else { - srcTaskIndex = edgeManager.routeInputErrorEventToSource(event, - destTaskIndex, event.getIndex()); + srcTaskIndex = edgeManager.routeInputErrorEventToSource(inputReadErrorEvent, + destTaskIndex, inputReadErrorEvent.getIndex()); } Preconditions.checkArgument(srcTaskIndex >= 0, "SourceTaskIndex should not be negative," @@ -414,11 +414,10 @@ public void sendTezEventToSourceTasks(TezEvent tezEvent) throws AMUserCodeExcept " edgeManager=" + edgeManager.getClass().getName()); } TezTaskID srcTaskId = srcTask.getTaskId(); - int taskAttemptIndex = event.getVersion(); + int srcTaskAttemptIndex = inputReadErrorEvent.getVersion(); TezTaskAttemptID srcTaskAttemptId = TezTaskAttemptID.getInstance(srcTaskId, - taskAttemptIndex); - sendEvent(new TaskAttemptEventOutputFailed(srcTaskAttemptId, - tezEvent, numConsumers)); + srcTaskAttemptIndex); + sendEvent(new TaskAttemptEventOutputFailed(srcTaskAttemptId, tezEvent, numConsumers)); break; default: throw new TezUncheckedException("Unhandled tez event type: " 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 593ea6cabf..e6b9e82000 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 @@ -194,7 +194,7 @@ public static DataEventDependencyInfo fromProto(DataEventDependencyInfoProto pro private Container container; private long allocationTime; private ContainerId containerId; - private NodeId containerNodeId; + protected NodeId containerNodeId; private String nodeHttpAddress; private String nodeRackName; @@ -1793,62 +1793,95 @@ protected static class OutputReportedFailedTransition implements MultipleArcTransition { @Override - public TaskAttemptStateInternal transition(TaskAttemptImpl attempt, + public TaskAttemptStateInternal transition(TaskAttemptImpl sourceAttempt, TaskAttemptEvent event) { TaskAttemptEventOutputFailed outputFailedEvent = (TaskAttemptEventOutputFailed) event; - TezEvent tezEvent = outputFailedEvent.getInputFailedEvent(); - TezTaskAttemptID failedDestTaId = tezEvent.getSourceInfo().getTaskAttemptID(); - InputReadErrorEvent readErrorEvent = (InputReadErrorEvent)tezEvent.getEvent(); + TezEvent inputFailedEvent = outputFailedEvent.getInputFailedEvent(); + TezTaskAttemptID failedDestTaId = inputFailedEvent.getSourceInfo().getTaskAttemptID(); + + InputReadErrorEvent readErrorEvent = (InputReadErrorEvent)inputFailedEvent.getEvent(); int failedInputIndexOnDestTa = readErrorEvent.getIndex(); - if (readErrorEvent.getVersion() != attempt.getID().getId()) { - throw new TezUncheckedException(attempt.getID() + + if (readErrorEvent.getVersion() != sourceAttempt.getID().getId()) { + throw new TezUncheckedException(sourceAttempt.getID() + " incorrectly blamed for read error from " + failedDestTaId + " at inputIndex " + failedInputIndexOnDestTa + " version" + readErrorEvent.getVersion()); } - LOG.info(attempt.getID() - + " blamed for read error from " + failedDestTaId - + " at inputIndex " + failedInputIndexOnDestTa); - long time = attempt.clock.getTime(); - Long firstErrReportTime = attempt.uniquefailedOutputReports.get(failedDestTaId); + // source host: where the data input is supposed to come from + String sHost = sourceAttempt.getNodeId().getHost(); + // destination: where the data is tried to be fetched to + String dHost = readErrorEvent.getDestinationLocalhostName(); + + LOG.info("{} (on {}) blamed for read error from {} (on {}) at inputIndex {}", sourceAttempt.getID(), + sHost, failedDestTaId, dHost, failedInputIndexOnDestTa); + + boolean tooManyDownstreamHostsBlamedTheSameUpstreamHost = false; + Map> downstreamBlamingHosts = sourceAttempt.getVertex().getDownstreamBlamingHosts(); + if (!downstreamBlamingHosts.containsKey(sHost)) { + LOG.info("Host {} is blamed for fetch failure from {} for the first time", sHost, dHost); + downstreamBlamingHosts.put(sHost, new HashSet()); + } + + downstreamBlamingHosts.get(sHost).add(dHost); + int currentNumberOfFailingDownstreamHosts = downstreamBlamingHosts.get(sHost).size(); + int numNodes = getNumNodes(sourceAttempt); + float hostFailureFraction = numNodes > 0 ? ((float) currentNumberOfFailingDownstreamHosts) / numNodes : 0; + double maxAllowedHostFailureFraction = sourceAttempt.getVertex().getVertexConfig() + .getMaxAllowedDownstreamHostFailuresFraction(); + + if (hostFailureFraction > maxAllowedHostFailureFraction) { + LOG.info("Host will be marked fail: {} because of host failure fraction {} is beyond the limit {}", sHost, + hostFailureFraction, maxAllowedHostFailureFraction); + tooManyDownstreamHostsBlamedTheSameUpstreamHost = true; + } + long time = sourceAttempt.clock.getTime(); + + Long firstErrReportTime = sourceAttempt.uniquefailedOutputReports.get(failedDestTaId); if (firstErrReportTime == null) { - attempt.uniquefailedOutputReports.put(failedDestTaId, time); + sourceAttempt.uniquefailedOutputReports.put(failedDestTaId, time); firstErrReportTime = time; } - int maxAllowedOutputFailures = attempt.getVertex().getVertexConfig() + int maxAllowedOutputFailures = sourceAttempt.getVertex().getVertexConfig() .getMaxAllowedOutputFailures(); - int maxAllowedTimeForTaskReadErrorSec = attempt.getVertex() + int maxAllowedTimeForTaskReadErrorSec = sourceAttempt.getVertex() .getVertexConfig().getMaxAllowedTimeForTaskReadErrorSec(); - double maxAllowedOutputFailuresFraction = attempt.getVertex() + double maxAllowedOutputFailuresFraction = sourceAttempt.getVertex() .getVertexConfig().getMaxAllowedOutputFailuresFraction(); int readErrorTimespanSec = (int)((time - firstErrReportTime)/1000); boolean crossTimeDeadline = readErrorTimespanSec >= maxAllowedTimeForTaskReadErrorSec; - int runningTasks = attempt.appContext.getCurrentDAG().getVertex( + int runningTasks = sourceAttempt.appContext.getCurrentDAG().getVertex( failedDestTaId.getTaskID().getVertexID()).getRunningTasks(); - float failureFraction = runningTasks > 0 ? ((float) attempt.uniquefailedOutputReports.size()) / runningTasks : 0; + float failureFraction = + runningTasks > 0 ? ((float) sourceAttempt.uniquefailedOutputReports.size()) / runningTasks : 0; boolean withinFailureFractionLimits = (failureFraction <= maxAllowedOutputFailuresFraction); boolean withinOutputFailureLimits = - (attempt.uniquefailedOutputReports.size() < maxAllowedOutputFailures); + (sourceAttempt.uniquefailedOutputReports.size() < maxAllowedOutputFailures); // If needed we can launch a background task without failing this task // to generate a copy of the output just in case. // If needed we can consider only running consumer tasks if (!crossTimeDeadline && withinFailureFractionLimits && withinOutputFailureLimits - && !(readErrorEvent.isLocalFetch() || readErrorEvent.isDiskErrorAtSource())) { - return attempt.getInternalState(); + && !(readErrorEvent.isLocalFetch() || readErrorEvent.isDiskErrorAtSource()) + && !tooManyDownstreamHostsBlamedTheSameUpstreamHost) { + return sourceAttempt.getInternalState(); } - String message = attempt.getID() + " being failed for too many output errors. " + String message = sourceAttempt.getID() + " being failed for too many output errors. " + "failureFraction=" + failureFraction + ", MAX_ALLOWED_OUTPUT_FAILURES_FRACTION=" + maxAllowedOutputFailuresFraction - + ", uniquefailedOutputReports=" + attempt.uniquefailedOutputReports.size() + + ", uniquefailedOutputReports=" + sourceAttempt.uniquefailedOutputReports.size() + ", MAX_ALLOWED_OUTPUT_FAILURES=" + maxAllowedOutputFailures + + ", hostFailureFraction=" + hostFailureFraction + + " (" + currentNumberOfFailingDownstreamHosts + " / " + numNodes + ")" + + ", MAX_ALLOWED_DOWNSTREAM_HOST_FAILURES_FRACTION=" + + maxAllowedHostFailureFraction + ", MAX_ALLOWED_TIME_FOR_TASK_READ_ERROR_SEC=" + maxAllowedTimeForTaskReadErrorSec + ", readErrorTimespan=" + readErrorTimespanSec @@ -1856,22 +1889,34 @@ public TaskAttemptStateInternal transition(TaskAttemptImpl attempt, + ", isDiskErrorAtSource=" + readErrorEvent.isDiskErrorAtSource(); LOG.info(message); - attempt.addDiagnosticInfo(message); + sourceAttempt.addDiagnosticInfo(message); // send input failed event - attempt.sendInputFailedToConsumers(); + sourceAttempt.sendInputFailedToConsumers(); // Not checking for leafVertex since a READ_ERROR should only be reported for intermediate tasks. - if (attempt.getInternalState() == TaskAttemptStateInternal.SUCCEEDED) { + if (sourceAttempt.getInternalState() == TaskAttemptStateInternal.SUCCEEDED) { (new TerminatedAfterSuccessHelper(FAILED_HELPER)).transition( - attempt, event); + sourceAttempt, event); return TaskAttemptStateInternal.FAILED; } else { (new TerminatedWhileRunningTransition(FAILED_HELPER)).transition( - attempt, event); + sourceAttempt, event); return TaskAttemptStateInternal.FAIL_IN_PROGRESS; } // TODO at some point. Nodes may be interested in FetchFailure info. // Can be used to blacklist nodes. } + + private int getNumNodes(TaskAttemptImpl sourceAttempt) { + Vertex vertex = sourceAttempt.getVertex(); + String taskSchedulerName = vertex.getServicePluginInfo().getTaskSchedulerName(); + int sourceIndex = vertex.getAppContext().getTaskScheduerIdentifier(taskSchedulerName); + int numActiveNodes = vertex.getAppContext().getNodeTracker().getNumActiveNodes(sourceIndex); + if (LOG.isDebugEnabled()) { + int numAllNodes = vertex.getAppContext().getNodeTracker().getNumNodes(sourceIndex); + LOG.debug("Getting nodes, active/all: {}/{}", numActiveNodes, numAllNodes); + } + return numActiveNodes; + } } @VisibleForTesting 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 912339e15e..f22f6de250 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 @@ -264,6 +264,13 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex, EventHandl final ServicePluginInfo servicePluginInfo; + /* + * For every upstream host (as map keys) contains every unique downstream hostnames that reported INPUT_READ_ERROR. + * This map helps to decide if there is a problem with the host that produced the map outputs. There is an assumption + * that if multiple downstream hosts report input errors for the same upstream host, then it's likely that the output + * has to be blamed and needs to rerun. + */ + private final Map> downstreamBlamingHosts = Maps.newHashMap(); private final float maxFailuresPercent; private boolean logSuccessDiagnostics = false; @@ -4833,6 +4840,10 @@ static class VertexConfigImpl implements VertexConfig { * See tez.am.max.allowed.time-sec.for-read-error. */ private final int maxAllowedTimeForTaskReadErrorSec; + /** + * See tez.am.max.allowed.downstream.host.failures.fraction. + */ + private final double maxAllowedDownstreamHostFailuresFraction; public VertexConfigImpl(Configuration conf) { this.maxFailedTaskAttempts = conf.getInt(TezConfiguration.TEZ_AM_TASK_MAX_FAILED_ATTEMPTS, @@ -4857,6 +4868,10 @@ public VertexConfigImpl(Configuration conf) { this.maxAllowedTimeForTaskReadErrorSec = conf.getInt( TezConfiguration.TEZ_AM_MAX_ALLOWED_TIME_FOR_TASK_READ_ERROR_SEC, TezConfiguration.TEZ_AM_MAX_ALLOWED_TIME_FOR_TASK_READ_ERROR_SEC_DEFAULT); + + this.maxAllowedDownstreamHostFailuresFraction = conf.getDouble( + TezConfiguration.TEZ_AM_MAX_ALLOWED_DOWNSTREAM_HOST_FAILURES_FRACTION, + TezConfiguration.TEZ_AM_MAX_ALLOWED_DOWNSTREAM_HOST_FAILURES_FRACTION_DEFAULT); } @Override @@ -4899,8 +4914,20 @@ public boolean getTaskRescheduleRelaxedLocality() { @Override public int getMaxAllowedTimeForTaskReadErrorSec() { return maxAllowedTimeForTaskReadErrorSec; } + + /** + * @return maxAllowedDownstreamHostsReportingFetchFailure. + */ + @Override public double getMaxAllowedDownstreamHostFailuresFraction() { + return maxAllowedDownstreamHostFailuresFraction; + } } @Override public AbstractService getSpeculator() { return speculator; } + + @Override + public Map> getDownstreamBlamingHosts(){ + return downstreamBlamingHosts; + } } 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 df19534bfc..26796d2af4 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 @@ -509,4 +509,10 @@ public void dagComplete(DAG dag) { this.writeLock.unlock(); } } + + public String toString() { + return String.format( + "{AMNodeImpl: nodeId: %s, state: %s, containers: %d, completed containers: %d, healthy: %s, blackListed: %s}", + nodeId, getState(), getContainers().size(), completedContainers.size(), !isUnhealthy(), isBlacklisted()); + } } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeTracker.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeTracker.java index 1536170fac..8c81cb52c9 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeTracker.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/AMNodeTracker.java @@ -134,6 +134,17 @@ public int getNumNodes(int schedulerId) { return perSourceNodeTrackers.get(schedulerId).getNumNodes(); } + /** + * Retrieve the number of nodes in ACTIVE state. This number is suitable for deciding + * how many nodes can be potentially used for running containers at the moment. + * + * @param schedulerId the schedulerId for which the node count is required + * @return the number of nodes from the scheduler being in ACTIVE state + */ + public int getNumActiveNodes(int schedulerId) { + return perSourceNodeTrackers.get(schedulerId).getNumActiveNodes(); + } + @Private @VisibleForTesting public boolean isBlacklistingIgnored(int schedulerId) { @@ -158,6 +169,4 @@ private PerSourceNodeTracker getAndCreateIfNeededPerSourceTracker(int schedulerI } return nodeTracker; } - - } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/PerSourceNodeTracker.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/PerSourceNodeTracker.java index 74c6176e4c..9906644fe1 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/PerSourceNodeTracker.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/PerSourceNodeTracker.java @@ -84,6 +84,10 @@ public int getNumNodes() { return nodeMap.size(); } + public int getNumActiveNodes() { + return (int) nodeMap.values().stream().filter(node -> node.getState() == AMNodeState.ACTIVE).count(); + } + public void handle(AMNodeEvent rEvent) { // No synchronization required until there's multiple dispatchers. NodeId nodeId = rEvent.getNodeId(); 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 6862bec2ee..7a2a05fb26 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 @@ -108,6 +108,7 @@ import org.apache.tez.dag.app.rm.AMSchedulerEventTALaunchRequest; import org.apache.tez.dag.app.rm.container.AMContainerMap; import org.apache.tez.dag.app.rm.container.ContainerContextMatcher; +import org.apache.tez.dag.app.rm.node.AMNodeTracker; import org.apache.tez.dag.history.DAGHistoryEvent; import org.apache.tez.dag.history.HistoryEventHandler; import org.apache.tez.dag.history.events.TaskAttemptFinishedEvent; @@ -131,6 +132,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import com.google.common.collect.Maps; + @SuppressWarnings({ "unchecked", "rawtypes" }) public class TestTaskAttempt { @@ -174,9 +177,16 @@ public void setupTest() { private void createMockVertex(Configuration conf) { mockVertex = mock(Vertex.class); + when(mockVertex.getDownstreamBlamingHosts()).thenReturn(Maps.newHashMap()); when(mockVertex.getServicePluginInfo()).thenReturn(servicePluginInfo); - when(mockVertex.getVertexConfig()).thenReturn( - new VertexImpl.VertexConfigImpl(conf)); + when(mockVertex.getVertexConfig()).thenReturn(new VertexImpl.VertexConfigImpl(conf)); + AppContext appContext = mock(AppContext.class); + when(appContext.getTaskScheduerIdentifier(Mockito.anyString())).thenReturn(0); + when(mockVertex.getAppContext()).thenReturn(appContext); + AMNodeTracker nodeTracker = mock(AMNodeTracker.class); + when(nodeTracker.getNumNodes(Mockito.anyInt())).thenReturn(10); + when(nodeTracker.getNumActiveNodes(Mockito.anyInt())).thenReturn(8); + when(appContext.getNodeTracker()).thenReturn(nodeTracker); } @Test(timeout = 5000) @@ -2173,11 +2183,11 @@ private void testMapTaskFailingForFetchFailureType(boolean isLocalFetch, TezTaskID.getInstance(TezVertexID.getInstance(TezDAGID.getInstance("1", 1, 1), 1), 1); TaskAttemptImpl sourceAttempt = new MockTaskAttemptImpl(taskID, 1, eventHandler, null, new Configuration(), SystemClock.getInstance(), mock(TaskHeartbeatHandler.class), appCtx, - false, null, null, false); + false, null, null, false).setNodeId(NodeId.newInstance("somehost", 0)); // the original read error event, sent by reducer task InputReadErrorEvent inputReadErrorEvent = - InputReadErrorEvent.create("", 0, 1, 1, isLocalFetch, isDiskErrorAtSource); + InputReadErrorEvent.create("", 0, 1, 1, isLocalFetch, isDiskErrorAtSource, null); TezTaskAttemptID destTaskAttemptId = mock(TezTaskAttemptID.class); when(destTaskAttemptId.getTaskID()).thenReturn(mock(TezTaskID.class)); when(destTaskAttemptId.getTaskID().getVertexID()).thenReturn(mock(TezVertexID.class)); @@ -2201,6 +2211,56 @@ private void testMapTaskFailingForFetchFailureType(boolean isLocalFetch, Assert.assertEquals(expectedState, resultState); } + @Test + public void testMapTaskIsBlamedByDownstreamAttemptsFromDifferentHosts() { + EventHandler eventHandler = mock(EventHandler.class); + TezTaskID taskID = TezTaskID.getInstance(TezVertexID.getInstance(TezDAGID.getInstance("1", 1, 1), 1), 1); + TaskAttemptImpl sourceAttempt = new MockTaskAttemptImpl(taskID, 1, eventHandler, null, new Configuration(), + SystemClock.getInstance(), mock(TaskHeartbeatHandler.class), appCtx, false, null, null, false) + .setNodeId(NodeId.newInstance("somehost", 0)); + + // input read error events from 2 different hosts + InputReadErrorEvent inputReadErrorEvent1 = + InputReadErrorEvent.create("", 0, 1, 1, false, false, "downstream_host_1"); + InputReadErrorEvent inputReadErrorEvent2 = + InputReadErrorEvent.create("", 1, 1, 1, false, false, "downstream_host_2"); + + TezTaskAttemptID destTaskAttemptId = mock(TezTaskAttemptID.class); + when(destTaskAttemptId.getTaskID()).thenReturn(mock(TezTaskID.class)); + when(destTaskAttemptId.getTaskID().getVertexID()).thenReturn(mock(TezVertexID.class)); + when(appCtx.getCurrentDAG()).thenReturn(mock(DAG.class)); + when(appCtx.getCurrentDAG().getVertex(Mockito.any(TezVertexID.class))).thenReturn(mock(Vertex.class)); + when(appCtx.getCurrentDAG().getVertex(Mockito.any(TezVertexID.class)).getRunningTasks()).thenReturn(100); + + EventMetaData mockMeta = mock(EventMetaData.class); + when(mockMeta.getTaskAttemptID()).thenReturn(destTaskAttemptId); + + // mapper task succeeded earlier + sourceAttempt.handle(new TaskAttemptEvent(sourceAttempt.getID(), TaskAttemptEventType.TA_DONE)); + Assert.assertEquals(TaskAttemptStateInternal.SUCCEEDED, sourceAttempt.getInternalState()); + + // the event is propagated to map task's event handler + TezEvent tezEvent = new TezEvent(inputReadErrorEvent1, mockMeta); + TaskAttemptEventOutputFailed outputFailedEvent = + new TaskAttemptEventOutputFailed(sourceAttempt.getID(), tezEvent, 11); + TaskAttemptStateInternal resultState = + new TaskAttemptImpl.OutputReportedFailedTransition().transition(sourceAttempt, outputFailedEvent); + // SUCCEEDED, as we haven't reached the host limit fraction + // active nodes: 8, failed hosts: 1, fraction 0.125 (< 0.2) + Assert.assertEquals(TaskAttemptStateInternal.SUCCEEDED, resultState); + + // the second event is propagated to map task's event handler + TezEvent tezEvent2 = new TezEvent(inputReadErrorEvent2, mockMeta); + TaskAttemptEventOutputFailed outputFailedEvent2 = + new TaskAttemptEventOutputFailed(sourceAttempt.getID(), tezEvent2, 11); + TaskAttemptStateInternal resultState2 = + new TaskAttemptImpl.OutputReportedFailedTransition().transition(sourceAttempt, outputFailedEvent2); + + // now it's marked as FAILED + // active nodes: 8, failed hosts: 2, fraction 0.25 (> 0.2) + Assert.assertEquals(TaskAttemptStateInternal.FAILED, resultState2); + } + private Event verifyEventType(List events, Class eventClass, int expectedOccurences) { int count = 0; @@ -2247,9 +2307,14 @@ public MockTaskAttemptImpl(TezTaskID taskId, int attemptNumber, isRescheduled, resource, containerContext, leafVertex, mockTask, locationHint, null, null); } - + boolean inputFailedReported = false; - + + public MockTaskAttemptImpl setNodeId(NodeId nodeId) { + this.containerNodeId = nodeId; + return this; + } + @Override protected Vertex getVertex() { return mockVertex; diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezEvent.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezEvent.java index ebea9a4f3f..f96a437a49 100644 --- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezEvent.java +++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezEvent.java @@ -193,6 +193,7 @@ private void serializeEvent(DataOutput out) throws IOException { .setVersion(ideEvt.getVersion()) .setIsLocalFetch(ideEvt.isLocalFetch()) .setIsDiskErrorAtSource(ideEvt.isDiskErrorAtSource()) + .setDestinationLocalhostName(ideEvt.getDestinationLocalhostName()) .build(); break; case TASK_ATTEMPT_FAILED_EVENT: @@ -298,7 +299,8 @@ private void deserializeEvent(DataInput in) throws IOException { case INPUT_READ_ERROR_EVENT: InputReadErrorEventProto ideProto = InputReadErrorEventProto.parseFrom(input); event = InputReadErrorEvent.create(ideProto.getDiagnostics(), ideProto.getIndex(), - ideProto.getVersion(), ideProto.getIsLocalFetch(), ideProto.getIsDiskErrorAtSource()); + ideProto.getVersion(), ideProto.getIsLocalFetch(), ideProto.getIsDiskErrorAtSource(), + ideProto.getDestinationLocalhostName()); break; case TASK_ATTEMPT_FAILED_EVENT: TaskAttemptFailedEventProto tfProto = 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 64e6a71d7e..1c747af2bb 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 @@ -414,6 +414,29 @@ public class TezRuntimeConfiguration { public static final float TEZ_RUNTIME_SHUFFLE_FETCH_BUFFER_PERCENT_DEFAULT = 0.90f; + /** + * Enables fetch failures by a configuration. Should be used for testing only. + */ + @ConfigurationProperty(type = "boolean") + public static final String TEZ_RUNTIME_SHUFFLE_FETCH_ENABLE_TESTING_ERRORS = + TEZ_RUNTIME_PREFIX + "shuffle.fetch.testing.errors.enable"; + public static final boolean TEZ_RUNTIME_SHUFFLE_FETCH_ENABLE_TESTING_ERRORS_DEFAULT = false; + + /** + * Configures the injectable fetch failures, in a form of: + * maphost#mapvertex#probability#comma,separated,features + * Possible values are (fetch fails...): + * "*#*#50": from all map hosts with 50% likelihood + * "_first_#*#80": for the first ever seen map host with 80% likelihood (user doesn't want to use hostnames) + * "host1#*#100": from host1 with 100% likelihood (simulates single node failure) + * "host1#Map_1#100": from host1 for Map 1 source tasks with 100% likelihood + * "host1#Map_1#100#fail_only_first": as above but only for input attempts with index 0 + */ + @ConfigurationProperty(type = "string") + public static final String TEZ_RUNTIME_SHUFFLE_FETCH_TESTING_ERRORS_CONFIG = + TEZ_RUNTIME_PREFIX + "shuffle.fetch.testing.errors.config"; + public static final String TEZ_RUNTIME_SHUFFLE_FETCH_TESTING_ERRORS_CONFIG_DEFAULT = "*#50"; + @ConfigurationProperty(type = "float") public static final String TEZ_RUNTIME_SHUFFLE_MEMORY_LIMIT_PERCENT = TEZ_RUNTIME_PREFIX + "shuffle.memory.limit.percent"; @@ -543,7 +566,7 @@ public class TezRuntimeConfiguration { /** - * Share data fetched between tasks running on the same host if applicable + * Share data fetched between tasks running on the same host if applicable. */ @ConfigurationProperty(type = "boolean") public static final String TEZ_RUNTIME_OPTIMIZE_SHARED_FETCH = TEZ_RUNTIME_PREFIX @@ -626,6 +649,8 @@ public class TezRuntimeConfiguration { tezRuntimeKeys.add(TEZ_RUNTIME_SHUFFLE_MAX_ALLOWED_FAILED_FETCH_ATTEMPT_FRACTION); tezRuntimeKeys.add(TEZ_RUNTIME_SHUFFLE_MIN_REQUIRED_PROGRESS_FRACTION); tezRuntimeKeys.add(TEZ_RUNTIME_SHUFFLE_FAILED_CHECK_SINCE_LAST_COMPLETION); + tezRuntimeKeys.add(TEZ_RUNTIME_SHUFFLE_FETCH_TESTING_ERRORS_CONFIG); + tezRuntimeKeys.add(TEZ_RUNTIME_SHUFFLE_FETCH_ENABLE_TESTING_ERRORS); tezRuntimeKeys.add(TEZ_RUNTIME_REPORT_PARTITION_STATS); tezRuntimeKeys.add(TEZ_RUNTIME_INPUT_POST_MERGE_BUFFER_PERCENT); tezRuntimeKeys.add(TEZ_RUNTIME_GROUP_COMPARATOR_CLASS); 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 f295c06b8e..6039df3a2f 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 @@ -58,6 +58,7 @@ import org.apache.tez.common.CallableWithNdc; import org.apache.tez.common.security.JobTokenSecretManager; import org.apache.tez.dag.api.TezUncheckedException; +import org.apache.tez.runtime.api.ObjectRegistry; import org.apache.tez.runtime.library.api.TezRuntimeConfiguration; import org.apache.tez.runtime.library.common.Constants; import org.apache.tez.runtime.library.common.InputAttemptIdentifier; @@ -144,7 +145,7 @@ public String toString() { private final AtomicBoolean isShutDown = new AtomicBoolean(false); - private final int fetcherIdentifier; + protected final int fetcherIdentifier; // Parameters to track work. private List srcAttempts; @@ -192,7 +193,7 @@ public String getHost() { private final boolean isDebugEnabled = LOG.isDebugEnabled(); - private Fetcher(FetcherCallback fetcherCallback, HttpConnectionParams params, + protected Fetcher(FetcherCallback fetcherCallback, HttpConnectionParams params, FetchedInputAllocator inputManager, ApplicationId appId, int dagIdentifier, JobTokenSecretManager jobTokenSecretManager, String srcNameTrimmed, Configuration conf, RawLocalFileSystem localFs, @@ -527,8 +528,7 @@ private HostFetchResult setupConnection(Collection attem } try { - input = httpConnection.getInputStream(); - httpConnection.validate(); + setupConnectionInternal(host, attempts); //validateConnectionResponse(msgToEncode, encHash); } catch (IOException e) { // ioErrs.increment(1); @@ -556,6 +556,13 @@ private HostFetchResult setupConnection(Collection attem return null; } + + protected void setupConnectionInternal(String host, Collection attempts) + throws IOException, InterruptedException { + input = httpConnection.getInputStream(); + httpConnection.validate(); + } + @VisibleForTesting protected HostFetchResult doHttpFetch(CachingCallBack callback) { @@ -1141,11 +1148,19 @@ public FetcherBuilder(FetcherCallback fetcherCallback, Configuration conf, RawLocalFileSystem localFs, LocalDirAllocator localDirAllocator, Path lockPath, boolean localDiskFetchEnabled, boolean sharedFetchEnabled, - String localHostname, int shufflePort, boolean asyncHttp, boolean verifyDiskChecksum, boolean compositeFetch) { - this.fetcher = new Fetcher(fetcherCallback, params, inputManager, appId, dagIdentifier, - jobTokenSecretMgr, srcNameTrimmed, conf, localFs, localDirAllocator, - lockPath, localDiskFetchEnabled, sharedFetchEnabled, localHostname, shufflePort, asyncHttp, - verifyDiskChecksum, compositeFetch); + String localHostname, int shufflePort, boolean asyncHttp, boolean verifyDiskChecksum, boolean compositeFetch, + boolean enableFetcherTestingErrors, ObjectRegistry objectRegistry) { + if (enableFetcherTestingErrors) { + this.fetcher = new FetcherWithInjectableErrors(fetcherCallback, params, inputManager, appId, dagIdentifier, + jobTokenSecretMgr, srcNameTrimmed, conf, localFs, localDirAllocator, + lockPath, localDiskFetchEnabled, sharedFetchEnabled, localHostname, shufflePort, asyncHttp, + verifyDiskChecksum, compositeFetch, objectRegistry); + } else { + this.fetcher = new Fetcher(fetcherCallback, params, inputManager, appId, dagIdentifier, + jobTokenSecretMgr, srcNameTrimmed, conf, localFs, localDirAllocator, + lockPath, localDiskFetchEnabled, sharedFetchEnabled, localHostname, shufflePort, asyncHttp, + verifyDiskChecksum, compositeFetch); + } } public FetcherBuilder setHttpConnectionParameters(HttpConnectionParams httpParams) { diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/FetcherErrorTestingConfig.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/FetcherErrorTestingConfig.java new file mode 100644 index 0000000000..ce15a87fb9 --- /dev/null +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/FetcherErrorTestingConfig.java @@ -0,0 +1,111 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.tez.runtime.library.common.shuffle; + +import java.util.Arrays; +import java.util.List; +import java.util.Random; + +import org.apache.hadoop.conf.Configuration; +import org.apache.tez.common.TezUtilsInternal; +import org.apache.tez.runtime.api.ObjectRegistry; +import org.apache.tez.runtime.library.api.TezRuntimeConfiguration; +import org.apache.tez.runtime.library.common.InputAttemptIdentifier; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class FetcherErrorTestingConfig { + private static final Logger LOG = LoggerFactory.getLogger(FetcherErrorTestingConfig.class); + private static final String KEY_CACHED_HOSTNAME = "FetcherErrorTestingConfig.host"; + + private String hostToFail = "*"; + private String srcNameTrimmedToFail = "*"; + private int probabilityPercent = 50; + private Random random = new Random(); + /** + * Whether to fail only in case of input attempts with index 0, + * this prevents continuous failure, and helps simulating a real-life node failure. + */ + private boolean failForFirstAttemptOnly = false; + private ObjectRegistry objectRegistry; + + public FetcherErrorTestingConfig(Configuration conf, ObjectRegistry objectRegistry) { + String errorConfig = conf.get(TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_FETCH_TESTING_ERRORS_CONFIG, + TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_FETCH_TESTING_ERRORS_CONFIG_DEFAULT); + String[] configParts = errorConfig.split("#"); + + // e.g. host_1 + if (configParts.length > 0) { + hostToFail = configParts[0]; + } + + // e.g. Map 1 or Map_1, both will work + if (configParts.length > 1) { + srcNameTrimmedToFail = TezUtilsInternal.cleanVertexName(configParts[1]); + } + + // e.g. 50 + if (configParts.length > 2) { + probabilityPercent = Integer.parseInt(configParts[2]); + } + + // e.g. fail_only_first + if (configParts.length > 3) { + List features = Arrays.asList(configParts[3].split(",")); + if (features.contains("fail_only_first")) { + failForFirstAttemptOnly = true; + } + } + + this.objectRegistry = objectRegistry; + if (hostToFail.equals("_first_")) { + String host = (String) objectRegistry.get(KEY_CACHED_HOSTNAME); + if (host != null) { + LOG.info("Get already stored hostname for fetcher test failures: " + host); + hostToFail = host; + } + } + } + + public boolean shouldFail(String host, String srcNameTrimmed, InputAttemptIdentifier inputAttemptIdentifier) { + if (matchHost(host) && matchSourceVertex(srcNameTrimmed)) { + return (!failForFirstAttemptOnly || failForFirstAttemptOnly && inputAttemptIdentifier.getAttemptNumber() == 0) + && random.nextInt(100) < probabilityPercent; + } + return false; + } + + private boolean matchHost(String host) { + if (hostToFail.equals("_first_")) { + objectRegistry.cacheForVertex(KEY_CACHED_HOSTNAME, host); + hostToFail = host; + } + return "*".equals(hostToFail) || host.equalsIgnoreCase(hostToFail); + } + + private boolean matchSourceVertex(String srcNameTrimmed) { + return "*".equals(srcNameTrimmedToFail) || srcNameTrimmed.equalsIgnoreCase(srcNameTrimmedToFail); + } + + @Override + public String toString() { + return String.format( + "[FetcherErrorTestingConfig: host: %s, source vertex: %s, probability: %d%%, failForFirstAttemptOnly: %s]", + hostToFail, srcNameTrimmedToFail, probabilityPercent, failForFirstAttemptOnly); + } +} diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/FetcherWithInjectableErrors.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/FetcherWithInjectableErrors.java new file mode 100644 index 0000000000..951adf9f5d --- /dev/null +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/FetcherWithInjectableErrors.java @@ -0,0 +1,87 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.tez.runtime.library.common.shuffle; + +import java.io.IOException; +import java.util.Collection; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.LocalDirAllocator; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.RawLocalFileSystem; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.tez.common.security.JobTokenSecretManager; +import org.apache.tez.http.HttpConnectionParams; +import org.apache.tez.runtime.api.ObjectRegistry; +import org.apache.tez.runtime.library.common.InputAttemptIdentifier; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class FetcherWithInjectableErrors extends Fetcher { + private static final Logger LOG = LoggerFactory.getLogger(FetcherWithInjectableErrors.class); + + private FetcherErrorTestingConfig fetcherErrorTestingConfig; + private String srcNameTrimmed; + + protected FetcherWithInjectableErrors(FetcherCallback fetcherCallback, HttpConnectionParams params, + FetchedInputAllocator inputManager, ApplicationId appId, int dagIdentifier, + JobTokenSecretManager jobTokenSecretManager, String srcNameTrimmed, Configuration conf, + RawLocalFileSystem localFs, LocalDirAllocator localDirAllocator, Path lockPath, boolean localDiskFetchEnabled, + boolean sharedFetchEnabled, String localHostname, int shufflePort, boolean asyncHttp, boolean verifyDiskChecksum, + boolean compositeFetch, ObjectRegistry objectRegistry) { + super(fetcherCallback, params, inputManager, appId, dagIdentifier, jobTokenSecretManager, srcNameTrimmed, conf, + localFs, localDirAllocator, lockPath, localDiskFetchEnabled, sharedFetchEnabled, localHostname, shufflePort, + asyncHttp, verifyDiskChecksum, compositeFetch); + this.fetcherErrorTestingConfig = new FetcherErrorTestingConfig(conf, objectRegistry); + this.srcNameTrimmed = srcNameTrimmed; + LOG.info("Initialized FetcherWithInjectableErrors with config: {}", fetcherErrorTestingConfig); + } + + @Override + protected void setupConnectionInternal(String host, Collection attempts) + throws IOException, InterruptedException { + LOG.info("Checking if fetcher should fail for host: {} ...", host); + for (InputAttemptIdentifier inputAttemptIdentifier : attempts) { + if (fetcherErrorTestingConfig.shouldFail(host, srcNameTrimmed, inputAttemptIdentifier)) { + throw new IOException(String.format( + "FetcherWithInjectableErrors tester made failure for host: %s, input attempt: %s", host, + inputAttemptIdentifier.getAttemptNumber())); + } + } + super.setupConnectionInternal(host, attempts); + } + + @Override + public int hashCode() { + return fetcherIdentifier; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (obj == null) { + return false; + } + if (getClass() != obj.getClass()) { + return false; + } + FetcherWithInjectableErrors other = (FetcherWithInjectableErrors) obj; + if (fetcherIdentifier != other.fetcherIdentifier) { + return false; + } + return true; + } +} 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 56195a8641..fa539c8374 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 @@ -174,7 +174,8 @@ public class ShuffleManager implements FetcherCallback { private final boolean sharedFetchEnabled; private final boolean verifyDiskChecksum; private final boolean compositeFetch; - + private final boolean enableFetcherTestingErrors; + private final int ifileBufferSize; private final boolean ifileReadAhead; private final int ifileReadAheadLength; @@ -259,6 +260,10 @@ public ShuffleManager(InputContext inputContext, Configuration conf, int numInpu this.lastEventReceived = inputContext.getCounters().findCounter(TaskCounter.LAST_EVENT_RECEIVED); this.compositeFetch = ShuffleUtils.isTezShuffleHandler(conf); + this.enableFetcherTestingErrors = + conf.getBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_FETCH_ENABLE_TESTING_ERRORS, + TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_FETCH_ENABLE_TESTING_ERRORS_DEFAULT); + this.srcNameTrimmed = TezUtilsInternal.cleanVertexName(inputContext.getSourceVertexName()); completedInputSet = new BitSet(numInputs); @@ -395,7 +400,7 @@ protected Void callInternal() throws Exception { for (InputReadErrorEvent key : failedEvents.keySet()) { failedEventsToSend.add(InputReadErrorEvent.create(key.getDiagnostics(), key.getIndex(), key.getVersion(), failedEvents.get(key), key.isLocalFetch(), - key.isDiskErrorAtSource())); + key.isDiskErrorAtSource(), localhostName)); } inputContext.sendEvents(failedEventsToSend); failedEvents.clear(); @@ -543,7 +548,8 @@ Fetcher constructFetcherForHost(InputHost inputHost, Configuration conf) { httpConnectionParams, inputManager, inputContext.getApplicationId(), inputContext.getDagIdentifier(), jobTokenSecretMgr, srcNameTrimmed, conf, localFs, localDirAllocator, lockDisk, localDiskFetchEnabled, sharedFetchEnabled, - localhostName, shufflePort, asyncHttp, verifyDiskChecksum, compositeFetch); + localhostName, shufflePort, asyncHttp, verifyDiskChecksum, compositeFetch, enableFetcherTestingErrors, + inputContext.getObjectRegistry()); if (codec != null) { fetcherBuilder.setCompressionParameters(codec); @@ -960,7 +966,7 @@ public void fetchFailed(String host, srcAttemptIdentifier.getInputIdentifier(), srcAttemptIdentifier.getAttemptNumber(), inputAttemptFetchFailure.isLocalFetch(), - inputAttemptFetchFailure.isDiskErrorAtSource()); + inputAttemptFetchFailure.isDiskErrorAtSource(), localhostName); if (maxTimeToWaitForReportMillis > 0) { try { reportLock.lock(); 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 a4328af44a..2725debb56 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 @@ -81,7 +81,7 @@ class FetcherOrderedGrouped extends CallableWithNdc { private final int localShufflePort; private final String applicationId; private final int dagId; - private final MapHost mapHost; + protected final MapHost mapHost; private final int minPartition; private final int maxPartition; @@ -350,8 +350,7 @@ boolean setupConnection(MapHost host, Collection attempt LOG.debug("Detected fetcher has been shutdown after connection establishment. Returning"); return false; } - input = httpConnection.getInputStream(); - httpConnection.validate(); + setupConnectionInternal(host, attempts); return true; } catch (IOException | InterruptedException ie) { if (ie instanceof InterruptedException) { @@ -385,6 +384,12 @@ boolean setupConnection(MapHost host, Collection attempt } } + protected void setupConnectionInternal(MapHost host, Collection attempts) + throws IOException, InterruptedException { + input = httpConnection.getInputStream(); + httpConnection.validate(); + } + @VisibleForTesting protected void putBackRemainingMapOutputs(MapHost host) { // Cycle through remaining MapOutputs @@ -426,7 +431,7 @@ public String toString() { } protected InputAttemptFetchFailure[] copyMapOutput(MapHost host, DataInputStream input, - InputAttemptIdentifier inputAttemptIdentifier) throws FetcherReadTimeoutException { + InputAttemptIdentifier inputAttemptIdentifier) throws FetcherReadTimeoutException, IOException { MapOutput mapOutput = null; InputAttemptIdentifier srcAttemptId = null; long decompressedLength = 0; diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/FetcherOrderedGroupedWithInjectableErrors.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/FetcherOrderedGroupedWithInjectableErrors.java new file mode 100644 index 0000000000..94172d12b9 --- /dev/null +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/FetcherOrderedGroupedWithInjectableErrors.java @@ -0,0 +1,68 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.tez.runtime.library.common.shuffle.orderedgrouped; + +import java.io.IOException; +import java.util.Collection; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.RawLocalFileSystem; +import org.apache.hadoop.io.compress.CompressionCodec; +import org.apache.tez.common.counters.TezCounter; +import org.apache.tez.common.security.JobTokenSecretManager; +import org.apache.tez.http.HttpConnectionParams; +import org.apache.tez.runtime.api.ObjectRegistry; +import org.apache.tez.runtime.library.common.InputAttemptIdentifier; +import org.apache.tez.runtime.library.common.shuffle.FetcherErrorTestingConfig; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class FetcherOrderedGroupedWithInjectableErrors extends FetcherOrderedGrouped { + private static final Logger LOG = LoggerFactory.getLogger(FetcherOrderedGroupedWithInjectableErrors.class); + + private FetcherErrorTestingConfig fetcherErrorTestingConfig; + private String srcNameTrimmed; + + public FetcherOrderedGroupedWithInjectableErrors(HttpConnectionParams httpConnectionParams, + ShuffleScheduler scheduler, FetchedInputAllocatorOrderedGrouped allocator, ExceptionReporter exceptionReporter, + JobTokenSecretManager jobTokenSecretMgr, boolean ifileReadAhead, int ifileReadAheadLength, CompressionCodec codec, + Configuration conf, RawLocalFileSystem localFs, boolean localDiskFetchEnabled, String localHostname, + int shufflePort, String srcNameTrimmed, MapHost mapHost, TezCounter ioErrsCounter, + TezCounter wrongLengthErrsCounter, TezCounter badIdErrsCounter, TezCounter wrongMapErrsCounter, + TezCounter connectionErrsCounter, TezCounter wrongReduceErrsCounter, String applicationId, int dagId, + boolean asyncHttp, boolean sslShuffle, boolean verifyDiskChecksum, boolean compositeFetch, + ObjectRegistry objectRegistry) { + super(httpConnectionParams, scheduler, allocator, exceptionReporter, jobTokenSecretMgr, ifileReadAhead, + ifileReadAheadLength, codec, conf, localFs, localDiskFetchEnabled, localHostname, shufflePort, srcNameTrimmed, + mapHost, ioErrsCounter, wrongLengthErrsCounter, badIdErrsCounter, wrongMapErrsCounter, connectionErrsCounter, + wrongReduceErrsCounter, applicationId, dagId, asyncHttp, sslShuffle, verifyDiskChecksum, compositeFetch); + this.fetcherErrorTestingConfig = new FetcherErrorTestingConfig(conf, objectRegistry); + this.srcNameTrimmed = srcNameTrimmed; + LOG.info("Initialized FetcherOrderedGroupedWithInjectableErrors with config: {}", fetcherErrorTestingConfig); + } + + @Override + protected void setupConnectionInternal(MapHost host, Collection attempts) + throws IOException, InterruptedException { + LOG.info("Checking if fetcher should fail for host: {} ...", mapHost.getHost()); + for (InputAttemptIdentifier inputAttemptIdentifier : attempts) { + if (fetcherErrorTestingConfig.shouldFail(mapHost.getHost(), srcNameTrimmed, inputAttemptIdentifier)) { + throw new IOException(String.format( + "FetcherOrderedGroupedWithInjectableErrors tester made failure for host: %s, input attempt: %s", + mapHost.getHost(), inputAttemptIdentifier.getAttemptNumber())); + } + } + super.setupConnectionInternal(host, attempts); + } +} 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 dd27d45aed..cd4b14ca1b 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 @@ -248,7 +248,7 @@ enum ShuffleErrors { private final boolean checkFailedFetchSinceLastCompletion; private final boolean verifyDiskChecksum; private final boolean compositeFetch; - + private final boolean enableFetcherTestingErrors; private volatile Thread shuffleSchedulerThread = null; private final int maxPenaltyTime; @@ -427,6 +427,10 @@ public ShuffleScheduler(InputContext inputContext, this.maxPenaltyTime = conf.getInt(TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_HOST_PENALTY_TIME_LIMIT_MS, TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_HOST_PENALTY_TIME_LIMIT_MS_DEFAULT); + this.enableFetcherTestingErrors = + conf.getBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_FETCH_ENABLE_TESTING_ERRORS, + TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_FETCH_ENABLE_TESTING_ERRORS_DEFAULT); + pipelinedShuffleInfoEventsMap = Maps.newConcurrentMap(); LOG.info("ShuffleScheduler running for sourceVertex: " + inputContext.getSourceVertexName() + " with configuration: " @@ -443,6 +447,7 @@ public ShuffleScheduler(InputContext inputContext, + ", minReqProgressFraction=" + minReqProgressFraction + ", checkFailedFetchSinceLastCompletion=" + checkFailedFetchSinceLastCompletion + ", asyncHttp=" + asyncHttp + + ", enableFetcherTestingErrors=" + enableFetcherTestingErrors ); } @@ -885,7 +890,7 @@ private void informAM(InputAttemptFetchFailure fetchFailure) { srcAttempt.getInputIdentifier(), srcAttempt.getAttemptNumber()) + " to jobtracker.", srcAttempt.getInputIdentifier(), srcAttempt.getAttemptNumber(), - fetchFailure.isLocalFetch(), fetchFailure.isDiskErrorAtSource())); + fetchFailure.isLocalFetch(), fetchFailure.isDiskErrorAtSource(), localHostname)); inputContext.sendEvents(failedEvents); } @@ -1466,12 +1471,21 @@ private synchronized void waitAndNotifyProgress() throws InterruptedException { @VisibleForTesting FetcherOrderedGrouped constructFetcherForHost(MapHost mapHost) { - return new FetcherOrderedGrouped(httpConnectionParams, ShuffleScheduler.this, allocator, - exceptionReporter, jobTokenSecretManager, ifileReadAhead, ifileReadAheadLength, - codec, conf, localFs, localDiskFetchEnabled, localHostname, shufflePort, srcNameTrimmed, mapHost, - ioErrsCounter, wrongLengthErrsCounter, badIdErrsCounter, wrongMapErrsCounter, - connectionErrsCounter, wrongReduceErrsCounter, applicationId, dagId, asyncHttp, sslShuffle, - verifyDiskChecksum, compositeFetch); + if (enableFetcherTestingErrors) { + return new FetcherOrderedGroupedWithInjectableErrors(httpConnectionParams, ShuffleScheduler.this, allocator, + exceptionReporter, jobTokenSecretManager, ifileReadAhead, ifileReadAheadLength, + codec, conf, localFs, localDiskFetchEnabled, localHostname, shufflePort, srcNameTrimmed, mapHost, + ioErrsCounter, wrongLengthErrsCounter, badIdErrsCounter, wrongMapErrsCounter, + connectionErrsCounter, wrongReduceErrsCounter, applicationId, dagId, asyncHttp, sslShuffle, + verifyDiskChecksum, compositeFetch, inputContext.getObjectRegistry()); + } else { + return new FetcherOrderedGrouped(httpConnectionParams, ShuffleScheduler.this, allocator, + exceptionReporter, jobTokenSecretManager, ifileReadAhead, ifileReadAheadLength, + codec, conf, localFs, localDiskFetchEnabled, localHostname, shufflePort, srcNameTrimmed, mapHost, + ioErrsCounter, wrongLengthErrsCounter, badIdErrsCounter, wrongMapErrsCounter, + connectionErrsCounter, wrongReduceErrsCounter, applicationId, dagId, asyncHttp, sslShuffle, + verifyDiskChecksum, compositeFetch); + } } private class FetchFutureCallback implements FutureCallback { From 8a75317bb9f954108aac44fc8ca30bfde0c0e462 Mon Sep 17 00:00:00 2001 From: dmmkr Date: Thu, 25 Nov 2021 17:52:26 +0530 Subject: [PATCH 074/137] TEZ-4353: Update commons-io to 2.8.0 (#165) (D M Murali Krishna Reddy reviewed by Laszlo Bodor) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 3785352bf5..27b3e399fe 100644 --- a/pom.xml +++ b/pom.xml @@ -253,7 +253,7 @@ commons-io commons-io - 2.4 + 2.8.0 commons-lang From bf616b22012b5e0f60242c80c697d58fc2eed005 Mon Sep 17 00:00:00 2001 From: jteagles Date: Sat, 25 Dec 2021 16:30:24 -0600 Subject: [PATCH 075/137] TEZ-4294. Tez UI counters missing in settings view (#144) (Jonathan Eagles reviewed by Laszlo Bodor) --- .../apache/tez/common/counters/DAGCounter.java | 1 + .../apache/tez/common/counters/TaskCounter.java | 1 + .../src/main/webapp/config/default-app-conf.js | 16 ++++++++++++++++ 3 files changed, 18 insertions(+) diff --git a/tez-api/src/main/java/org/apache/tez/common/counters/DAGCounter.java b/tez-api/src/main/java/org/apache/tez/common/counters/DAGCounter.java index 0a32d388bc..1ef56778c3 100644 --- a/tez-api/src/main/java/org/apache/tez/common/counters/DAGCounter.java +++ b/tez-api/src/main/java/org/apache/tez/common/counters/DAGCounter.java @@ -22,6 +22,7 @@ import org.apache.hadoop.classification.InterfaceStability; // Per-job counters +// Keep in sync with tez-ui/src/main/webapp/config/default-app-conf.js @InterfaceAudience.Public @InterfaceStability.Evolving public enum DAGCounter { 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 2ee82a3965..a65388ffd0 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 @@ -21,6 +21,7 @@ import org.apache.hadoop.classification.InterfaceAudience.Private; // Counters used by Task classes +// Keep in sync with tez-ui/src/main/webapp/config/default-app-conf.js @Private public enum TaskCounter { // TODO Eventually, rename counters to be non-MR specific and map them to MR equivalent. diff --git a/tez-ui/src/main/webapp/config/default-app-conf.js b/tez-ui/src/main/webapp/config/default-app-conf.js index 388ca4ce0f..a49a3e0134 100644 --- a/tez-ui/src/main/webapp/config/default-app-conf.js +++ b/tez-ui/src/main/webapp/config/default-app-conf.js @@ -194,6 +194,10 @@ module.exports = { // Tez App configurations counterName: "INPUT_RECORDS_PROCESSED", counterGroupName: "org.apache.tez.common.counters.TaskCounter", }, + { + counterName: "INPUT_SPLIT_LENGTH_BYTES", + counterGroupName: "org.apache.tez.common.counters.TaskCounter", + }, { counterName: "OUTPUT_RECORDS", counterGroupName: "org.apache.tez.common.counters.TaskCounter", @@ -226,6 +230,10 @@ module.exports = { // Tez App configurations counterName: "ADDITIONAL_SPILL_COUNT", counterGroupName: "org.apache.tez.common.counters.TaskCounter", }, + { + counterName: "SHUFFLE_CHUNK_COUNT", + counterGroupName: "org.apache.tez.common.counters.TaskCounter", + }, { counterName: "SHUFFLE_BYTES", counterGroupName: "org.apache.tez.common.counters.TaskCounter", @@ -321,6 +329,14 @@ module.exports = { // Tez App configurations counterName :"NUM_FAILED_UBERTASKS", counterGroupName :"org.apache.tez.common.counters.DAGCounter", }, + { + counterName :"AM_CPU_MILLISECONDS", + counterGroupName :"org.apache.tez.common.counters.DAGCounter", + }, + { + counterName :"AM_GC_TIME_MILLIS", + counterGroupName :"org.apache.tez.common.counters.DAGCounter", + }, { counterName: "REDUCE_OUTPUT_RECORDS", From 8f846dcfe131aca69f8d3cb5cc1b47c6413e3b01 Mon Sep 17 00:00:00 2001 From: Bodor Laszlo Date: Sun, 26 Dec 2021 18:10:36 +0100 Subject: [PATCH 076/137] TEZ-4303: Exclude compile-time httpclient dependency from Tez (#119) (Laszlo Bodor reviewed by Jonathan Eagles) --- pom.xml | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/pom.xml b/pom.xml index 27b3e399fe..5a2c4fa45c 100644 --- a/pom.xml +++ b/pom.xml @@ -341,6 +341,10 @@ io.netty netty + + org.apache.httpcomponents + httpclient + @@ -352,6 +356,10 @@ io.netty netty + + org.apache.httpcomponents + httpclient + From c9b8e90db5af2b9648fccaa1501ad6dda82f31e0 Mon Sep 17 00:00:00 2001 From: belugabehr <12578579+belugabehr@users.noreply.github.com> Date: Sun, 26 Dec 2021 12:35:43 -0500 Subject: [PATCH 077/137] TEZ-4270: HeartbeatHandlerBase Does Not Need Stop Condition (#93) (David Mollitor reviewed by Laszlo Bodor) --- .../java/org/apache/tez/dag/app/HeartbeatHandlerBase.java | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/HeartbeatHandlerBase.java b/tez-dag/src/main/java/org/apache/tez/dag/app/HeartbeatHandlerBase.java index 5c786b2232..b5ded81500 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/HeartbeatHandlerBase.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/HeartbeatHandlerBase.java @@ -42,7 +42,6 @@ public abstract class HeartbeatHandlerBase extends AbstractService { protected final AppContext appContext; private ConcurrentMap runningMap; - private volatile boolean stopped; public HeartbeatHandlerBase(AppContext appContext, int expectedConcurrency, String name) { super(name); @@ -70,7 +69,6 @@ public void serviceStart() { @Override public void serviceStop() { - stopped = true; if (timeOutCheckerThread != null) { timeOutCheckerThread.interrupt(); } @@ -140,7 +138,7 @@ private class PingChecker implements Runnable { @Override public void run() { - while (!stopped && !Thread.currentThread().isInterrupted()) { + while (!Thread.currentThread().isInterrupted()) { Iterator> iterator = runningMap.entrySet().iterator(); @@ -158,7 +156,7 @@ public void run() { try { Thread.sleep(timeOutCheckInterval); } catch (InterruptedException e) { - break; + Thread.currentThread().interrupt(); } } } From a602a07dc2c18f2a59eddbcdade9d116d06174ec Mon Sep 17 00:00:00 2001 From: Ahmed Hussein <50450311+amahussein@users.noreply.github.com> Date: Mon, 3 Jan 2022 01:39:40 -0600 Subject: [PATCH 078/137] TEZ-4349. DAGClient gets stuck with invalid cached DAGStatus (#161) (Ahmed Hussein reviewed by Laszlo Bodor, Jonathan Eagles) --- .../org/apache/tez/common/CachedEntity.java | 104 +++++++++++++ .../apache/tez/dag/api/TezConfiguration.java | 12 ++ .../tez/dag/api/client/DAGClientImpl.java | 66 +++++--- .../tez/dag/api/client/rpc/TestDAGClient.java | 144 +++++++++++++++++- 4 files changed, 303 insertions(+), 23 deletions(-) create mode 100644 tez-api/src/main/java/org/apache/tez/common/CachedEntity.java diff --git a/tez-api/src/main/java/org/apache/tez/common/CachedEntity.java b/tez-api/src/main/java/org/apache/tez/common/CachedEntity.java new file mode 100644 index 0000000000..47ccd77e48 --- /dev/null +++ b/tez-api/src/main/java/org/apache/tez/common/CachedEntity.java @@ -0,0 +1,104 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.tez.common; + +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; + +import org.apache.hadoop.yarn.util.Clock; +import org.apache.hadoop.yarn.util.MonotonicClock; + +/** + * A thread safe implementation used as a container for cacheable entries with Expiration times. + * It supports custom {@link Clock} to control the elapsed time calculation. + * @param the data object type. + */ +public class CachedEntity { + private final AtomicReference entryDataRef; + private final Clock cacheClock; + private final long expiryDurationMS; + private volatile long entryTimeStamp; + + public CachedEntity(TimeUnit expiryTimeUnit, long expiryLength, Clock clock) { + entryDataRef = new AtomicReference<>(null); + cacheClock = clock; + expiryDurationMS = TimeUnit.MILLISECONDS.convert(expiryLength, expiryTimeUnit); + entryTimeStamp = 0; + } + + public CachedEntity(TimeUnit expiryTimeUnit, long expiryLength) { + this(expiryTimeUnit, expiryLength, new MonotonicClock()); + } + + /** + * + * @return true if expiration timestamp is 0, or the elapsed time since last update is + * greater than {@link #expiryDurationMS} + */ + public boolean isExpired() { + return (entryTimeStamp == 0) + || ((cacheClock.getTime() - entryTimeStamp) > expiryDurationMS); + } + + /** + * If the entry has expired, it reset the cache reference through {@link #clearExpiredEntry()}. + * @return cached data if the timestamp is valid. Null, if the timestamp has expired. + */ + public T getValue() { + if (isExpired()) { // quick check for expiration + if (clearExpiredEntry()) { // remove reference to the expired entry + return null; + } + } + return entryDataRef.get(); + } + + /** + * Safely sets the cached data. + * @param newEntry + */ + public void setValue(T newEntry) { + T currentEntry = entryDataRef.get(); + while (!entryDataRef.compareAndSet(currentEntry, newEntry)) { + currentEntry = entryDataRef.get(); + } + entryTimeStamp = cacheClock.getTime(); + } + + /** + * Enforces the expiration of the cached entry. + */ + public void enforceExpiration() { + entryTimeStamp = 0; + } + + /** + * Safely deletes the reference to the data if it was not null. + * @return true if the reference is set to Null. False indicates that another thread + * updated the cache. + */ + private boolean clearExpiredEntry() { + T currentEntry = entryDataRef.get(); + if (currentEntry == null) { + return true; + } + // the current value is not null: try to reset it. + // if the CAS is successful, then we won't override a recent update to the cache. + return (entryDataRef.compareAndSet(currentEntry, null)); + } +} 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 5063b04eac..ee991ccf53 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 @@ -1993,6 +1993,18 @@ static Set getPropertySet() { TEZ_PREFIX + "test.minicluster.app.wait.on.shutdown.secs"; public static final long TEZ_TEST_MINI_CLUSTER_APP_WAIT_ON_SHUTDOWN_SECS_DEFAULT = 30; + /** + * Long value + * Status Cache timeout window in minutes for the DAGClient. + */ + @Private + @ConfigurationScope(Scope.CLIENT) + @ConfigurationProperty(type="long") + public static final String TEZ_CLIENT_DAG_STATUS_CACHE_TIMEOUT_SECS = TEZ_PREFIX + + "client.dag.status.cache.timeout-secs"; + // Default timeout is 60 seconds. + public static final long TEZ_CLIENT_DAG_STATUS_CACHE_TIMEOUT_SECS_DEFAULT = 60; + /** * Long value * Time to wait (in milliseconds) for yarn app's diagnotics is available 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 e58863f614..3c0de00819 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 @@ -27,9 +27,11 @@ import java.util.HashMap; import java.util.Map; import java.util.Set; +import java.util.concurrent.TimeUnit; import com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.security.UserGroupInformation; +import org.apache.tez.common.CachedEntity; import org.apache.tez.common.Preconditions; import org.apache.hadoop.yarn.exceptions.ApplicationNotFoundException; @@ -58,13 +60,15 @@ public class DAGClientImpl extends DAGClient { private final String dagId; private final TezConfiguration conf; private final FrameworkClient frameworkClient; - + /** + * Container to cache the last {@link DAGStatus}. + */ + private final CachedEntity cachedDAGStatusRef; @VisibleForTesting protected DAGClientInternal realClient; - private boolean dagCompleted = false; + private volatile boolean dagCompleted = false; @VisibleForTesting protected boolean isATSEnabled = false; - private DAGStatus cachedDagStatus = null; Map cachedVertexStatus = new HashMap(); private static final long SLEEP_FOR_COMPLETION = 500; @@ -110,6 +114,28 @@ public DAGClientImpl(ApplicationId appId, String dagId, TezConfiguration conf, this.diagnoticsWaitTimeout = conf.getLong( TezConfiguration.TEZ_CLIENT_DIAGNOSTICS_WAIT_TIMEOUT_MS, TezConfiguration.TEZ_CLIENT_DIAGNOSTICS_WAIT_TIMEOUT_MS_DEFAULT); + cachedDAGStatusRef = initCacheDAGRefFromConf(conf); + } + + /** + * Constructs a new {@link CachedEntity} for {@link DAGStatus}. + * @param tezConf TEZ configuration parameters. + * @return a caching entry to hold the {@link DAGStatus}. + */ + protected CachedEntity initCacheDAGRefFromConf(TezConfiguration tezConf) { + long clientDAGStatusCacheTimeOut = tezConf.getLong( + TezConfiguration.TEZ_CLIENT_DAG_STATUS_CACHE_TIMEOUT_SECS, + TezConfiguration.TEZ_CLIENT_DAG_STATUS_CACHE_TIMEOUT_SECS_DEFAULT); + if (clientDAGStatusCacheTimeOut <= 0) { + LOG.error("DAG Status cache timeout interval should be positive. Enforcing default value."); + clientDAGStatusCacheTimeOut = + TezConfiguration.TEZ_CLIENT_DAG_STATUS_CACHE_TIMEOUT_SECS_DEFAULT; + } + return new CachedEntity<>(TimeUnit.SECONDS, clientDAGStatusCacheTimeOut); + } + + protected CachedEntity getCachedDAGStatusRef() { + return cachedDAGStatusRef; } @Override @@ -133,13 +159,11 @@ public DAGStatus getDAGStatus(@Nullable Set statusOptions, } long startTime = System.currentTimeMillis(); - boolean refreshStatus; - DAGStatus dagStatus; - if(cachedDagStatus != null) { - dagStatus = cachedDagStatus; - refreshStatus = true; - } else { - // For the first lookup only. After this cachedDagStatus should be populated. + + DAGStatus dagStatus = cachedDAGStatusRef.getValue(); + boolean refreshStatus = true; + if (dagStatus == null) { + // the first lookup only or when the cachedDAG has expired dagStatus = getDAGStatus(statusOptions); refreshStatus = false; } @@ -221,13 +245,14 @@ protected DAGStatus getDAGStatusInternal(@Nullable Set statusOpti final DAGStatus dagStatus = getDAGStatusViaAM(statusOptions, timeout); if (!dagCompleted) { - if (dagStatus != null) { - cachedDagStatus = dagStatus; + if (dagStatus != null) { // update the cached DAGStatus + cachedDAGStatusRef.setValue(dagStatus); return dagStatus; } - if (cachedDagStatus != null) { + DAGStatus cachedDAG = cachedDAGStatusRef.getValue(); + if (cachedDAG != null) { // could not get from AM (not reachable/ was killed). return cached status. - return cachedDagStatus; + return cachedDAG; } } @@ -253,8 +278,11 @@ protected DAGStatus getDAGStatusInternal(@Nullable Set statusOpti // dag completed and Timeline service is either not enabled or does not have completion status // return cached status if completion info is present. - if (dagCompleted && cachedDagStatus != null && cachedDagStatus.isCompleted()) { - return cachedDagStatus; + if (dagCompleted) { + DAGStatus cachedDag = cachedDAGStatusRef.getValue(); + if (cachedDag != null && cachedDag.isCompleted()) { + return cachedDag; + } } // everything else fails rely on RM. @@ -377,9 +405,11 @@ private DAGStatus getDAGStatusViaAM(@Nullable Set statusOptions, LOG.info("DAG is no longer running - application not found by YARN", e); dagCompleted = true; } catch (TezException e) { - // can be either due to a n/w issue of due to AM completed. + // can be either due to a n/w issue or due to AM completed. + LOG.info("Cannot retrieve DAG Status due to TezException: {}", e.getMessage()); } catch (IOException e) { - // can be either due to a n/w issue of due to AM completed. + // can be either due to a n/w issue or due to AM completed. + LOG.info("Cannot retrieve DAG Status due to IOException: {}", e.getMessage()); } if (dagStatus == null && !dagCompleted) { diff --git a/tez-api/src/test/java/org/apache/tez/dag/api/client/rpc/TestDAGClient.java b/tez-api/src/test/java/org/apache/tez/dag/api/client/rpc/TestDAGClient.java index 50c9a6061c..c1ebaa8db1 100644 --- a/tez-api/src/test/java/org/apache/tez/dag/api/client/rpc/TestDAGClient.java +++ b/tez-api/src/test/java/org/apache/tez/dag/api/client/rpc/TestDAGClient.java @@ -34,6 +34,7 @@ import java.lang.reflect.Field; import java.util.EnumSet; import java.util.Set; +import java.util.concurrent.atomic.AtomicReference; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.ssl.KeyStoreTestUtil; @@ -41,12 +42,14 @@ import org.apache.hadoop.yarn.api.records.ApplicationReport; import org.apache.hadoop.yarn.exceptions.YarnException; import org.apache.tez.client.FrameworkClient; +import org.apache.tez.common.CachedEntity; import org.apache.tez.dag.api.TezConfiguration; import org.apache.tez.dag.api.TezException; import org.apache.tez.dag.api.client.DAGClient; import org.apache.tez.dag.api.client.DAGClientImpl; import org.apache.tez.dag.api.client.DAGClientTimelineImpl; import org.apache.tez.dag.api.client.DAGStatus; +import org.apache.tez.dag.api.client.DAGStatus.State; import org.apache.tez.dag.api.client.DagStatusSource; import org.apache.tez.dag.api.client.StatusGetOpts; import org.apache.tez.dag.api.client.VertexStatus; @@ -374,7 +377,7 @@ public void testGetDagStatusWithTimeout() throws Exception { // Fetch from AM. RUNNING dagClient.resetCounters(); - dagClientRpc.resetCountesr(); + dagClientRpc.resetCounters(); rmDagStatus = new DAGStatus(constructDagStatusProto(DAGStatusStateProto.DAG_RUNNING), DagStatusSource.RM); dagClient.setRmDagStatus(rmDagStatus); @@ -394,7 +397,7 @@ public void testGetDagStatusWithTimeout() throws Exception { // Fetch from AM. Success. dagClient.resetCounters(); - dagClientRpc.resetCountesr(); + dagClientRpc.resetCounters(); rmDagStatus = new DAGStatus(constructDagStatusProto(DAGStatusStateProto.DAG_RUNNING), DagStatusSource.RM); dagClient.setRmDagStatus(rmDagStatus); @@ -438,20 +441,21 @@ private static void testAtsEnabled(ApplicationId appId, String dagIdStr, boolean } private static class DAGClientRPCImplForTest extends DAGClientRPCImpl { - + private AtomicReference faultAMInjectedRef; int numGetStatusViaAmInvocations = 0; public DAGClientRPCImplForTest(ApplicationId appId, String dagId, TezConfiguration conf, @Nullable FrameworkClient frameworkClient) throws IOException { super(appId, dagId, conf, frameworkClient, UserGroupInformation.getCurrentUser()); + faultAMInjectedRef = new AtomicReference<>(null); } void setAMProxy(DAGClientAMProtocolBlockingPB proxy) { this.proxy = proxy; } - void resetCountesr() { + void resetCounters() { numGetStatusViaAmInvocations = 0; } @@ -468,15 +472,22 @@ boolean createAMProxyIfNeeded() throws IOException, TezException { DAGStatus getDAGStatusViaAM(Set statusOptions, long timeout) throws IOException, TezException { numGetStatusViaAmInvocations++; + if (faultAMInjectedRef.get() != null) { + throw faultAMInjectedRef.get(); + } return super.getDAGStatusViaAM(statusOptions, timeout); } + + void injectAMFault(IOException exception) { + faultAMInjectedRef.set(exception); + } } private static class DAGClientImplForTest extends DAGClientImpl { private DAGStatus rmDagStatus; int numGetStatusViaRmInvocations = 0; - + private volatile boolean faultInjected; public DAGClientImplForTest(ApplicationId appId, String dagId, TezConfiguration conf, @Nullable FrameworkClient frameworkClient) throws IOException { super(appId, dagId, conf, frameworkClient, UserGroupInformation.getCurrentUser()); @@ -497,6 +508,9 @@ void resetCounters() { @Override protected DAGStatus getDAGStatusViaRM() throws TezException, IOException { numGetStatusViaRmInvocations++; + if (faultInjected) { + throw new IOException("Fault Injected for RM"); + } return rmDagStatus; } @@ -504,6 +518,18 @@ public boolean getIsATSEnabled() { return isATSEnabled; } + void injectFault() { + faultInjected = true; + } + + DAGStatus getCachedDAGStatus() { + CachedEntity cacheRef = getCachedDAGStatusRef(); + return cacheRef.getValue(); + } + + void enforceExpirationCachedDAGStatus() { + getCachedDAGStatusRef().enforceExpiration(); + } } private DAGProtos.DAGStatusProto.Builder constructDagStatusProto(DAGStatusStateProto stateProto) { @@ -578,4 +604,112 @@ public void testTimelineClientCleanup() throws Exception { } Assert.assertFalse("Reloader is still alive", reloaderStillAlive); } + + @Test(timeout = 50000) + public void testGetDagStatusWithCachedStatusExpiration() throws Exception { + long startTime; + long endTime; + long diff; + + TezConfiguration tezConf = new TezConfiguration(); + tezConf.setLong(TezConfiguration.TEZ_DAG_STATUS_POLLINTERVAL_MS, 800L); + tezConf.setLong(TezConfiguration.TEZ_CLIENT_DAG_STATUS_CACHE_TIMEOUT_SECS, 100000L); + try (DAGClientImplForTest dagClientImpl = + new DAGClientImplForTest(mockAppId, dagIdStr, tezConf, null)) { + DAGClientRPCImplForTest dagClientRpc = + new DAGClientRPCImplForTest(mockAppId, dagIdStr, tezConf, null); + dagClientImpl.setRealClient(dagClientRpc); + + DAGStatus dagStatus; + DAGStatus rmDagStatus; + + // Fetch from AM. RUNNING + rmDagStatus = + new DAGStatus(constructDagStatusProto(DAGStatusStateProto.DAG_RUNNING), + DagStatusSource.RM); + dagClientImpl.setRmDagStatus(rmDagStatus); + dagClientRpc.setAMProxy(createMockProxy(DAGStatusStateProto.DAG_RUNNING, -1)); + + startTime = System.currentTimeMillis(); + dagStatus = dagClientImpl.getDAGStatus(EnumSet.noneOf(StatusGetOpts.class), 2000L); + endTime = System.currentTimeMillis(); + diff = endTime - startTime; + assertTrue(diff > 1500L && diff < 2500L); + // Directly from AM + assertEquals(0, dagClientImpl.numGetStatusViaRmInvocations); + // Directly from AM - one refresh. One with timeout. + assertEquals(2, dagClientRpc.numGetStatusViaAmInvocations); + assertEquals(DAGStatus.State.RUNNING, dagStatus.getState()); + + // Fetch from AM. Success. + dagClientImpl.resetCounters(); + dagClientRpc.resetCounters(); + rmDagStatus = + new DAGStatus(constructDagStatusProto(DAGStatusStateProto.DAG_RUNNING), + DagStatusSource.RM); + dagClientImpl.setRmDagStatus(rmDagStatus); + dagClientRpc.setAMProxy(createMockProxy(DAGStatusStateProto.DAG_SUCCEEDED, 1000L)); + + startTime = System.currentTimeMillis(); + dagStatus = dagClientImpl.getDAGStatus(EnumSet.noneOf(StatusGetOpts.class), 2000L); + endTime = System.currentTimeMillis(); + diff = endTime - startTime; + assertTrue("diff is " + diff, diff > 500L && diff < 1500L); + // Directly from AM + assertEquals(0, dagClientImpl.numGetStatusViaRmInvocations); + // Directly from AM - previous request cached, so single invocation only. + assertEquals(1, dagClientRpc.numGetStatusViaAmInvocations); + assertEquals(DAGStatus.State.SUCCEEDED, dagStatus.getState()); + + // verify that the cachedDAGStatus is correct + DAGStatus cachedDagStatus = dagClientImpl.getCachedDAGStatus(); + Assert.assertNotNull(cachedDagStatus); + Assert.assertSame(dagStatus, cachedDagStatus); + + // When AM proxy throws an exception, the cachedDAGStatus should be returned + dagClientImpl.resetCounters(); + dagClientRpc.resetCounters(); + dagClientRpc.injectAMFault(new IOException("injected Fault")); + dagStatus = dagClientImpl.getDAGStatus(EnumSet.noneOf(StatusGetOpts.class)); + // get the Status from the cache + assertEquals(0, dagClientImpl.numGetStatusViaRmInvocations); + // Directly from AM - previous request cached, so single invocation only. + assertEquals(1, dagClientRpc.numGetStatusViaAmInvocations); + assertEquals(DAGStatus.State.SUCCEEDED, dagStatus.getState()); + Assert.assertSame(dagStatus, cachedDagStatus); + + // test that RM is invoked when the cacheExpires and the AM fails. + dagClientRpc.setAMProxy(createMockProxy(DAGStatusStateProto.DAG_SUCCEEDED, 1000L)); + dagClientRpc.injectAMFault(new IOException("injected AM Fault")); + dagClientImpl.resetCounters(); + dagClientRpc.resetCounters(); + dagClientImpl.enforceExpirationCachedDAGStatus(); + dagStatus = dagClientImpl.getDAGStatus(EnumSet.noneOf(StatusGetOpts.class)); + // get the Status from the cache + assertEquals(1, dagClientImpl.numGetStatusViaRmInvocations); + assertEquals(1, dagClientRpc.numGetStatusViaAmInvocations); + assertEquals(State.RUNNING, dagStatus.getState()); + Assert.assertNotSame(dagStatus, cachedDagStatus); + + // verify that the cachedDAGStatus is null because AM threw exception before setting the + // cache. + cachedDagStatus = dagClientImpl.getCachedDAGStatus(); + Assert.assertNull(cachedDagStatus); + Assert.assertNotNull(dagStatus); + + // inject fault in RM too. getDAGStatus should return null; + dagClientImpl.resetCounters(); + dagClientRpc.resetCounters(); + dagClientRpc.setAMProxy(createMockProxy(DAGStatusStateProto.DAG_SUCCEEDED, 1000L)); + dagClientImpl.injectFault(); + try { + dagClientImpl.getDAGStatus(EnumSet.noneOf(StatusGetOpts.class)); + Assert.fail("The RM should throw IOException"); + } catch (IOException ioException) { + Assert.assertEquals(ioException.getMessage(), "Fault Injected for RM"); + assertEquals(1, dagClientImpl.numGetStatusViaRmInvocations); + assertEquals(1, dagClientRpc.numGetStatusViaAmInvocations); + } + } + } } From 56a7bf46cda95d9de2a3ef0c0e1ec73a68a7e95d Mon Sep 17 00:00:00 2001 From: dmmkr Date: Mon, 3 Jan 2022 13:12:00 +0530 Subject: [PATCH 079/137] TEZ-4354: Update netty to 4.1.61.Final (#164) (D M Murali Krishna Reddy reviewed by Laszlo Bodor) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 5a2c4fa45c..3f219eaa43 100644 --- a/pom.xml +++ b/pom.xml @@ -59,7 +59,7 @@ ${user.home}/clover.license 27.0-jre 3.2.2 - 4.0.52.Final + 4.1.61.Final 0.13.0 1.19 1.7.30 From 7e09dfa5081797b721f219472bd511b1633b55a7 Mon Sep 17 00:00:00 2001 From: Bodor Laszlo Date: Mon, 3 Jan 2022 10:57:11 +0100 Subject: [PATCH 080/137] TEZ-4364: TestFaultTolerance timeout on master (#171) (Laszlo Bodor reviewed by Rajesh Balamohan) --- .../runtime/api/events/InputReadErrorEvent.java | 15 ++++++++++----- .../test/java/org/apache/tez/test/TestInput.java | 7 ++++--- 2 files changed, 14 insertions(+), 8 deletions(-) diff --git a/tez-api/src/main/java/org/apache/tez/runtime/api/events/InputReadErrorEvent.java b/tez-api/src/main/java/org/apache/tez/runtime/api/events/InputReadErrorEvent.java index 1d0c44d6c1..6c9f1fbe6b 100644 --- a/tez-api/src/main/java/org/apache/tez/runtime/api/events/InputReadErrorEvent.java +++ b/tez-api/src/main/java/org/apache/tez/runtime/api/events/InputReadErrorEvent.java @@ -21,6 +21,8 @@ import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.tez.runtime.api.Event; +import com.google.common.annotations.VisibleForTesting; + import java.util.Objects; /** @@ -79,11 +81,14 @@ private InputReadErrorEvent(final String diagnostics, final int index, final int this.destinationLocalhostName = destinationLocalhostName; } - public static InputReadErrorEvent create(String diagnostics, int index, int version, - boolean isLocalFetch, boolean isDiskErrorAtSource) { - return create(diagnostics, index, version, 1, isLocalFetch, isDiskErrorAtSource, null); - } - + /** + * Creates an InputReadErrorEvent from less parameters, supposed to be used from tests only. + * @param diagnostics + * @param index + * @param version + * @return InputReadErrorEvent instance + */ + @VisibleForTesting public static InputReadErrorEvent create(String diagnostics, int index, int version) { return create(diagnostics, index, version, 1, false, false, null); } diff --git a/tez-tests/src/test/java/org/apache/tez/test/TestInput.java b/tez-tests/src/test/java/org/apache/tez/test/TestInput.java index 811ca3cc17..56bfe49dc8 100644 --- a/tez-tests/src/test/java/org/apache/tez/test/TestInput.java +++ b/tez-tests/src/test/java/org/apache/tez/test/TestInput.java @@ -175,7 +175,7 @@ public int doRead() { for (int i=0; i Date: Wed, 5 Jan 2022 21:40:06 +0100 Subject: [PATCH 081/137] TEZ-4351: ShuffleHandler port should respect value in config (#163) (Laszlo Bodor reviewed by Jonathan Eagles) --- .../tez/auxservices/ShuffleHandler.java | 7 ++-- .../tez/auxservices/TestShuffleHandler.java | 33 +++++++++++++++++++ 2 files changed, 38 insertions(+), 2 deletions(-) diff --git a/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java b/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java index fb28a0f4b1..a54a7cfe01 100644 --- a/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java +++ b/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java @@ -464,6 +464,10 @@ static Token deserializeServiceData(ByteBuffer secret) throw return jt; } + public int getPort() { + return port; + } + @Override public void initializeApplication(ApplicationInitializationContext context) { @@ -537,7 +541,7 @@ public Thread newThread(Runnable r) { return new Thread(r, WORKER_THREAD_NAME_PREFIX + workerThreadCounter.incrementAndGet()); } }); - + port = conf.getInt(SHUFFLE_PORT_CONFIG_KEY, DEFAULT_SHUFFLE_PORT); super.serviceInit(new YarnConfiguration(conf)); } @@ -556,7 +560,6 @@ protected void serviceStart() throws Exception { conf.getInt(SHUFFLE_LISTEN_QUEUE_SIZE, DEFAULT_SHUFFLE_LISTEN_QUEUE_SIZE)) .childOption(ChannelOption.SO_KEEPALIVE, true); initPipeline(bootstrap, conf); - port = conf.getInt(SHUFFLE_PORT_CONFIG_KEY, DEFAULT_SHUFFLE_PORT); Channel ch = bootstrap.bind().sync().channel(); accepted.add(ch); diff --git a/tez-plugins/tez-aux-services/src/test/java/org/apache/tez/auxservices/TestShuffleHandler.java b/tez-plugins/tez-aux-services/src/test/java/org/apache/tez/auxservices/TestShuffleHandler.java index 21addd393e..6a2e1cc68e 100644 --- a/tez-plugins/tez-aux-services/src/test/java/org/apache/tez/auxservices/TestShuffleHandler.java +++ b/tez-plugins/tez-aux-services/src/test/java/org/apache/tez/auxservices/TestShuffleHandler.java @@ -42,6 +42,7 @@ import java.util.ArrayList; import java.util.List; import java.util.Map; +import java.util.Random; import java.util.concurrent.atomic.AtomicBoolean; import java.util.zip.Checksum; @@ -1432,4 +1433,36 @@ public FullHttpRequest createHttpRequest() { } return new DefaultFullHttpRequest(HttpVersion.HTTP_1_1, HttpMethod.GET, uri); } + + @Test + public void testConfigPortStatic() throws Exception { + Random rand = new Random(); + int port = rand.nextInt(10) + 50000; + Configuration conf = new Configuration(); + // provide a port for ShuffleHandler + conf.setInt(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY, port); + MockShuffleHandler2 shuffleHandler = new MockShuffleHandler2(); + shuffleHandler.serviceInit(conf); + try { + shuffleHandler.serviceStart(); + Assert.assertEquals(port, shuffleHandler.getPort()); + } finally { + shuffleHandler.stop(); + } + } + + @Test + public void testConfigPortDynamic() throws Exception { + Configuration conf = new Configuration(); + // 0 as config, should be dynamically chosen by netty + conf.setInt(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY, 0); + MockShuffleHandler2 shuffleHandler = new MockShuffleHandler2(); + shuffleHandler.serviceInit(conf); + try { + shuffleHandler.serviceStart(); + Assert.assertTrue("ShuffleHandler should use a random chosen port", shuffleHandler.getPort() > 0); + } finally { + shuffleHandler.stop(); + } + } } From 4953f2983b315bbb008de434017d6ff41121260d Mon Sep 17 00:00:00 2001 From: jteagles Date: Thu, 6 Jan 2022 14:09:29 -0600 Subject: [PATCH 082/137] TEZ-4167. Speed up TestPipelinedSorter (#174) (Jonathan Eagles reviewed by Laszlo Bodor) --- .../common/sort/impl/TestPipelinedSorter.java | 45 ++++++++++++------- 1 file changed, 29 insertions(+), 16 deletions(-) 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 eed423b494..bb0f22fabd 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 @@ -60,6 +60,7 @@ import java.nio.ByteBuffer; import java.util.List; import java.util.Map; +import java.util.Random; import java.util.TreeMap; import java.util.UUID; @@ -83,7 +84,7 @@ public class TestPipelinedSorter { private long initialAvailableMem; //TODO: Need to make it nested structure so that multiple partition cases can be validated - private static TreeMap sortedDataMap = Maps.newTreeMap(); + private static TreeMap sortedDataMap = Maps.newTreeMap(); static { conf = getConf(); @@ -541,10 +542,10 @@ private void writeData2(ExternalSorter sorter, int counter = 0; for (int numkey : numKeys) { int curKeyLen = keyLen[counter]; + char[] buffer = new char[curKeyLen]; for (int i = 0; i < numkey; i++) { - Text key = new Text(RandomStringUtils.randomAlphanumeric(curKeyLen)); - Text value = new Text(RandomStringUtils.randomAlphanumeric(curKeyLen)); - sorter.write(key, value); + Text random = new Text(randomAlphanumeric(buffer)); + sorter.write(random, random); } counter++; } @@ -876,29 +877,41 @@ private void writeData(ExternalSorter sorter, int numKeys, int keyLen) throws IO private void writeSimilarKeys(ExternalSorter sorter, int numKeys, int keyLen, boolean autoClose) throws IOException { sortedDataMap.clear(); - String keyStr = RandomStringUtils.randomAlphanumeric(keyLen); + char[] buffer = new char[keyLen]; + String keyStr = randomAlphanumeric(buffer); for (int i = 0; i < numKeys; i++) { if (i % 4 == 0) { - keyStr = RandomStringUtils.randomAlphanumeric(keyLen); + keyStr = randomAlphanumeric(buffer); } Text key = new Text(keyStr); Text value = new Text(RandomStringUtils.randomAlphanumeric(keyLen)); sorter.write(key, value); - sortedDataMap.put(key.toString(), value.toString()); //for verifying data later + sortedDataMap.put(key, value); //for verifying data later } if (autoClose) { closeSorter(sorter); } } + static private final Random RANDOM = new Random(); + int start = ' '; + int end = 'z' + 1; + int gap = end - start; + private String randomAlphanumeric(char[] buffer) { + for (int i = 0; i < buffer.length; ++i) { + buffer[i] = (char)(RANDOM.nextInt(gap) + start); + } + return new String(buffer); + } private void writeData(ExternalSorter sorter, int numKeys, int keyLen, boolean autoClose) throws IOException { + char[] buffer = new char[keyLen]; sortedDataMap.clear(); for (int i = 0; i < numKeys; i++) { - Text key = new Text(RandomStringUtils.randomAlphanumeric(keyLen)); - Text value = new Text(RandomStringUtils.randomAlphanumeric(keyLen)); - sorter.write(key, value); - sortedDataMap.put(key.toString(), value.toString()); //for verifying data later + String randomStr = randomAlphanumeric(buffer); + Text random = new Text(randomStr); + sorter.write(random, random); + sortedDataMap.put(random, random); //for verifying data later } if (autoClose) { closeSorter(sorter); @@ -926,15 +939,15 @@ private void verifyData(IFile.Reader reader) int numRecordsRead = 0; - for (Map.Entry entry : sortedDataMap.entrySet()) { - String key = entry.getKey(); - String val = entry.getValue(); + for (Map.Entry entry : sortedDataMap.entrySet()) { + Text key = entry.getKey(); + Text val = entry.getValue(); if (reader.nextRawKey(keyIn)) { reader.nextRawValue(valIn); readKey = keyDeserializer.deserialize(readKey); readValue = valDeserializer.deserialize(readValue); - Assert.assertTrue(key.equalsIgnoreCase(readKey.toString())); - Assert.assertTrue(val.equalsIgnoreCase(readValue.toString())); + Assert.assertTrue(key.equals(readKey)); + Assert.assertTrue(val.equals(readValue)); numRecordsRead++; } } From a6a936dad34397226adcb672f25184169ecbcb71 Mon Sep 17 00:00:00 2001 From: Bodor Laszlo Date: Thu, 6 Jan 2022 21:30:58 +0100 Subject: [PATCH 083/137] TEZ-4311: Bump hadoop dependency version to 3.3.x (#132) (Laszlo Bodor reviewed by Jonathan Eagles) --- pom.xml | 11 +++- .../tez/dag/api/client/rpc/TestDAGClient.java | 6 ++- .../ats/acls/TestATSHistoryWithACLs.java | 54 +++++++++++++++++-- tez-runtime-library/pom.xml | 4 ++ .../apache/tez/test/TestSecureShuffle.java | 5 +- 5 files changed, 71 insertions(+), 9 deletions(-) diff --git a/pom.xml b/pom.xml index 3f219eaa43..5cf77deedf 100644 --- a/pom.xml +++ b/pom.xml @@ -58,7 +58,7 @@ ${user.home}/clover.license 27.0-jre - 3.2.2 + 3.3.1 4.1.61.Final 0.13.0 1.19 @@ -244,6 +244,12 @@ org.asynchttpclient async-http-client 2.12.1 + + + io.netty + * + + org.slf4j @@ -337,9 +343,10 @@ commons-el commons-el + io.netty - netty + * org.apache.httpcomponents diff --git a/tez-api/src/test/java/org/apache/tez/dag/api/client/rpc/TestDAGClient.java b/tez-api/src/test/java/org/apache/tez/dag/api/client/rpc/TestDAGClient.java index c1ebaa8db1..f0cc20b062 100644 --- a/tez-api/src/test/java/org/apache/tez/dag/api/client/rpc/TestDAGClient.java +++ b/tez-api/src/test/java/org/apache/tez/dag/api/client/rpc/TestDAGClient.java @@ -587,7 +587,11 @@ public void testTimelineClientCleanup() throws Exception { threadGroup.enumerate(threads); Thread reloaderThread = null; for (Thread thread : threads) { - if ((thread.getName() != null) && (thread.getName().contains("Truststore reloader thread"))) { + /* Since HADOOP-16524, the reloader thread's name is changed, let's handle the backward compatibility + * with a simple OR, as this is just a unit test, it's not worth involving a hadoop version check. + */ + if ((thread.getName() != null) && (thread.getName().contains("Truststore reloader thread")) + || (thread.getName().contains("SSL Certificates Store Monitor"))) { reloaderThread = thread; } } diff --git a/tez-plugins/tez-yarn-timeline-history-with-acls/src/test/java/org/apache/tez/dag/history/ats/acls/TestATSHistoryWithACLs.java b/tez-plugins/tez-yarn-timeline-history-with-acls/src/test/java/org/apache/tez/dag/history/ats/acls/TestATSHistoryWithACLs.java index 4b5702ef7e..f79a78b9fa 100644 --- a/tez-plugins/tez-yarn-timeline-history-with-acls/src/test/java/org/apache/tez/dag/history/ats/acls/TestATSHistoryWithACLs.java +++ b/tez-plugins/tez-yarn-timeline-history-with-acls/src/test/java/org/apache/tez/dag/history/ats/acls/TestATSHistoryWithACLs.java @@ -25,8 +25,10 @@ import static org.mockito.Mockito.*; import java.io.IOException; +import java.util.ArrayList; import java.util.Collection; import java.util.Collections; +import java.util.List; import java.util.Random; import javax.ws.rs.core.MediaType; @@ -42,6 +44,7 @@ import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.api.records.timeline.TimelineDomain; import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity; +import org.apache.hadoop.yarn.api.records.timeline.TimelineEvent; import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.tez.client.TezClient; import org.apache.tez.common.ReflectionUtils; @@ -63,6 +66,9 @@ import org.apache.tez.runtime.library.processor.SleepProcessor; import org.apache.tez.runtime.library.processor.SleepProcessor.SleepProcessorConfig; import org.apache.tez.tests.MiniTezClusterWithTimeline; +import org.codehaus.jettison.json.JSONArray; +import org.codehaus.jettison.json.JSONException; +import org.codehaus.jettison.json.JSONObject; import org.junit.AfterClass; import org.junit.Assert; import org.junit.BeforeClass; @@ -149,9 +155,48 @@ private K getTimelineData(String url, Class clazz) { assertEquals(200, response.getStatus()); assertTrue(MediaType.APPLICATION_JSON_TYPE.isCompatible(response.getType())); - K entity = response.getEntity(clazz); - assertNotNull(entity); - return entity; + JSONObject entity = response.getEntity(JSONObject.class); + K converted = null; + try { + converted = convertJSONObjectToTimelineObject(entity, clazz); + } catch (JSONException e) { + throw new RuntimeException(e); + } + assertNotNull(converted); + return converted; + } + + private K convertJSONObjectToTimelineObject(JSONObject jsonObj, Class clazz) throws JSONException { + LOG.info("convertJSONObjectToEntity got object: " + jsonObj); + if (clazz == TimelineDomain.class) { + TimelineDomain domain = new TimelineDomain(); + domain.setId(jsonObj.getString("id")); + domain.setOwner(jsonObj.getString("owner")); + domain.setReaders(jsonObj.getString("readers")); + domain.setWriters(jsonObj.getString("writers")); + return (K) domain; + } else if (clazz == TimelineEntity.class) { + TimelineEntity entity = new TimelineEntity(); + entity.setEntityId(jsonObj.getString("entity")); + entity.setEntityType(jsonObj.getString("entitytype")); + entity.setDomainId(jsonObj.getString("domain")); + entity.setEvents(getEventsFromJSON(jsonObj)); + return (K) entity; + } else { + throw new RuntimeException( + "convertJSONObjectToTimelineObject doesn't support conversion from JSONObject to " + clazz); + } + } + + private List getEventsFromJSON(JSONObject jsonObj) throws JSONException { + List events = new ArrayList<>(); + JSONArray arrEvents = jsonObj.getJSONArray("events"); + for (int i = 0; i < arrEvents.length(); i++) { + TimelineEvent event = new TimelineEvent(); + event.setEventType(((JSONObject) arrEvents.get(i)).getString("eventtype")); + events.add(event); + } + return events; } private TimelineDomain getDomain(String domainId) { @@ -459,7 +504,8 @@ public void testDagLoggingEnabled() throws Exception { .get(ClientResponse.class); assertEquals(200, response.getStatus()); assertTrue(MediaType.APPLICATION_JSON_TYPE.isCompatible(response.getType())); - TimelineEntity entity = response.getEntity(TimelineEntity.class); + JSONObject entityJson = response.getEntity(JSONObject.class); + TimelineEntity entity = convertJSONObjectToTimelineObject(entityJson, TimelineEntity.class); assertEquals(entity.getEntityType(), "TEZ_DAG_ID"); assertEquals(entity.getEvents().get(0).getEventType(), HistoryEventType.DAG_SUBMITTED.toString()); } diff --git a/tez-runtime-library/pom.xml b/tez-runtime-library/pom.xml index 2caff344ec..e43f90f415 100644 --- a/tez-runtime-library/pom.xml +++ b/tez-runtime-library/pom.xml @@ -33,6 +33,10 @@ org.asynchttpclient async-http-client + + io.netty + netty-all + org.apache.tez tez-api 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 2b2221230f..8e298506b8 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 @@ -114,6 +114,7 @@ public static Collection getParameters() { public static void setupDFSCluster() throws Exception { conf = new Configuration(); conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_EDITS_NOEDITLOGCHANNELFLUSH, false); + conf.setBoolean("fs.hdfs.impl.disable.cache", true); EditLogFileOutputStream.setShouldSkipFsyncForTesting(true); conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, TEST_ROOT_DIR); miniDFSCluster = @@ -301,8 +302,8 @@ public static X509Certificate generateCertificate(String dn, KeyPair pair, int d String hostAddress = InetAddress.getLocalHost().getHostAddress(); certGen.addExtension(X509Extensions.SubjectAlternativeName, false, - new GeneralNames(new GeneralName(GeneralName.iPAddress, hostAddress))); - + new GeneralNames(new GeneralName[] { new GeneralName(GeneralName.iPAddress, hostAddress), + new GeneralName(GeneralName.dNSName, "localhost") })); X500Principal dnName = new X500Principal(dn); certGen.setSerialNumber(sn); From 41cbc17aab3a2cc5fb177fc6a15dec867e795b72 Mon Sep 17 00:00:00 2001 From: belugabehr <12578579+belugabehr@users.noreply.github.com> Date: Wed, 12 Jan 2022 08:11:58 -0500 Subject: [PATCH 084/137] TEZ-4276: Clean Up Use of Preconditions (#96) (David Mollitor reviewed by Laszlo Bodor) --- .../tez/dag/api/NamedEntityDescriptor.java | 5 +- .../java/org/apache/tez/dag/api/Vertex.java | 5 +- .../dag/api/VertexManagerPluginContext.java | 5 +- .../api/events/VertexManagerEvent.java | 17 ++++--- .../tez/common/TezContainerLogAppender.java | 13 ++--- .../org/apache/tez/dag/records/TezDAGID.java | 10 ++-- .../tez/dag/records/TezTaskAttemptID.java | 6 +-- .../org/apache/tez/dag/records/TezTaskID.java | 6 +-- .../apache/tez/dag/records/TezVertexID.java | 5 +- .../dag/app/TaskCommunicatorContextImpl.java | 8 ++-- .../tez/dag/app/dag/impl/VertexManager.java | 1 - .../tez/dag/app/rm/node/ExtendedNodeId.java | 4 +- .../split/TezGroupedSplitsInputFormat.java | 7 ++- .../mapred/split/TezMapredSplitsGrouper.java | 4 +- .../split/TezGroupedSplitsInputFormat.java | 7 ++- .../split/TezMapReduceSplitsGrouper.java | 4 +- .../mapreduce/grouper/TezSplitGrouper.java | 3 +- .../apache/tez/mapreduce/input/MRInput.java | 3 +- .../tez/mapreduce/input/MultiMRInput.java | 3 +- .../org/apache/tez/history/ATSImportTool.java | 4 +- .../tez/history/parser/ATSFileParser.java | 47 +++++++++---------- .../parser/datamodel/TaskAttemptInfo.java | 4 +- .../history/parser/datamodel/TaskInfo.java | 4 +- .../history/parser/datamodel/VertexInfo.java | 4 +- .../tez/runtime/api/impl/TaskStatistics.java | 4 +- .../java/org/apache/tez/http/SSLFactory.java | 9 ++-- .../runtime/library/common/ConfigUtils.java | 37 +++++++-------- .../library/common/shuffle/ShuffleUtils.java | 3 +- .../conf/OrderedGroupedKVInputConfig.java | 14 ++---- .../OrderedPartitionedKVOutputConfig.java | 18 +++---- .../library/conf/UnorderedKVInputConfig.java | 11 ++--- .../library/conf/UnorderedKVOutputConfig.java | 11 ++--- .../UnorderedPartitionedKVOutputConfig.java | 11 ++--- .../org/apache/tez/analyzer/CSVResult.java | 3 +- .../tez/analyzer/plugins/SkewAnalyzer.java | 5 +- 35 files changed, 149 insertions(+), 156 deletions(-) diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/NamedEntityDescriptor.java b/tez-api/src/main/java/org/apache/tez/dag/api/NamedEntityDescriptor.java index 452deaa77b..86ae26c876 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/NamedEntityDescriptor.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/NamedEntityDescriptor.java @@ -17,8 +17,8 @@ import java.io.DataInput; import java.io.DataOutput; import java.io.IOException; +import java.util.Objects; -import org.apache.tez.common.Preconditions; import org.apache.hadoop.classification.InterfaceAudience; @SuppressWarnings("unchecked") @@ -28,8 +28,7 @@ public class NamedEntityDescriptor> extends E @InterfaceAudience.Private public NamedEntityDescriptor(String entityName, String className) { super(className); - Preconditions.checkArgument(entityName != null, "EntityName must be specified"); - this.entityName = entityName; + this.entityName = Objects.requireNonNull(entityName, "EntityName must be specified"); } public String getEntityName() { diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/Vertex.java b/tez-api/src/main/java/org/apache/tez/dag/api/Vertex.java index ccfa92a9f6..8ce3fb67e6 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/Vertex.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/Vertex.java @@ -23,6 +23,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Objects; import org.apache.commons.lang.StringUtils; import org.apache.hadoop.classification.InterfaceAudience; @@ -276,10 +277,10 @@ public Map getTaskLocalFiles() { * set environment for all vertices via Tezconfiguration#TEZ_TASK_LAUNCH_ENV * @param environment * @return this Vertex + * NullPointerException if {@code environment} is {@code null} */ public Vertex setTaskEnvironment(Map environment) { - Preconditions.checkArgument(environment != null); - this.taskEnvironment.putAll(environment); + this.taskEnvironment.putAll(Objects.requireNonNull(environment)); return this; } diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/VertexManagerPluginContext.java b/tez-api/src/main/java/org/apache/tez/dag/api/VertexManagerPluginContext.java index ba43ecd9c1..0862f49034 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/VertexManagerPluginContext.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/VertexManagerPluginContext.java @@ -21,6 +21,7 @@ import java.util.Collection; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Set; import javax.annotation.Nullable; @@ -29,7 +30,6 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable; import org.apache.hadoop.yarn.api.records.Resource; import org.apache.tez.dag.api.event.VertexState; -import org.apache.tez.runtime.api.Event; import org.apache.tez.runtime.api.InputSpecUpdate; import org.apache.tez.runtime.api.VertexStatistics; import org.apache.tez.runtime.api.events.CustomProcessorEvent; @@ -72,8 +72,7 @@ public class TaskWithLocationHint { Integer taskIndex; TaskLocationHint locationHint; public TaskWithLocationHint(Integer taskIndex, @Nullable TaskLocationHint locationHint) { - Preconditions.checkState(taskIndex != null); - this.taskIndex = taskIndex; + this.taskIndex = Objects.requireNonNull(taskIndex); this.locationHint = locationHint; } diff --git a/tez-api/src/main/java/org/apache/tez/runtime/api/events/VertexManagerEvent.java b/tez-api/src/main/java/org/apache/tez/runtime/api/events/VertexManagerEvent.java index 6e5738918c..7a450c6cdf 100644 --- a/tez-api/src/main/java/org/apache/tez/runtime/api/events/VertexManagerEvent.java +++ b/tez-api/src/main/java/org/apache/tez/runtime/api/events/VertexManagerEvent.java @@ -19,6 +19,7 @@ package org.apache.tez.runtime.api.events; import java.nio.ByteBuffer; +import java.util.Objects; import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.classification.InterfaceAudience.Public; @@ -27,8 +28,6 @@ import org.apache.tez.runtime.api.Event; import org.apache.tez.runtime.api.TaskAttemptIdentifier; -import org.apache.tez.common.Preconditions; - /** * Event used to send information from a Task to the VertexManager for a vertex. * This may be used to send statistics like samples etc to the VertexManager for @@ -50,11 +49,17 @@ public class VertexManagerEvent extends Event { */ private final ByteBuffer userPayload; + /** + * Constructor. + * + * @param vertexName + * @param userPayload + * @throws NullPointerException if {@code vertexName} or {@code userPayload} + * is {@code null} + */ private VertexManagerEvent(String vertexName, ByteBuffer userPayload) { - Preconditions.checkArgument(vertexName != null); - Preconditions.checkArgument(userPayload != null); - this.targetVertexName = vertexName; - this.userPayload = userPayload; + this.targetVertexName = Objects.requireNonNull(vertexName); + this.userPayload = Objects.requireNonNull(userPayload); } /** diff --git a/tez-common/src/main/java/org/apache/tez/common/TezContainerLogAppender.java b/tez-common/src/main/java/org/apache/tez/common/TezContainerLogAppender.java index 2cfacfb08e..1649a0b318 100644 --- a/tez-common/src/main/java/org/apache/tez/common/TezContainerLogAppender.java +++ b/tez-common/src/main/java/org/apache/tez/common/TezContainerLogAppender.java @@ -19,6 +19,7 @@ package org.apache.tez.common; import java.io.File; +import java.util.Objects; import org.apache.hadoop.classification.InterfaceStability.Unstable; import org.apache.log4j.FileAppender; @@ -48,14 +49,14 @@ public void activateOptions() { * The file will be created within the container's log directory. * * @param fileName + * @throws NullPointerException if {@code fileName} is {@code null} + * @throws IllegalArgumentException if {@code fileName} is an absolute path */ public void setLogFileName(String fileName) { - if (fileName == null || fileName.contains(File.pathSeparator)) { - throw new RuntimeException( - "Invalid filename specified: " - + fileName - + " . FileName should not have a path component and should not be empty."); - } + Objects.requireNonNull(fileName); + Preconditions.checkArgument(!fileName.contains(File.pathSeparator), + "Invalid filename specified: " + fileName + + " . FileName should not have a path component and should not be empty."); this.logFileName = fileName; } 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 68184fc8c0..c46aa6088b 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 @@ -21,10 +21,10 @@ import java.io.DataInput; import java.io.DataOutput; import java.io.IOException; +import java.util.Objects; import org.apache.hadoop.yarn.api.records.ApplicationId; -import org.apache.tez.common.Preconditions; import org.apache.tez.util.FastNumberFormat; import com.google.common.collect.Interner; @@ -49,12 +49,13 @@ public class TezDAGID extends TezID { * Get a DAGID object from given {@link ApplicationId}. * @param applicationId Application that this dag belongs to * @param id the dag number + * @throws NullPointerException if {@code obj} is {@code applicationId} */ public static TezDAGID getInstance(ApplicationId applicationId, int id) { // The newly created TezDAGIds are primarily for their hashCode method, and // will be short-lived. // Alternately the cache can be keyed by the hash of the incoming paramters. - Preconditions.checkArgument(applicationId != null, "ApplicationID cannot be null"); + Objects.requireNonNull(applicationId, "ApplicationID cannot be null"); return tezDAGIDCache.intern(new TezDAGID(applicationId, id)); } @@ -63,15 +64,16 @@ public static TezDAGID getInstance(ApplicationId applicationId, int id) { * @param yarnRMIdentifier YARN RM identifier * @param appId application number * @param id the dag number + * @throws NullPointerException if {@code yarnRMIdentifier} is {@code null} */ public static TezDAGID getInstance(String yarnRMIdentifier, int appId, int id) { // The newly created TezDAGIds are primarily for their hashCode method, and // will be short-lived. // Alternately the cache can be keyed by the hash of the incoming paramters. - Preconditions.checkArgument(yarnRMIdentifier != null, "yarnRMIdentifier cannot be null"); + Objects.requireNonNull(yarnRMIdentifier, "yarnRMIdentifier cannot be null"); return tezDAGIDCache.intern(new TezDAGID(yarnRMIdentifier, appId, id)); } - + // Public for Writable serialization. Verify if this is actually required. public TezDAGID() { } diff --git a/tez-common/src/main/java/org/apache/tez/dag/records/TezTaskAttemptID.java b/tez-common/src/main/java/org/apache/tez/dag/records/TezTaskAttemptID.java index 9ce1b10aa0..7c92be9449 100644 --- a/tez-common/src/main/java/org/apache/tez/dag/records/TezTaskAttemptID.java +++ b/tez-common/src/main/java/org/apache/tez/dag/records/TezTaskAttemptID.java @@ -21,6 +21,7 @@ import java.io.DataInput; import java.io.DataOutput; import java.io.IOException; +import java.util.Objects; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; @@ -59,16 +60,15 @@ public TezTaskAttemptID() { * Constructs a TaskAttemptID object from given {@link TezTaskID}. * @param taskID TaskID that this task belongs to * @param id the task attempt number + * @throws NullPointerException if {@code taskID} is {@code null} */ public static TezTaskAttemptID getInstance(TezTaskID taskID, int id) { + Objects.requireNonNull(taskID); return tezTaskAttemptIDCache.intern(new TezTaskAttemptID(taskID, id)); } private TezTaskAttemptID(TezTaskID taskId, int id) { super(id); - if(taskId == null) { - throw new IllegalArgumentException("taskId cannot be null"); - } this.taskId = taskId; } diff --git a/tez-common/src/main/java/org/apache/tez/dag/records/TezTaskID.java b/tez-common/src/main/java/org/apache/tez/dag/records/TezTaskID.java index 993df7c87a..a1a556c9ba 100644 --- a/tez-common/src/main/java/org/apache/tez/dag/records/TezTaskID.java +++ b/tez-common/src/main/java/org/apache/tez/dag/records/TezTaskID.java @@ -21,11 +21,11 @@ import java.io.DataInput; import java.io.DataOutput; import java.io.IOException; +import java.util.Objects; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; -import org.apache.tez.common.Preconditions; import org.apache.tez.util.FastNumberFormat; import com.google.common.collect.Interner; @@ -61,15 +61,15 @@ public FastNumberFormat initialValue() { * Constructs a TezTaskID object from given {@link TezVertexID}. * @param vertexID the vertexID object for this TezTaskID * @param id the tip number + * @throws NullPointerException if {@code vertexID} is {@code null} */ public static TezTaskID getInstance(TezVertexID vertexID, int id) { - Preconditions.checkArgument(vertexID != null, "vertexID cannot be null"); + Objects.requireNonNull(vertexID, "vertexID cannot be null"); return tezTaskIDCache.intern(new TezTaskID(vertexID, id)); } private TezTaskID(TezVertexID vertexID, int id) { super(id); - Preconditions.checkArgument(vertexID != null, "vertexID cannot be null"); this.vertexId = vertexID; this.serializingHash = getHashCode(true); } diff --git a/tez-common/src/main/java/org/apache/tez/dag/records/TezVertexID.java b/tez-common/src/main/java/org/apache/tez/dag/records/TezVertexID.java index 5ecfca6a49..e9ddd77d38 100644 --- a/tez-common/src/main/java/org/apache/tez/dag/records/TezVertexID.java +++ b/tez-common/src/main/java/org/apache/tez/dag/records/TezVertexID.java @@ -21,11 +21,11 @@ import java.io.DataInput; import java.io.DataOutput; import java.io.IOException; +import java.util.Objects; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; -import org.apache.tez.common.Preconditions; import org.apache.tez.util.FastNumberFormat; import com.google.common.collect.Interner; @@ -67,9 +67,10 @@ public TezVertexID() { * Constructs a TezVertexID object from given {@link TezDAGID}. * @param dagId TezDAGID object for this TezVertexID * @param id the tip number + * @throws NullPointerException if {@code dagId} is {@code null} */ public static TezVertexID getInstance(TezDAGID dagId, int id) { - Preconditions.checkArgument(dagId != null, "DagID cannot be null"); + Objects.requireNonNull(dagId, "DagID cannot be null"); return tezVertexIDCache.intern(new TezVertexID(dagId, id)); } 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 faa6fe1967..4f9e8723aa 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 @@ -191,7 +191,7 @@ public String apply(Vertex input) { @Override public int getVertexTotalTaskCount(String vertexName) { - Preconditions.checkArgument(vertexName != null, "VertexName must be specified"); + Objects.requireNonNull(vertexName, "VertexName must be specified"); DAG dag = getDag(); Vertex vertex = dag.getVertex(vertexName); return vertex.getTotalTasks(); @@ -199,7 +199,7 @@ public int getVertexTotalTaskCount(String vertexName) { @Override public int getVertexCompletedTaskCount(String vertexName) { - Preconditions.checkArgument(vertexName != null, "VertexName must be specified"); + Objects.requireNonNull(vertexName, "VertexName must be specified"); DAG dag = getDag(); Vertex vertex = dag.getVertex(vertexName); return vertex.getCompletedTasks(); @@ -207,7 +207,7 @@ public int getVertexCompletedTaskCount(String vertexName) { @Override public int getVertexRunningTaskCount(String vertexName) { - Preconditions.checkArgument(vertexName != null, "VertexName must be specified"); + Objects.requireNonNull(vertexName, "VertexName must be specified"); DAG dag = getDag(); Vertex vertex = dag.getVertex(vertexName); return vertex.getRunningTasks(); @@ -215,7 +215,7 @@ public int getVertexRunningTaskCount(String vertexName) { @Override public long getFirstAttemptStartTime(String vertexName, int taskIndex) { - Preconditions.checkArgument(vertexName != null, "VertexName must be specified"); + Objects.requireNonNull(vertexName, "VertexName must be specified"); Preconditions.checkArgument(taskIndex >=0, "TaskIndex must be > 0"); DAG dag = getDag(); Vertex vertex = dag.getVertex(vertexName); 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 292742530a..57c8c72be1 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 @@ -425,7 +425,6 @@ public VertexManager(VertexManagerPluginDescriptor pluginDesc, UserGroupInformat this.rootInputInitEventQueue = new LinkedBlockingQueue(); pluginContext = new VertexManagerPluginContextImpl(); - Preconditions.checkArgument(pluginDesc != null); payload = pluginDesc.getUserPayload(); pluginFailed = new AtomicBoolean(false); plugin = ReflectionUtils.createClazzInstance(pluginDesc.getClassName(), diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/ExtendedNodeId.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/ExtendedNodeId.java index ea58d8624c..07b2dd4252 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/ExtendedNodeId.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/node/ExtendedNodeId.java @@ -21,7 +21,6 @@ import java.util.Objects; import org.apache.hadoop.yarn.api.records.NodeId; -import org.apache.tez.common.Preconditions; /** * ExtendedNodeId extends NodeId with unique identifier in addition to hostname and port. @@ -33,8 +32,7 @@ public class ExtendedNodeId extends NodeId { private final String uniqueIdentifier; public ExtendedNodeId(NodeId nodeId, String uniqueIdentifier) { - Preconditions.checkArgument(nodeId != null); - this.nodeId = nodeId; + this.nodeId = Objects.requireNonNull(nodeId); this.uniqueIdentifier = uniqueIdentifier == null ? "" : uniqueIdentifier.trim(); } diff --git a/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/split/TezGroupedSplitsInputFormat.java b/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/split/TezGroupedSplitsInputFormat.java index 61e1f6c431..61ba560300 100644 --- a/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/split/TezGroupedSplitsInputFormat.java +++ b/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/split/TezGroupedSplitsInputFormat.java @@ -19,6 +19,7 @@ package org.apache.hadoop.mapred.split; import java.io.IOException; +import java.util.Objects; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -66,14 +67,12 @@ public void setInputFormat(InputFormat wrappedInputFormat) { } public void setSplitSizeEstimator(SplitSizeEstimator estimator) { - Preconditions.checkArgument(estimator != null); - this.estimator = estimator; + this.estimator = Objects.requireNonNull(estimator); LOG.debug("Split size estimator : {}", estimator); } public void setSplitLocationProvider(SplitLocationProvider locationProvider) { - Preconditions.checkArgument(locationProvider != null); - this.locationProvider = locationProvider; + this.locationProvider = Objects.requireNonNull(locationProvider); LOG.debug("Split size location provider: {}", locationProvider); } 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 ccaccd627f..38a213ec9a 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 @@ -21,9 +21,9 @@ import java.io.IOException; import java.util.Arrays; import java.util.List; +import java.util.Objects; import com.google.common.base.Function; -import org.apache.tez.common.Preconditions; import com.google.common.collect.Lists; import org.apache.tez.mapreduce.grouper.GroupedSplitContainer; import org.apache.tez.mapreduce.grouper.MapredSplitContainer; @@ -66,7 +66,7 @@ public InputSplit[] getGroupedSplits(Configuration conf, public InputSplit[] getGroupedSplits(Configuration conf, InputSplit[] originalSplits, int desiredNumSplits, String wrappedInputFormatName, SplitSizeEstimator estimator, SplitLocationProvider locationProvider) throws IOException { - Preconditions.checkArgument(originalSplits != null, "Splits must be specified"); + Objects.requireNonNull(originalSplits, "Splits must be specified"); List originalSplitContainers = Lists.transform(Arrays.asList(originalSplits), new Function() { diff --git a/tez-mapreduce/src/main/java/org/apache/hadoop/mapreduce/split/TezGroupedSplitsInputFormat.java b/tez-mapreduce/src/main/java/org/apache/hadoop/mapreduce/split/TezGroupedSplitsInputFormat.java index 863f9aa792..b85e97e467 100644 --- a/tez-mapreduce/src/main/java/org/apache/hadoop/mapreduce/split/TezGroupedSplitsInputFormat.java +++ b/tez-mapreduce/src/main/java/org/apache/hadoop/mapreduce/split/TezGroupedSplitsInputFormat.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.util.ArrayList; import java.util.List; +import java.util.Objects; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -74,14 +75,12 @@ public void setDesiredNumberOfSplits(int num) { } public void setSplitSizeEstimator(SplitSizeEstimator estimator) { - Preconditions.checkArgument(estimator != null); - this.estimator = estimator; + this.estimator = Objects.requireNonNull(estimator); LOG.debug("Split size estimator : {}", estimator); } public void setSplitLocationProvider(SplitLocationProvider locationProvider) { - Preconditions.checkArgument(locationProvider != null); - this.locationProvider = locationProvider; + this.locationProvider = Objects.requireNonNull(locationProvider); LOG.debug("Split location provider : {}", locationProvider); } 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 629f01e8e5..4f638e1400 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 @@ -20,11 +20,11 @@ import java.io.IOException; import java.util.List; +import java.util.Objects; import javax.annotation.Nullable; import com.google.common.base.Function; -import org.apache.tez.common.Preconditions; import com.google.common.collect.Lists; import org.apache.tez.mapreduce.grouper.GroupedSplitContainer; import org.apache.tez.mapreduce.grouper.MapReduceSplitContainer; @@ -154,7 +154,7 @@ public List getGroupedSplits(Configuration conf, SplitSizeEstimator estimator, SplitLocationProvider locationProvider) throws IOException, InterruptedException { - Preconditions.checkArgument(originalSplits != null, "Splits must be specified"); + Objects.requireNonNull(originalSplits, "Splits must be specified"); List originalSplitContainers = Lists.transform(originalSplits, new Function() { @Override diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/grouper/TezSplitGrouper.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/grouper/TezSplitGrouper.java index 6f3d9df033..a1d6b6c806 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/grouper/TezSplitGrouper.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/grouper/TezSplitGrouper.java @@ -22,6 +22,7 @@ import java.util.LinkedList; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Set; import java.util.TreeMap; @@ -166,7 +167,7 @@ public List getGroupedSplits(Configuration conf, SplitLocationProviderWrapper locationProvider) throws IOException, InterruptedException { LOG.info("Grouping splits in Tez"); - Preconditions.checkArgument(originalSplits != null, "Splits must be specified"); + Objects.requireNonNull(originalSplits, "Splits must be specified"); int configNumSplits = conf.getInt(TEZ_GROUPING_SPLIT_COUNT, 0); if (configNumSplits > 0) { diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/MRInput.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/MRInput.java index 891249b63e..964c0b8301 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/MRInput.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/MRInput.java @@ -23,6 +23,7 @@ import java.net.URI; import java.util.Collection; import java.util.List; +import java.util.Objects; import java.util.concurrent.locks.Condition; import java.util.concurrent.locks.ReentrantLock; @@ -672,7 +673,7 @@ private void initFromEventInternal(InputDataInformationEvent initEvent) throws I if (LOG.isDebugEnabled()) { LOG.debug(getContext().getSourceVertexName() + " initializing RecordReader from event"); } - Preconditions.checkState(initEvent != null, "InitEvent must be specified"); + Objects.requireNonNull(initEvent, "InitEvent must be specified"); MRSplitProto splitProto = MRSplitProto.parseFrom(ByteString.copyFrom(initEvent.getUserPayload())); Object splitObj = null; long splitLength = -1; diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/MultiMRInput.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/MultiMRInput.java index ee776a59f2..3bc8629565 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/MultiMRInput.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/MultiMRInput.java @@ -25,6 +25,7 @@ import java.util.Collections; import java.util.LinkedList; import java.util.List; +import java.util.Objects; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.locks.Condition; import java.util.concurrent.locks.ReentrantLock; @@ -164,7 +165,7 @@ public void handleEvents(List inputEvents) throws Exception { } private MRReader initFromEvent(InputDataInformationEvent event) throws IOException { - Preconditions.checkState(event != null, "Event must be specified"); + Objects.requireNonNull(event, "Event must be specified"); if (LOG.isDebugEnabled()) { LOG.debug(getContext().getSourceVertexName() + " initializing Reader: " + eventCount.get()); } diff --git a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/ATSImportTool.java b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/ATSImportTool.java index f05e946acc..c909f7aa0d 100644 --- a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/ATSImportTool.java +++ b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/ATSImportTool.java @@ -69,6 +69,7 @@ import java.net.URL; import java.net.URLEncoder; import java.util.Iterator; +import java.util.Objects; import java.util.zip.ZipEntry; import java.util.zip.ZipOutputStream; @@ -242,11 +243,12 @@ private void downloadData(ZipOutputStream zos) throws TezException, JSONExceptio * @throws IOException * @throws TezException * @throws JSONException + * @throws NullPointerException if {@code zos} is {@code null} */ private void downloadJSONArrayFromATS(String url, ZipOutputStream zos, String tag) throws IOException, TezException, JSONException { - Preconditions.checkArgument(zos != null, "ZipOutputStream can not be null"); + Objects.requireNonNull(zos, "ZipOutputStream can not be null"); String baseUrl = url; JSONArray entities; diff --git a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/ATSFileParser.java b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/ATSFileParser.java index e64fb43f94..caeb406f21 100644 --- a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/ATSFileParser.java +++ b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/ATSFileParser.java @@ -42,6 +42,7 @@ import java.util.Enumeration; import java.util.Iterator; import java.util.List; +import java.util.Objects; import java.util.zip.ZipEntry; import java.util.zip.ZipFile; @@ -89,18 +90,17 @@ public DagInfo getDAGData(String dagId) throws TezException { * * @param verticesJson * @throws JSONException + * @throws NullPointerException if {@code verticesJson} is {@code null} */ private void processVertices(JSONArray verticesJson) throws JSONException { //Process vertex information - Preconditions.checkState(verticesJson != null, "Vertex json can not be null"); - if (verticesJson != null) { - LOG.debug("Started parsing vertex"); - for (int i = 0; i < verticesJson.length(); i++) { - VertexInfo vertexInfo = VertexInfo.create(verticesJson.getJSONObject(i)); - vertexList.add(vertexInfo); - } - LOG.debug("Finished parsing vertex"); + Objects.requireNonNull(verticesJson, "Vertex json cannot be null"); + LOG.debug("Started parsing vertex"); + for (int i = 0; i < verticesJson.length(); i++) { + VertexInfo vertexInfo = VertexInfo.create(verticesJson.getJSONObject(i)); + vertexList.add(vertexInfo); } + LOG.debug("Finished parsing vertex"); } /** @@ -108,18 +108,17 @@ private void processVertices(JSONArray verticesJson) throws JSONException { * * @param tasksJson * @throws JSONException + * @throws NullPointerException if {@code verticesJson} is {@code null} */ private void processTasks(JSONArray tasksJson) throws JSONException { //Process Task information - Preconditions.checkState(tasksJson != null, "Task json can not be null"); - if (tasksJson != null) { - LOG.debug("Started parsing task"); - for (int i = 0; i < tasksJson.length(); i++) { - TaskInfo taskInfo = TaskInfo.create(tasksJson.getJSONObject(i)); - taskList.add(taskInfo); - } - LOG.debug("Finished parsing task"); + Objects.requireNonNull(tasksJson, "Task json can not be null"); + LOG.debug("Started parsing task"); + for (int i = 0; i < tasksJson.length(); i++) { + TaskInfo taskInfo = TaskInfo.create(tasksJson.getJSONObject(i)); + taskList.add(taskInfo); } + LOG.debug("Finished parsing task"); } /** @@ -127,18 +126,18 @@ private void processTasks(JSONArray tasksJson) throws JSONException { * * @param taskAttemptsJson * @throws JSONException + * @throws NullPointerException if {@code taskAttemptsJson} is {@code null} */ private void processAttempts(JSONArray taskAttemptsJson) throws JSONException { //Process TaskAttempt information - Preconditions.checkState(taskAttemptsJson != null, "Attempts json can not be null"); - if (taskAttemptsJson != null) { - LOG.debug("Started parsing task attempts"); - for (int i = 0; i < taskAttemptsJson.length(); i++) { - TaskAttemptInfo attemptInfo = TaskAttemptInfo.create(taskAttemptsJson.getJSONObject(i)); - attemptList.add(attemptInfo); - } - LOG.debug("Finished parsing task attempts"); + Objects.requireNonNull(taskAttemptsJson, "Attempts json can not be null"); + LOG.debug("Started parsing task attempts"); + for (int i = 0; i < taskAttemptsJson.length(); i++) { + TaskAttemptInfo attemptInfo = + TaskAttemptInfo.create(taskAttemptsJson.getJSONObject(i)); + attemptList.add(attemptInfo); } + LOG.debug("Finished parsing task attempts"); } /** diff --git a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/TaskAttemptInfo.java b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/TaskAttemptInfo.java index c8d400cc35..c45fcf5ff3 100644 --- a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/TaskAttemptInfo.java +++ b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/TaskAttemptInfo.java @@ -41,6 +41,7 @@ import java.util.Comparator; import java.util.List; import java.util.Map; +import java.util.Objects; import static org.apache.hadoop.classification.InterfaceStability.Evolving; import static org.apache.hadoop.classification.InterfaceAudience.Public; @@ -169,8 +170,7 @@ public int compare(TaskAttemptInfo o1, TaskAttemptInfo o2) { } void setTaskInfo(TaskInfo taskInfo) { - Preconditions.checkArgument(taskInfo != null, "Provide valid taskInfo"); - this.taskInfo = taskInfo; + this.taskInfo = Objects.requireNonNull(taskInfo, "Provide valid taskInfo"); taskInfo.addTaskAttemptInfo(this); } diff --git a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/TaskInfo.java b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/TaskInfo.java index 43886fa5c6..fd01093244 100644 --- a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/TaskInfo.java +++ b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/TaskInfo.java @@ -41,6 +41,7 @@ import java.util.Comparator; import java.util.List; import java.util.Map; +import java.util.Objects; import static org.apache.hadoop.classification.InterfaceAudience.Public; import static org.apache.hadoop.classification.InterfaceStability.Evolving; @@ -153,8 +154,7 @@ void addTaskAttemptInfo(TaskAttemptInfo taskAttemptInfo) { } void setVertexInfo(VertexInfo vertexInfo) { - Preconditions.checkArgument(vertexInfo != null, "Provide valid vertexInfo"); - this.vertexInfo = vertexInfo; + this.vertexInfo = Objects.requireNonNull(vertexInfo, "Provide valid vertexInfo"); //link it to vertex vertexInfo.addTaskInfo(this); } diff --git a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/VertexInfo.java b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/VertexInfo.java index c196c48cd4..106f2bc729 100644 --- a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/VertexInfo.java +++ b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/VertexInfo.java @@ -41,6 +41,7 @@ import java.util.Comparator; import java.util.List; import java.util.Map; +import java.util.Objects; import static org.apache.hadoop.classification.InterfaceAudience.Public; import static org.apache.hadoop.classification.InterfaceStability.Evolving; @@ -198,8 +199,7 @@ void addOutEdge(EdgeInfo edgeInfo) { } void setDagInfo(DagInfo dagInfo) { - Preconditions.checkArgument(dagInfo != null, "Provide valid dagInfo"); - this.dagInfo = dagInfo; + this.dagInfo = Objects.requireNonNull(dagInfo, "Provide valid dagInfo"); //link vertex to dagInfo dagInfo.addVertexInfo(this); updateEdgeInfo(); diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TaskStatistics.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TaskStatistics.java index d9825c9f43..961c28ef60 100644 --- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TaskStatistics.java +++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TaskStatistics.java @@ -21,11 +21,11 @@ import java.io.DataOutput; import java.io.IOException; import java.util.Map; +import java.util.Objects; import org.apache.hadoop.io.Text; import org.apache.hadoop.io.Writable; -import org.apache.tez.common.Preconditions; import org.apache.tez.util.StringInterner; import com.google.common.collect.Maps; @@ -40,7 +40,7 @@ public void addIO(String edgeName) { } public void addIO(String edgeName, IOStatistics stats) { - Preconditions.checkArgument(stats != null, edgeName); + Objects.requireNonNull(stats, edgeName); ioStatistics.put(StringInterner.intern(edgeName), stats); } 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 4147be8980..8c98764bfe 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 @@ -41,6 +41,7 @@ import java.io.IOException; import java.net.HttpURLConnection; import java.security.GeneralSecurityException; +import java.util.Objects; import static org.apache.hadoop.security.ssl.SSLFactory.KEYSTORES_FACTORY_CLASS_KEY; import static org.apache.hadoop.security.ssl.SSLFactory.SSL_CLIENT_CONF_KEY; @@ -81,13 +82,11 @@ public class SSLFactory implements ConnectionConfigurator { * @param mode SSLFactory mode, client or server. * @param conf Hadoop configuration from where the SSLFactory configuration * will be read. + * @throws NullPointerException if {@code mode} or {@code conf} is {@code null} */ public SSLFactory(Mode mode, Configuration conf) { - this.conf = conf; - if (mode == null) { - throw new IllegalArgumentException("mode cannot be NULL"); - } - this.mode = mode; + this.conf = Objects.requireNonNull(conf); + this.mode = Objects.requireNonNull(mode, "mode cannot be NULL"); requireClientCert = conf.getBoolean(SSL_REQUIRE_CLIENT_CERT_KEY, DEFAULT_SSL_REQUIRE_CLIENT_CERT); // Rest of ssl configs are pre-populated in incoming conf payload diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/ConfigUtils.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/ConfigUtils.java index f83fdc99d7..74e74f2780 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/ConfigUtils.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/ConfigUtils.java @@ -22,10 +22,9 @@ import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Set; -import org.apache.tez.common.Preconditions; - import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.io.RawComparator; @@ -134,8 +133,8 @@ public static boolean useNewApi(Configuration conf) { @InterfaceAudience.Private public static Map extractConfigurationMap(Map confMap, Set allowedKeys) { - Preconditions.checkArgument(confMap != null, "ConfMap cannot be null"); - Preconditions.checkArgument(allowedKeys != null, "Valid key set cannot be empty"); + Objects.requireNonNull(confMap, "ConfMap cannot be null"); + Objects.requireNonNull(allowedKeys, "Valid key set cannot be empty"); Map map = new HashMap(); for (Map.Entry entry : confMap.entrySet()) { if (allowedKeys.contains(entry.getKey())) { @@ -147,8 +146,8 @@ public static Map extractConfigurationMap(Map co @InterfaceAudience.Private public static void addConfigMapToConfiguration(Configuration conf, Map confMap) { - Preconditions.checkArgument(conf != null, "Configuration cannot be null"); - Preconditions.checkArgument(confMap != null, "Configuration map cannot be null"); + Objects.requireNonNull(conf, "Configuration cannot be null"); + Objects.requireNonNull(confMap, "Configuration map cannot be null"); for (Map.Entry entry : confMap.entrySet()) { conf.set(entry.getKey(), entry.getValue()); } @@ -158,9 +157,9 @@ public static void addConfigMapToConfiguration(Configuration conf, Map extractConfigurationMap(Map confMap, List> validKeySets, List allowedPrefixes) { - Preconditions.checkArgument(confMap != null, "ConfMap cannot be null"); - Preconditions.checkArgument(validKeySets != null, "Valid key set cannot be empty"); - Preconditions.checkArgument(allowedPrefixes != null, "Allowed prefixes cannot be null"); + Objects.requireNonNull(confMap, "ConfMap cannot be null"); + Objects.requireNonNull(validKeySets, "Valid key set cannot be empty"); + Objects.requireNonNull(allowedPrefixes, "Allowed prefixes cannot be null"); return extractConfigurationMapInternal(confMap.entrySet(), validKeySets, allowedPrefixes); } @@ -169,17 +168,17 @@ public static Map extractConfigurationMap(Map co public static Map extractConfigurationMap(Configuration conf, List> validKeySets, List allowedPrefixes) { - Preconditions.checkArgument(conf != null, "conf cannot be null"); - Preconditions.checkArgument(validKeySets != null, "Valid key set cannot be empty"); - Preconditions.checkArgument(allowedPrefixes != null, "Allowed prefixes cannot be null"); + Objects.requireNonNull(conf, "conf cannot be null"); + Objects.requireNonNull(validKeySets, "Valid key set cannot be empty"); + Objects.requireNonNull(allowedPrefixes, "Allowed prefixes cannot be null"); return extractConfigurationMapInternal(conf, validKeySets, allowedPrefixes); } @InterfaceAudience.Private public static boolean doesKeyQualify(String key, List> validKeySets, List allowedPrefixes) { - Preconditions.checkArgument(key != null, "key cannot be null"); - Preconditions.checkArgument(validKeySets != null, "Valid key set cannot be empty"); - Preconditions.checkArgument(allowedPrefixes != null, "Allowed prefixes cannot be null"); + Objects.requireNonNull(key, "key cannot be null"); + Objects.requireNonNull(validKeySets, "Valid key set cannot be empty"); + Objects.requireNonNull(allowedPrefixes, "Allowed prefixes cannot be null"); for (Set set : validKeySets) { if (set.contains(key)) { return true; @@ -195,8 +194,8 @@ public static boolean doesKeyQualify(String key, List> validKeySets, @InterfaceAudience.Private public static void mergeConfsWithExclusions(Configuration destConf, Map srcConf, Set excludedKeySet) { - Preconditions.checkState(destConf != null, "Destination conf cannot be null"); - Preconditions.checkState(srcConf != null, "Source conf cannot be null"); + Objects.requireNonNull(destConf, "Destination conf cannot be null"); + Objects.requireNonNull(srcConf, "Source conf cannot be null"); for (Map.Entry entry : srcConf.entrySet()) { if (!excludedKeySet.contains(entry.getKey())) { destConf.set(entry.getKey(), entry.getValue()); @@ -205,8 +204,8 @@ public static void mergeConfsWithExclusions(Configuration destConf, Map entry : srcConf) { // Explicit get to have parameter replacement work. String val = srcConf.get(entry.getKey()); 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 6a61474a5a..ca8b84724e 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 @@ -29,6 +29,7 @@ import java.util.BitSet; import java.util.Collection; import java.util.List; +import java.util.Objects; import java.util.concurrent.atomic.AtomicLong; import java.util.zip.Deflater; @@ -407,7 +408,7 @@ public static void generateEventOnSpill(List eventList, boolean finalMerg int numPhysicalOutputs, boolean sendEmptyPartitionDetails, String pathComponent, @Nullable long[] partitionStats, boolean reportDetailedPartitionStats, String auxiliaryService, Deflater deflater) throws IOException { - Preconditions.checkArgument(eventList != null, "EventList can't be null"); + Objects.requireNonNull(eventList, "EventList can't be null"); context.notifyProgress(); if (finalMergeEnabled) { diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/conf/OrderedGroupedKVInputConfig.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/conf/OrderedGroupedKVInputConfig.java index e7b96930a9..d9c1d21403 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/conf/OrderedGroupedKVInputConfig.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/conf/OrderedGroupedKVInputConfig.java @@ -27,7 +27,6 @@ import java.util.Objects; import com.google.common.annotations.VisibleForTesting; -import org.apache.tez.common.Preconditions; import com.google.common.collect.Lists; import org.apache.hadoop.classification.InterfaceAudience; @@ -448,7 +447,7 @@ public Builder setAdditionalConfiguration(Map confMap) { @Override public Builder setFromConfiguration(Configuration conf) { // Maybe ensure this is the first call ? Otherwise this can end up overriding other parameters - Preconditions.checkArgument(conf != null, "Configuration cannot be null"); + Objects.requireNonNull(conf, "Configuration cannot be null"); Map map = ConfigUtils.extractConfigurationMap(conf, Lists.newArrayList(OrderedGroupedKVInput.getConfigurationKeySet(), TezRuntimeConfiguration.getRuntimeAdditionalConfigKeySet()), TezRuntimeConfiguration.getAllowedPrefixes()); @@ -460,7 +459,7 @@ public Builder setFromConfiguration(Configuration conf) { @Override public Builder setFromConfigurationUnfiltered(Configuration conf) { // Maybe ensure this is the first call ? Otherwise this can end up overriding other parameters - Preconditions.checkArgument(conf != null, "Configuration cannot be null"); + Objects.requireNonNull(conf, "Configuration cannot be null"); ConfigUtils.mergeConfs(this.conf, conf); return this; } @@ -494,10 +493,8 @@ public Builder setCompression(boolean enabled, @Nullable String compressionCodec */ public Builder setKeySerializationClass(String serializationClassName, String comparatorClassName, @Nullable Map serializerConf) { - Preconditions.checkArgument(serializationClassName != null, - "serializationClassName cannot be null"); - Preconditions.checkArgument(comparatorClassName != null, - "comparator cannot be null"); + Objects.requireNonNull(serializationClassName, "serializationClassName cannot be null"); + Objects.requireNonNull(comparatorClassName, "comparator cannot be null"); this.conf.set(CommonConfigurationKeys.IO_SERIALIZATIONS_KEY, serializationClassName + "," + conf.get(CommonConfigurationKeys.IO_SERIALIZATIONS_KEY)); setKeyComparatorClass(comparatorClassName, null); @@ -520,8 +517,7 @@ public Builder setKeySerializationClass(String serializationClassName, */ public Builder setValueSerializationClass(String serializationClassName, @Nullable Map serializerConf) { - Preconditions.checkArgument(serializationClassName != null, - "serializationClassName cannot be null"); + Objects.requireNonNull(serializationClassName, "serializationClassName cannot be null"); this.conf.set(CommonConfigurationKeys.IO_SERIALIZATIONS_KEY, serializationClassName + "," + conf.get(CommonConfigurationKeys.IO_SERIALIZATIONS_KEY)); if (serializerConf != null) { diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/conf/OrderedPartitionedKVOutputConfig.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/conf/OrderedPartitionedKVOutputConfig.java index 7c92f38260..9671feb8b9 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/conf/OrderedPartitionedKVOutputConfig.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/conf/OrderedPartitionedKVOutputConfig.java @@ -27,7 +27,6 @@ import java.util.Objects; import com.google.common.annotations.VisibleForTesting; -import org.apache.tez.common.Preconditions; import com.google.common.collect.Lists; import org.apache.hadoop.classification.InterfaceAudience; @@ -374,7 +373,7 @@ public Builder setAdditionalConfiguration(Map confMap) { @Override public Builder setFromConfiguration(Configuration conf) { // Maybe ensure this is the first call ? Otherwise this can end up overriding other parameters - Preconditions.checkArgument(conf != null, "Configuration cannot be null"); + Objects.requireNonNull(conf, "Configuration cannot be null"); Map map = ConfigUtils.extractConfigurationMap(conf, Lists.newArrayList(OrderedPartitionedKVOutput.getConfigurationKeySet(), TezRuntimeConfiguration.getRuntimeAdditionalConfigKeySet()), TezRuntimeConfiguration.getAllowedPrefixes()); @@ -386,7 +385,7 @@ public Builder setFromConfiguration(Configuration conf) { @Override public Builder setFromConfigurationUnfiltered(Configuration conf) { // Maybe ensure this is the first call ? Otherwise this can end up overriding other parameters - Preconditions.checkArgument(conf != null, "Configuration cannot be null"); + Objects.requireNonNull(conf, "Configuration cannot be null"); ConfigUtils.mergeConfs(this.conf, conf); return this; } @@ -412,6 +411,7 @@ public Builder setKeyComparatorClass(String comparatorClassName) { * java.util.Map} of key-value pairs. The keys should be limited to * the ones required by the comparator. * @return instance of the current builder + * @throws NullPointerException if {@code comparatorClassName} is {@code null} */ public Builder setKeyComparatorClass(String comparatorClassName, @Nullable Map comparatorConf) { @@ -452,13 +452,13 @@ public Builder setCompression(boolean enabled, @Nullable String compressionCodec * {@link java.util.Map} of key-value pairs. The keys should be limited * to the ones required by the comparator. * @return this object for further chained method calls + * @throws NullPointerException if {@code serializationClassName} or + * {@code comparatorClassName} is {@code null} */ public Builder setKeySerializationClass(String serializationClassName, String comparatorClassName, @Nullable Map serializerConf) { - Preconditions.checkArgument(serializationClassName != null, - "serializationClassName cannot be null"); - Preconditions.checkArgument(comparatorClassName != null, - "comparator cannot be null"); + Objects.requireNonNull(serializationClassName, "serializationClassName cannot be null"); + Objects.requireNonNull(comparatorClassName, "comparator cannot be null"); this.conf.set(CommonConfigurationKeys.IO_SERIALIZATIONS_KEY, serializationClassName + "," + conf.get(CommonConfigurationKeys.IO_SERIALIZATIONS_KEY)); setKeyComparatorClass(comparatorClassName, null); @@ -478,11 +478,11 @@ public Builder setKeySerializationClass(String serializationClassName, * {@link java.util.Map} of key-value pairs. The keys should be limited * to the ones required by the comparator. * @return this object for further chained method calls + * @throws NullPointerException if {@code serializationClassName} is {@code null} */ public Builder setValueSerializationClass(String serializationClassName, @Nullable Map serializerConf) { - Preconditions.checkArgument(serializationClassName != null, - "serializationClassName cannot be null"); + Objects.requireNonNull(serializationClassName, "serializationClassName cannot be null"); this.conf.set(CommonConfigurationKeys.IO_SERIALIZATIONS_KEY, serializationClassName + "," + conf.get(CommonConfigurationKeys.IO_SERIALIZATIONS_KEY)); if (serializerConf != null) { diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/conf/UnorderedKVInputConfig.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/conf/UnorderedKVInputConfig.java index 6ac0dbb3a8..6f285e7995 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/conf/UnorderedKVInputConfig.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/conf/UnorderedKVInputConfig.java @@ -27,7 +27,6 @@ import java.util.Objects; import com.google.common.annotations.VisibleForTesting; -import org.apache.tez.common.Preconditions; import com.google.common.collect.Lists; import org.apache.hadoop.classification.InterfaceAudience; @@ -295,7 +294,7 @@ public Builder setAdditionalConfiguration(Map confMap) { @Override public Builder setFromConfiguration(Configuration conf) { // Maybe ensure this is the first call ? Otherwise this can end up overriding other parameters - Preconditions.checkArgument(conf != null, "Configuration cannot be null"); + Objects.requireNonNull(conf, "Configuration cannot be null"); Map map = ConfigUtils.extractConfigurationMap(conf, Lists.newArrayList(UnorderedKVInput.getConfigurationKeySet(), TezRuntimeConfiguration.getRuntimeAdditionalConfigKeySet()), TezRuntimeConfiguration.getAllowedPrefixes()); @@ -307,7 +306,7 @@ public Builder setFromConfiguration(Configuration conf) { @Override public Builder setFromConfigurationUnfiltered(Configuration conf) { // Maybe ensure this is the first call ? Otherwise this can end up overriding other parameters - Preconditions.checkArgument(conf != null, "Configuration cannot be null"); + Objects.requireNonNull(conf, "Configuration cannot be null"); ConfigUtils.mergeConfs(this.conf, conf); return this; } @@ -339,8 +338,7 @@ public Builder setCompression(boolean enabled, @Nullable String compressionCodec */ public Builder setKeySerializationClass(String serializationClassName, @Nullable Map serializerConf) { - Preconditions.checkArgument(serializationClassName != null, - "serializationClassName cannot be null"); + Objects.requireNonNull(serializationClassName, "serializationClassName cannot be null"); this.conf.set(CommonConfigurationKeys.IO_SERIALIZATIONS_KEY, serializationClassName + "," + conf.get(CommonConfigurationKeys.IO_SERIALIZATIONS_KEY)); if (serializerConf != null) { @@ -362,8 +360,7 @@ public Builder setKeySerializationClass(String serializationClassName, */ public Builder setValueSerializationClass(String serializationClassName, @Nullable Map serializerConf) { - Preconditions.checkArgument(serializationClassName != null, - "serializationClassName cannot be null"); + Objects.requireNonNull(serializationClassName, "serializationClassName cannot be null"); this.conf.set(CommonConfigurationKeys.IO_SERIALIZATIONS_KEY, serializationClassName + "," + conf.get(CommonConfigurationKeys.IO_SERIALIZATIONS_KEY)); if (serializerConf != null) { diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/conf/UnorderedKVOutputConfig.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/conf/UnorderedKVOutputConfig.java index 34513badf4..a6ac7f2b5a 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/conf/UnorderedKVOutputConfig.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/conf/UnorderedKVOutputConfig.java @@ -27,7 +27,6 @@ import java.util.Objects; import com.google.common.annotations.VisibleForTesting; -import org.apache.tez.common.Preconditions; import com.google.common.collect.Lists; import org.apache.hadoop.classification.InterfaceAudience; @@ -222,7 +221,7 @@ public Builder setAdditionalConfiguration(Map confMap) { @Override public Builder setFromConfiguration(Configuration conf) { // Maybe ensure this is the first call ? Otherwise this can end up overriding other parameters - Preconditions.checkArgument(conf != null, "Configuration cannot be null"); + Objects.requireNonNull(conf, "Configuration cannot be null"); Map map = ConfigUtils.extractConfigurationMap(conf, Lists.newArrayList(UnorderedKVOutput.getConfigurationKeySet(), TezRuntimeConfiguration.getRuntimeAdditionalConfigKeySet()), TezRuntimeConfiguration.getAllowedPrefixes()); @@ -234,7 +233,7 @@ public Builder setFromConfiguration(Configuration conf) { @Override public Builder setFromConfigurationUnfiltered(Configuration conf) { // Maybe ensure this is the first call ? Otherwise this can end up overriding other parameters - Preconditions.checkArgument(conf != null, "Configuration cannot be null"); + Objects.requireNonNull(conf, "Configuration cannot be null"); ConfigUtils.mergeConfs(this.conf, conf); return this; } @@ -250,8 +249,7 @@ public Builder setFromConfigurationUnfiltered(Configuration conf) { */ public Builder setKeySerializationClass(String serializationClassName, @Nullable Map serializerConf) { - Preconditions.checkArgument(serializationClassName != null, - "serializationClassName cannot be null"); + Objects.requireNonNull(serializationClassName, "serializationClassName cannot be null"); this.conf.set(CommonConfigurationKeys.IO_SERIALIZATIONS_KEY, serializationClassName + "," + conf.get(CommonConfigurationKeys.IO_SERIALIZATIONS_KEY)); if (serializerConf != null) { @@ -273,8 +271,7 @@ public Builder setKeySerializationClass(String serializationClassName, */ public Builder setValueSerializationClass(String serializationClassName, @Nullable Map serializerConf) { - Preconditions.checkArgument(serializationClassName != null, - "serializationClassName cannot be null"); + Objects.requireNonNull(serializationClassName, "serializationClassName cannot be null"); this.conf.set(CommonConfigurationKeys.IO_SERIALIZATIONS_KEY, serializationClassName + "," + conf.get(CommonConfigurationKeys.IO_SERIALIZATIONS_KEY)); if (serializerConf != null) { diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/conf/UnorderedPartitionedKVOutputConfig.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/conf/UnorderedPartitionedKVOutputConfig.java index c924e7db66..dbbfb9ea6d 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/conf/UnorderedPartitionedKVOutputConfig.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/conf/UnorderedPartitionedKVOutputConfig.java @@ -27,7 +27,6 @@ import java.util.Objects; import com.google.common.annotations.VisibleForTesting; -import org.apache.tez.common.Preconditions; import com.google.common.collect.Lists; import org.apache.hadoop.classification.InterfaceAudience; @@ -266,7 +265,7 @@ public Builder setAdditionalConfiguration(Map confMap) { @Override public Builder setFromConfiguration(Configuration conf) { // Maybe ensure this is the first call ? Otherwise this can end up overriding other parameters - Preconditions.checkArgument(conf != null, "Configuration cannot be null"); + Objects.requireNonNull(conf, "Configuration cannot be null"); Map map = ConfigUtils.extractConfigurationMap(conf, Lists.newArrayList(UnorderedPartitionedKVOutput.getConfigurationKeySet(), TezRuntimeConfiguration.getRuntimeAdditionalConfigKeySet()), TezRuntimeConfiguration.getAllowedPrefixes()); @@ -278,7 +277,7 @@ public Builder setFromConfiguration(Configuration conf) { @Override public Builder setFromConfigurationUnfiltered(Configuration conf) { // Maybe ensure this is the first call ? Otherwise this can end up overriding other parameters - Preconditions.checkArgument(conf != null, "Configuration cannot be null"); + Objects.requireNonNull(conf, "Configuration cannot be null"); ConfigUtils.mergeConfs(this.conf, conf); return this; } @@ -309,8 +308,7 @@ public Builder setCompression(boolean enabled, @Nullable String compressionCodec */ public Builder setKeySerializationClass(String serializationClassName, @Nullable Map serializerConf) { - Preconditions.checkArgument(serializationClassName != null, - "serializationClassName cannot be null"); + Objects.requireNonNull(serializationClassName, "serializationClassName cannot be null"); this.conf.set(CommonConfigurationKeys.IO_SERIALIZATIONS_KEY, serializationClassName + "," + conf.get(CommonConfigurationKeys.IO_SERIALIZATIONS_KEY)); if (serializerConf != null) { @@ -331,8 +329,7 @@ public Builder setKeySerializationClass(String serializationClassName, */ public Builder setValueSerializationClass(String serializationClassName, @Nullable Map serializerConf) { - Preconditions.checkArgument(serializationClassName != null, - "serializationClassName cannot be null"); + Objects.requireNonNull(serializationClassName, "serializationClassName cannot be null"); this.conf.set(CommonConfigurationKeys.IO_SERIALIZATIONS_KEY, serializationClassName + "," + conf.get(CommonConfigurationKeys.IO_SERIALIZATIONS_KEY)); if (serializerConf != null) { diff --git a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/CSVResult.java b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/CSVResult.java index 0e167b2dcf..1144f7643c 100644 --- a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/CSVResult.java +++ b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/CSVResult.java @@ -36,6 +36,7 @@ import java.util.Comparator; import java.util.Iterator; import java.util.List; +import java.util.Objects; /** * Simple placeholder for storing CSV results. @@ -57,7 +58,7 @@ public String[] getHeaders() { } public void addRecord(String[] record) { - Preconditions.checkArgument(record != null, "Record can't be null"); + Objects.requireNonNull(record, "Record cannot be null"); Preconditions.checkArgument(record.length == headers.length, "Record length" + record.length + " does not match headers length " + headers.length); recordsList.add(record); diff --git a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/SkewAnalyzer.java b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/SkewAnalyzer.java index a7d14fae25..b0387d1973 100644 --- a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/SkewAnalyzer.java +++ b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/SkewAnalyzer.java @@ -18,7 +18,6 @@ package org.apache.tez.analyzer.plugins; -import org.apache.tez.common.Preconditions; import com.google.common.collect.Lists; import org.apache.hadoop.conf.Configuration; @@ -34,6 +33,7 @@ import java.util.List; import java.util.Map; +import java.util.Objects; /** @@ -101,8 +101,7 @@ public SkewAnalyzer(Configuration config) { @Override public void analyze(DagInfo dagInfo) throws TezException { - Preconditions.checkArgument(dagInfo != null, "DAG can't be null"); - analyzeReducers(dagInfo); + analyzeReducers(Objects.requireNonNull(dagInfo, "DAG can't be null")); } private void analyzeReducers(DagInfo dagInfo) { From 0b3f3b69f9eefc883b69439a37d93122904ee574 Mon Sep 17 00:00:00 2001 From: Bodor Laszlo Date: Sun, 16 Jan 2022 09:42:08 +0100 Subject: [PATCH 085/137] TEZ-4376: Remove javax.security from the tez-auxservices shaded jar (#177) (Laszlo Bodor reviewed by Jonathan Eagles) --- tez-plugins/tez-aux-services/pom.xml | 1 + 1 file changed, 1 insertion(+) diff --git a/tez-plugins/tez-aux-services/pom.xml b/tez-plugins/tez-aux-services/pom.xml index 2a936d6fa5..76f431b3ac 100644 --- a/tez-plugins/tez-aux-services/pom.xml +++ b/tez-plugins/tez-aux-services/pom.xml @@ -246,6 +246,7 @@ org.apache.tez.shaded.$0 javax.crypto.* + javax.security.** From cc8249e932c8c5b117c901188dbd92bf6f8d5391 Mon Sep 17 00:00:00 2001 From: csjuhasz-c <66361392+csjuhasz-c@users.noreply.github.com> Date: Mon, 17 Jan 2022 14:15:27 +0100 Subject: [PATCH 086/137] TEZ-4340: Show convenient input -> output vertex names in input messages (#170) (Csaba Juhasz reviewed by Laszlo Bodor) --- .../apache/tez/runtime/api/InputContext.java | 6 ++ .../apache/tez/mapreduce/input/MRInput.java | 16 ++-- .../tez/mapreduce/input/MRInputLegacy.java | 4 +- .../tez/mapreduce/input/MultiMRInput.java | 10 +-- .../runtime/api/impl/TezInputContextImpl.java | 5 ++ .../library/common/shuffle/Fetcher.java | 44 ++++----- .../shuffle/FetcherWithInjectableErrors.java | 16 ++-- .../impl/ShuffleInputEventHandlerImpl.java | 2 +- .../common/shuffle/impl/ShuffleManager.java | 62 ++++++------- .../orderedgrouped/FetcherOrderedGrouped.java | 16 ++-- ...herOrderedGroupedWithInjectableErrors.java | 18 ++-- .../shuffle/orderedgrouped/MergeManager.java | 17 ++-- .../shuffle/orderedgrouped/Shuffle.java | 35 ++++---- ...huffleInputEventHandlerOrderedGrouped.java | 2 +- .../orderedgrouped/ShuffleScheduler.java | 18 ++-- .../library/input/OrderedGroupedKVInput.java | 4 +- .../library/input/UnorderedKVInput.java | 4 +- .../library/common/shuffle/TestFetcher.java | 25 ++++-- .../TestShuffleInputEventHandlerImpl.java | 1 + .../shuffle/impl/TestShuffleManager.java | 1 + .../shuffle/orderedgrouped/TestFetcher.java | 90 ++++++++----------- .../shuffle/orderedgrouped/TestShuffle.java | 1 + .../java/org/apache/tez/test/TestInput.java | 3 +- 23 files changed, 209 insertions(+), 191 deletions(-) diff --git a/tez-api/src/main/java/org/apache/tez/runtime/api/InputContext.java b/tez-api/src/main/java/org/apache/tez/runtime/api/InputContext.java index 479a7dba1b..6eac2dfc95 100644 --- a/tez-api/src/main/java/org/apache/tez/runtime/api/InputContext.java +++ b/tez-api/src/main/java/org/apache/tez/runtime/api/InputContext.java @@ -32,6 +32,12 @@ public interface InputContext extends TaskContext { * @return Name of the Source Vertex */ public String getSourceVertexName(); + + /** + * Returns a convenient, human-readable string describing the input and output vertices. + * @return the convenient string + */ + String getInputOutputVertexNames(); /** * Get the index of the input in the set of all inputs for the task. The diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/MRInput.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/MRInput.java index 964c0b8301..e64d273b44 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/MRInput.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/MRInput.java @@ -461,7 +461,7 @@ public List initialize() throws IOException { getContext().inputIsReady(); this.splitInfoViaEvents = jobConf.getBoolean(MRJobConfig.MR_TEZ_SPLITS_VIA_EVENTS, MRJobConfig.MR_TEZ_SPLITS_VIA_EVENTS_DEFAULT); - LOG.info(getContext().getSourceVertexName() + " using newmapreduce API=" + useNewApi + + LOG.info(getContext().getInputOutputVertexNames() + " using newmapreduce API=" + useNewApi + ", split via event=" + splitInfoViaEvents + ", numPhysicalInputs=" + getNumPhysicalInputs()); initializeInternal(); @@ -526,7 +526,7 @@ inputRecordCounter, getContext().getApplicationId().getClusterTimestamp(), } finally { rrLock.unlock(); } - LOG.info("Initialized MRInput: " + getContext().getSourceVertexName()); + LOG.info("Initialized MRInput: " + getContext().getInputOutputVertexNames()); } /** @@ -634,7 +634,7 @@ void processSplitEvent(InputDataInformationEvent event) try { initFromEventInternal(event); if (LOG.isDebugEnabled()) { - LOG.debug(getContext().getSourceVertexName() + " notifying on RecordReader initialized"); + LOG.debug(getContext().getInputOutputVertexNames() + " notifying on RecordReader initialized"); } rrInited.signal(); } finally { @@ -647,7 +647,7 @@ void checkAndAwaitRecordReaderInitialization() throws IOException { rrLock.lock(); try { if (LOG.isDebugEnabled()) { - LOG.debug(getContext().getSourceVertexName() + " awaiting RecordReader initialization"); + LOG.debug(getContext().getInputOutputVertexNames() + " awaiting RecordReader initialization"); } rrInited.await(); } catch (Exception e) { @@ -671,7 +671,7 @@ void initFromEvent(InputDataInformationEvent initEvent) private void initFromEventInternal(InputDataInformationEvent initEvent) throws IOException { if (LOG.isDebugEnabled()) { - LOG.debug(getContext().getSourceVertexName() + " initializing RecordReader from event"); + LOG.debug(getContext().getInputOutputVertexNames() + " initializing RecordReader from event"); } Objects.requireNonNull(initEvent, "InitEvent must be specified"); MRSplitProto splitProto = MRSplitProto.parseFrom(ByteString.copyFrom(initEvent.getUserPayload())); @@ -686,7 +686,7 @@ private void initFromEventInternal(InputDataInformationEvent initEvent) throws I LOG.warn("Thread interrupted while getting split length: ", e); } if (LOG.isDebugEnabled()) { - LOG.debug(getContext().getSourceVertexName() + " split Details -> SplitClass: " + + LOG.debug(getContext().getInputOutputVertexNames() + " split Details -> SplitClass: " + split.getClass().getName() + ", NewSplit: " + split + ", length: " + splitLength); } @@ -696,7 +696,7 @@ private void initFromEventInternal(InputDataInformationEvent initEvent) throws I splitObj = split; splitLength = split.getLength(); if (LOG.isDebugEnabled()) { - LOG.debug(getContext().getSourceVertexName() + " split Details -> SplitClass: " + + LOG.debug(getContext().getInputOutputVertexNames() + " split Details -> SplitClass: " + split.getClass().getName() + ", OldSplit: " + split + ", length: " + splitLength); } } @@ -705,7 +705,7 @@ private void initFromEventInternal(InputDataInformationEvent initEvent) throws I .increment(splitLength); } mrReader.setSplit(splitObj); - LOG.info(getContext().getSourceVertexName() + " initialized RecordReader from event"); + LOG.info(getContext().getInputOutputVertexNames() + " initialized RecordReader from event"); } private static class MRInputHelpersInternal extends MRInputHelpers { diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/MRInputLegacy.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/MRInputLegacy.java index 70be7ee444..bbf145eb6c 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/MRInputLegacy.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/MRInputLegacy.java @@ -74,7 +74,7 @@ public MRInputLegacy(InputContext inputContext, int numPhysicalInputs) { @Private protected void initializeInternal() throws IOException { - LOG.info(getContext().getSourceVertexName() + " MRInputLegacy deferring initialization"); + LOG.info(getContext().getInputOutputVertexNames() + " MRInputLegacy deferring initialization"); } @Private @@ -136,7 +136,7 @@ void checkAndAwaitRecordReaderInitialization() throws IOException { if (splitInfoViaEvents && !inited) { if (initEvent == null) { if (LOG.isDebugEnabled()) { - LOG.debug(getContext().getSourceVertexName() + + LOG.debug(getContext().getInputOutputVertexNames() + " awaiting init event before initializing record reader"); } diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/MultiMRInput.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/MultiMRInput.java index 3bc8629565..4a98052211 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/MultiMRInput.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/MultiMRInput.java @@ -110,7 +110,7 @@ private MultiMRInputConfigBuilder(Configuration conf, Class inputFormat) { @Override public List initialize() throws IOException { super.initialize(); - LOG.info(getContext().getSourceVertexName() + " using newmapreduce API=" + useNewApi + + LOG.info(getContext().getInputOutputVertexNames() + " using newmapreduce API=" + useNewApi + ", numPhysicalInputs=" + getNumPhysicalInputs()); if (getNumPhysicalInputs() == 0) { getContext().inputIsReady(); @@ -167,7 +167,7 @@ public void handleEvents(List inputEvents) throws Exception { private MRReader initFromEvent(InputDataInformationEvent event) throws IOException { Objects.requireNonNull(event, "Event must be specified"); if (LOG.isDebugEnabled()) { - LOG.debug(getContext().getSourceVertexName() + " initializing Reader: " + eventCount.get()); + LOG.debug(getContext().getInputOutputVertexNames() + " initializing Reader: " + eventCount.get()); } MRSplitProto splitProto = MRSplitProto.parseFrom(ByteString.copyFrom(event.getUserPayload())); MRReader reader = null; @@ -186,7 +186,7 @@ private MRReader initFromEvent(InputDataInformationEvent event) throws IOExcepti .getApplicationId().getId(), getContext().getTaskIndex(), getContext() .getTaskAttemptNumber(), getContext()); if (LOG.isDebugEnabled()) { - LOG.debug(getContext().getSourceVertexName() + " split Details -> SplitClass: " + + LOG.debug(getContext().getInputOutputVertexNames() + " split Details -> SplitClass: " + split.getClass().getName() + ", NewSplit: " + split + ", length: " + splitLength); } } else { @@ -196,7 +196,7 @@ private MRReader initFromEvent(InputDataInformationEvent event) throws IOExcepti reader = new MRReaderMapred(localJobConf, split, getContext().getCounters(), inputRecordCounter, getContext()); if (LOG.isDebugEnabled()) { - LOG.debug(getContext().getSourceVertexName() + " split Details -> SplitClass: " + + LOG.debug(getContext().getInputOutputVertexNames() + " split Details -> SplitClass: " + split.getClass().getName() + ", OldSplit: " + split + ", length: " + splitLength); } } @@ -204,7 +204,7 @@ private MRReader initFromEvent(InputDataInformationEvent event) throws IOExcepti getContext().getCounters().findCounter(TaskCounter.INPUT_SPLIT_LENGTH_BYTES) .increment(splitLength); } - LOG.info(getContext().getSourceVertexName() + " initialized RecordReader from event"); + LOG.info(getContext().getInputOutputVertexNames() + " initialized RecordReader from event"); return reader; } diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezInputContextImpl.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezInputContextImpl.java index f28573a649..1c1c10bf86 100644 --- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezInputContextImpl.java +++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezInputContextImpl.java @@ -152,6 +152,11 @@ public String getSourceVertexName() { return sourceVertexName; } + @Override + public String getInputOutputVertexNames() { + return String.format("%s -> %s", getSourceVertexName(), getTaskVertexName()); + } + @Override public void fatalError(Throwable exception, String message) { super.signalFatalError(exception, message, sourceInfo); 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 6039df3a2f..eb34ec2993 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 @@ -58,7 +58,7 @@ import org.apache.tez.common.CallableWithNdc; import org.apache.tez.common.security.JobTokenSecretManager; import org.apache.tez.dag.api.TezUncheckedException; -import org.apache.tez.runtime.api.ObjectRegistry; +import org.apache.tez.runtime.api.InputContext; import org.apache.tez.runtime.library.api.TezRuntimeConfiguration; import org.apache.tez.runtime.library.common.Constants; import org.apache.tez.runtime.library.common.InputAttemptIdentifier; @@ -69,6 +69,7 @@ import org.apache.tez.runtime.library.common.shuffle.FetchedInput.Type; import org.apache.tez.runtime.library.common.shuffle.api.ShuffleHandlerError; import org.apache.tez.common.Preconditions; +import org.apache.tez.common.TezUtilsInternal; /** * Responsible for fetching inputs served by the ShuffleHandler for a single @@ -194,8 +195,8 @@ public String getHost() { private final boolean isDebugEnabled = LOG.isDebugEnabled(); protected Fetcher(FetcherCallback fetcherCallback, HttpConnectionParams params, - FetchedInputAllocator inputManager, ApplicationId appId, int dagIdentifier, - JobTokenSecretManager jobTokenSecretManager, String srcNameTrimmed, Configuration conf, + FetchedInputAllocator inputManager, InputContext inputContext, + JobTokenSecretManager jobTokenSecretManager, Configuration conf, RawLocalFileSystem localFs, LocalDirAllocator localDirAllocator, Path lockPath, @@ -208,8 +209,8 @@ protected Fetcher(FetcherCallback fetcherCallback, HttpConnectionParams params, this.fetcherCallback = fetcherCallback; this.inputManager = inputManager; this.jobTokenSecretMgr = jobTokenSecretManager; - this.appId = appId; - this.dagIdentifier = dagIdentifier; + this.appId = inputContext.getApplicationId(); + this.dagIdentifier = inputContext.getDagIdentifier(); this.pathToAttemptMap = new HashMap(); this.httpConnectionParams = params; this.conf = conf; @@ -218,7 +219,10 @@ protected Fetcher(FetcherCallback fetcherCallback, HttpConnectionParams params, this.sharedFetchEnabled = sharedFetchEnabled; this.fetcherIdentifier = fetcherIdGen.getAndIncrement(); - this.logIdentifier = " fetcher [" + srcNameTrimmed +"] " + fetcherIdentifier; + + String sourceDestNameTrimmed = TezUtilsInternal.cleanVertexName(inputContext.getSourceVertexName()) + " -> " + + TezUtilsInternal.cleanVertexName(inputContext.getTaskVertexName()); + this.logIdentifier = " fetcher [" + sourceDestNameTrimmed +"] " + fetcherIdentifier; this.localFs = localFs; this.localDirAllocator = localDirAllocator; @@ -1133,31 +1137,29 @@ public static class FetcherBuilder { private boolean workAssigned = false; public FetcherBuilder(FetcherCallback fetcherCallback, - HttpConnectionParams params, FetchedInputAllocator inputManager, - ApplicationId appId, int dagIdentifier, JobTokenSecretManager jobTokenSecretMgr, String srcNameTrimmed, - Configuration conf, boolean localDiskFetchEnabled, String localHostname, int shufflePort, - boolean asyncHttp, boolean verifyDiskChecksum, boolean compositeFetch) { - this.fetcher = new Fetcher(fetcherCallback, params, inputManager, appId, dagIdentifier, - jobTokenSecretMgr, srcNameTrimmed, conf, null, null, null, localDiskFetchEnabled, + HttpConnectionParams params, FetchedInputAllocator inputManager, InputContext inputContext, + JobTokenSecretManager jobTokenSecretMgr, Configuration conf, boolean localDiskFetchEnabled, + String localHostname, int shufflePort, boolean asyncHttp, boolean verifyDiskChecksum, boolean compositeFetch) { + this.fetcher = new Fetcher(fetcherCallback, params, inputManager, inputContext, + jobTokenSecretMgr, conf, null, null, null, localDiskFetchEnabled, false, localHostname, shufflePort, asyncHttp, verifyDiskChecksum, compositeFetch); } public FetcherBuilder(FetcherCallback fetcherCallback, - HttpConnectionParams params, FetchedInputAllocator inputManager, - ApplicationId appId, int dagIdentifier, JobTokenSecretManager jobTokenSecretMgr, String srcNameTrimmed, - Configuration conf, RawLocalFileSystem localFs, + HttpConnectionParams params, FetchedInputAllocator inputManager, InputContext inputContext, + JobTokenSecretManager jobTokenSecretMgr, Configuration conf, RawLocalFileSystem localFs, LocalDirAllocator localDirAllocator, Path lockPath, boolean localDiskFetchEnabled, boolean sharedFetchEnabled, String localHostname, int shufflePort, boolean asyncHttp, boolean verifyDiskChecksum, boolean compositeFetch, - boolean enableFetcherTestingErrors, ObjectRegistry objectRegistry) { + boolean enableFetcherTestingErrors) { if (enableFetcherTestingErrors) { - this.fetcher = new FetcherWithInjectableErrors(fetcherCallback, params, inputManager, appId, dagIdentifier, - jobTokenSecretMgr, srcNameTrimmed, conf, localFs, localDirAllocator, + this.fetcher = new FetcherWithInjectableErrors(fetcherCallback, params, inputManager, inputContext, + jobTokenSecretMgr, conf, localFs, localDirAllocator, lockPath, localDiskFetchEnabled, sharedFetchEnabled, localHostname, shufflePort, asyncHttp, - verifyDiskChecksum, compositeFetch, objectRegistry); + verifyDiskChecksum, compositeFetch); } else { - this.fetcher = new Fetcher(fetcherCallback, params, inputManager, appId, dagIdentifier, - jobTokenSecretMgr, srcNameTrimmed, conf, localFs, localDirAllocator, + this.fetcher = new Fetcher(fetcherCallback, params, inputManager, inputContext, + jobTokenSecretMgr, conf, localFs, localDirAllocator, lockPath, localDiskFetchEnabled, sharedFetchEnabled, localHostname, shufflePort, asyncHttp, verifyDiskChecksum, compositeFetch); } diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/FetcherWithInjectableErrors.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/FetcherWithInjectableErrors.java index 951adf9f5d..cf53a57a9e 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/FetcherWithInjectableErrors.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/FetcherWithInjectableErrors.java @@ -20,10 +20,10 @@ import org.apache.hadoop.fs.LocalDirAllocator; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.RawLocalFileSystem; -import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.tez.common.TezUtilsInternal; import org.apache.tez.common.security.JobTokenSecretManager; import org.apache.tez.http.HttpConnectionParams; -import org.apache.tez.runtime.api.ObjectRegistry; +import org.apache.tez.runtime.api.InputContext; import org.apache.tez.runtime.library.common.InputAttemptIdentifier; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -35,16 +35,16 @@ public class FetcherWithInjectableErrors extends Fetcher { private String srcNameTrimmed; protected FetcherWithInjectableErrors(FetcherCallback fetcherCallback, HttpConnectionParams params, - FetchedInputAllocator inputManager, ApplicationId appId, int dagIdentifier, - JobTokenSecretManager jobTokenSecretManager, String srcNameTrimmed, Configuration conf, + FetchedInputAllocator inputManager, InputContext inputContext, + JobTokenSecretManager jobTokenSecretManager, Configuration conf, RawLocalFileSystem localFs, LocalDirAllocator localDirAllocator, Path lockPath, boolean localDiskFetchEnabled, boolean sharedFetchEnabled, String localHostname, int shufflePort, boolean asyncHttp, boolean verifyDiskChecksum, - boolean compositeFetch, ObjectRegistry objectRegistry) { - super(fetcherCallback, params, inputManager, appId, dagIdentifier, jobTokenSecretManager, srcNameTrimmed, conf, + boolean compositeFetch) { + super(fetcherCallback, params, inputManager, inputContext, jobTokenSecretManager, conf, localFs, localDirAllocator, lockPath, localDiskFetchEnabled, sharedFetchEnabled, localHostname, shufflePort, asyncHttp, verifyDiskChecksum, compositeFetch); - this.fetcherErrorTestingConfig = new FetcherErrorTestingConfig(conf, objectRegistry); - this.srcNameTrimmed = srcNameTrimmed; + this.fetcherErrorTestingConfig = new FetcherErrorTestingConfig(conf, inputContext.getObjectRegistry()); + this.srcNameTrimmed = TezUtilsInternal.cleanVertexName(inputContext.getSourceVertexName()); LOG.info("Initialized FetcherWithInjectableErrors with config: {}", fetcherErrorTestingConfig); } diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleInputEventHandlerImpl.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleInputEventHandlerImpl.java index bcb7bb58ea..ca1259f30a 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleInputEventHandlerImpl.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleInputEventHandlerImpl.java @@ -161,7 +161,7 @@ private void handleEvent(Event event) throws IOException { @Override public void logProgress(boolean updateOnClose) { - LOG.info(inputContext.getSourceVertexName() + ": " + LOG.info(inputContext.getInputOutputVertexNames() + ": " + "numDmeEventsSeen=" + numDmeEvents.get() + ", numDmeEventsSeenWithNoData=" + numDmeEventsNoData.get() + ", numObsoletionEventsSeen=" + numObsoletionEvents.get() 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 fa539c8374..769ac68f7e 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 @@ -185,7 +185,7 @@ public class ShuffleManager implements FetcherCallback { */ private final int maxTimeToWaitForReportMillis; - private final String srcNameTrimmed; + private final String sourceDestNameTrimmed; private final int maxTaskOutputAtOnce; @@ -264,8 +264,9 @@ public ShuffleManager(InputContext inputContext, Configuration conf, int numInpu conf.getBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_FETCH_ENABLE_TESTING_ERRORS, TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_FETCH_ENABLE_TESTING_ERRORS_DEFAULT); - this.srcNameTrimmed = TezUtilsInternal.cleanVertexName(inputContext.getSourceVertexName()); - + this.sourceDestNameTrimmed = TezUtilsInternal.cleanVertexName(inputContext.getSourceVertexName()) + " -> " + + TezUtilsInternal.cleanVertexName(inputContext.getTaskVertexName()); + completedInputSet = new BitSet(numInputs); /** * In case of pipelined shuffle, it is possible to get multiple FetchedInput per attempt. @@ -288,15 +289,15 @@ public ShuffleManager(InputContext inputContext, Configuration conf, int numInpu if (conf.getBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_FETCHER_USE_SHARED_POOL, TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_FETCHER_USE_SHARED_POOL_DEFAULT)) { fetcherRawExecutor = inputContext.createTezFrameworkExecutorService(numFetchers, - "Fetcher_B {" + srcNameTrimmed + "} #%d"); + "Fetcher_B {" + sourceDestNameTrimmed + "} #%d"); } else { fetcherRawExecutor = Executors.newFixedThreadPool(numFetchers, new ThreadFactoryBuilder() - .setDaemon(true).setNameFormat("Fetcher_B {" + srcNameTrimmed + "} #%d").build()); + .setDaemon(true).setNameFormat("Fetcher_B {" + sourceDestNameTrimmed + "} #%d").build()); } this.fetcherExecutor = MoreExecutors.listeningDecorator(fetcherRawExecutor); ExecutorService schedulerRawExecutor = Executors.newFixedThreadPool(1, new ThreadFactoryBuilder() - .setDaemon(true).setNameFormat("ShuffleRunner {" + srcNameTrimmed + "}").build()); + .setDaemon(true).setNameFormat("ShuffleRunner {" + sourceDestNameTrimmed + "}").build()); this.schedulerExecutor = MoreExecutors.listeningDecorator(schedulerRawExecutor); this.schedulerCallable = new RunShuffleCallable(conf); @@ -336,7 +337,7 @@ public ShuffleManager(InputContext inputContext, Configuration conf, int numInpu shuffleInfoEventsMap = new ConcurrentHashMap(); - LOG.info(srcNameTrimmed + ": numInputs=" + numInputs + ", compressionCodec=" + LOG.info(sourceDestNameTrimmed + ": numInputs=" + numInputs + ", compressionCodec=" + (codec == null ? "NoCompressionCodec" : codec.getClass().getName()) + ", numFetchers=" + numFetchers + ", ifileBufferSize=" + ifileBufferSize + ", ifileReadAheadEnabled=" + ifileReadAhead + ", ifileReadAheadLength=" + ifileReadAheadLength +", " @@ -361,7 +362,7 @@ public void run() throws IOException { if (maxTimeToWaitForReportMillis > 0) { reporterExecutor = Executors.newSingleThreadExecutor( new ThreadFactoryBuilder().setDaemon(true) - .setNameFormat("ShuffleRunner {" + srcNameTrimmed + "}") + .setNameFormat("ShuffleRunner {" + sourceDestNameTrimmed + "}") .build()); Future reporterFuture = reporterExecutor.submit(new ReporterCallable()); } @@ -446,7 +447,7 @@ protected Void callInternal() throws Exception { break; } - LOG.debug("{}: NumCompletedInputs: {}", srcNameTrimmed, numCompletedInputs); + LOG.debug("{}: NumCompletedInputs: {}", sourceDestNameTrimmed, numCompletedInputs); if (numCompletedInputs.get() < numInputs && !isShutdown.get()) { lock.lock(); try { @@ -458,7 +459,8 @@ protected Void callInternal() throws Exception { inputHost = pendingHosts.take(); } catch (InterruptedException e) { if (isShutdown.get()) { - LOG.info(srcNameTrimmed + ": " + "Interrupted and hasBeenShutdown, Breaking out of ShuffleScheduler Loop"); + LOG.info(sourceDestNameTrimmed + ": " + + "Interrupted and hasBeenShutdown, Breaking out of ShuffleScheduler Loop"); Thread.currentThread().interrupt(); break; } else { @@ -466,14 +468,14 @@ protected Void callInternal() throws Exception { } } if (LOG.isDebugEnabled()) { - LOG.debug(srcNameTrimmed + ": " + "Processing pending host: " + + LOG.debug(sourceDestNameTrimmed + ": " + "Processing pending host: " + inputHost.toDetailedString()); } if (inputHost.getNumPendingPartitions() > 0 && !isShutdown.get()) { Fetcher fetcher = constructFetcherForHost(inputHost, conf); runningFetchers.add(fetcher); if (isShutdown.get()) { - LOG.info(srcNameTrimmed + ": " + "hasBeenShutdown," + + LOG.info(sourceDestNameTrimmed + ": " + "hasBeenShutdown," + "Breaking out of ShuffleScheduler Loop"); break; } @@ -485,7 +487,7 @@ protected Void callInternal() throws Exception { } } else { if (LOG.isDebugEnabled()) { - LOG.debug(srcNameTrimmed + ": " + "Skipping host: " + + LOG.debug(sourceDestNameTrimmed + ": " + "Skipping host: " + inputHost.getIdentifier() + " since it has no inputs to process"); } @@ -497,7 +499,8 @@ protected Void callInternal() throws Exception { } } shufflePhaseTime.setValue(System.currentTimeMillis() - startTime); - LOG.info(srcNameTrimmed + ": " + "Shutting down FetchScheduler, Was Interrupted: " + Thread.currentThread().isInterrupted()); + LOG.info(sourceDestNameTrimmed + ": " + "Shutting down FetchScheduler, Was Interrupted: " + + Thread.currentThread().isInterrupted()); if (!fetcherExecutor.isShutdown()) { fetcherExecutor.shutdownNow(); } @@ -540,16 +543,15 @@ Fetcher constructFetcherForHost(InputHost inputHost, Configuration conf) { if (sharedFetchEnabled) { // pick a single lock disk from the edge name's hashcode + host hashcode - final int h = Math.abs(Objects.hashCode(this.srcNameTrimmed, inputHost.getHost())); + final int h = Math.abs(Objects.hashCode(this.sourceDestNameTrimmed, inputHost.getHost())); lockDisk = new Path(this.localDisks[h % this.localDisks.length], "locks"); } FetcherBuilder fetcherBuilder = new FetcherBuilder(ShuffleManager.this, - httpConnectionParams, inputManager, inputContext.getApplicationId(), inputContext.getDagIdentifier(), - jobTokenSecretMgr, srcNameTrimmed, conf, localFs, localDirAllocator, + httpConnectionParams, inputManager, inputContext, + jobTokenSecretMgr, conf, localFs, localDirAllocator, lockDisk, localDiskFetchEnabled, sharedFetchEnabled, - localhostName, shufflePort, asyncHttp, verifyDiskChecksum, compositeFetch, enableFetcherTestingErrors, - inputContext.getObjectRegistry()); + localhostName, shufflePort, asyncHttp, verifyDiskChecksum, compositeFetch, enableFetcherTestingErrors); if (codec != null) { fetcherBuilder.setCompressionParameters(codec); @@ -632,7 +634,7 @@ public void addKnownInput(String hostName, int port, } } if (LOG.isDebugEnabled()) { - LOG.debug(srcNameTrimmed + ": " + "Adding input: " + + LOG.debug(sourceDestNameTrimmed + ": " + "Adding input: " + srcAttemptIdentifier + ", to host: " + host); } @@ -878,7 +880,7 @@ private void adjustCompletedInputs(FetchedInput fetchedInput) { if (fetchedInput instanceof NullFetchedInput) { completedInputs.add(fetchedInput); } - LOG.info("All inputs fetched for input vertex : " + inputContext.getSourceVertexName()); + LOG.info("All inputs fetched for input vertex : " + inputContext.getInputOutputVertexNames()); } } finally { lock.unlock(); @@ -950,7 +952,7 @@ public void fetchFailed(String host, LOG.info( "{}: Fetch failed for src: {} InputIdentifier: {}, connectFailed: {}, " + "local fetch: {}, remote fetch failure reported as local failure: {})", - srcNameTrimmed, srcAttemptIdentifier, srcAttemptIdentifier, connectFailed, + sourceDestNameTrimmed, srcAttemptIdentifier, srcAttemptIdentifier, connectFailed, inputAttemptFetchFailure.isLocalFetch(), inputAttemptFetchFailure.isDiskErrorAtSource()); failedShufflesCounter.increment(1); inputContext.notifyProgress(); @@ -988,11 +990,11 @@ public void shutdown() throws InterruptedException { if (Thread.currentThread().isInterrupted()) { //TODO: need to cleanup all FetchedInput (DiskFetchedInput, LocalDisFetchedInput), lockFile //As of now relying on job cleanup (when all directories would be cleared) - LOG.info(srcNameTrimmed + ": " + "Thread interrupted. Need to cleanup the local dirs"); + LOG.info(sourceDestNameTrimmed + ": " + "Thread interrupted. Need to cleanup the local dirs"); } if (!isShutdown.getAndSet(true)) { // Shut down any pending fetchers - LOG.info("Shutting down pending fetchers on source" + srcNameTrimmed + ": " + LOG.info("Shutting down pending fetchers on source" + sourceDestNameTrimmed + ": " + runningFetchers.size()); lock.lock(); try { @@ -1140,15 +1142,15 @@ private class SchedulerFutureCallback implements FutureCallback { @Override public void onSuccess(Void result) { - LOG.info(srcNameTrimmed + ": " + "Scheduler thread completed"); + LOG.info(sourceDestNameTrimmed + ": " + "Scheduler thread completed"); } @Override public void onFailure(Throwable t) { if (isShutdown.get()) { - LOG.debug("{}: Already shutdown. Ignoring error.", srcNameTrimmed, t); + LOG.debug("{}: Already shutdown. Ignoring error.", sourceDestNameTrimmed, t); } else { - LOG.error(srcNameTrimmed + ": " + "Scheduler failed with error: ", t); + LOG.error(sourceDestNameTrimmed + ": " + "Scheduler failed with error: ", t); inputContext.reportFailure(TaskFailureType.NON_FATAL, t, "Shuffle Scheduler Failed"); } } @@ -1177,7 +1179,7 @@ private void doBookKeepingForFetcherComplete() { public void onSuccess(FetchResult result) { fetcher.shutdown(); if (isShutdown.get()) { - LOG.debug("{}: Already shutdown. Ignoring event from fetcher", srcNameTrimmed); + LOG.debug("{}: Already shutdown. Ignoring event from fetcher", sourceDestNameTrimmed); } else { Iterable pendingInputs = result.getPendingInputs(); if (pendingInputs != null && pendingInputs.iterator().hasNext()) { @@ -1200,9 +1202,9 @@ public void onFailure(Throwable t) { // Unsuccessful - the fetcher may not have shutdown correctly. Try shutting it down. fetcher.shutdown(); if (isShutdown.get()) { - LOG.debug("{}: Already shutdown. Ignoring error from fetcher.", srcNameTrimmed, t); + LOG.debug("{}: Already shutdown. Ignoring error from fetcher.", sourceDestNameTrimmed, t); } else { - LOG.error(srcNameTrimmed + ": " + "Fetcher failed with error: ", t); + LOG.error(sourceDestNameTrimmed + ": " + "Fetcher failed with error: ", t); shuffleError = t; inputContext.reportFailure(TaskFailureType.NON_FATAL, t, "Fetch failed"); doBookKeepingForFetcherComplete(); 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 2725debb56..5887dcb3ca 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 @@ -42,8 +42,10 @@ import org.apache.hadoop.fs.RawLocalFileSystem; import org.apache.hadoop.io.compress.CompressionCodec; import org.apache.tez.common.TezRuntimeFrameworkConfigs; +import org.apache.tez.common.TezUtilsInternal; import org.apache.tez.common.counters.TezCounter; import org.apache.tez.common.security.JobTokenSecretManager; +import org.apache.tez.runtime.api.InputContext; import org.apache.tez.runtime.library.common.Constants; import org.apache.tez.runtime.library.common.InputAttemptIdentifier; import org.apache.tez.runtime.library.common.shuffle.orderedgrouped.MapOutput.Type; @@ -121,7 +123,6 @@ public FetcherOrderedGrouped(HttpConnectionParams httpConnectionParams, boolean localDiskFetchEnabled, String localHostname, int shufflePort, - String srcNameTrimmed, MapHost mapHost, TezCounter ioErrsCounter, TezCounter wrongLengthErrsCounter, @@ -129,12 +130,11 @@ public FetcherOrderedGrouped(HttpConnectionParams httpConnectionParams, TezCounter wrongMapErrsCounter, TezCounter connectionErrsCounter, TezCounter wrongReduceErrsCounter, - String applicationId, - int dagId, boolean asyncHttp, boolean sslShuffle, boolean verifyDiskChecksum, - boolean compositeFetch) { + boolean compositeFetch, + InputContext inputContext) { this.scheduler = scheduler; this.allocator = allocator; this.exceptionReporter = exceptionReporter; @@ -149,8 +149,8 @@ public FetcherOrderedGrouped(HttpConnectionParams httpConnectionParams, this.badIdErrs = badIdErrsCounter; this.connectionErrs = connectionErrsCounter; this.wrongReduceErrs = wrongReduceErrsCounter; - this.applicationId = applicationId; - this.dagId = dagId; + this.applicationId = inputContext.getApplicationId().toString(); + this.dagId = inputContext.getDagIdentifier(); this.ifileReadAhead = ifileReadAhead; this.ifileReadAheadLength = ifileReadAheadLength; @@ -171,7 +171,9 @@ public FetcherOrderedGrouped(HttpConnectionParams httpConnectionParams, this.verifyDiskChecksum = verifyDiskChecksum; this.compositeFetch = compositeFetch; - this.logIdentifier = "fetcher [" + srcNameTrimmed + "] #" + id; + String sourceDestNameTrimmed = TezUtilsInternal.cleanVertexName(inputContext.getSourceVertexName()) + " -> " + + TezUtilsInternal.cleanVertexName(inputContext.getTaskVertexName()); + this.logIdentifier = "fetcher [" + sourceDestNameTrimmed + "] #" + id; } @VisibleForTesting diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/FetcherOrderedGroupedWithInjectableErrors.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/FetcherOrderedGroupedWithInjectableErrors.java index 94172d12b9..9c782f6585 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/FetcherOrderedGroupedWithInjectableErrors.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/FetcherOrderedGroupedWithInjectableErrors.java @@ -19,10 +19,11 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.RawLocalFileSystem; import org.apache.hadoop.io.compress.CompressionCodec; +import org.apache.tez.common.TezUtilsInternal; import org.apache.tez.common.counters.TezCounter; import org.apache.tez.common.security.JobTokenSecretManager; import org.apache.tez.http.HttpConnectionParams; -import org.apache.tez.runtime.api.ObjectRegistry; +import org.apache.tez.runtime.api.InputContext; import org.apache.tez.runtime.library.common.InputAttemptIdentifier; import org.apache.tez.runtime.library.common.shuffle.FetcherErrorTestingConfig; import org.slf4j.Logger; @@ -38,17 +39,16 @@ public FetcherOrderedGroupedWithInjectableErrors(HttpConnectionParams httpConnec ShuffleScheduler scheduler, FetchedInputAllocatorOrderedGrouped allocator, ExceptionReporter exceptionReporter, JobTokenSecretManager jobTokenSecretMgr, boolean ifileReadAhead, int ifileReadAheadLength, CompressionCodec codec, Configuration conf, RawLocalFileSystem localFs, boolean localDiskFetchEnabled, String localHostname, - int shufflePort, String srcNameTrimmed, MapHost mapHost, TezCounter ioErrsCounter, + int shufflePort, MapHost mapHost, TezCounter ioErrsCounter, TezCounter wrongLengthErrsCounter, TezCounter badIdErrsCounter, TezCounter wrongMapErrsCounter, - TezCounter connectionErrsCounter, TezCounter wrongReduceErrsCounter, String applicationId, int dagId, - boolean asyncHttp, boolean sslShuffle, boolean verifyDiskChecksum, boolean compositeFetch, - ObjectRegistry objectRegistry) { + TezCounter connectionErrsCounter, TezCounter wrongReduceErrsCounter, boolean asyncHttp, + boolean sslShuffle, boolean verifyDiskChecksum, boolean compositeFetch, InputContext inputContext) { super(httpConnectionParams, scheduler, allocator, exceptionReporter, jobTokenSecretMgr, ifileReadAhead, - ifileReadAheadLength, codec, conf, localFs, localDiskFetchEnabled, localHostname, shufflePort, srcNameTrimmed, + ifileReadAheadLength, codec, conf, localFs, localDiskFetchEnabled, localHostname, shufflePort, mapHost, ioErrsCounter, wrongLengthErrsCounter, badIdErrsCounter, wrongMapErrsCounter, connectionErrsCounter, - wrongReduceErrsCounter, applicationId, dagId, asyncHttp, sslShuffle, verifyDiskChecksum, compositeFetch); - this.fetcherErrorTestingConfig = new FetcherErrorTestingConfig(conf, objectRegistry); - this.srcNameTrimmed = srcNameTrimmed; + wrongReduceErrsCounter, asyncHttp, sslShuffle, verifyDiskChecksum, compositeFetch, inputContext); + this.fetcherErrorTestingConfig = new FetcherErrorTestingConfig(conf, inputContext.getObjectRegistry()); + this.srcNameTrimmed = TezUtilsInternal.cleanVertexName(inputContext.getSourceVertexName()); LOG.info("Initialized FetcherOrderedGroupedWithInjectableErrors with config: {}", fetcherErrorTestingConfig); } diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/MergeManager.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/MergeManager.java index 46360e1287..8d3f37dbcd 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/MergeManager.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/MergeManager.java @@ -250,7 +250,7 @@ public MergeManager(Configuration conf, if (LOG.isDebugEnabled()) { LOG.debug( - inputContext.getSourceVertexName() + ": " + "InitialRequest: ShuffleMem=" + memLimit + + inputContext.getInputOutputVertexNames() + ": " + "InitialRequest: ShuffleMem=" + memLimit + ", postMergeMem=" + maxRedBuffer + ", RuntimeTotalAvailable=" + this.initialMemoryAvailable + ". Updated to: ShuffleMem=" @@ -285,7 +285,7 @@ public MergeManager(Configuration conf, conf.getFloat( TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_MERGE_PERCENT, TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_MERGE_PERCENT_DEFAULT)); - LOG.info(inputContext.getSourceVertexName() + ": MergerManager: memoryLimit=" + memoryLimit + ", " + + LOG.info(inputContext.getInputOutputVertexNames() + ": MergerManager: memoryLimit=" + memoryLimit + ", " + "maxSingleShuffleLimit=" + maxSingleShuffleLimit + ", " + "mergeThreshold=" + mergeThreshold + ", " + "ioSortFactor=" + ioSortFactor + ", " + @@ -549,8 +549,9 @@ private void trackAndLogCloseInMemoryFile(MapOutput mapOutput) { private void startMemToDiskMerge() { synchronized (inMemoryMerger) { if (!inMemoryMerger.isInProgress()) { - LOG.info(inputContext.getSourceVertexName() + ": " + "Starting inMemoryMerger's merge since commitMemory=" + - commitMemory + " > mergeThreshold=" + mergeThreshold + + LOG.info(inputContext.getInputOutputVertexNames() + ": " + + "Starting inMemoryMerger's merge since commitMemory=" + commitMemory + + " > mergeThreshold=" + mergeThreshold + ". Current usedMemory=" + usedMemory); inMemoryMapOutputs.addAll(inMemoryMergedMapOutputs); inMemoryMergedMapOutputs.clear(); @@ -788,8 +789,8 @@ public void merge(List inputs) throws IOException, InterruptedExcepti Writer writer = new InMemoryWriter(mergedMapOutputs.getMemory()); - LOG.info(inputContext.getSourceVertexName() + ": " + "Initiating Memory-to-Memory merge with " + noInMemorySegments + - " segments of total-size: " + mergeOutputSize); + LOG.info(inputContext.getInputOutputVertexNames() + ": " + "Initiating Memory-to-Memory merge with " + + noInMemorySegments + " segments of total-size: " + mergeOutputSize); if (Thread.currentThread().isInterrupted()) { return; // early exit @@ -808,7 +809,7 @@ public void merge(List inputs) throws IOException, InterruptedExcepti TezMerger.writeFile(rIter, writer, progressable, TezRuntimeConfiguration.TEZ_RUNTIME_RECORDS_BEFORE_PROGRESS_DEFAULT); writer.close(); - LOG.info(inputContext.getSourceVertexName() + + LOG.info(inputContext.getInputOutputVertexNames() + " Memory-to-Memory merge of the " + noInMemorySegments + " files in-memory complete with mergeOutputSize=" + mergeOutputSize); @@ -1051,7 +1052,7 @@ public void merge(List inputs) throws IOException, InterruptedExcepti final long outputLen = localFS.getFileStatus(outputPath).getLen(); closeOnDiskFile(new FileChunk(outputPath, 0, outputLen)); - LOG.info(inputContext.getSourceVertexName() + + LOG.info(inputContext.getInputOutputVertexNames() + " Finished merging " + inputs.size() + " map output files on disk of total-size " + approxOutputSize + "." + diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/Shuffle.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/Shuffle.java index db5ef734de..2b99739a58 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/Shuffle.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/Shuffle.java @@ -93,7 +93,7 @@ public class Shuffle implements ExceptionReporter { private volatile ListenableFuture runShuffleFuture; private final ListeningExecutorService executor; - private final String srcNameTrimmed; + private final String sourceDestNameTrimmed; private AtomicBoolean isShutDown = new AtomicBoolean(false); private AtomicBoolean fetchersClosed = new AtomicBoolean(false); @@ -109,7 +109,8 @@ public Shuffle(InputContext inputContext, Configuration conf, int numInputs, this.inputContext = inputContext; this.conf = conf; - this.srcNameTrimmed = TezUtilsInternal.cleanVertexName(inputContext.getSourceVertexName()); + this.sourceDestNameTrimmed = TezUtilsInternal.cleanVertexName(inputContext.getSourceVertexName()) + " -> " + + TezUtilsInternal.cleanVertexName(inputContext.getTaskVertexName()); this.codec = CodecUtils.getCodec(conf); @@ -138,7 +139,7 @@ public Shuffle(InputContext inputContext, Configuration conf, int numInputs, TezCounter mergedMapOutputsCounter = inputContext.getCounters().findCounter(TaskCounter.MERGED_MAP_OUTPUTS); - LOG.info(srcNameTrimmed + ": " + "Shuffle assigned with " + numInputs + " inputs" + ", codec: " + LOG.info(sourceDestNameTrimmed + ": " + "Shuffle assigned with " + numInputs + " inputs" + ", codec: " + (codec == null ? "None" : codec.getClass().getName()) + ", ifileReadAhead: " + ifileReadAhead); @@ -169,7 +170,7 @@ public Shuffle(InputContext inputContext, Configuration conf, int numInputs, codec, ifileReadAhead, ifileReadAheadLength, - srcNameTrimmed); + sourceDestNameTrimmed); this.mergePhaseTime = inputContext.getCounters().findCounter(TaskCounter.MERGE_PHASE_TIME); this.shufflePhaseTime = inputContext.getCounters().findCounter(TaskCounter.SHUFFLE_PHASE_TIME); @@ -182,7 +183,7 @@ public Shuffle(InputContext inputContext, Configuration conf, int numInputs, ShuffleUtils.isTezShuffleHandler(conf)); ExecutorService rawExecutor = Executors.newFixedThreadPool(1, new ThreadFactoryBuilder() - .setDaemon(true).setNameFormat("ShuffleAndMergeRunner {" + srcNameTrimmed + "}").build()); + .setDaemon(true).setNameFormat("ShuffleAndMergeRunner {" + sourceDestNameTrimmed + "}").build()); executor = MoreExecutors.listeningDecorator(rawExecutor); @@ -193,7 +194,7 @@ public void handleEvents(List events) throws IOException { if (!isShutDown.get()) { eventHandler.handleEvents(events); } else { - LOG.info(srcNameTrimmed + ": " + "Ignoring events since already shutdown. EventCount: " + events.size()); + LOG.info(sourceDestNameTrimmed + ": " + "Ignoring events since already shutdown. EventCount: " + events.size()); } } @@ -267,7 +268,7 @@ public void run() throws IOException { public void shutdown() { if (!isShutDown.getAndSet(true)) { // Interrupt so that the scheduler / merger sees this interrupt. - LOG.info("Shutting down Shuffle for source: " + srcNameTrimmed); + LOG.info("Shutting down Shuffle for source: " + sourceDestNameTrimmed); runShuffleFuture.cancel(true); cleanupIgnoreErrors(); } @@ -323,7 +324,7 @@ protected TezRawKeyValueIterator callInternal() throws IOException, InterruptedE } inputContext.inputIsReady(); - LOG.info("merge complete for input vertex : " + srcNameTrimmed); + LOG.info("merge complete for input vertex : " + sourceDestNameTrimmed); return kvIter; } } @@ -333,7 +334,8 @@ private void cleanupShuffleSchedulerIgnoreErrors() { cleanupShuffleScheduler(); } catch (InterruptedException e) { Thread.currentThread().interrupt(); - LOG.info(srcNameTrimmed + ": " + "Interrupted while attempting to close the scheduler during cleanup. Ignoring"); + LOG.info(sourceDestNameTrimmed + ": " + + "Interrupted while attempting to close the scheduler during cleanup. Ignoring"); } } @@ -351,13 +353,14 @@ private void cleanupMerger(boolean ignoreErrors) throws Throwable { if (ignoreErrors) { //Reset the status Thread.currentThread().interrupt(); - LOG.info(srcNameTrimmed + ": " + "Interrupted while attempting to close the merger during cleanup. Ignoring"); + LOG.info(sourceDestNameTrimmed + ": " + + "Interrupted while attempting to close the merger during cleanup. Ignoring"); } else { throw e; } } catch (Throwable e) { if (ignoreErrors) { - LOG.info(srcNameTrimmed + ": " + "Exception while trying to shutdown merger, Ignoring", e); + LOG.info(sourceDestNameTrimmed + ": " + "Exception while trying to shutdown merger, Ignoring", e); } else { throw e; } @@ -379,7 +382,7 @@ private void cleanupIgnoreErrors() { } cleanupMerger(true); } catch (Throwable t) { - LOG.info(srcNameTrimmed + ": " + "Error in cleaning up.., ", t); + LOG.info(sourceDestNameTrimmed + ": " + "Error in cleaning up.., ", t); } } @@ -388,7 +391,7 @@ private void cleanupIgnoreErrors() { public synchronized void reportException(Throwable t) { // RunShuffleCallable onFailure deals with ignoring errors on shutdown. if (throwable.get() == null) { - LOG.info(srcNameTrimmed + ": " + "Setting throwable in reportException with message [" + t.getMessage() + + LOG.info(sourceDestNameTrimmed + ": " + "Setting throwable in reportException with message [" + t.getMessage() + "] from thread [" + Thread.currentThread().getName()); throwable.set(t); throwingThreadName = Thread.currentThread().getName(); @@ -423,15 +426,15 @@ public static long getInitialMemoryRequirement(Configuration conf, long maxAvail private class ShuffleRunnerFutureCallback implements FutureCallback { @Override public void onSuccess(TezRawKeyValueIterator result) { - LOG.info(srcNameTrimmed + ": " + "Shuffle Runner thread complete"); + LOG.info(sourceDestNameTrimmed + ": " + "Shuffle Runner thread complete"); } @Override public void onFailure(Throwable t) { if (isShutDown.get()) { - LOG.info(srcNameTrimmed + ": " + "Already shutdown. Ignoring error"); + LOG.info(sourceDestNameTrimmed + ": " + "Already shutdown. Ignoring error"); } else { - LOG.error(srcNameTrimmed + ": " + "ShuffleRunner failed with error", t); + LOG.error(sourceDestNameTrimmed + ": " + "ShuffleRunner failed with error", t); // In case of an abort / Interrupt - the runtime makes sure that this is ignored. inputContext.reportFailure(TaskFailureType.NON_FATAL, t, "Shuffle Runner Failed"); cleanupIgnoreErrors(); 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 c97cfdf3ce..9984c5af10 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 @@ -77,7 +77,7 @@ public void handleEvents(List events) throws IOException { @Override public void logProgress(boolean updateOnClose) { - LOG.info(inputContext.getSourceVertexName() + ": " + LOG.info(inputContext.getInputOutputVertexNames() + ": " + "numDmeEventsSeen=" + numDmeEvents.get() + ", numDmeEventsSeenWithNoData=" + numDmeEventsNoData.get() + ", numObsoletionEventsSeen=" + numObsoletionEvents.get() 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 cd4b14ca1b..470b04cc5f 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 @@ -222,8 +222,6 @@ enum ShuffleErrors { private final boolean localDiskFetchEnabled; private final String localHostname; private final int shufflePort; - private final String applicationId; - private final int dagId; private final boolean asyncHttp; private final boolean sslShuffle; @@ -338,8 +336,6 @@ public ShuffleScheduler(InputContext inputContext, TezRuntimeConfiguration .TEZ_RUNTIME_SHUFFLE_FAILED_CHECK_SINCE_LAST_COMPLETION_DEFAULT); - this.applicationId = inputContext.getApplicationId().toString(); - this.dagId = inputContext.getDagIdentifier(); this.localHostname = inputContext.getExecutionContext().getHostName(); String auxiliaryService = conf.get(TezConfiguration.TEZ_AM_SHUFFLE_AUXILIARY_SERVICE_ID, TezConfiguration.TEZ_AM_SHUFFLE_AUXILIARY_SERVICE_ID_DEFAULT); @@ -673,7 +669,7 @@ public synchronized void copySucceeded(InputAttemptIdentifier srcAttemptIdentifi if (remainingMaps.get() == 0) { notifyAll(); // Notify the getHost() method. - LOG.info("All inputs fetched for input vertex : " + inputContext.getSourceVertexName()); + LOG.info("All inputs fetched for input vertex : " + inputContext.getInputOutputVertexNames()); } // update the status @@ -1474,17 +1470,17 @@ FetcherOrderedGrouped constructFetcherForHost(MapHost mapHost) { if (enableFetcherTestingErrors) { return new FetcherOrderedGroupedWithInjectableErrors(httpConnectionParams, ShuffleScheduler.this, allocator, exceptionReporter, jobTokenSecretManager, ifileReadAhead, ifileReadAheadLength, - codec, conf, localFs, localDiskFetchEnabled, localHostname, shufflePort, srcNameTrimmed, mapHost, + codec, conf, localFs, localDiskFetchEnabled, localHostname, shufflePort, mapHost, ioErrsCounter, wrongLengthErrsCounter, badIdErrsCounter, wrongMapErrsCounter, - connectionErrsCounter, wrongReduceErrsCounter, applicationId, dagId, asyncHttp, sslShuffle, - verifyDiskChecksum, compositeFetch, inputContext.getObjectRegistry()); + connectionErrsCounter, wrongReduceErrsCounter, asyncHttp, sslShuffle, + verifyDiskChecksum, compositeFetch, inputContext); } else { return new FetcherOrderedGrouped(httpConnectionParams, ShuffleScheduler.this, allocator, exceptionReporter, jobTokenSecretManager, ifileReadAhead, ifileReadAheadLength, - codec, conf, localFs, localDiskFetchEnabled, localHostname, shufflePort, srcNameTrimmed, mapHost, + codec, conf, localFs, localDiskFetchEnabled, localHostname, shufflePort, mapHost, ioErrsCounter, wrongLengthErrsCounter, badIdErrsCounter, wrongMapErrsCounter, - connectionErrsCounter, wrongReduceErrsCounter, applicationId, dagId, asyncHttp, sslShuffle, - verifyDiskChecksum, compositeFetch); + connectionErrsCounter, wrongReduceErrsCounter, asyncHttp, sslShuffle, + verifyDiskChecksum, compositeFetch, inputContext); } } 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 313c13d188..1463cfabbe 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 @@ -104,7 +104,7 @@ public synchronized List initialize() throws IOException { isStarted.set(true); getContext().inputIsReady(); LOG.info("input fetch not required since there are 0 physical inputs for input vertex: " - + getContext().getSourceVertexName()); + + getContext().getInputOutputVertexNames()); return Collections.emptyList(); } @@ -305,7 +305,7 @@ protected synchronized void createValuesIterator() RawComparator rawComparator = ConfigUtils.getIntermediateInputKeyComparator(conf); Class keyClass = ConfigUtils.getIntermediateInputKeyClass(conf); Class valClass = ConfigUtils.getIntermediateInputValueClass(conf); - LOG.info(getContext().getSourceVertexName() + ": " + "creating ValuesIterator with " + LOG.info(getContext().getInputOutputVertexNames() + ": " + "creating ValuesIterator with " + "comparator=" + rawComparator.getClass().getName() + ", keyClass=" + keyClass.getName() + ", valClass=" + valClass.getName()); 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 c67c405b43..38d5295094 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 @@ -92,7 +92,7 @@ public synchronized List initialize() throws Exception { isStarted.set(true); getContext().inputIsReady(); LOG.info("input fetch not required since there are 0 physical inputs for input vertex: " - + getContext().getSourceVertexName()); + + getContext().getInputOutputVertexNames()); return Collections.emptyList(); } else { long initalMemReq = getInitialMemoryReq(); @@ -148,7 +148,7 @@ public synchronized void start() throws IOException { pendingEvents.drainTo(pending); if (pending.size() > 0) { if (LOG.isDebugEnabled()) { - LOG.debug(getContext().getSourceVertexName() + ": " + "NoAutoStart delay in processing first event: " + LOG.debug(getContext().getInputOutputVertexNames() + ": " + "NoAutoStart delay in processing first event: " + (System.currentTimeMillis() - firstEventReceivedTime)); } inputEventHandler.handleEvents(pending); 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 cea3272818..338f640f91 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 @@ -81,7 +81,7 @@ public void testLocalFetchModeSetting() throws Exception { final boolean DISABLE_LOCAL_FETCH = false; Fetcher.FetcherBuilder builder = new Fetcher.FetcherBuilder(fetcherCallback, null, null, - ApplicationId.newInstance(0, 1), 1, null, "fetcherTest", conf, ENABLE_LOCAL_FETCH, HOST, + createMockInputContext(), null, conf, ENABLE_LOCAL_FETCH, HOST, PORT, false, true, false); builder.assignWork(HOST, PORT, 0, 1, Arrays.asList(srcAttempts)); Fetcher fetcher = spy(builder.build()); @@ -100,7 +100,7 @@ public void testLocalFetchModeSetting() throws Exception { // when enabled and hostname does not match use http fetch. builder = new Fetcher.FetcherBuilder(fetcherCallback, null, null, - ApplicationId.newInstance(0, 1), -1, null, "fetcherTest", conf, ENABLE_LOCAL_FETCH, HOST, + createMockInputContext(), null, conf, ENABLE_LOCAL_FETCH, HOST, PORT, false, true, false); builder.assignWork(HOST + "_OTHER", PORT, 0, 1, Arrays.asList(srcAttempts)); fetcher = spy(builder.build()); @@ -116,7 +116,7 @@ public void testLocalFetchModeSetting() throws Exception { // when enabled and port does not match use http fetch. builder = new Fetcher.FetcherBuilder(fetcherCallback, null, null, - ApplicationId.newInstance(0, 1), -1, null, "fetcherTest", conf, ENABLE_LOCAL_FETCH, HOST, + createMockInputContext(), null, conf, ENABLE_LOCAL_FETCH, HOST, PORT, false, true, false); builder.assignWork(HOST, PORT + 1, 0, 1, Arrays.asList(srcAttempts)); fetcher = spy(builder.build()); @@ -133,7 +133,7 @@ public void testLocalFetchModeSetting() throws Exception { // 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), 1, null, "fetcherTest", conf, DISABLE_LOCAL_FETCH, HOST, + createMockInputContext(), null, conf, DISABLE_LOCAL_FETCH, HOST, PORT, false, true, false); builder.assignWork(HOST, PORT, 0, 1, Arrays.asList(srcAttempts)); fetcher = spy(builder.build()); @@ -167,7 +167,7 @@ public void testSetupLocalDiskFetch() throws Exception { int partition = 42; FetcherCallback callback = mock(FetcherCallback.class); Fetcher.FetcherBuilder builder = new Fetcher.FetcherBuilder(callback, null, null, - ApplicationId.newInstance(0, 1), 1, null, "fetcherTest", conf, true, HOST, PORT, + createMockInputContext(), null, conf, true, HOST, PORT, false, true, true); ArrayList inputAttemptIdentifiers = new ArrayList<>(); for(CompositeInputAttemptIdentifier compositeInputAttemptIdentifier : srcAttempts) { @@ -306,7 +306,7 @@ public void testInputAttemptIdentifierMap() { int partition = 42; FetcherCallback callback = mock(FetcherCallback.class); Fetcher.FetcherBuilder builder = new Fetcher.FetcherBuilder(callback, null, null, - ApplicationId.newInstance(0, 1), 1, null, "fetcherTest", conf, true, HOST, PORT, + createMockInputContext(), null, conf, true, HOST, PORT, false, true, false); builder.assignWork(HOST, PORT, partition, 1, Arrays.asList(srcAttempts)); Fetcher fetcher = spy(builder.build()); @@ -330,7 +330,7 @@ public void testShuffleHandlerDiskErrorUnordered() doReturn("vertex").when(inputContext).getSourceVertexName(); Fetcher.FetcherBuilder builder = new Fetcher.FetcherBuilder(mock(ShuffleManager.class), null, - null, ApplicationId.newInstance(0, 1), 1, null, "fetcherTest", conf, true, HOST, PORT, + null, createMockInputContext(), null, conf, true, HOST, PORT, false, true, false); builder.assignWork(HOST, PORT, 0, 1, Arrays.asList(new InputAttemptIdentifier(0, 0))); @@ -345,4 +345,15 @@ public void testShuffleHandlerDiskErrorUnordered() Assert.assertTrue(failures[0].isDiskErrorAtSource()); Assert.assertFalse(failures[0].isLocalFetch()); } + + private InputContext createMockInputContext() { + InputContext inputContext = mock(InputContext.class); + + doReturn(ApplicationId.newInstance(0, 1)).when(inputContext).getApplicationId(); + doReturn(1).when(inputContext).getDagIdentifier(); + doReturn("sourceVertex").when(inputContext).getSourceVertexName(); + doReturn("taskVertex").when(inputContext).getTaskVertexName(); + + return inputContext; + } } diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/impl/TestShuffleInputEventHandlerImpl.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/impl/TestShuffleInputEventHandlerImpl.java index 683422b1a7..8a774dc1a5 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/impl/TestShuffleInputEventHandlerImpl.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/impl/TestShuffleInputEventHandlerImpl.java @@ -189,6 +189,7 @@ private InputContext createInputContext() throws IOException { InputContext inputContext = mock(InputContext.class); doReturn(new TezCounters()).when(inputContext).getCounters(); doReturn("sourceVertex").when(inputContext).getSourceVertexName(); + doReturn("taskVertex").when(inputContext).getTaskVertexName(); doReturn(shuffleMetaData).when(inputContext) .getServiceProviderMetaData(conf.get(TezConfiguration.TEZ_AM_SHUFFLE_AUXILIARY_SERVICE_ID, TezConfiguration.TEZ_AM_SHUFFLE_AUXILIARY_SERVICE_ID_DEFAULT)); diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/impl/TestShuffleManager.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/impl/TestShuffleManager.java index 041fd03854..fda2c896d1 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/impl/TestShuffleManager.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/impl/TestShuffleManager.java @@ -169,6 +169,7 @@ private InputContext createInputContext() throws IOException { InputContext inputContext = mock(InputContext.class); doReturn(new TezCounters()).when(inputContext).getCounters(); doReturn("sourceVertex").when(inputContext).getSourceVertexName(); + doReturn("taskVertex").when(inputContext).getTaskVertexName(); doReturn(shuffleMetaData).when(inputContext) .getServiceProviderMetaData(conf.get(TezConfiguration.TEZ_AM_SHUFFLE_AUXILIARY_SERVICE_ID, TezConfiguration.TEZ_AM_SHUFFLE_AUXILIARY_SERVICE_ID_DEFAULT)); 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 028fbce96a..f283780c12 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 @@ -90,7 +90,7 @@ public class TestFetcher { public static final String HOST = "localhost"; public static final int PORT = 65; public static final int DAG_ID = 1; - public static final String APP_ID = "application_1234_1"; + public static final ApplicationId APP_ID = ApplicationId.newInstance(0, 1); private TezCounters tezCounters = new TezCounters(); private TezCounter ioErrsCounter = tezCounters.findCounter(ShuffleScheduler.SHUFFLE_ERR_GRP_NAME, @@ -121,10 +121,6 @@ public void testInputsReturnedOnConnectionException() throws Exception { Shuffle shuffle = mock(Shuffle.class); - InputContext inputContext = mock(InputContext.class); - doReturn(new TezCounters()).when(inputContext).getCounters(); - doReturn("src vertex").when(inputContext).getSourceVertexName(); - MapHost mapHost = new MapHost(HOST, PORT, 0, 1); InputAttemptIdentifier inputAttemptIdentifier = new InputAttemptIdentifier(0, 0, "attempt"); mapHost.addKnownMap(inputAttemptIdentifier); @@ -133,9 +129,9 @@ public void testInputsReturnedOnConnectionException() throws Exception { FetcherOrderedGrouped fetcher = new FetcherOrderedGrouped(null, scheduler, merger, shuffle, null, false, 0, null, conf, - getRawFs(conf), false, HOST, PORT, "src vertex", mapHost, ioErrsCounter, + getRawFs(conf), false, HOST, PORT, mapHost, ioErrsCounter, wrongLengthErrsCounter, badIdErrsCounter, wrongMapErrsCounter, connectionErrsCounter, - wrongReduceErrsCounter, APP_ID, DAG_ID, false, false, true, false); + wrongReduceErrsCounter, false, false, true, false, createMockInputContext()); fetcher.call(); verify(scheduler).getMapsForHost(mapHost); @@ -151,18 +147,14 @@ public void testLocalFetchModeSetting1() throws Exception { MergeManager merger = mock(MergeManager.class); Shuffle shuffle = mock(Shuffle.class); - InputContext inputContext = mock(InputContext.class); - doReturn(new TezCounters()).when(inputContext).getCounters(); - doReturn("src vertex").when(inputContext).getSourceVertexName(); - final boolean ENABLE_LOCAL_FETCH = true; final boolean DISABLE_LOCAL_FETCH = false; MapHost mapHost = new MapHost(HOST, PORT, 0, 1); FetcherOrderedGrouped fetcher = new FetcherOrderedGrouped(null, scheduler, merger, shuffle, null, false, 0, null, conf, - getRawFs(conf), ENABLE_LOCAL_FETCH, HOST, PORT, "src vertex", mapHost, ioErrsCounter, + getRawFs(conf), ENABLE_LOCAL_FETCH, HOST, PORT, mapHost, ioErrsCounter, wrongLengthErrsCounter, badIdErrsCounter, wrongMapErrsCounter, connectionErrsCounter, - wrongReduceErrsCounter, APP_ID, DAG_ID, false, false, true, false); + wrongReduceErrsCounter, false, false, true, false, createMockInputContext()); // when local mode is enabled and host and port matches use local fetch FetcherOrderedGrouped spyFetcher = spy(fetcher); @@ -177,9 +169,9 @@ public void testLocalFetchModeSetting1() throws Exception { mapHost = new MapHost(HOST + "_OTHER", PORT, 0, 1); fetcher = new FetcherOrderedGrouped(null, scheduler, merger, shuffle, null, false, 0, null, conf, - getRawFs(conf), ENABLE_LOCAL_FETCH, HOST, PORT, "src vertex", mapHost, ioErrsCounter, + getRawFs(conf), ENABLE_LOCAL_FETCH, HOST, PORT, mapHost, ioErrsCounter, wrongLengthErrsCounter, badIdErrsCounter, wrongMapErrsCounter, connectionErrsCounter, - wrongReduceErrsCounter, APP_ID, DAG_ID, false, false, true, false); + wrongReduceErrsCounter, false, false, true, false, createMockInputContext()); spyFetcher = spy(fetcher); doNothing().when(spyFetcher).setupLocalDiskFetch(mapHost); @@ -192,9 +184,9 @@ public void testLocalFetchModeSetting1() throws Exception { mapHost = new MapHost(HOST, PORT + 1, 0, 1); fetcher = new FetcherOrderedGrouped(null, scheduler, merger, shuffle, null, false, 0, null, conf, - getRawFs(conf), ENABLE_LOCAL_FETCH, HOST, PORT, "src vertex", mapHost, ioErrsCounter, + getRawFs(conf), ENABLE_LOCAL_FETCH, HOST, PORT, mapHost, ioErrsCounter, wrongLengthErrsCounter, badIdErrsCounter, wrongMapErrsCounter, connectionErrsCounter, - wrongReduceErrsCounter, APP_ID, DAG_ID, false, false, true, false); + wrongReduceErrsCounter, false, false, true, false, createMockInputContext()); spyFetcher = spy(fetcher); doNothing().when(spyFetcher).setupLocalDiskFetch(mapHost); @@ -206,9 +198,9 @@ public void testLocalFetchModeSetting1() throws Exception { //if local fetch is not enabled mapHost = new MapHost(HOST, PORT, 0, 1); fetcher = new FetcherOrderedGrouped(null, scheduler, merger, shuffle, null, false, 0, null, - conf, getRawFs(conf), DISABLE_LOCAL_FETCH, HOST, PORT, "src vertex", mapHost, ioErrsCounter, + conf, getRawFs(conf), DISABLE_LOCAL_FETCH, HOST, PORT, mapHost, ioErrsCounter, wrongLengthErrsCounter, badIdErrsCounter, wrongMapErrsCounter, connectionErrsCounter, - wrongReduceErrsCounter, APP_ID, DAG_ID, false, false, true, false); + wrongReduceErrsCounter, false, false, true, false, createMockInputContext()); spyFetcher = spy(fetcher); doNothing().when(spyFetcher).setupLocalDiskFetch(mapHost); @@ -224,15 +216,12 @@ public void testSetupLocalDiskFetch() throws Exception { ShuffleScheduler scheduler = mock(ShuffleScheduler.class); MergeManager merger = mock(MergeManager.class); Shuffle shuffle = mock(Shuffle.class); - InputContext inputContext = mock(InputContext.class); - when(inputContext.getCounters()).thenReturn(new TezCounters()); - when(inputContext.getSourceVertexName()).thenReturn(""); MapHost host = new MapHost(HOST, PORT, 1, 1); FetcherOrderedGrouped fetcher = new FetcherOrderedGrouped(null, scheduler, merger, shuffle, - null, false, 0, null, conf, getRawFs(conf), true, HOST, PORT, "src vertex", host, + null, false, 0, null, conf, getRawFs(conf), true, HOST, PORT, host, ioErrsCounter, wrongLengthErrsCounter, badIdErrsCounter, wrongMapErrsCounter, - connectionErrsCounter, wrongReduceErrsCounter, APP_ID, DAG_ID, false, false, true, false); + connectionErrsCounter, wrongReduceErrsCounter, false, false, true, false, createMockInputContext()); FetcherOrderedGrouped spyFetcher = spy(fetcher); @@ -338,15 +327,12 @@ public void testSetupLocalDiskFetchEmptyPartitions() throws Exception { ShuffleScheduler scheduler = mock(ShuffleScheduler.class); MergeManager merger = mock(MergeManager.class); Shuffle shuffle = mock(Shuffle.class); - InputContext inputContext = mock(InputContext.class); - when(inputContext.getCounters()).thenReturn(new TezCounters()); - when(inputContext.getSourceVertexName()).thenReturn(""); MapHost host = new MapHost(HOST, PORT, 1, 1); FetcherOrderedGrouped fetcher = new FetcherOrderedGrouped(null, scheduler, merger, shuffle, - null, false, 0, null, conf, getRawFs(conf), true, HOST, PORT, "src vertex", host, + null, false, 0, null, conf, getRawFs(conf), true, HOST, PORT, host, ioErrsCounter, wrongLengthErrsCounter, badIdErrsCounter, wrongMapErrsCounter, - connectionErrsCounter, wrongReduceErrsCounter, APP_ID, DAG_ID, false, false, true, false); + connectionErrsCounter, wrongReduceErrsCounter, false, false, true, false, createMockInputContext()); FetcherOrderedGrouped spyFetcher = spy(fetcher); final List srcAttempts = Arrays.asList( @@ -413,15 +399,12 @@ public void testSetupLocalDiskFetchAutoReduce() throws Exception { ShuffleScheduler scheduler = mock(ShuffleScheduler.class); MergeManager merger = mock(MergeManager.class); Shuffle shuffle = mock(Shuffle.class); - InputContext inputContext = mock(InputContext.class); - when(inputContext.getCounters()).thenReturn(new TezCounters()); - when(inputContext.getSourceVertexName()).thenReturn(""); MapHost host = new MapHost(HOST, PORT, 1, 2); FetcherOrderedGrouped fetcher = new FetcherOrderedGrouped(null, scheduler, merger, shuffle, - null, false, 0, null, conf, getRawFs(conf), true, HOST, PORT, "src vertex", host, + null, false, 0, null, conf, getRawFs(conf), true, HOST, PORT, host, ioErrsCounter, wrongLengthErrsCounter, badIdErrsCounter, wrongMapErrsCounter, - connectionErrsCounter, wrongReduceErrsCounter, APP_ID, DAG_ID, false, false, true, false); + connectionErrsCounter, wrongReduceErrsCounter, false, false, true, false, createMockInputContext()); FetcherOrderedGrouped spyFetcher = spy(fetcher); @@ -593,17 +576,12 @@ public void testWithRetry() throws Exception { MergeManager merger = mock(MergeManager.class); Shuffle shuffle = mock(Shuffle.class); - InputContext inputContext = mock(InputContext.class); - when(inputContext.getCounters()).thenReturn(new TezCounters()); - when(inputContext.getSourceVertexName()).thenReturn(""); - when(inputContext.getApplicationId()).thenReturn(ApplicationId.newInstance(0, 1)); - HttpConnectionParams httpConnectionParams = ShuffleUtils.getHttpConnectionParams(conf); final MapHost host = new MapHost(HOST, PORT, 1, 1); FetcherOrderedGrouped mockFetcher = new FetcherOrderedGrouped(null, scheduler, merger, shuffle, - null, false, 0, null, conf, getRawFs(conf), false, HOST, PORT, "src vertex", host, + null, false, 0, null, conf, getRawFs(conf), false, HOST, PORT, host, ioErrsCounter, wrongLengthErrsCounter, badIdErrsCounter, wrongMapErrsCounter, - connectionErrsCounter, wrongReduceErrsCounter, APP_ID, DAG_ID, false, false, true, false); + connectionErrsCounter, wrongReduceErrsCounter, false, false, true, false, createMockInputContext()); final FetcherOrderedGrouped fetcher = spy(mockFetcher); @@ -676,11 +654,6 @@ public void testAsyncWithException() throws Exception { MergeManager merger = mock(MergeManager.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)); @@ -688,9 +661,9 @@ public void testAsyncWithException() throws Exception { final MapHost host = new MapHost(HOST, PORT, 1, 1); FetcherOrderedGrouped mockFetcher = new FetcherOrderedGrouped(httpConnectionParams, scheduler, merger, shuffle, jobMgr, false, - 0, null, conf, getRawFs(conf), false, HOST, PORT, "src vertex", host, ioErrsCounter, + 0, null, conf, getRawFs(conf), false, HOST, PORT, host, ioErrsCounter, wrongLengthErrsCounter, badIdErrsCounter, wrongMapErrsCounter, connectionErrsCounter, - wrongReduceErrsCounter, APP_ID, DAG_ID, true, false, true, false); + wrongReduceErrsCounter, true, false, true, false, createMockInputContext()); final FetcherOrderedGrouped fetcher = spy(mockFetcher); fetcher.remaining = new LinkedHashMap(); final List srcAttempts = Arrays.asList( @@ -753,9 +726,9 @@ public void testInputAttemptIdentifierMap() { Shuffle shuffle = mock(Shuffle.class); MapHost mapHost = new MapHost(HOST, PORT, 0, 1); FetcherOrderedGrouped fetcher = new FetcherOrderedGrouped(null, scheduler, merger, shuffle, - null, false, 0, null, conf, getRawFs(conf), false, HOST, PORT, "src vertex", mapHost, + null, false, 0, null, conf, getRawFs(conf), false, HOST, PORT, mapHost, ioErrsCounter, wrongLengthErrsCounter, badIdErrsCounter, wrongMapErrsCounter, - connectionErrsCounter, wrongReduceErrsCounter, APP_ID, DAG_ID, false, false, true, false); + connectionErrsCounter, wrongReduceErrsCounter, false, false, true, false, createMockInputContext()); fetcher.populateRemainingMap(new LinkedList(Arrays.asList(srcAttempts))); Assert.assertEquals(expectedSrcAttempts.length, fetcher.remaining.size()); Iterator> iterator = fetcher.remaining.entrySet().iterator(); @@ -773,9 +746,9 @@ public void testShuffleHandlerDiskErrorOrdered() InputAttemptIdentifier inputAttemptIdentifier = new InputAttemptIdentifier(0, 0, "attempt"); FetcherOrderedGrouped fetcher = new FetcherOrderedGrouped(null, null, null, null, null, false, - 0, null, new TezConfiguration(), null, false, HOST, PORT, "src vertex", mapHost, + 0, null, new TezConfiguration(), null, false, HOST, PORT, mapHost, ioErrsCounter, wrongLengthErrsCounter, badIdErrsCounter, wrongMapErrsCounter, - connectionErrsCounter, wrongReduceErrsCounter, APP_ID, DAG_ID, false, false, true, false); + connectionErrsCounter, wrongReduceErrsCounter, false, false, true, false, createMockInputContext()); fetcher.remaining = new HashMap(); ShuffleHeader header = @@ -800,4 +773,17 @@ private RawLocalFileSystem getRawFs(Configuration conf) { throw new RuntimeException(e); } } + + + private InputContext createMockInputContext() { + InputContext inputContext = mock(InputContext.class); + + doReturn(APP_ID).when(inputContext).getApplicationId(); + doReturn(DAG_ID).when(inputContext).getDagIdentifier(); + doReturn(new TezCounters()).when(inputContext).getCounters(); + doReturn("src vertex").when(inputContext).getSourceVertexName(); + doReturn("task_Vertex").when(inputContext).getTaskVertexName(); + + return inputContext; + } } diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestShuffle.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestShuffle.java index a28b1fa1a5..0a9c37e9ea 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestShuffle.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestShuffle.java @@ -144,6 +144,7 @@ private InputContext createTezInputContext() throws IOException { InputContext inputContext = mock(InputContext.class); doReturn(applicationId).when(inputContext).getApplicationId(); doReturn("sourceVertex").when(inputContext).getSourceVertexName(); + doReturn("taskVertex").when(inputContext).getTaskVertexName(); when(inputContext.getCounters()).thenReturn(new TezCounters()); ExecutionContext executionContext = new ExecutionContextImpl("localhost"); doReturn(executionContext).when(inputContext).getExecutionContext(); diff --git a/tez-tests/src/test/java/org/apache/tez/test/TestInput.java b/tez-tests/src/test/java/org/apache/tez/test/TestInput.java index 56bfe49dc8..36bb983d1d 100644 --- a/tez-tests/src/test/java/org/apache/tez/test/TestInput.java +++ b/tez-tests/src/test/java/org/apache/tez/test/TestInput.java @@ -352,7 +352,8 @@ public void handleEvents(List inputEvents) throws Exception { if (event instanceof DataMovementEvent) { DataMovementEvent dmEvent = (DataMovementEvent) event; numCompletedInputs++; - LOG.info(getContext().getSourceVertexName() + " Received DataMovement event sourceId : " + dmEvent.getSourceIndex() + + LOG.info(getContext().getInputOutputVertexNames() + + " Received DataMovement event sourceId : " + dmEvent.getSourceIndex() + " targetId: " + dmEvent.getTargetIndex() + " version: " + dmEvent.getVersion() + " numInputs: " + getNumPhysicalInputs() + From 0210dab45042b20d40437aa416740df9d2482309 Mon Sep 17 00:00:00 2001 From: Bodor Laszlo Date: Fri, 21 Jan 2022 18:07:49 +0100 Subject: [PATCH 087/137] TEZ-4310: Avoid duplicated precommit comments (#176) --- Jenkinsfile | 4 ---- 1 file changed, 4 deletions(-) diff --git a/Jenkinsfile b/Jenkinsfile index aadfaa953e..4ce236598d 100644 --- a/Jenkinsfile +++ b/Jenkinsfile @@ -108,10 +108,6 @@ pipeline { YETUS_ARGS+=(--github-user="${GITHUB_USER}") YETUS_ARGS+=(--github-token="${GITHUB_TOKEN}") - # enable writing back to ASF JIRA - YETUS_ARGS+=(--jira-password="${JIRA_PASSWORD}") - YETUS_ARGS+=(--jira-user="${JIRA_USER}") - # auto-kill any surefire stragglers during unit test runs YETUS_ARGS+=("--reapermode=kill") From 02d574fd6a4681eca05cd1d6aba19304d6ada3d1 Mon Sep 17 00:00:00 2001 From: Bodor Laszlo Date: Fri, 21 Jan 2022 18:08:50 +0100 Subject: [PATCH 088/137] TEZ-4369: Upgrade netty to 4.1.72 due to CVE-2021-37136, CVE-2021-37137 (#175) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 5cf77deedf..aff586bad9 100644 --- a/pom.xml +++ b/pom.xml @@ -59,7 +59,7 @@ ${user.home}/clover.license 27.0-jre 3.3.1 - 4.1.61.Final + 4.1.72.Final 0.13.0 1.19 1.7.30 From 267ca11b497d080042feca5b98cf0fb7ca9b6e15 Mon Sep 17 00:00:00 2001 From: Bodor Laszlo Date: Fri, 21 Jan 2022 18:09:49 +0100 Subject: [PATCH 089/137] TEZ-4366: Disk utilization limit fix should be applied to all tez mini clusters (#173) * TEZ-4366: Disk utilization limit fix should be applied to all tez mini clusters --- .../org/apache/tez/common/TezTestUtils.java | 38 +++++++++++++++++++ .../tez/tests/MiniTezClusterWithTimeline.java | 7 ++++ .../org/apache/tez/test/MiniTezCluster.java | 6 +-- 3 files changed, 47 insertions(+), 4 deletions(-) create mode 100644 tez-common/src/main/java/org/apache/tez/common/TezTestUtils.java diff --git a/tez-common/src/main/java/org/apache/tez/common/TezTestUtils.java b/tez-common/src/main/java/org/apache/tez/common/TezTestUtils.java new file mode 100644 index 0000000000..4b9d5c3218 --- /dev/null +++ b/tez-common/src/main/java/org/apache/tez/common/TezTestUtils.java @@ -0,0 +1,38 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tez.common; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.yarn.conf.YarnConfiguration; + +public class TezTestUtils { + /** + * Ensures a reasonably high limit for yarn disk utilization. This is very important for tests, + * as devs keep bumping into silent test hangs where yarn simply considers their machines as unhealthy, + * as the default limit is 90%, even if a machine with 90% full disk is still able to function. + * @param conf + */ + public static void ensureHighDiskUtilizationLimit(Configuration conf) { + if (conf.getFloat(YarnConfiguration.NM_MAX_PER_DISK_UTILIZATION_PERCENTAGE, + YarnConfiguration.DEFAULT_NM_MAX_PER_DISK_UTILIZATION_PERCENTAGE) == + YarnConfiguration.DEFAULT_NM_MAX_PER_DISK_UTILIZATION_PERCENTAGE) { + conf.setFloat(YarnConfiguration.NM_MAX_PER_DISK_UTILIZATION_PERCENTAGE, 99.0f); + } + } +} diff --git a/tez-plugins/tez-yarn-timeline-history/src/test/java/org/apache/tez/tests/MiniTezClusterWithTimeline.java b/tez-plugins/tez-yarn-timeline-history/src/test/java/org/apache/tez/tests/MiniTezClusterWithTimeline.java index d13ebdbee2..f8a35a29c3 100644 --- a/tez-plugins/tez-yarn-timeline-history/src/test/java/org/apache/tez/tests/MiniTezClusterWithTimeline.java +++ b/tez-plugins/tez-yarn-timeline-history/src/test/java/org/apache/tez/tests/MiniTezClusterWithTimeline.java @@ -49,6 +49,7 @@ import org.apache.hadoop.yarn.server.MiniYARNCluster; import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor; import org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor; +import org.apache.tez.common.TezTestUtils; import org.apache.tez.dag.api.TezConfiguration; import org.apache.tez.dag.api.TezUncheckedException; import org.apache.tez.dag.app.DAGAppMaster; @@ -92,6 +93,12 @@ public MiniTezClusterWithTimeline(String testName, int noOfNMs, super(testName, 1, noOfNMs, numLocalDirs, numLogDirs, enableAHS); } + @Override + public void init(Configuration conf) { + TezTestUtils.ensureHighDiskUtilizationLimit(conf); + super.init(conf); + } + @Override public void serviceInit(Configuration conf) throws Exception { conf.set(MRConfig.FRAMEWORK_NAME, MRConfig.YARN_TEZ_FRAMEWORK_NAME); diff --git a/tez-tests/src/test/java/org/apache/tez/test/MiniTezCluster.java b/tez-tests/src/test/java/org/apache/tez/test/MiniTezCluster.java index 17c688590c..9af1e604b2 100644 --- a/tez-tests/src/test/java/org/apache/tez/test/MiniTezCluster.java +++ b/tez-tests/src/test/java/org/apache/tez/test/MiniTezCluster.java @@ -50,6 +50,7 @@ import org.apache.hadoop.yarn.server.MiniYARNCluster; import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor; import org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor; +import org.apache.tez.common.TezTestUtils; import org.apache.tez.dag.api.TezConfiguration; import org.apache.tez.dag.api.TezUncheckedException; import org.apache.tez.dag.app.DAGAppMaster; @@ -90,10 +91,7 @@ public MiniTezCluster(String testName, int noOfNMs, @Override public void init(Configuration conf) { - if (conf.getFloat(YarnConfiguration.NM_MAX_PER_DISK_UTILIZATION_PERCENTAGE, - YarnConfiguration.DEFAULT_NM_MAX_PER_DISK_UTILIZATION_PERCENTAGE) == YarnConfiguration.DEFAULT_NM_MAX_PER_DISK_UTILIZATION_PERCENTAGE) { - conf.setFloat(YarnConfiguration.NM_MAX_PER_DISK_UTILIZATION_PERCENTAGE, 99.0f); - } + TezTestUtils.ensureHighDiskUtilizationLimit(conf); super.init(conf); } From 1176386c9519d90b69371e954f8b60b20dabc2f7 Mon Sep 17 00:00:00 2001 From: Bodor Laszlo Date: Fri, 21 Jan 2022 21:40:28 +0100 Subject: [PATCH 090/137] TEZ-4350: Remove synchronized from DAGAppMaster.serviceInit (#162) --- tez-dag/findbugs-exclude.xml | 18 +--- .../org/apache/tez/dag/app/DAGAppMaster.java | 94 +++++++++---------- 2 files changed, 47 insertions(+), 65 deletions(-) diff --git a/tez-dag/findbugs-exclude.xml b/tez-dag/findbugs-exclude.xml index 50422ff0e0..e8755832f4 100644 --- a/tez-dag/findbugs-exclude.xml +++ b/tez-dag/findbugs-exclude.xml @@ -154,24 +154,8 @@ - - - - - - - - - - - - - - - - - + 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 abc10bd86c..f15c73c04a 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 @@ -416,7 +416,7 @@ private long getAMGCTime() { } @Override - public synchronized void serviceInit(final Configuration conf) throws Exception { + protected void serviceInit(final Configuration conf) throws Exception { this.amConf = conf; initResourceCalculatorPlugins(); @@ -1923,7 +1923,7 @@ private DAGRecoveryData recoverDAG() throws IOException, TezException { } @Override - public synchronized void serviceStart() throws Exception { + public void serviceStart() throws Exception { //start all the components startServices(); super.serviceStart(); @@ -2115,57 +2115,55 @@ public void serviceStop() throws Exception { if (isSession) { sessionStopped.set(true); } - synchronized (this) { - if (this.dagSubmissionTimer != null) { - this.dagSubmissionTimer.cancel(); - } - if (this.clientAMHeartBeatTimeoutService != null) { - this.clientAMHeartBeatTimeoutService.shutdownNow(); - } - // release all the held containers before stop services TEZ-2687 - initiateStop(); - stopServices(); - - // Given pre-emption, we should delete tez scratch dir only if unregister is - // successful - boolean deleteTezScratchData = this.amConf.getBoolean( - TezConfiguration.TEZ_AM_STAGING_SCRATCH_DATA_AUTO_DELETE, - TezConfiguration.TEZ_AM_STAGING_SCRATCH_DATA_AUTO_DELETE_DEFAULT); - LOG.debug("Checking whether tez scratch data dir should be deleted, deleteTezScratchData={}", - deleteTezScratchData); - if (deleteTezScratchData && this.taskSchedulerManager != null - && this.taskSchedulerManager.hasUnregistered()) { - // Delete tez scratch data dir - if (this.tezSystemStagingDir != null) { - try { - this.appMasterUgi.doAs(new PrivilegedExceptionAction() { - @Override - public Void run() throws Exception { - FileSystem fs = tezSystemStagingDir.getFileSystem(amConf); - boolean deletedStagingDir = fs.delete(tezSystemStagingDir, true); - if (!deletedStagingDir) { - LOG.warn("Failed to delete tez scratch data dir, path=" - + tezSystemStagingDir); - } else { - LOG.info("Completed deletion of tez scratch data dir, path=" - + tezSystemStagingDir); - } - return null; + if (this.dagSubmissionTimer != null) { + this.dagSubmissionTimer.cancel(); + } + if (this.clientAMHeartBeatTimeoutService != null) { + this.clientAMHeartBeatTimeoutService.shutdownNow(); + } + // release all the held containers before stop services TEZ-2687 + initiateStop(); + stopServices(); + + // Given pre-emption, we should delete tez scratch dir only if unregister is + // successful + boolean deleteTezScratchData = this.amConf.getBoolean( + TezConfiguration.TEZ_AM_STAGING_SCRATCH_DATA_AUTO_DELETE, + TezConfiguration.TEZ_AM_STAGING_SCRATCH_DATA_AUTO_DELETE_DEFAULT); + LOG.debug("Checking whether tez scratch data dir should be deleted, deleteTezScratchData={}", + deleteTezScratchData); + if (deleteTezScratchData && this.taskSchedulerManager != null + && this.taskSchedulerManager.hasUnregistered()) { + // Delete tez scratch data dir + if (this.tezSystemStagingDir != null) { + try { + this.appMasterUgi.doAs(new PrivilegedExceptionAction() { + @Override + public Void run() throws Exception { + FileSystem fs = tezSystemStagingDir.getFileSystem(amConf); + boolean deletedStagingDir = fs.delete(tezSystemStagingDir, true); + if (!deletedStagingDir) { + LOG.warn("Failed to delete tez scratch data dir, path=" + + tezSystemStagingDir); + } else { + LOG.info("Completed deletion of tez scratch data dir, path=" + + tezSystemStagingDir); } - }); - } catch (IOException e) { - // Best effort to delete tez scratch data dir - LOG.warn("Failed to delete tez scratch data dir", e); - } + return null; + } + }); + } catch (IOException e) { + // Best effort to delete tez scratch data dir + LOG.warn("Failed to delete tez scratch data dir", e); } } + } - if (execService != null) { - execService.shutdownNow(); - } - - super.serviceStop(); + if (execService != null) { + execService.shutdownNow(); } + + super.serviceStop(); } private class DagEventDispatcher implements EventHandler { From 4e3eb9574a9c59ea534a70021531337b2de13ed3 Mon Sep 17 00:00:00 2001 From: Syed Shameerur Rahman Date: Thu, 27 Jan 2022 19:47:53 +0530 Subject: [PATCH 091/137] TEZ-4129: Delete intermediate attempt data for failed attempts for Shuffle Handler (#72) (Syed Shameerur Rahman reviewed by Laszlo Bodor) --- .../apache/tez/dag/api/TezConfiguration.java | 12 ++- .../tez/common/DagContainerLauncher.java | 5 ++ .../org/apache/tez/dag/app/DAGAppMaster.java | 5 ++ .../apache/tez/dag/app/dag/impl/TaskImpl.java | 9 +- .../launcher/ContainerLauncherManager.java | 8 ++ .../launcher/ContainerLauncherWrapper.java | 9 ++ .../tez/dag/app/launcher/DeletionTracker.java | 6 ++ .../dag/app/launcher/DeletionTrackerImpl.java | 21 +++++ .../app/launcher/LocalContainerLauncher.java | 23 +++++- .../TaskAttemptFailedDeleteRunnable.java | 82 +++++++++++++++++++ .../launcher/TezContainerLauncherImpl.java | 25 ++++-- .../TestContainerLauncherWrapper.java | 2 +- .../tez/auxservices/ShuffleHandler.java | 54 +++++++++++- .../tez/auxservices/TestShuffleHandler.java | 82 +++++++++++++++++++ .../library/common/TezRuntimeUtils.java | 19 +++++ 15 files changed, 348 insertions(+), 14 deletions(-) create mode 100644 tez-dag/src/main/java/org/apache/tez/dag/app/launcher/TaskAttemptFailedDeleteRunnable.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 ee991ccf53..16d1dfcb70 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 @@ -884,7 +884,17 @@ public TezConfiguration(boolean loadDefaults) { public static final boolean TEZ_AM_DAG_CLEANUP_ON_COMPLETION_DEFAULT = false; /** - * Int value. Upper limit on the number of threads used to delete DAG directories on nodes. + * Boolean value. Instructs AM to delete intermediate attempt data for failed task attempts. + */ + @ConfigurationScope(Scope.AM) + @ConfigurationProperty(type="boolean") + public static final String TEZ_AM_TASK_ATTEMPT_CLEANUP_ON_FAILURE = TEZ_AM_PREFIX + + "task.attempt.cleanup.on.failure"; + public static final boolean TEZ_AM_TASK_ATTEMPT_CLEANUP_ON_FAILURE_DEFAULT = false; + + /** + * Int value. Upper limit on the number of threads used to delete DAG directories and failed task attempts + * directories on nodes. */ @ConfigurationScope(Scope.AM) @ConfigurationProperty(type="integer") diff --git a/tez-common/src/main/java/org/apache/tez/common/DagContainerLauncher.java b/tez-common/src/main/java/org/apache/tez/common/DagContainerLauncher.java index e3bd385a1a..6bda0a8da3 100644 --- a/tez-common/src/main/java/org/apache/tez/common/DagContainerLauncher.java +++ b/tez-common/src/main/java/org/apache/tez/common/DagContainerLauncher.java @@ -20,8 +20,10 @@ import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.tez.common.security.JobTokenSecretManager; import org.apache.tez.dag.records.TezDAGID; +import org.apache.tez.dag.records.TezTaskAttemptID; import org.apache.tez.serviceplugins.api.ContainerLauncher; import org.apache.tez.serviceplugins.api.ContainerLauncherContext; @@ -40,4 +42,7 @@ public DagContainerLauncher(ContainerLauncherContext containerLauncherContext) { } public abstract void dagComplete(TezDAGID dag, JobTokenSecretManager jobTokenSecretManager); + + public abstract void taskAttemptFailed(TezTaskAttemptID taskAttemptID, + JobTokenSecretManager jobTokenSecretManager, NodeId nodeId); } 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 f15c73c04a..3be9aacec5 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 @@ -67,6 +67,7 @@ import org.apache.commons.cli.GnuParser; import org.apache.commons.cli.Options; import org.apache.commons.lang.exception.ExceptionUtils; +import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.tez.Utils; import org.apache.tez.client.CallerContext; import org.apache.tez.client.TezClientUtils; @@ -178,6 +179,7 @@ import org.apache.tez.dag.history.events.DAGSubmittedEvent; import org.apache.tez.dag.history.utils.DAGUtils; import org.apache.tez.dag.records.TezDAGID; +import org.apache.tez.dag.records.TezTaskAttemptID; import org.apache.tez.dag.records.TezVertexID; import org.apache.tez.dag.utils.RelocalizationUtils; import org.apache.tez.dag.utils.Simple2LevelVersionComparator; @@ -2738,4 +2740,7 @@ String buildPluginComponentLog(List namedEntityDescriptor return sb.toString(); } + public void taskAttemptFailed(TezTaskAttemptID attemptID, NodeId nodeId) { + getContainerLauncherManager().taskAttemptFailed(attemptID, jobTokenSecretManager, nodeId); + } } 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 cb8545f8ca..b840798c26 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 @@ -39,13 +39,13 @@ import org.apache.commons.lang.StringUtils; import org.apache.commons.lang.exception.ExceptionUtils; import org.apache.hadoop.classification.InterfaceAudience.Private; -import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.tez.dag.app.dag.event.TaskEventTAFailed; import org.apache.tez.runtime.api.TaskFailureType; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.event.EventHandler; import org.apache.hadoop.yarn.state.InvalidStateTransitonException; @@ -1263,6 +1263,13 @@ public TaskStateInternal transition(TaskImpl task, TaskEvent event) { } // The attempt would have informed the scheduler about it's failure + // Delete the intermediate shuffle data for failed task attempt + TaskAttempt taskAttempt = task.getAttempt(castEvent.getTaskAttemptID()); + if (taskAttempt.getAssignedContainer() != null) { + NodeId nodeId = taskAttempt.getAssignedContainer().getNodeId(); + task.appContext.getAppMaster().taskAttemptFailed(taskAttempt.getID(), nodeId); + } + task.taskAttemptStatus.put(castEvent.getTaskAttemptID().getId(), true); if (task.failedAttempts < task.maxFailedAttempts && castEvent.getTaskFailureType() == TaskFailureType.NON_FATAL) { diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherManager.java index d55787e25a..b0e0f0cf0d 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherManager.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherManager.java @@ -22,6 +22,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.service.AbstractService; +import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.hadoop.yarn.event.Event; import org.apache.hadoop.yarn.event.EventHandler; import org.apache.tez.Utils; @@ -35,6 +36,7 @@ import org.apache.tez.dag.app.dag.event.DAGAppMasterEventType; import org.apache.tez.dag.app.dag.event.DAGAppMasterEventUserServiceFatalError; import org.apache.tez.dag.records.TezDAGID; +import org.apache.tez.dag.records.TezTaskAttemptID; import org.apache.tez.serviceplugins.api.ContainerLaunchRequest; import org.apache.tez.serviceplugins.api.ContainerLauncher; import org.apache.tez.serviceplugins.api.ContainerLauncherContext; @@ -200,6 +202,12 @@ public void dagComplete(TezDAGID dag, JobTokenSecretManager secretManager) { } } + public void taskAttemptFailed(TezTaskAttemptID taskAttemptID, JobTokenSecretManager secretManager, NodeId nodeId) { + for (int i = 0; i < containerLaunchers.length; i++) { + containerLaunchers[i].taskAttemptFailed(taskAttemptID, secretManager, nodeId); + } + } + public void dagSubmitted() { // Nothing to do right now. Indicates that a new DAG has been submitted and // the context has updated information. diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherWrapper.java b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherWrapper.java index 8ecac14856..5d262bdab4 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherWrapper.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherWrapper.java @@ -15,8 +15,10 @@ package org.apache.tez.dag.app.launcher; import org.apache.tez.common.DagContainerLauncher; +import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.tez.common.security.JobTokenSecretManager; import org.apache.tez.dag.records.TezDAGID; +import org.apache.tez.dag.records.TezTaskAttemptID; import org.apache.tez.serviceplugins.api.ContainerLaunchRequest; import org.apache.tez.serviceplugins.api.ContainerLauncher; import org.apache.tez.serviceplugins.api.ContainerStopRequest; @@ -46,4 +48,11 @@ public void dagComplete(TezDAGID dag, JobTokenSecretManager jobTokenSecretManage ((DagContainerLauncher)real).dagComplete(dag, jobTokenSecretManager); } } + + public void taskAttemptFailed(TezTaskAttemptID taskAttemptID, JobTokenSecretManager jobTokenSecretManager, + NodeId nodeId) { + if (real instanceof DagContainerLauncher) { + ((DagContainerLauncher) real).taskAttemptFailed(taskAttemptID, jobTokenSecretManager, nodeId); + } + } } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/DeletionTracker.java b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/DeletionTracker.java index 27ece70513..87b7366bfc 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/DeletionTracker.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/DeletionTracker.java @@ -22,6 +22,7 @@ import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.tez.common.security.JobTokenSecretManager; import org.apache.tez.dag.records.TezDAGID; +import org.apache.tez.dag.records.TezTaskAttemptID; public abstract class DeletionTracker { @@ -35,6 +36,11 @@ public void dagComplete(TezDAGID dag, JobTokenSecretManager jobTokenSecretManage //do nothing } + public void taskAttemptFailed(TezTaskAttemptID taskAttemptID, JobTokenSecretManager jobTokenSecretManager, + NodeId nodeId) { + //do nothing + } + public void addNodeShufflePort(NodeId nodeId, int port) { //do nothing } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/DeletionTrackerImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/DeletionTrackerImpl.java index 06dae2d2b0..e4204bfc71 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/DeletionTrackerImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/DeletionTrackerImpl.java @@ -34,6 +34,7 @@ import org.apache.tez.dag.records.TezDAGID; import org.apache.hadoop.conf.Configuration; import org.apache.tez.dag.api.TezConfiguration; +import org.apache.tez.dag.records.TezTaskAttemptID; import org.apache.tez.runtime.library.common.TezRuntimeUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -70,6 +71,26 @@ public void dagComplete(TezDAGID dag, JobTokenSecretManager jobTokenSecretManage } } + @Override + public void taskAttemptFailed(TezTaskAttemptID taskAttemptID, JobTokenSecretManager jobTokenSecretManager, + NodeId nodeId) { + super.taskAttemptFailed(taskAttemptID, jobTokenSecretManager, nodeId); + if (nodeIdShufflePortMap == null || nodeIdShufflePortMap.get(nodeId) == null) { + LOG.warn("Unable to find the shuffle port for shuffle data deletion of failed task attempt."); + return; + } + int shufflePort = nodeIdShufflePortMap.get(nodeId); + if (shufflePort != TezRuntimeUtils.INVALID_PORT) { + TaskAttemptFailedRunnable taskAttemptFailedRunnable = new TaskAttemptFailedRunnable(nodeId, shufflePort, + taskAttemptID, TezRuntimeUtils.getHttpConnectionParams(conf), jobTokenSecretManager); + try { + dagCleanupService.submit(taskAttemptFailedRunnable); + } catch (RejectedExecutionException rejectedException) { + LOG.info("Ignoring failed task attempt deletion request for " + taskAttemptFailedRunnable); + } + } + } + @Override public void addNodeShufflePort(NodeId nodeId, int port) { if (port != TezRuntimeUtils.INVALID_PORT) { 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 ae308098a7..ebc8f95566 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 @@ -43,11 +43,13 @@ import com.google.common.util.concurrent.MoreExecutors; import com.google.common.util.concurrent.ThreadFactoryBuilder; +import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.tez.common.DagContainerLauncher; import org.apache.tez.common.ReflectionUtils; import org.apache.tez.common.TezUtils; import org.apache.tez.dag.records.TezDAGID; import org.apache.tez.common.security.JobTokenSecretManager; +import org.apache.tez.dag.records.TezTaskAttemptID; import org.apache.tez.runtime.library.common.TezRuntimeUtils; import org.apache.tez.runtime.library.common.shuffle.ShuffleUtils; import org.apache.tez.serviceplugins.api.ContainerLaunchRequest; @@ -93,6 +95,8 @@ public class LocalContainerLauncher extends DagContainerLauncher { private final boolean isLocalMode; int shufflePort = TezRuntimeUtils.INVALID_PORT; private DeletionTracker deletionTracker; + private boolean dagDelete; + private boolean failedTaskAttemptDelete; private final ConcurrentHashMap> runningContainers = @@ -155,10 +159,14 @@ public LocalContainerLauncher(ContainerLauncherContext containerLauncherContext, new ThreadFactoryBuilder().setDaemon(true).setNameFormat("LocalTaskExecutionThread #%d") .build()); this.taskExecutorService = MoreExecutors.listeningDecorator(rawExecutor); - boolean cleanupDagDataOnComplete = ShuffleUtils.isTezShuffleHandler(conf) - && conf.getBoolean(TezConfiguration.TEZ_AM_DAG_CLEANUP_ON_COMPLETION, + dagDelete = ShuffleUtils.isTezShuffleHandler(conf) && + conf.getBoolean(TezConfiguration.TEZ_AM_DAG_CLEANUP_ON_COMPLETION, TezConfiguration.TEZ_AM_DAG_CLEANUP_ON_COMPLETION_DEFAULT); - if (cleanupDagDataOnComplete) { + failedTaskAttemptDelete = ShuffleUtils.isTezShuffleHandler(conf) && + conf.getBoolean(TezConfiguration.TEZ_AM_TASK_ATTEMPT_CLEANUP_ON_FAILURE, + TezConfiguration.TEZ_AM_TASK_ATTEMPT_CLEANUP_ON_FAILURE_DEFAULT); + + if (dagDelete || failedTaskAttemptDelete) { String deletionTrackerClassName = conf.get(TezConfiguration.TEZ_AM_DELETION_TRACKER_CLASS, TezConfiguration.TEZ_AM_DELETION_TRACKER_CLASS_DEFAULT); deletionTracker = ReflectionUtils.createClazzInstance( @@ -441,9 +449,16 @@ public void stopContainer(ContainerStopRequest stopRequest) { @Override public void dagComplete(TezDAGID dag, JobTokenSecretManager jobTokenSecretManager) { - if (deletionTracker != null) { + if (dagDelete && deletionTracker != null) { deletionTracker.dagComplete(dag, jobTokenSecretManager); } } + @Override + public void taskAttemptFailed(TezTaskAttemptID taskAttemptID, JobTokenSecretManager jobTokenSecretManager, + NodeId nodeId) { + if (failedTaskAttemptDelete && deletionTracker != null) { + deletionTracker.taskAttemptFailed(taskAttemptID, jobTokenSecretManager, nodeId); + } + } } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/TaskAttemptFailedDeleteRunnable.java b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/TaskAttemptFailedDeleteRunnable.java new file mode 100644 index 0000000000..d2587b5190 --- /dev/null +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/TaskAttemptFailedDeleteRunnable.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 + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tez.dag.app.launcher; + +import org.apache.hadoop.yarn.api.records.NodeId; +import org.apache.tez.common.security.JobTokenSecretManager; +import org.apache.tez.dag.records.TezTaskAttemptID; +import org.apache.tez.http.BaseHttpConnection; +import org.apache.tez.http.HttpConnectionParams; +import org.apache.tez.runtime.library.common.TezRuntimeUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.net.URL; + +class TaskAttemptFailedRunnable implements Runnable { + private static final Logger LOG = LoggerFactory.getLogger(TaskAttemptFailedRunnable.class); + private final NodeId nodeId; + private final TezTaskAttemptID taskAttemptID; + private final JobTokenSecretManager jobTokenSecretManager; + private final int shufflePort; + private final HttpConnectionParams httpConnectionParams; + + TaskAttemptFailedRunnable(NodeId nodeId, int shufflePort, TezTaskAttemptID taskAttemptID, + HttpConnectionParams httpConnectionParams, + JobTokenSecretManager jobTokenSecretMgr) { + this.nodeId = nodeId; + this.shufflePort = shufflePort; + this.taskAttemptID = taskAttemptID; + this.httpConnectionParams = httpConnectionParams; + this.jobTokenSecretManager = jobTokenSecretMgr; + } + + @Override + public void run() { + BaseHttpConnection httpConnection = null; + try { + URL baseURL = TezRuntimeUtils.constructBaseURIForShuffleHandlerTaskAttemptFailed( + nodeId.getHost(), shufflePort, taskAttemptID.getTaskID().getVertexID().getDAGId(). + getApplicationId().toString(), taskAttemptID.getTaskID().getVertexID().getDAGId().getId(), + taskAttemptID.toString(), false); + httpConnection = TezRuntimeUtils.getHttpConnection(true, baseURL, httpConnectionParams, + "FailedTaskAttemptDelete", jobTokenSecretManager); + httpConnection.connect(); + httpConnection.getInputStream(); + } catch (Exception e) { + LOG.warn("Could not setup HTTP Connection to the node " + nodeId.getHost() + + " for failed task attempt delete. ", e); + } finally { + try { + if (httpConnection != null) { + httpConnection.cleanup(true); + } + } catch (IOException ioe) { + LOG.warn("Encountered IOException for " + nodeId.getHost() + " during close. ", ioe); + } + } + } + + @Override + public String toString() { + return "TaskAttemptFailedRunnable nodeId=" + nodeId + ", shufflePort=" + shufflePort + ", taskAttemptId=" + + taskAttemptID.toString(); + } +} diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/TezContainerLauncherImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/TezContainerLauncherImpl.java index f5be74683e..88ed4f7b89 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/TezContainerLauncherImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/TezContainerLauncherImpl.java @@ -34,6 +34,7 @@ import org.apache.commons.lang.exception.ExceptionUtils; import org.apache.hadoop.io.DataInputByteBuffer; +import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.tez.common.DagContainerLauncher; import org.apache.tez.common.ReflectionUtils; import org.apache.tez.common.TezUtils; @@ -41,6 +42,7 @@ import org.apache.tez.dag.api.TezConstants; import org.apache.tez.dag.api.TezException; import org.apache.tez.dag.records.TezDAGID; +import org.apache.tez.dag.records.TezTaskAttemptID; import org.apache.tez.runtime.library.common.TezRuntimeUtils; import org.apache.tez.runtime.library.common.shuffle.ShuffleUtils; import org.apache.tez.serviceplugins.api.ContainerLaunchRequest; @@ -90,6 +92,8 @@ public class TezContainerLauncherImpl extends DagContainerLauncher { private ContainerManagementProtocolProxy cmProxy; private AtomicBoolean serviceStopped = new AtomicBoolean(false); private DeletionTracker deletionTracker = null; + private boolean dagDelete; + private boolean failedTaskAttemptDelete; private Container getContainer(ContainerOp event) { ContainerId id = event.getBaseOperation().getContainerId(); @@ -332,10 +336,14 @@ public void run() { }; eventHandlingThread.setName("ContainerLauncher Event Handler"); eventHandlingThread.start(); - boolean cleanupDagDataOnComplete = ShuffleUtils.isTezShuffleHandler(conf) - && conf.getBoolean(TezConfiguration.TEZ_AM_DAG_CLEANUP_ON_COMPLETION, - TezConfiguration.TEZ_AM_DAG_CLEANUP_ON_COMPLETION_DEFAULT); - if (cleanupDagDataOnComplete) { + dagDelete = ShuffleUtils.isTezShuffleHandler(conf) && + conf.getBoolean(TezConfiguration.TEZ_AM_DAG_CLEANUP_ON_COMPLETION, + TezConfiguration.TEZ_AM_DAG_CLEANUP_ON_COMPLETION_DEFAULT); + failedTaskAttemptDelete = ShuffleUtils.isTezShuffleHandler(conf) && + conf.getBoolean(TezConfiguration.TEZ_AM_TASK_ATTEMPT_CLEANUP_ON_FAILURE, + TezConfiguration.TEZ_AM_TASK_ATTEMPT_CLEANUP_ON_FAILURE_DEFAULT); + + if (dagDelete || failedTaskAttemptDelete) { String deletionTrackerClassName = conf.get(TezConfiguration.TEZ_AM_DELETION_TRACKER_CLASS, TezConfiguration.TEZ_AM_DELETION_TRACKER_CLASS_DEFAULT); deletionTracker = ReflectionUtils.createClazzInstance( @@ -441,9 +449,16 @@ public void stopContainer(ContainerStopRequest stopRequest) { @Override public void dagComplete(TezDAGID dag, JobTokenSecretManager jobTokenSecretManager) { - if (deletionTracker != null) { + if (dagDelete && deletionTracker != null) { deletionTracker.dagComplete(dag, jobTokenSecretManager); } } + @Override + public void taskAttemptFailed(TezTaskAttemptID taskAttemptID, JobTokenSecretManager jobTokenSecretManager, + NodeId nodeId) { + if (failedTaskAttemptDelete && deletionTracker != null) { + deletionTracker.taskAttemptFailed(taskAttemptID, jobTokenSecretManager, nodeId); + } + } } diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/launcher/TestContainerLauncherWrapper.java b/tez-dag/src/test/java/org/apache/tez/dag/app/launcher/TestContainerLauncherWrapper.java index 8778f32d58..c4f4eff0cf 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/launcher/TestContainerLauncherWrapper.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/launcher/TestContainerLauncherWrapper.java @@ -24,7 +24,7 @@ public class TestContainerLauncherWrapper { @Test(timeout = 5000) public void testDelegation() throws Exception { PluginWrapperTestHelpers.testDelegation(ContainerLauncherWrapper.class, ContainerLauncher.class, - Sets.newHashSet("getContainerLauncher", "dagComplete")); + Sets.newHashSet("getContainerLauncher", "dagComplete", "taskAttemptFailed")); } } diff --git a/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java b/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java index a54a7cfe01..9730be1869 100644 --- a/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java +++ b/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java @@ -57,6 +57,8 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileContext; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.LocalDirAllocator; import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.DataInputByteBuffer; @@ -1007,6 +1009,7 @@ private void handleRequest(ChannelHandlerContext ctx, FullHttpRequest request) final Map> q = new QueryStringDecoder(request.getUri()).parameters(); final List keepAliveList = q.get("keepAlive"); final List dagCompletedQ = q.get("dagAction"); + final List taskAttemptFailedQ = q.get("taskAttemptAction"); boolean keepAliveParam = false; if (keepAliveList != null && keepAliveList.size() == 1) { keepAliveParam = Boolean.parseBoolean(keepAliveList.get(0)); @@ -1028,6 +1031,9 @@ private void handleRequest(ChannelHandlerContext ctx, FullHttpRequest request) if (deleteDagDirectories(ctx.channel(), dagCompletedQ, jobQ, dagIdQ)) { return; } + if (deleteTaskAttemptDirectories(ctx.channel(), taskAttemptFailedQ, jobQ, dagIdQ, mapIds)) { + return; + } if (mapIds == null || reduceRange == null || jobQ == null || dagIdQ == null) { sendError(ctx, "Required param job, dag, map and reduce", BAD_REQUEST); return; @@ -1115,14 +1121,24 @@ private void handleRequest(ChannelHandlerContext ctx, FullHttpRequest request) ch.writeAndFlush(LastHttpContent.EMPTY_LAST_CONTENT); } + private boolean isNullOrEmpty(List entries) { + return entries == null || entries.isEmpty(); + } + + private boolean notEmptyAndContains(List entries, String key) { + if (entries == null || entries.isEmpty()) { + return false; + } + return entries.get(0).contains(key); + } + private boolean deleteDagDirectories(Channel channel, List dagCompletedQ, List jobQ, List dagIdQ) { if (jobQ == null || jobQ.isEmpty()) { return false; } - if (dagCompletedQ != null && !dagCompletedQ.isEmpty() && dagCompletedQ.get(0).contains("delete") - && dagIdQ != null && !dagIdQ.isEmpty()) { + if (notEmptyAndContains(dagCompletedQ,"delete") && !isNullOrEmpty(dagIdQ)) { String base = getDagLocation(jobQ.get(0), dagIdQ.get(0), userRsrc.get(jobQ.get(0))); try { FileContext lfc = FileContext.getLocalFSFileContext(); @@ -1139,6 +1155,40 @@ private boolean deleteDagDirectories(Channel channel, return false; } + private boolean deleteTaskAttemptDirectories(Channel channel, List taskAttemptFailedQ, + List jobQ, List dagIdQ, List taskAttemptIdQ) { + if (jobQ == null || jobQ.isEmpty()) { + return false; + } + if (notEmptyAndContains(taskAttemptFailedQ,"delete") && !isNullOrEmpty(taskAttemptIdQ)) { + for (String taskAttemptId : taskAttemptIdQ) { + String baseStr = getBaseLocation(jobQ.get(0), dagIdQ.get(0), userRsrc.get(jobQ.get(0))); + try { + FileSystem fs = FileSystem.getLocal(conf).getRaw(); + for (Path basePath : lDirAlloc.getAllLocalPathsToRead(baseStr, conf)) { + for (FileStatus fileStatus : fs.listStatus(basePath)) { + Path taskAttemptPath = fileStatus.getPath(); + if (taskAttemptPath.getName().startsWith(taskAttemptId)) { + if (fs.delete(taskAttemptPath, true)) { + LOG.info("Deleted directory : " + taskAttemptPath); + // remove entry from IndexCache + indexCache.removeMap(taskAttemptPath.getName()); + break; + } + } + } + } + } catch (IOException e) { + LOG.warn("Encountered exception during failed task attempt delete " + e); + } + } + channel.writeAndFlush(new DefaultHttpResponse(HTTP_1_1, OK)) + .addListener(ChannelFutureListener.CLOSE); + return true; + } + return false; + } + /** * Calls sendMapOutput for the mapId pointed by ReduceContext.mapsToSend * and increments it. This method is first called by channelRead() diff --git a/tez-plugins/tez-aux-services/src/test/java/org/apache/tez/auxservices/TestShuffleHandler.java b/tez-plugins/tez-aux-services/src/test/java/org/apache/tez/auxservices/TestShuffleHandler.java index 6a2e1cc68e..45dd0adb06 100644 --- a/tez-plugins/tez-aux-services/src/test/java/org/apache/tez/auxservices/TestShuffleHandler.java +++ b/tez-plugins/tez-aux-services/src/test/java/org/apache/tez/auxservices/TestShuffleHandler.java @@ -1312,6 +1312,88 @@ protected void sendError(ChannelHandlerContext ctx, String message, } } + @Test(timeout = 5000) + public void testFailedTaskAttemptDelete() throws Exception { + final ArrayList failures = new ArrayList(1); + Configuration conf = new Configuration(); + conf.setInt(ShuffleHandler.MAX_SHUFFLE_CONNECTIONS, 3); + conf.setInt(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY, 0); + conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION, + "simple"); + UserGroupInformation.setConfiguration(conf); + File absLogDir = new File("target", TestShuffleHandler.class. + getSimpleName() + "LocDir").getAbsoluteFile(); + conf.set(YarnConfiguration.NM_LOCAL_DIRS, absLogDir.getAbsolutePath()); + ApplicationId appId = ApplicationId.newInstance(12345, 1); + String appAttemptId = "attempt_12345_1_m_1_0"; + String user = "randomUser"; + List fileMap = new ArrayList(); + String taskAttemptDirStr = + StringUtils.join(Path.SEPARATOR, + new String[] {absLogDir.getAbsolutePath(), + ShuffleHandler.USERCACHE, user, + ShuffleHandler.APPCACHE, appId.toString(), "dag_1/output/", appAttemptId}); + File taskAttemptDir = new File(taskAttemptDirStr); + Assert.assertFalse("Task Attempt Directory should not exist", taskAttemptDir.exists()); + createShuffleHandlerFiles(absLogDir, user, appId.toString(), appAttemptId, + conf, fileMap); + ShuffleHandler shuffleHandler = new ShuffleHandler() { + @Override + protected Shuffle getShuffle(Configuration conf) { + // replace the shuffle handler with one stubbed for testing + return new Shuffle(conf) { + @Override + protected void sendError(ChannelHandlerContext ctx, String message, + HttpResponseStatus status) { + if (failures.size() == 0) { + failures.add(new Error(message)); + ctx.channel().close(); + } + } + }; + } + }; + shuffleHandler.init(conf); + try { + shuffleHandler.start(); + DataOutputBuffer outputBuffer = new DataOutputBuffer(); + outputBuffer.reset(); + Token jt = + new Token("identifier".getBytes(), + "password".getBytes(), new Text(user), new Text("shuffleService")); + jt.write(outputBuffer); + shuffleHandler + .initializeApplication(new ApplicationInitializationContext(user, + appId, ByteBuffer.wrap(outputBuffer.getData(), 0, + outputBuffer.getLength()))); + URL url = + new URL( + "http://127.0.0.1:" + + shuffleHandler.getConfig().get( + ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY) + + "/mapOutput?taskAttemptAction=delete&job=job_12345_0001&dag=1&map=" + appAttemptId); + HttpURLConnection conn = (HttpURLConnection) url.openConnection(); + conn.setRequestProperty(ShuffleHeader.HTTP_HEADER_NAME, + ShuffleHeader.DEFAULT_HTTP_HEADER_NAME); + conn.setRequestProperty(ShuffleHeader.HTTP_HEADER_VERSION, + ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION); + Assert.assertTrue("Task Attempt Directory does not exist!", taskAttemptDir.exists()); + conn.connect(); + try { + DataInputStream is = new DataInputStream(conn.getInputStream()); + is.close(); + Assert.assertFalse("Task Attempt file was not deleted!", taskAttemptDir.exists()); + } catch (EOFException e) { + // ignore + } + Assert.assertEquals("sendError called due to shuffle error", + 0, failures.size()); + } finally { + shuffleHandler.stop(); + FileUtil.fullyDelete(absLogDir); + } + } + @Test(timeout = 4000) public void testSendMapCount() throws Exception { final List listenerList = diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/TezRuntimeUtils.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/TezRuntimeUtils.java index 9d9b8c16c0..48b23bc694 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/TezRuntimeUtils.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/TezRuntimeUtils.java @@ -187,6 +187,25 @@ public static URL constructBaseURIForShuffleHandlerDagComplete( return new URL(sb.toString()); } + public static URL constructBaseURIForShuffleHandlerTaskAttemptFailed( + String host, int port, String appId, int dagIdentifier, String taskAttemptIdentifier, boolean sslShuffle) + throws MalformedURLException { + String httpProtocol = (sslShuffle) ? "https://" : "http://"; + StringBuilder sb = new StringBuilder(httpProtocol); + sb.append(host); + sb.append(":"); + sb.append(port); + sb.append("/"); + sb.append("mapOutput?taskAttemptAction=delete"); + sb.append("&job="); + sb.append(appId.replace("application", "job")); + sb.append("&dag="); + sb.append(String.valueOf(dagIdentifier)); + sb.append("&map="); + sb.append(String.valueOf(taskAttemptIdentifier)); + return new URL(sb.toString()); + } + public static HttpConnectionParams getHttpConnectionParams(Configuration conf) { int connectionTimeout = conf.getInt(TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_CONNECT_TIMEOUT, From 6d7ef203c69f00c47831cd3cb9760a94242eee64 Mon Sep 17 00:00:00 2001 From: Mark Bathori <57758037+mbathori-cloudera@users.noreply.github.com> Date: Sun, 30 Jan 2022 11:48:56 +0100 Subject: [PATCH 092/137] TEZ-4381: Speed up TestSecureShuffle (#180) (Mark Bathori reviewed by Laszlo Bodor) --- .../src/test/java/org/apache/tez/test/TestSecureShuffle.java | 2 ++ 1 file changed, 2 insertions(+) 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 8e298506b8..7857c1ca2f 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 @@ -146,6 +146,8 @@ public void setupTezCluster() throws Exception { conf.setInt(TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_READ_TIMEOUT, 3 * 1000); //set to low value so that it can detect failures quickly conf.setInt(TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_FETCH_FAILURES_LIMIT, 2); + //reduce the maximum number of failed attempts per task + conf.setInt(TezConfiguration.TEZ_AM_TASK_MAX_FAILED_ATTEMPTS, 1); conf.setLong(TezConfiguration.TEZ_AM_SLEEP_TIME_BEFORE_EXIT_MILLIS, 500); conf.setBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_USE_ASYNC_HTTP, asyncHttp); From 7b66d3da714ee09b434b8d8d438ba30a0026af3c Mon Sep 17 00:00:00 2001 From: ghanko <54805928+ghanko@users.noreply.github.com> Date: Thu, 10 Feb 2022 11:05:47 +0100 Subject: [PATCH 093/137] TEZ-4227 Introduce convenient methods in TezID subclasses (#166) (Gergely Hanko reviewed by Laszlo Bodor) --- .../apache/tez/dag/records/DAGIDAware.java | 29 ++ .../tez/dag/records/TaskAttemptIDAware.java | 28 ++ .../apache/tez/dag/records/TaskIDAware.java | 28 ++ .../tez/dag/records/TezTaskAttemptID.java | 3 +- .../org/apache/tez/dag/records/TezTaskID.java | 3 +- .../apache/tez/dag/records/TezVertexID.java | 6 +- .../apache/tez/dag/records/VertexIDAware.java | 28 ++ .../tez/dag/records/VertexIdentifierImpl.java | 2 +- .../apache/tez/dag/records/TestTezIds.java | 2 +- .../org/apache/tez/dag/app/DAGAppMaster.java | 17 +- .../apache/tez/dag/app/RecoveryParser.java | 12 +- .../tez/dag/app/TaskCommunicatorManager.java | 6 +- .../apache/tez/dag/app/dag/DAGScheduler.java | 8 +- .../app/dag/RootInputInitializerManager.java | 6 +- .../java/org/apache/tez/dag/app/dag/Task.java | 5 +- .../apache/tez/dag/app/dag/TaskAttempt.java | 11 +- .../tez/dag/app/dag/event/DAGEvent.java | 6 +- .../dag/event/DAGEventSchedulerUpdate.java | 13 +- .../dag/event/DAGEventVertexCompleted.java | 4 +- .../dag/event/DAGEventVertexReRunning.java | 2 +- ...peculatorEventTaskAttemptStatusUpdate.java | 2 +- .../dag/app/dag/event/TaskAttemptEvent.java | 4 +- .../tez/dag/app/dag/event/TaskEvent.java | 4 +- .../tez/dag/app/dag/event/VertexEvent.java | 6 +- .../VertexEventTaskAttemptCompleted.java | 2 +- .../apache/tez/dag/app/dag/impl/DAGImpl.java | 4 +- .../dag/impl/DAGSchedulerNaturalOrder.java | 6 +- .../DAGSchedulerNaturalOrderControlled.java | 15 +- .../org/apache/tez/dag/app/dag/impl/Edge.java | 2 +- .../tez/dag/app/dag/impl/TaskAttemptImpl.java | 62 ++-- .../apache/tez/dag/app/dag/impl/TaskImpl.java | 88 ++--- .../TezRootInputInitializerContextImpl.java | 2 +- .../tez/dag/app/dag/impl/VertexImpl.java | 26 +- .../tez/dag/app/dag/impl/VertexManager.java | 6 +- .../speculation/legacy/LegacySpeculator.java | 8 +- .../TaskAttemptFailedDeleteRunnable.java | 4 +- .../dag/app/rm/AMSchedulerEventTAEnded.java | 2 +- .../tez/dag/app/rm/TaskSchedulerManager.java | 28 +- .../AMContainerEventLaunchRequest.java | 2 +- .../tez/dag/app/web/AMWebController.java | 6 +- .../tez/dag/history/DAGHistoryEvent.java | 6 +- .../tez/dag/history/HistoryEventHandler.java | 6 +- .../dag/history/events/DAGFinishedEvent.java | 6 +- .../history/events/DAGInitializedEvent.java | 6 +- .../dag/history/events/DAGStartedEvent.java | 6 +- .../dag/history/events/DAGSubmittedEvent.java | 6 +- .../events/TaskAttemptFinishedEvent.java | 4 +- .../events/TaskAttemptStartedEvent.java | 4 +- .../dag/history/events/TaskFinishedEvent.java | 4 +- .../dag/history/events/TaskStartedEvent.java | 4 +- .../events/VertexCommitStartedEvent.java | 2 +- .../events/VertexConfigurationDoneEvent.java | 4 +- .../history/events/VertexFinishedEvent.java | 6 +- .../events/VertexInitializedEvent.java | 6 +- .../history/events/VertexStartedEvent.java | 6 +- .../impl/HistoryEventJsonConversion.java | 16 +- .../dag/history/recovery/RecoveryService.java | 12 +- .../tez/dag/app/TestMockDAGAppMaster.java | 12 +- .../apache/tez/dag/app/TestSpeculation.java | 10 +- .../tez/dag/app/dag/impl/TestCommit.java | 136 ++++---- .../tez/dag/app/dag/impl/TestDAGImpl.java | 40 +-- .../tez/dag/app/dag/impl/TestDAGRecovery.java | 7 +- .../dag/app/dag/impl/TestDAGScheduler.java | 41 ++- ...estDAGSchedulerNaturalOrderControlled.java | 6 +- .../apache/tez/dag/app/dag/impl/TestEdge.java | 6 +- .../tez/dag/app/dag/impl/TestTaskAttempt.java | 73 +++-- .../tez/dag/app/dag/impl/TestTaskImpl.java | 300 +++++++++--------- .../tez/dag/app/dag/impl/TestVertexImpl.java | 44 +-- .../tez/dag/app/rm/TestContainerReuse.java | 12 +- .../dag/app/rm/TestTaskSchedulerManager.java | 6 +- .../tez/dag/app/web/TestAMWebController.java | 14 +- .../TestHistoryEventsProtoConversion.java | 20 +- .../tez/service/impl/ContainerRunnerImpl.java | 4 +- .../tez/mapreduce/hadoop/IDConverter.java | 2 +- .../mapreduce/TaskAttemptContextImpl.java | 6 +- .../tez/mapreduce/processor/MRTask.java | 4 +- .../history/parser/SimpleHistoryParser.java | 12 +- .../history/parser/datamodel/BaseParser.java | 5 +- .../proto/HistoryEventProtoConverter.java | 16 +- .../proto/ProtoHistoryLoggingService.java | 12 +- .../logging/ats/TimelineCachePluginImpl.java | 6 +- .../ats/acls/TestATSHistoryWithACLs.java | 4 +- .../ats/ATSV15HistoryLoggingService.java | 12 +- .../logging/ats/ATSHistoryLoggingService.java | 8 +- .../ats/HistoryEventTimelineConversion.java | 81 +++-- .../tez/runtime/api/impl/EventMetaData.java | 4 +- .../apache/tez/runtime/api/impl/TaskSpec.java | 7 +- .../runtime/api/impl/TezTaskContextImpl.java | 7 +- .../org/apache/tez/runtime/task/TezChild.java | 6 +- .../RecoveryServiceWithEventHandlingHook.java | 16 +- .../org/apache/tez/test/TestAMRecovery.java | 4 +- 91 files changed, 834 insertions(+), 711 deletions(-) create mode 100644 tez-common/src/main/java/org/apache/tez/dag/records/DAGIDAware.java create mode 100644 tez-common/src/main/java/org/apache/tez/dag/records/TaskAttemptIDAware.java create mode 100644 tez-common/src/main/java/org/apache/tez/dag/records/TaskIDAware.java create mode 100644 tez-common/src/main/java/org/apache/tez/dag/records/VertexIDAware.java diff --git a/tez-common/src/main/java/org/apache/tez/dag/records/DAGIDAware.java b/tez-common/src/main/java/org/apache/tez/dag/records/DAGIDAware.java new file mode 100644 index 0000000000..1234a30053 --- /dev/null +++ b/tez-common/src/main/java/org/apache/tez/dag/records/DAGIDAware.java @@ -0,0 +1,29 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tez.dag.records; + +import org.apache.hadoop.yarn.api.records.ApplicationId; + +public interface DAGIDAware { + TezDAGID getDAGID(); + + default ApplicationId getApplicationId() { + return getDAGID().getApplicationId(); + } +} diff --git a/tez-common/src/main/java/org/apache/tez/dag/records/TaskAttemptIDAware.java b/tez-common/src/main/java/org/apache/tez/dag/records/TaskAttemptIDAware.java new file mode 100644 index 0000000000..924fd07109 --- /dev/null +++ b/tez-common/src/main/java/org/apache/tez/dag/records/TaskAttemptIDAware.java @@ -0,0 +1,28 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tez.dag.records; + +public interface TaskAttemptIDAware extends TaskIDAware { + TezTaskAttemptID getTaskAttemptID(); + + @Override + default TezTaskID getTaskID() { + return getTaskAttemptID().getTaskID(); + } +} diff --git a/tez-common/src/main/java/org/apache/tez/dag/records/TaskIDAware.java b/tez-common/src/main/java/org/apache/tez/dag/records/TaskIDAware.java new file mode 100644 index 0000000000..0bee45dfa1 --- /dev/null +++ b/tez-common/src/main/java/org/apache/tez/dag/records/TaskIDAware.java @@ -0,0 +1,28 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tez.dag.records; + +public interface TaskIDAware extends VertexIDAware { + TezTaskID getTaskID(); + + @Override + default TezVertexID getVertexID() { + return getTaskID().getVertexID(); + } +} diff --git a/tez-common/src/main/java/org/apache/tez/dag/records/TezTaskAttemptID.java b/tez-common/src/main/java/org/apache/tez/dag/records/TezTaskAttemptID.java index 7c92be9449..fe2b84449f 100644 --- a/tez-common/src/main/java/org/apache/tez/dag/records/TezTaskAttemptID.java +++ b/tez-common/src/main/java/org/apache/tez/dag/records/TezTaskAttemptID.java @@ -46,7 +46,7 @@ */ @InterfaceAudience.Public @InterfaceStability.Stable -public class TezTaskAttemptID extends TezID { +public class TezTaskAttemptID extends TezID implements TaskIDAware { public static final String ATTEMPT = "attempt"; private TezTaskID taskId; @@ -73,6 +73,7 @@ private TezTaskAttemptID(TezTaskID taskId, int id) { } /** Returns the {@link TezTaskID} object that this task attempt belongs to */ + @Override public TezTaskID getTaskID() { return taskId; } diff --git a/tez-common/src/main/java/org/apache/tez/dag/records/TezTaskID.java b/tez-common/src/main/java/org/apache/tez/dag/records/TezTaskID.java index a1a556c9ba..08310f3dfc 100644 --- a/tez-common/src/main/java/org/apache/tez/dag/records/TezTaskID.java +++ b/tez-common/src/main/java/org/apache/tez/dag/records/TezTaskID.java @@ -41,7 +41,7 @@ */ @InterfaceAudience.Public @InterfaceStability.Stable -public class TezTaskID extends TezID { +public class TezTaskID extends TezID implements VertexIDAware { public static final String TASK = "task"; private final int serializingHash; @@ -79,6 +79,7 @@ public int getSerializingHash() { } /** Returns the {@link TezVertexID} object that this task belongs to */ + @Override public TezVertexID getVertexID() { return vertexId; } diff --git a/tez-common/src/main/java/org/apache/tez/dag/records/TezVertexID.java b/tez-common/src/main/java/org/apache/tez/dag/records/TezVertexID.java index e9ddd77d38..f7becc250f 100644 --- a/tez-common/src/main/java/org/apache/tez/dag/records/TezVertexID.java +++ b/tez-common/src/main/java/org/apache/tez/dag/records/TezVertexID.java @@ -44,7 +44,7 @@ */ @InterfaceAudience.Public @InterfaceStability.Stable -public class TezVertexID extends TezID { +public class TezVertexID extends TezID implements DAGIDAware { public static final String VERTEX = "vertex"; static final ThreadLocal tezVertexIdFormat = new ThreadLocal() { @@ -80,7 +80,8 @@ private TezVertexID(TezDAGID dagId, int id) { } /** Returns the {@link TezDAGID} object that this tip belongs to */ - public TezDAGID getDAGId() { + @Override + public TezDAGID getDAGID() { return dagId; } @@ -159,5 +160,4 @@ public static TezVertexID fromString(String vertexIdStr) { } return null; } - } diff --git a/tez-common/src/main/java/org/apache/tez/dag/records/VertexIDAware.java b/tez-common/src/main/java/org/apache/tez/dag/records/VertexIDAware.java new file mode 100644 index 0000000000..01bbe859b6 --- /dev/null +++ b/tez-common/src/main/java/org/apache/tez/dag/records/VertexIDAware.java @@ -0,0 +1,28 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tez.dag.records; + +public interface VertexIDAware extends DAGIDAware { + TezVertexID getVertexID(); + + @Override + default TezDAGID getDAGID() { + return getVertexID().getDAGID(); + } +} diff --git a/tez-common/src/main/java/org/apache/tez/dag/records/VertexIdentifierImpl.java b/tez-common/src/main/java/org/apache/tez/dag/records/VertexIdentifierImpl.java index 4480f742fc..83b503203c 100644 --- a/tez-common/src/main/java/org/apache/tez/dag/records/VertexIdentifierImpl.java +++ b/tez-common/src/main/java/org/apache/tez/dag/records/VertexIdentifierImpl.java @@ -30,7 +30,7 @@ public class VertexIdentifierImpl implements VertexIdentifier { public VertexIdentifierImpl(String dagName, String vertexName, TezVertexID vertexId) { this.vertexId = vertexId; this.vertexName = vertexName; - this.dagIdentifier = new DagIdentifierImpl(dagName, vertexId.getDAGId()); + this.dagIdentifier = new DagIdentifierImpl(dagName, vertexId.getDAGID()); } @Override diff --git a/tez-common/src/test/java/org/apache/tez/dag/records/TestTezIds.java b/tez-common/src/test/java/org/apache/tez/dag/records/TestTezIds.java index 5e1552d345..10b62b2bd1 100644 --- a/tez-common/src/test/java/org/apache/tez/dag/records/TestTezIds.java +++ b/tez-common/src/test/java/org/apache/tez/dag/records/TestTezIds.java @@ -38,7 +38,7 @@ private void verifyDagInfo(String[] splits, TezDAGID dagId) { } private void verifyVertexInfo(String[] splits, TezVertexID vId) { - verifyDagInfo(splits, vId.getDAGId()); + verifyDagInfo(splits, vId.getDAGID()); Assert.assertEquals(vId.getId(), Integer.valueOf(splits[4]).intValue()); } 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 3be9aacec5..972fadf854 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 @@ -50,7 +50,6 @@ import java.util.TimerTask; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; -import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; @@ -2173,7 +2172,7 @@ private class DagEventDispatcher implements EventHandler { @Override public void handle(DAGEvent event) { DAG dag = context.getCurrentDAG(); - int eventDagIndex = event.getDAGId().getId(); + int eventDagIndex = event.getDAGID().getId(); if (dag == null || eventDagIndex != dag.getID().getId()) { return; // event not relevant any more } @@ -2187,12 +2186,12 @@ private class TaskEventDispatcher implements EventHandler { public void handle(TaskEvent event) { DAG dag = context.getCurrentDAG(); int eventDagIndex = - event.getTaskID().getVertexID().getDAGId().getId(); + event.getDAGID().getId(); if (dag == null || eventDagIndex != dag.getID().getId()) { return; // event not relevant any more } Task task = - dag.getVertex(event.getTaskID().getVertexID()). + dag.getVertex(event.getVertexID()). getTask(event.getTaskID()); ((EventHandler)task).handle(event); } @@ -2217,13 +2216,13 @@ private class TaskAttemptEventDispatcher public void handle(TaskAttemptEvent event) { DAG dag = context.getCurrentDAG(); int eventDagIndex = - event.getTaskAttemptID().getTaskID().getVertexID().getDAGId().getId(); + event.getDAGID().getId(); if (dag == null || eventDagIndex != dag.getID().getId()) { return; // event not relevant any more } Task task = - dag.getVertex(event.getTaskAttemptID().getTaskID().getVertexID()). - getTask(event.getTaskAttemptID().getTaskID()); + dag.getVertex(event.getVertexID()). + getTask(event.getTaskID()); TaskAttempt attempt = task.getAttempt(event.getTaskAttemptID()); ((EventHandler) attempt).handle(event); } @@ -2236,13 +2235,13 @@ private class VertexEventDispatcher public void handle(VertexEvent event) { DAG dag = context.getCurrentDAG(); int eventDagIndex = - event.getVertexId().getDAGId().getId(); + event.getDAGID().getId(); if (dag == null || eventDagIndex != dag.getID().getId()) { return; // event not relevant any more } Vertex vertex = - dag.getVertex(event.getVertexId()); + dag.getVertex(event.getVertexID()); ((EventHandler) vertex).handle(event); } } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/RecoveryParser.java b/tez-dag/src/main/java/org/apache/tez/dag/app/RecoveryParser.java index 19c24f300c..ce3b62bbd5 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/RecoveryParser.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/RecoveryParser.java @@ -846,9 +846,9 @@ public DAGRecoveryData parseRecoveryData() throws IOException { case TASK_STARTED: { TaskStartedEvent taskStartedEvent = (TaskStartedEvent) event; - VertexRecoveryData vertexRecoveryData = recoveredDAGData.vertexRecoveryDataMap.get(taskStartedEvent.getTaskID().getVertexID()); + VertexRecoveryData vertexRecoveryData = recoveredDAGData.vertexRecoveryDataMap.get(taskStartedEvent.getVertexID()); Preconditions.checkArgument(vertexRecoveryData != null, - "Invalid TaskStartedEvent, its vertex does not exist:" + taskStartedEvent.getTaskID().getVertexID()); + "Invalid TaskStartedEvent, its vertex does not exist:" + taskStartedEvent.getVertexID()); TaskRecoveryData taskRecoveryData = vertexRecoveryData.maybeCreateTaskRecoveryData(taskStartedEvent.getTaskID()); taskRecoveryData.taskStartedEvent = taskStartedEvent; break; @@ -856,9 +856,9 @@ public DAGRecoveryData parseRecoveryData() throws IOException { case TASK_FINISHED: { TaskFinishedEvent taskFinishedEvent = (TaskFinishedEvent) event; - VertexRecoveryData vertexRecoveryData = recoveredDAGData.vertexRecoveryDataMap.get(taskFinishedEvent.getTaskID().getVertexID()); + VertexRecoveryData vertexRecoveryData = recoveredDAGData.vertexRecoveryDataMap.get(taskFinishedEvent.getVertexID()); Preconditions.checkArgument(vertexRecoveryData != null, - "Invalid TaskFinishedEvent, its vertex does not exist:" + taskFinishedEvent.getTaskID().getVertexID()); + "Invalid TaskFinishedEvent, its vertex does not exist:" + taskFinishedEvent.getVertexID()); TaskRecoveryData taskRecoveryData = vertexRecoveryData.maybeCreateTaskRecoveryData(taskFinishedEvent.getTaskID()); taskRecoveryData.taskFinishedEvent = taskFinishedEvent; break; @@ -867,7 +867,7 @@ public DAGRecoveryData parseRecoveryData() throws IOException { { TaskAttemptStartedEvent taStartedEvent = (TaskAttemptStartedEvent)event; VertexRecoveryData vertexRecoveryData = recoveredDAGData.vertexRecoveryDataMap.get( - taStartedEvent.getTaskAttemptID().getTaskID().getVertexID()); + taStartedEvent.getVertexID()); Preconditions.checkArgument(vertexRecoveryData != null, "Invalid TaskAttemptStartedEvent, its vertexId does not exist, taId=" + taStartedEvent.getTaskAttemptID()); TaskRecoveryData taskRecoveryData = vertexRecoveryData.taskRecoveryDataMap @@ -882,7 +882,7 @@ public DAGRecoveryData parseRecoveryData() throws IOException { { TaskAttemptFinishedEvent taFinishedEvent = (TaskAttemptFinishedEvent)event; VertexRecoveryData vertexRecoveryData = recoveredDAGData.vertexRecoveryDataMap.get( - taFinishedEvent.getTaskAttemptID().getTaskID().getVertexID()); + taFinishedEvent.getVertexID()); Preconditions.checkArgument(vertexRecoveryData != null, "Invalid TaskAttemtFinishedEvent, its vertexId does not exist, taId=" + taFinishedEvent.getTaskAttemptID()); TaskRecoveryData taskRecoveryData = vertexRecoveryData.taskRecoveryDataMap diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorManager.java index ac2f7605ae..51895f4afd 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorManager.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorManager.java @@ -350,14 +350,14 @@ public TaskHeartbeatResponse heartbeat(TaskHeartbeatRequest request) } } if (!eventsForVertex.isEmpty()) { - TezVertexID vertexId = taskAttemptID.getTaskID().getVertexID(); + TezVertexID vertexId = taskAttemptID.getVertexID(); sendEvent( new VertexEventRouteEvent(vertexId, Collections.unmodifiableList(eventsForVertex))); } taskHeartbeatHandler.pinged(taskAttemptID); eventInfo = context .getCurrentDAG() - .getVertex(taskAttemptID.getTaskID().getVertexID()) + .getVertex(taskAttemptID.getVertexID()) .getTaskAttemptTezEvents(taskAttemptID, request.getStartIndex(), request.getPreRoutedStartIndex(), request.getMaxEvents()); } @@ -442,7 +442,7 @@ public boolean canCommit(TezTaskAttemptID taskAttemptId) throws IOException { DAG job = context.getCurrentDAG(); Task task = - job.getVertex(taskAttemptId.getTaskID().getVertexID()). + job.getVertex(taskAttemptId.getVertexID()). getTask(taskAttemptId.getTaskID()); return task.canCommit(taskAttemptId); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/DAGScheduler.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/DAGScheduler.java index 2fa735ef8f..ada7867a6a 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/DAGScheduler.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/DAGScheduler.java @@ -52,7 +52,7 @@ public void addVertexConcurrencyLimit(TezVertexID vId, int concurrency) { public void scheduleTask(DAGEventSchedulerUpdate event) { VertexInfo vInfo = null; if (vertexInfo != null) { - vInfo = vertexInfo.get(event.getAttempt().getID().getTaskID().getVertexID()); + vInfo = vertexInfo.get(event.getVertexID()); } scheduleTaskWithLimit(event, vInfo); } @@ -60,7 +60,7 @@ public void scheduleTask(DAGEventSchedulerUpdate event) { private void scheduleTaskWithLimit(DAGEventSchedulerUpdate event, VertexInfo vInfo) { if (vInfo != null) { if (vInfo.concurrency >= vInfo.concurrencyLimit) { - vInfo.pendingAttempts.put(event.getAttempt().getID(), event); + vInfo.pendingAttempts.put(event.getTaskAttemptID(), event); return; // already at max concurrency } vInfo.concurrency++; @@ -71,9 +71,9 @@ private void scheduleTaskWithLimit(DAGEventSchedulerUpdate event, VertexInfo vIn public void taskCompleted(DAGEventSchedulerUpdate event) { taskCompletedEx(event); if (vertexInfo != null) { - VertexInfo vInfo = vertexInfo.get(event.getAttempt().getID().getTaskID().getVertexID()); + VertexInfo vInfo = vertexInfo.get(event.getVertexID()); if (vInfo != null) { - if(vInfo.pendingAttempts.remove(event.getAttempt().getID()) == null) { + if(vInfo.pendingAttempts.remove(event.getTaskAttemptID()) == null) { vInfo.concurrency--; if(!vInfo.pendingAttempts.isEmpty()) { Iterator i = vInfo.pendingAttempts.values().iterator(); diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/RootInputInitializerManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/RootInputInitializerManager.java index 3c4a05eb84..cfbdb19e30 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/RootInputInitializerManager.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/RootInputInitializerManager.java @@ -457,7 +457,7 @@ public void onTaskSucceeded(String vertexName, TezTaskID taskId, int attemptId) Iterator eventIterator = events.iterator(); while (eventIterator.hasNext()) { TezEvent tezEvent = eventIterator.next(); - int taskIndex = tezEvent.getSourceInfo().getTaskAttemptID().getTaskID().getId(); + int taskIndex = tezEvent.getSourceInfo().getTaskID().getId(); int taskAttemptIndex = tezEvent.getSourceInfo().getTaskAttemptID().getId(); if (taskIndex == taskId.getId()) { // Process only if there's a pending event for the specific succeeded task @@ -476,7 +476,7 @@ public void handleInputInitializerEvents(Collection tezEvents) { List toForwardEvents = new LinkedList(); for (TezEvent tezEvent : tezEvents) { String srcVertexName = tezEvent.getSourceInfo().getTaskVertexName(); - int taskIndex = tezEvent.getSourceInfo().getTaskAttemptID().getTaskID().getId(); + int taskIndex = tezEvent.getSourceInfo().getTaskID().getId(); int taskAttemptIndex = tezEvent.getSourceInfo().getTaskAttemptID().getId(); Map vertexSuccessfulAttemptMap = @@ -496,7 +496,7 @@ public void handleInputInitializerEvents(Collection tezEvents) { Vertex srcVertex = appContext.getCurrentDAG().getVertex(srcVertexName); Task task = srcVertex.getTask(taskIndex); if (task.getState() == TaskState.SUCCEEDED) { - successfulAttemptInteger = task.getSuccessfulAttempt().getID().getId(); + successfulAttemptInteger = task.getSuccessfulAttempt().getTaskAttemptID().getId(); vertexSuccessfulAttemptMap.put(taskIndex, successfulAttemptInteger); } } 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 c1fe7f79ad..cb1bc33866 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 @@ -28,16 +28,15 @@ import org.apache.tez.dag.api.TaskLocationHint; import org.apache.tez.dag.api.oldrecords.TaskReport; import org.apache.tez.dag.api.oldrecords.TaskState; +import org.apache.tez.dag.records.TaskIDAware; import org.apache.tez.dag.records.TezTaskAttemptID; -import org.apache.tez.dag.records.TezTaskID; import org.apache.tez.runtime.api.impl.TaskSpec; import org.apache.tez.runtime.api.impl.TezEvent; /** * Read only view of Task. */ -public interface Task { - TezTaskID getTaskId(); +public interface Task extends TaskIDAware { TaskReport getReport(); TaskState getState(); TezCounters getCounters(); 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 0fc7013fea..563e4c7192 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 @@ -28,17 +28,15 @@ import org.apache.tez.common.counters.TezCounters; import org.apache.tez.dag.api.oldrecords.TaskAttemptReport; import org.apache.tez.dag.api.oldrecords.TaskAttemptState; +import org.apache.tez.dag.records.TaskAttemptIDAware; import org.apache.tez.dag.records.TaskAttemptTerminationCause; -import org.apache.tez.dag.records.TezDAGID; import org.apache.tez.dag.records.TezTaskAttemptID; -import org.apache.tez.dag.records.TezTaskID; -import org.apache.tez.dag.records.TezVertexID; import org.apache.tez.runtime.api.impl.TezEvent; /** * Read only view of TaskAttempt. */ -public interface TaskAttempt { +public interface TaskAttempt extends TaskAttemptIDAware { public static class TaskAttemptStatus { public TezTaskAttemptID id; @@ -66,11 +64,6 @@ public void setLocalityCounter(DAGCounter localityCounter) { } } } - - TezTaskAttemptID getID(); - TezTaskID getTaskID(); - TezVertexID getVertexID(); - TezDAGID getDAGID(); Task getTask(); TaskAttemptReport getReport(); diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/DAGEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/DAGEvent.java index a0a8a1af8e..4eed8ff984 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/DAGEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/DAGEvent.java @@ -19,13 +19,14 @@ package org.apache.tez.dag.app.dag.event; import org.apache.tez.common.TezAbstractEvent; +import org.apache.tez.dag.records.DAGIDAware; import org.apache.tez.dag.records.TezDAGID; /** * This class encapsulates job related events. * */ -public class DAGEvent extends TezAbstractEvent { +public class DAGEvent extends TezAbstractEvent implements DAGIDAware { private TezDAGID dagId; @@ -34,7 +35,8 @@ public DAGEvent(TezDAGID dagId, DAGEventType type) { this.dagId = dagId; } - public TezDAGID getDAGId() { + @Override + public TezDAGID getDAGID() { return dagId; } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/DAGEventSchedulerUpdate.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/DAGEventSchedulerUpdate.java index eda02b52da..1dedaecad9 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/DAGEventSchedulerUpdate.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/DAGEventSchedulerUpdate.java @@ -19,9 +19,11 @@ package org.apache.tez.dag.app.dag.event; import org.apache.tez.dag.app.dag.TaskAttempt; +import org.apache.tez.dag.records.TaskAttemptIDAware; +import org.apache.tez.dag.records.TezTaskAttemptID; + +public class DAGEventSchedulerUpdate extends DAGEvent implements TaskAttemptIDAware { -public class DAGEventSchedulerUpdate extends DAGEvent { - public enum UpdateType { TA_SCHEDULE, TA_COMPLETED @@ -31,7 +33,7 @@ public enum UpdateType { private final UpdateType updateType; public DAGEventSchedulerUpdate(UpdateType updateType, TaskAttempt attempt) { - super(attempt.getDAGID(), + super(attempt.getDAGID(), DAGEventType.DAG_SCHEDULER_UPDATE); this.attempt = attempt; this.updateType = updateType; @@ -44,4 +46,9 @@ public UpdateType getUpdateType() { public TaskAttempt getAttempt() { return attempt; } + + @Override + public TezTaskAttemptID getTaskAttemptID() { + return attempt.getTaskAttemptID(); + } } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/DAGEventVertexCompleted.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/DAGEventVertexCompleted.java index e58d46e834..2c18be85dd 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/DAGEventVertexCompleted.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/DAGEventVertexCompleted.java @@ -29,7 +29,7 @@ public class DAGEventVertexCompleted extends DAGEvent { private final VertexTerminationCause terminationCause; public DAGEventVertexCompleted(TezVertexID vertexId, VertexState vertexState) { - super(vertexId.getDAGId(), DAGEventType.DAG_VERTEX_COMPLETED); + super(vertexId.getDAGID(), DAGEventType.DAG_VERTEX_COMPLETED); this.vertexId = vertexId; this.vertexState = vertexState; this.terminationCause = null; @@ -37,7 +37,7 @@ public DAGEventVertexCompleted(TezVertexID vertexId, VertexState vertexState) { public DAGEventVertexCompleted(TezVertexID vertexId, VertexState vertexState, VertexTerminationCause terminationCause) { - super(vertexId.getDAGId(), DAGEventType.DAG_VERTEX_COMPLETED); + super(vertexId.getDAGID(), DAGEventType.DAG_VERTEX_COMPLETED); this.vertexId = vertexId; this.vertexState = vertexState; this.terminationCause = terminationCause; diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/DAGEventVertexReRunning.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/DAGEventVertexReRunning.java index 303d48d375..30e595906b 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/DAGEventVertexReRunning.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/DAGEventVertexReRunning.java @@ -25,7 +25,7 @@ public class DAGEventVertexReRunning extends DAGEvent { private TezVertexID vertexId; public DAGEventVertexReRunning(TezVertexID vertexId) { - super(vertexId.getDAGId(), DAGEventType.DAG_VERTEX_RERUNNING); + super(vertexId.getDAGID(), DAGEventType.DAG_VERTEX_RERUNNING); this.vertexId = vertexId; } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/SpeculatorEventTaskAttemptStatusUpdate.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/SpeculatorEventTaskAttemptStatusUpdate.java index d5745c4df4..7ab6141c39 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/SpeculatorEventTaskAttemptStatusUpdate.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/SpeculatorEventTaskAttemptStatusUpdate.java @@ -34,7 +34,7 @@ public SpeculatorEventTaskAttemptStatusUpdate(TezTaskAttemptID taId, TaskAttempt public SpeculatorEventTaskAttemptStatusUpdate(TezTaskAttemptID taId, TaskAttemptState state, long timestamp, boolean justStarted) { - super(SpeculatorEventType.S_TASK_ATTEMPT_STATUS_UPDATE, taId.getTaskID().getVertexID()); + super(SpeculatorEventType.S_TASK_ATTEMPT_STATUS_UPDATE, taId.getVertexID()); this.id = taId; this.state = state; this.timestamp = timestamp; diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEvent.java index 63ef70feb4..d5d8481a54 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEvent.java @@ -19,13 +19,14 @@ package org.apache.tez.dag.app.dag.event; import org.apache.tez.common.TezAbstractEvent; +import org.apache.tez.dag.records.TaskAttemptIDAware; import org.apache.tez.dag.records.TezTaskAttemptID; /** * This class encapsulates task attempt related events. * */ -public class TaskAttemptEvent extends TezAbstractEvent { +public class TaskAttemptEvent extends TezAbstractEvent implements TaskAttemptIDAware { private TezTaskAttemptID attemptID; @@ -39,6 +40,7 @@ public TaskAttemptEvent(TezTaskAttemptID id, TaskAttemptEventType type) { this.attemptID = id; } + @Override public TezTaskAttemptID getTaskAttemptID() { return attemptID; } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskEvent.java index def9ddfa7c..9e741388fa 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskEvent.java @@ -19,13 +19,14 @@ package org.apache.tez.dag.app.dag.event; import org.apache.tez.common.TezAbstractEvent; +import org.apache.tez.dag.records.TaskIDAware; import org.apache.tez.dag.records.TezTaskID; /** * this class encapsulates task related events. * */ -public class TaskEvent extends TezAbstractEvent { +public class TaskEvent extends TezAbstractEvent implements TaskIDAware { private TezTaskID taskId; @@ -34,6 +35,7 @@ public TaskEvent(TezTaskID taskId, TaskEventType type) { this.taskId = taskId; } + @Override public TezTaskID getTaskID() { return taskId; } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEvent.java index 33128e4536..6957a50dc7 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEvent.java @@ -20,12 +20,13 @@ import org.apache.tez.common.TezAbstractEvent; import org.apache.tez.dag.records.TezVertexID; +import org.apache.tez.dag.records.VertexIDAware; /** * this class encapsulates vertex related events. * */ -public class VertexEvent extends TezAbstractEvent { +public class VertexEvent extends TezAbstractEvent implements VertexIDAware { private TezVertexID vertexId; @@ -34,7 +35,8 @@ public VertexEvent(TezVertexID vertexId, VertexEventType type) { this.vertexId = vertexId; } - public TezVertexID getVertexId() { + @Override + public TezVertexID getVertexID() { return vertexId; } } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventTaskAttemptCompleted.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventTaskAttemptCompleted.java index 5b07674422..e23d27cf6e 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventTaskAttemptCompleted.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventTaskAttemptCompleted.java @@ -28,7 +28,7 @@ public class VertexEventTaskAttemptCompleted extends VertexEvent { public VertexEventTaskAttemptCompleted(TezTaskAttemptID taskAttemptId, TaskAttemptStateInternal state) { - super(taskAttemptId.getTaskID().getVertexID(), + super(taskAttemptId.getVertexID(), VertexEventType.V_TASK_ATTEMPT_COMPLETED); this.attemptId = taskAttemptId; this.attempState = state; 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 07715cdfe7..c9337e473a 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 @@ -1040,7 +1040,7 @@ public VertexStatusBuilder getVertexStatus(String vertexName, } public TaskAttemptImpl getTaskAttempt(TezTaskAttemptID taId) { - return (TaskAttemptImpl) getVertex(taId.getTaskID().getVertexID()).getTask(taId.getTaskID()) + return (TaskAttemptImpl) getVertex(taId.getVertexID()).getTask(taId.getTaskID()) .getAttempt(taId); } @@ -1206,7 +1206,7 @@ private void abortOutputs() { */ public void handle(DAGEvent event) { if (LOG.isDebugEnabled()) { - LOG.debug("Processing DAGEvent " + event.getDAGId() + " of type " + LOG.debug("Processing DAGEvent " + event.getDAGID() + " of type " + event.getType() + " while in state " + getInternalState() + ". Event: " + event); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGSchedulerNaturalOrder.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGSchedulerNaturalOrder.java index 2383db83b1..78860868dd 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGSchedulerNaturalOrder.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGSchedulerNaturalOrder.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -52,12 +52,12 @@ public void scheduleTaskEx(DAGEventSchedulerUpdate event) { int priorityHighLimit = getPriorityHighLimit(dag, vertex); if (LOG.isDebugEnabled()) { - LOG.debug("Scheduling " + attempt.getID() + " between priorityLow: " + priorityLowLimit + LOG.debug("Scheduling " + attempt.getTaskAttemptID() + " between priorityLow: " + priorityLowLimit + " and priorityHigh: " + priorityHighLimit); } TaskAttemptEventSchedule attemptEvent = new TaskAttemptEventSchedule( - attempt.getID(), priorityLowLimit, priorityHighLimit); + attempt.getTaskAttemptID(), priorityLowLimit, priorityHighLimit); sendEvent(attemptEvent); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGSchedulerNaturalOrderControlled.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGSchedulerNaturalOrderControlled.java index c51783bc2d..3167c9f3b7 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGSchedulerNaturalOrderControlled.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGSchedulerNaturalOrderControlled.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -62,9 +62,9 @@ public class DAGSchedulerNaturalOrderControlled extends DAGScheduler { LinkedListMultimap.create(); // Tacks vertices for which no additional scheduling checks are required. Once in this list, the // vertex is considered to be fully scheduled. - private final Set scheduledVertices = new HashSet(); + private final Set scheduledVertices = new HashSet<>(); // Tracks tasks scheduled for a vertex. - private final Map vertexScheduledTasks = new HashMap(); + private final Map vertexScheduledTasks = new HashMap<>(); public DAGSchedulerNaturalOrderControlled(DAG dag, EventHandler dispatcher) { this.dag = dag; @@ -82,14 +82,14 @@ public void scheduleTaskEx(DAGEventSchedulerUpdate event) { int priorityHighLimit = getPriorityHighLimit(dag, vertex); TaskAttemptEventSchedule attemptEvent = new TaskAttemptEventSchedule( - attempt.getID(), priorityLowLimit, priorityHighLimit); + attempt.getTaskAttemptID(), priorityLowLimit, priorityHighLimit); - taskAttemptSeen(vertex.getName(), attempt.getID()); + taskAttemptSeen(vertex.getName(), attempt.getTaskAttemptID()); if (vertexAlreadyScheduled(vertex)) { // Vertex previously marked ready for scheduling. if (LOG.isDebugEnabled()) { - LOG.debug("Scheduling " + attempt.getID() + " between priorityLow: " + priorityLowLimit + LOG.debug("Scheduling " + attempt.getTaskAttemptID() + " between priorityLow: " + priorityLowLimit + " and priorityHigh: " + priorityHighLimit); } sendEvent(attemptEvent); @@ -154,8 +154,7 @@ private void processDownstreamVertices(Vertex vertex) { List newlyScheduledVertices = Lists.newLinkedList(); Map outputVertexEdgeMap = vertex.getOutputVertices(); for (Vertex destVertex : outputVertexEdgeMap.keySet()) { - if (vertexAlreadyScheduled(destVertex)) { // Nothing to do if already scheduled. - } else { + if (!vertexAlreadyScheduled(destVertex)) { if (LOG.isDebugEnabled()) { LOG.debug("Attempting to schedule vertex: " + destVertex.getLogIdentifier() + " due to upstream event from " + vertex.getLogIdentifier()); 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 99b56fbf07..08e1c19cee 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 @@ -413,7 +413,7 @@ public void sendTezEventToSourceTasks(TezEvent tezEvent) throws AMUserCodeExcept " destIndex=" + destTaskIndex + " edgeManager=" + edgeManager.getClass().getName()); } - TezTaskID srcTaskId = srcTask.getTaskId(); + TezTaskID srcTaskId = srcTask.getTaskID(); int srcTaskAttemptIndex = inputReadErrorEvent.getVersion(); TezTaskAttemptID srcTaskAttemptId = TezTaskAttemptID.getInstance(srcTaskId, srcTaskAttemptIndex); 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 e6b9e82000..c8343c834b 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 @@ -104,10 +104,7 @@ import org.apache.tez.dag.history.events.TaskAttemptFinishedEvent; import org.apache.tez.dag.history.events.TaskAttemptStartedEvent; import org.apache.tez.dag.records.TaskAttemptTerminationCause; -import org.apache.tez.dag.records.TezDAGID; import org.apache.tez.dag.records.TezTaskAttemptID; -import org.apache.tez.dag.records.TezTaskID; -import org.apache.tez.dag.records.TezVertexID; import org.apache.tez.dag.recovery.records.RecoveryProtos.DataEventDependencyInfoProto; import org.apache.tez.runtime.api.events.InputFailedEvent; import org.apache.tez.runtime.api.events.InputReadErrorEvent; @@ -588,25 +585,10 @@ public TaskAttemptImpl(TezTaskAttemptID attemptId, EventHandler eventHandler, } @Override - public TezTaskAttemptID getID() { + public TezTaskAttemptID getTaskAttemptID() { return attemptId; } - @Override - public TezTaskID getTaskID() { - return attemptId.getTaskID(); - } - - @Override - public TezVertexID getVertexID() { - return attemptId.getTaskID().getVertexID(); - } - - @Override - public TezDAGID getDAGID() { - return getVertexID().getDAGId(); - } - public TezTaskAttemptID getSchedulingCausalTA() { return creationCausalTA; } @@ -884,12 +866,12 @@ public void handle(TaskAttemptEvent event) { LOG.error("Can't handle this event at current state for " + this.attemptId, e); eventHandler.handle(new DAGEventDiagnosticsUpdate( - this.attemptId.getTaskID().getVertexID().getDAGId(), + getDAGID(), "Invalid event " + event.getType() + " on TaskAttempt " + this.attemptId)); eventHandler.handle( new DAGEvent( - this.attemptId.getTaskID().getVertexID().getDAGId(), + getDAGID(), DAGEventType.INTERNAL_ERROR) ); } catch (RuntimeException e) { @@ -897,13 +879,13 @@ public void handle(TaskAttemptEvent event) { + " at current state " + oldState + " for " + this.attemptId, e); eventHandler.handle(new DAGEventDiagnosticsUpdate( - this.attemptId.getTaskID().getVertexID().getDAGId(), + getDAGID(), "Uncaught exception when handling event " + event.getType() + " on TaskAttempt " + this.attemptId + " at state " + oldState + ", error=" + e.getMessage())); eventHandler.handle( new DAGEvent( - this.attemptId.getTaskID().getVertexID().getDAGId(), + getDAGID(), DAGEventType.INTERNAL_ERROR) ); } @@ -1269,7 +1251,7 @@ public TaskAttemptStateInternal transition(TaskAttemptImpl ta, TaskAttemptEvent if (taFinishedEvent == null) { LOG.debug("Only TaskAttemptStartedEvent but no TaskAttemptFinishedEvent, " + "send out TaskAttemptEventAttemptKilled to move it to KILLED"); - ta.sendEvent(new TaskAttemptEventAttemptKilled(ta.getID(), + ta.sendEvent(new TaskAttemptEventAttemptKilled(ta.getTaskAttemptID(), "Task Attempt killed in recovery due to can't recover the running task attempt", TaskAttemptTerminationCause.TERMINATED_AT_RECOVERY, true)); return TaskAttemptStateInternal.NEW; @@ -1280,29 +1262,29 @@ public TaskAttemptStateInternal transition(TaskAttemptImpl ta, TaskAttemptEvent TaskAttemptFinishedEvent taFinishedEvent = ta.recoveryData.getTaskAttemptFinishedEvent(); Preconditions.checkArgument(taFinishedEvent != null, "Both of TaskAttemptStartedEvent and TaskFinishedEvent is null," - + "taskAttemptId=" + ta.getID()); + + "taskAttemptId=" + ta.getTaskAttemptID()); switch (taFinishedEvent.getState()) { case FAILED: LOG.debug("TaskAttemptFinishedEvent is seen with state of FAILED, " + "send TA_FAILED to itself, attemptId={}", ta.attemptId); - ta.sendEvent(new TaskAttemptEventAttemptFailed(ta.getID(), TaskAttemptEventType.TA_FAILED, + ta.sendEvent(new TaskAttemptEventAttemptFailed(ta.getTaskAttemptID(), TaskAttemptEventType.TA_FAILED, taFinishedEvent.getTaskFailureType(), taFinishedEvent.getDiagnostics(), taFinishedEvent.getTaskAttemptError(), true)); break; case KILLED: LOG.debug("TaskAttemptFinishedEvent is seen with state of KILLED, " + "send TA_KILLED to itself, attemptId={}", ta.attemptId); - ta.sendEvent(new TaskAttemptEventAttemptKilled(ta.getID(), + ta.sendEvent(new TaskAttemptEventAttemptKilled(ta.getTaskAttemptID(), taFinishedEvent.getDiagnostics(), taFinishedEvent.getTaskAttemptError(), true)); break; case SUCCEEDED: LOG.debug("TaskAttemptFinishedEvent is seen with state of SUCCEEDED, " + "send TA_DONE to itself, attemptId={}", ta.attemptId); - ta.sendEvent(new TaskAttemptEvent(ta.getID(), TaskAttemptEventType.TA_DONE)); + ta.sendEvent(new TaskAttemptEvent(ta.getTaskAttemptID(), TaskAttemptEventType.TA_DONE)); break; default: throw new TezUncheckedException("Invalid state in TaskAttemptFinishedEvent, state=" - + taFinishedEvent.getState() + ", taId=" + ta.getID()); + + taFinishedEvent.getState() + ", taId=" + ta.getTaskAttemptID()); } return TaskAttemptStateInternal.NEW; } @@ -1428,7 +1410,7 @@ public void transition(TaskAttemptImpl ta, TaskAttemptEvent event) { RecoveryEvent rEvent = (RecoveryEvent)event; if (rEvent.isFromRecovery()) { if (LOG.isDebugEnabled()) { - LOG.debug("Faked TerminateEvent from recovery, taskAttemptId=" + ta.getID()); + LOG.debug("Faked TerminateEvent from recovery, taskAttemptId=" + ta.getTaskAttemptID()); } } } @@ -1613,10 +1595,10 @@ public void transition(TaskAttemptImpl ta, TaskAttemptEvent event) { // task is hung String diagnostics = "Attempt failed because it appears to make no progress for " + ta.hungIntervalMax + "ms"; - LOG.info(diagnostics + " " + ta.getID()); + LOG.info(diagnostics + " " + ta.getTaskAttemptID()); // send event that will fail this attempt ta.sendEvent( - new TaskAttemptEventAttemptFailed(ta.getID(), + new TaskAttemptEventAttemptFailed(ta.getTaskAttemptID(), TaskAttemptEventType.TA_FAILED, TaskFailureType.NON_FATAL, diagnostics, @@ -1803,8 +1785,8 @@ public TaskAttemptStateInternal transition(TaskAttemptImpl sourceAttempt, InputReadErrorEvent readErrorEvent = (InputReadErrorEvent)inputFailedEvent.getEvent(); int failedInputIndexOnDestTa = readErrorEvent.getIndex(); - if (readErrorEvent.getVersion() != sourceAttempt.getID().getId()) { - throw new TezUncheckedException(sourceAttempt.getID() + if (readErrorEvent.getVersion() != sourceAttempt.getTaskAttemptID().getId()) { + throw new TezUncheckedException(sourceAttempt.getTaskAttemptID() + " incorrectly blamed for read error from " + failedDestTaId + " at inputIndex " + failedInputIndexOnDestTa + " version" + readErrorEvent.getVersion()); @@ -1814,7 +1796,7 @@ public TaskAttemptStateInternal transition(TaskAttemptImpl sourceAttempt, // destination: where the data is tried to be fetched to String dHost = readErrorEvent.getDestinationLocalhostName(); - LOG.info("{} (on {}) blamed for read error from {} (on {}) at inputIndex {}", sourceAttempt.getID(), + LOG.info("{} (on {}) blamed for read error from {} (on {}) at inputIndex {}", sourceAttempt.getTaskAttemptID(), sHost, failedDestTaId, dHost, failedInputIndexOnDestTa); boolean tooManyDownstreamHostsBlamedTheSameUpstreamHost = false; @@ -1856,7 +1838,7 @@ public TaskAttemptStateInternal transition(TaskAttemptImpl sourceAttempt, boolean crossTimeDeadline = readErrorTimespanSec >= maxAllowedTimeForTaskReadErrorSec; int runningTasks = sourceAttempt.appContext.getCurrentDAG().getVertex( - failedDestTaId.getTaskID().getVertexID()).getRunningTasks(); + failedDestTaId.getVertexID()).getRunningTasks(); float failureFraction = runningTasks > 0 ? ((float) sourceAttempt.uniquefailedOutputReports.size()) / runningTasks : 0; boolean withinFailureFractionLimits = @@ -1872,7 +1854,7 @@ public TaskAttemptStateInternal transition(TaskAttemptImpl sourceAttempt, && !tooManyDownstreamHostsBlamedTheSameUpstreamHost) { return sourceAttempt.getInternalState(); } - String message = sourceAttempt.getID() + " being failed for too many output errors. " + String message = sourceAttempt.getTaskAttemptID() + " being failed for too many output errors. " + "failureFraction=" + failureFraction + ", MAX_ALLOWED_OUTPUT_FAILURES_FRACTION=" + maxAllowedOutputFailuresFraction @@ -1929,8 +1911,8 @@ protected void sendInputFailedToConsumers() { tezIfEvents.add(new TezEvent(new InputFailedEvent(), new EventMetaData(EventProducerConsumerType.SYSTEM, vertex.getName(), - edgeVertex.getName(), - getID()), appContext.getClock().getTime())); + edgeVertex.getName(), + getTaskAttemptID()), appContext.getClock().getTime())); } sendEvent(new VertexEventRouteEvent(vertex.getVertexId(), tezIfEvents)); } @@ -2024,7 +2006,7 @@ public TaskFailureType getFailureType(TaskAttemptEvent event) { @Override public String toString() { - return getID().toString(); + return getTaskAttemptID().toString(); } 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 b840798c26..d2933c5b86 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskImpl.java @@ -40,6 +40,10 @@ import org.apache.commons.lang.exception.ExceptionUtils; import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.tez.dag.app.dag.event.TaskEventTAFailed; +import org.apache.tez.dag.records.TaskAttemptTerminationCause; +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.TaskFailureType; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -93,10 +97,6 @@ import org.apache.tez.dag.history.DAGHistoryEvent; import org.apache.tez.dag.history.events.TaskFinishedEvent; import org.apache.tez.dag.history.events.TaskStartedEvent; -import org.apache.tez.dag.records.TaskAttemptTerminationCause; -import org.apache.tez.dag.records.TezTaskAttemptID; -import org.apache.tez.dag.records.TezTaskID; -import org.apache.tez.dag.records.TezVertexID; import org.apache.tez.dag.utils.TezBuilderUtils; import org.apache.tez.runtime.api.OutputCommitter; import org.apache.tez.runtime.api.impl.TaskSpec; @@ -434,7 +434,7 @@ public Vertex getVertex() { } @Override - public TezTaskID getTaskId() { + public TezTaskID getTaskID() { return taskId; } @@ -523,7 +523,7 @@ public ArrayList getTaskAttemptTezEvents(TezTaskAttemptID attemptID, try { if (!attempts.containsKey(attemptID)) { throw new TezUncheckedException("Unknown TA: " + attemptID - + " asking for events from task:" + getTaskId()); + + " asking for events from task:" + getTaskID()); } if (tezEventsForTaskAttempts.size() > fromEventId) { @@ -775,11 +775,11 @@ public TaskAttempt getSuccessfulAttempt() { private boolean addAndScheduleAttempt(TezTaskAttemptID schedulingCausalTA) { TaskAttempt attempt = createAttempt(attempts.size(), schedulingCausalTA); if (LOG.isDebugEnabled()) { - LOG.debug("Created attempt " + attempt.getID()); + LOG.debug("Created attempt " + attempt.getTaskAttemptID()); } switch (attempts.size()) { case 0: - attempts = Collections.singletonMap(attempt.getID(), attempt); + attempts = Collections.singletonMap(attempt.getTaskAttemptID(), attempt); break; case 1: @@ -787,12 +787,12 @@ private boolean addAndScheduleAttempt(TezTaskAttemptID schedulingCausalTA) { = new LinkedHashMap(maxFailedAttempts); newAttempts.putAll(attempts); attempts = newAttempts; - Preconditions.checkArgument(attempts.put(attempt.getID(), attempt) == null, - attempt.getID() + " already existed"); + Preconditions.checkArgument(attempts.put(attempt.getTaskAttemptID(), attempt) == null, + attempt.getTaskAttemptID() + " already existed"); break; default: - Preconditions.checkArgument(attempts.put(attempt.getID(), attempt) == null, - attempt.getID() + " already existed"); + Preconditions.checkArgument(attempts.put(attempt.getTaskAttemptID(), attempt) == null, + attempt.getTaskAttemptID() + " already existed"); break; } @@ -819,7 +819,7 @@ private boolean addAndScheduleAttempt(TezTaskAttemptID schedulingCausalTA) { } */ - this.taskAttemptStatus.put(attempt.getID().getId(), false); + this.taskAttemptStatus.put(attempt.getTaskAttemptID().getId(), false); //schedule the nextAttemptNumber // send event to DAG to assign priority and schedule the attempt with global // picture in mind @@ -865,17 +865,17 @@ protected void internalError(TaskEventType type) { LOG.error("Invalid event " + type + " on Task " + this.taskId + " in state:" + getInternalState()); eventHandler.handle(new DAGEventDiagnosticsUpdate( - this.taskId.getVertexID().getDAGId(), "Invalid event " + type + + getDAGID(), "Invalid event " + type + " on Task " + this.taskId)); - eventHandler.handle(new DAGEvent(this.taskId.getVertexID().getDAGId(), + eventHandler.handle(new DAGEvent(getDAGID(), DAGEventType.INTERNAL_ERROR)); } protected void internalErrorUncaughtException(TaskEventType type, Exception e) { eventHandler.handle(new DAGEventDiagnosticsUpdate( - this.taskId.getVertexID().getDAGId(), "Uncaught exception when handling event " + type + + getDAGID(), "Uncaught exception when handling event " + type + " on Task " + this.taskId + ", error=" + e.getMessage())); - eventHandler.handle(new DAGEvent(this.taskId.getVertexID().getDAGId(), + eventHandler.handle(new DAGEvent(getDAGID(), DAGEventType.INTERNAL_ERROR)); } @@ -918,7 +918,7 @@ protected void logJobHistoryTaskStartedEvent() { TaskStartedEvent startEvt = new TaskStartedEvent(taskId, getVertex().getName(), scheduledTime, getLaunchTime()); this.appContext.getHistoryHandler().handle( - new DAGHistoryEvent(taskId.getVertexID().getDAGId(), startEvt)); + new DAGHistoryEvent(getDAGID(), startEvt)); } protected void logJobHistoryTaskFinishedEvent() { @@ -930,7 +930,7 @@ protected void logJobHistoryTaskFinishedEvent() { successfulAttempt, TaskState.SUCCEEDED, "", getCounters(), failedAttempts); this.appContext.getHistoryHandler().handle( - new DAGHistoryEvent(taskId.getVertexID().getDAGId(), finishEvt)); + new DAGHistoryEvent(getDAGID(), finishEvt)); } protected void logJobHistoryTaskFailedEvent(TaskState finalState) { @@ -941,7 +941,7 @@ protected void logJobHistoryTaskFailedEvent(TaskState finalState) { StringUtils.join(getDiagnostics(), LINE_SEPARATOR), getCounters(), failedAttempts); this.appContext.getHistoryHandler().handle( - new DAGHistoryEvent(taskId.getVertexID().getDAGId(), finishEvt)); + new DAGHistoryEvent(getDAGID(), finishEvt)); } private void addDiagnosticInfo(String diag) { @@ -994,7 +994,7 @@ public TaskStateInternal transition(TaskImpl task, TaskEvent event) { && tFinishedEvent != null) { Preconditions.checkArgument(tFinishedEvent.getState() == TaskState.KILLED, "TaskStartedEvent is not seen, but TaskFinishedEvent is seen and with invalid state=" - + tFinishedEvent.getState() + ", taskId=" + task.getTaskId()); + + tFinishedEvent.getState() + ", taskId=" + task.getTaskID()); // TODO (TEZ-2938) // use tFinishedEvent.getTerminationCause after adding TaskTerminationCause to TaskFinishedEvent task.eventHandler.handle(new TaskEventTermination(task.taskId, @@ -1043,22 +1043,22 @@ public void transition(TaskImpl task, TaskEvent event) { } else { if (TaskAttemptState.SUCCEEDED.equals(ta.getState())) { LOG.info("Ignore speculation scheduling for task {} since it has succeeded with attempt {}.", - task.getTaskId(), ta.getID()); + task.getTaskID(), ta.getTaskAttemptID()); return; } } } if (earliestUnfinishedAttempt == null) { // no running (or SUCCEEDED) task attempt at this moment, no need to schedule speculative attempt either - LOG.info("Ignore speculation scheduling since there is no running attempt on task {}.", task.getTaskId()); + LOG.info("Ignore speculation scheduling since there is no running attempt on task {}.", task.getTaskID()); return; } if (task.commitAttempt != null) { LOG.info("Ignore speculation scheduling for task {} since commit has started with commitAttempt {}.", - task.getTaskId(), task.commitAttempt); + task.getTaskID(), task.commitAttempt); return; } - task.addAndScheduleAttempt(earliestUnfinishedAttempt.getID()); + task.addAndScheduleAttempt(earliestUnfinishedAttempt.getTaskAttemptID()); } } @@ -1075,26 +1075,26 @@ private String recoverSuccessTaskAttempt(TaskImpl task) { for (Entry entry : task.getVertex().getOutputCommitters().entrySet()) { LOG.info("Recovering data for task from previous DAG attempt" - + ", taskId=" + task.getTaskId() + + ", taskId=" + task.getTaskID() + ", output=" + entry.getKey()); OutputCommitter committer = entry.getValue(); if (!committer.isTaskRecoverySupported()) { errorMsg = "Task recovery not supported by committer" + ", failing task attempt"; LOG.info(errorMsg - + ", taskId=" + task.getTaskId() + + ", taskId=" + task.getTaskID() + ", attemptId=" + task.successfulAttempt + ", output=" + entry.getKey()); break; } try { - committer.recoverTask(task.getTaskId().getId(), + committer.recoverTask(task.getTaskID().getId(), task.appContext.getApplicationAttemptId().getAttemptId()-1); } catch (Exception e) { errorMsg = "Task recovery failed by committer: " + ExceptionUtils.getStackTrace(e); LOG.warn("Task recovery failed by committer" - + ", taskId=" + task.getTaskId() + + ", taskId=" + task.getTaskID() + ", attemptId=" + task.successfulAttempt + ", output=" + entry.getKey(), e); break; @@ -1114,7 +1114,7 @@ public TaskStateInternal transition(TaskImpl task, TaskEvent event) { String errorMsg = recoverSuccessTaskAttempt(task); if (errorMsg != null) { LOG.info("Can not recover the successful task attempt, schedule new task attempt," - + "taskId=" + task.getTaskId()); + + "taskId=" + task.getTaskID()); task.successfulAttempt = null; if (!task.addAndScheduleAttempt(successTaId)) { task.finished(TaskStateInternal.FAILED); @@ -1150,12 +1150,12 @@ public TaskStateInternal transition(TaskImpl task, TaskEvent event) { // issue kill to all other attempts for (TaskAttempt attempt : task.attempts.values()) { - if (!attempt.getID().equals(task.successfulAttempt) && + if (!attempt.getTaskAttemptID().equals(task.successfulAttempt) && // This is okay because it can only talk us out of sending a // TA_KILL message to an attempt that doesn't need one for // other reasons. !attempt.isFinished()) { - LOG.info("Issuing kill to other attempt " + attempt.getID() + " as attempt: " + + LOG.info("Issuing kill to other attempt " + attempt.getTaskAttemptID() + " as attempt: " + task.successfulAttempt + " has succeeded"); String diagnostics = null; TaskAttemptTerminationCause errCause = null; @@ -1169,7 +1169,7 @@ public TaskStateInternal transition(TaskImpl task, TaskEvent event) { errCause = TaskAttemptTerminationCause.TERMINATED_INEFFECTIVE_SPECULATION; } task.eventHandler.handle(new TaskAttemptEventKillRequest(attempt - .getID(), diagnostics, errCause)); + .getTaskAttemptID(), diagnostics, errCause)); } } return task.finished(TaskStateInternal.SUCCEEDED); @@ -1267,7 +1267,7 @@ public TaskStateInternal transition(TaskImpl task, TaskEvent event) { TaskAttempt taskAttempt = task.getAttempt(castEvent.getTaskAttemptID()); if (taskAttempt.getAssignedContainer() != null) { NodeId nodeId = taskAttempt.getAssignedContainer().getNodeId(); - task.appContext.getAppMaster().taskAttemptFailed(taskAttempt.getID(), nodeId); + task.appContext.getAppMaster().taskAttemptFailed(taskAttempt.getTaskAttemptID(), nodeId); } task.taskAttemptStatus.put(castEvent.getTaskAttemptID().getId(), true); @@ -1278,7 +1278,7 @@ public TaskStateInternal transition(TaskImpl task, TaskEvent event) { TaskAttemptStateInternal.FAILED); // we don't need a new event if we already have a spare if (task.shouldScheduleNewAttempt()) { - LOG.info("Scheduling new attempt for task: " + task.getTaskId() + LOG.info("Scheduling new attempt for task: " + task.getTaskID() + ", currentFailedAttempts: " + task.failedAttempts + ", maxFailedAttempts: " + task.maxFailedAttempts + ", maxAttempts: " + task.maxAttempts); if (!task.addAndScheduleAttempt(getSchedulingCausalTA())){ @@ -1289,11 +1289,11 @@ public TaskStateInternal transition(TaskImpl task, TaskEvent event) { if (castEvent.getTaskFailureType() == TaskFailureType.NON_FATAL) { LOG.info( "Failing task: {} due to too many failed attempts. currentFailedAttempts={}, maxFailedAttempts={}", - task.getTaskId(), task.failedAttempts, task.maxFailedAttempts); + task.getTaskID(), task.failedAttempts, task.maxFailedAttempts); } else { LOG.info( "Failing task: {} due to {} error reported by TaskAttempt. CurrentFailedAttempts={}", - task.getTaskId(), TaskFailureType.FATAL, task.failedAttempts); + task.getTaskID(), TaskFailureType.FATAL, task.failedAttempts); } task.handleTaskAttemptCompletion( ((TaskEventTAUpdate) event).getTaskAttemptID(), @@ -1348,7 +1348,7 @@ public TaskStateInternal transition(TaskImpl task, TaskEvent event) { if (task.leafVertex) { LOG.error("Unexpected event for task of leaf vertex " + event.getType() + ", taskId: " - + task.getTaskId()); + + task.getTaskID()); task.internalError(event.getType()); } Preconditions.checkState(castEvent.getCausalEvent() != null); @@ -1418,7 +1418,7 @@ public void transition(TaskImpl task, TaskEvent event) { task.addDiagnosticInfo(terminateEvent.getDiagnosticInfo()); if (terminateEvent.isFromRecovery()) { if (LOG.isDebugEnabled()) { - LOG.debug("Recovered to KILLED, taskId=" + task.getTaskId()); + LOG.debug("Recovered to KILLED, taskId=" + task.getTaskID()); } } else { task.logJobHistoryTaskFailedEvent(TaskState.KILLED); @@ -1453,20 +1453,20 @@ public void onStateChanged(TaskImpl task, TaskStateInternal taskStateInternal) { // With this, recovery will end up failing for DAGs making use of InputInitializerEvents int succesfulAttemptInt = -1; if (successfulAttempt != null) { - succesfulAttemptInt = successfulAttempt.getID().getId(); + succesfulAttemptInt = successfulAttempt.getTaskAttemptID().getId(); } - task.stateChangeNotifier.taskSucceeded(task.getVertex().getName(), task.getTaskId(), + task.stateChangeNotifier.taskSucceeded(task.getVertex().getName(), task.getTaskID(), succesfulAttemptInt); } } private void killUnfinishedAttempt(TaskAttempt attempt, String logMsg, TaskAttemptTerminationCause errorCause) { - if (commitAttempt != null && commitAttempt.equals(attempt.getID())) { + if (commitAttempt != null && commitAttempt.equals(attempt.getTaskAttemptID())) { LOG.info("Unsetting commit attempt: " + commitAttempt + " since attempt is being killed"); commitAttempt = null; } if (attempt != null && !attempt.isFinished()) { - eventHandler.handle(new TaskAttemptEventKillRequest(attempt.getID(), logMsg, errorCause)); + eventHandler.handle(new TaskAttemptEventKillRequest(attempt.getTaskAttemptID(), logMsg, errorCause)); } } @@ -1485,7 +1485,7 @@ public long getFirstAttemptStartTime() { readLock.lock(); try { // The first attempt will always have an index of 0. - return getAttempt(TezTaskAttemptID.getInstance(getTaskId(), 0)).getScheduleTime(); + return getAttempt(TezTaskAttemptID.getInstance(getTaskID(), 0)).getScheduleTime(); } finally { readLock.unlock(); } 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 a994359354..be4ee6068e 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 @@ -64,7 +64,7 @@ public TezRootInputInitializerContextImpl( @Override public ApplicationId getApplicationId() { - return vertex.getVertexId().getDAGId().getApplicationId(); + return vertex.getVertexId().getDAGID().getApplicationId(); } @Override 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 f22f6de250..934dd4e76e 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 @@ -1590,7 +1590,7 @@ private void computeProgress() { if (LOG.isDebugEnabled()) { if (!ProgressHelper.isProgressWithinRange(taskProg)) { LOG.debug("progress update: vertex={}, task={} incorrect; range={}", - getName(), task.getTaskId(), taskProg); + getName(), task.getTaskID(), taskProg); } } accProg += ProgressHelper.processProgress(taskProg); @@ -2052,7 +2052,7 @@ public void doneReconfiguringVertex() { */ public void handle(VertexEvent event) { if (LOG.isDebugEnabled()) { - LOG.debug("Processing VertexEvent " + event.getVertexId() + LOG.debug("Processing VertexEvent " + event.getVertexID() + " of type " + event.getType() + " while in state " + getInternalState() + ". Event: " + event); } @@ -2114,7 +2114,7 @@ protected void addTask(Task task) { lazyTasksCopyNeeded = false; } } - tasks.put(task.getTaskId(), task); + tasks.put(task.getTaskID(), task); // TODO Metrics //metrics.waitingTask(task); } @@ -2410,7 +2410,7 @@ void tryEnactKill(VertexTerminationCause trigger, LOG.info(msg); for (Task task : tasks.values()) { eventHandler.handle( // attempt was terminated because the vertex is shutting down - new TaskEventTermination(task.getTaskId(), errCause, msg)); + new TaskEventTermination(task.getTaskID(), errCause, msg)); } } } @@ -2649,7 +2649,7 @@ private void createTasks() { this.addTask(task); if(LOG.isDebugEnabled()) { LOG.debug("Created task for vertex " + logIdentifier + ": " + - task.getTaskId()); + task.getTaskID()); } } } @@ -2663,7 +2663,7 @@ private void addTasks(int newNumTasks) { this.numTasks++; if(LOG.isDebugEnabled()) { LOG.debug("Created task for vertex " + logIdentifier + ": " + - task.getTaskId()); + task.getTaskID()); } } } @@ -2919,7 +2919,7 @@ private static List getTaskAttemptIdentifiers(DAG dag, List attempts = new ArrayList(taIds.size()); String dagName = dag.getName(); for (TezTaskAttemptID taId : taIds) { - String vertexName = dag.getVertex(taId.getTaskID().getVertexID()).getName(); + String vertexName = dag.getVertex(taId.getVertexID()).getName(); attempts.add(getTaskAttemptIdentifier(dagName, vertexName, taId)); } return attempts; @@ -3639,7 +3639,7 @@ public VertexState transition(VertexImpl vertex, VertexEvent event) { TezTaskAttemptID taId = completionEvent.getTaskAttemptId(); vertex.vertexManager.onSourceTaskCompleted( getTaskAttemptIdentifier(vertex.dag.getName(), - vertex.dag.getVertex(taId.getTaskID().getVertexID()).getName(), + vertex.dag.getVertex(taId.getVertexID()).getName(), taId)); } catch (AMUserCodeException e) { String msg = "Exception in " + e.getSource() + ", vertex:" + vertex.getLogIdentifier(); @@ -3692,8 +3692,8 @@ public VertexState transition(VertexImpl vertex, VertexEvent event) { Task task = vertex.tasks.get(taskEvent.getTaskID()); if (taskEvent.getState() == TaskState.SUCCEEDED) { taskSucceeded(vertex, task); - if (!vertex.completedTasksStatsCache.containsTask(task.getTaskId())) { - vertex.completedTasksStatsCache.addTask(task.getTaskId()); + if (!vertex.completedTasksStatsCache.containsTask(task.getTaskID())) { + vertex.completedTasksStatsCache.addTask(task.getTaskID()); vertex.completedTasksStatsCache.mergeFrom(((TaskImpl) task).getStatistics()); } } else if (taskEvent.getState() == TaskState.FAILED) { @@ -3725,7 +3725,7 @@ private void taskSucceeded(VertexImpl vertex, Task task) { private void taskFailed(VertexImpl vertex, Task task) { vertex.failedTaskCount++; vertex.addDiagnostic("Task failed" - + ", taskId=" + task.getTaskId() + + ", taskId=" + task.getTaskID() + ", diagnostics=" + task.getDiagnostics()); // TODO Metrics //vertex.metrics.failedTask(task); @@ -4090,7 +4090,7 @@ private void handleRoutedTezEvents(List tezEvents, boolean isPendingEv pendingTaskEvents.add(tezEvent); } else { // event not from this vertex. must have come from source vertex. - int srcTaskIndex = sourceMeta.getTaskAttemptID().getTaskID().getId(); + int srcTaskIndex = sourceMeta.getTaskID().getId(); Vertex edgeVertex = getDAG().getVertex(sourceMeta.getTaskVertexName()); Edge srcEdge = sourceVertices.get(edgeVertex); if (srcEdge == null) { @@ -4132,7 +4132,7 @@ private void handleRoutedTezEvents(List tezEvents, boolean isPendingEv Preconditions.checkArgument(target != null, "Event sent to unkown vertex: " + vmEvent.getTargetVertexName()); TezTaskAttemptID srcTaId = sourceMeta.getTaskAttemptID(); - if (srcTaId.getTaskID().getVertexID().equals(vertexId)) { + if (srcTaId.getVertexID().equals(vertexId)) { // this is the producer tasks' vertex vmEvent.setProducerAttemptIdentifier( getTaskAttemptIdentifier(dag.getName(), getName(), srcTaId)); 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 57c8c72be1..c3d49ea840 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 @@ -226,7 +226,7 @@ public synchronized void scheduleTasks(List tasks) { @Override public synchronized void scheduleVertexTasks(List tasks) { checkAndThrowIfDone(); - List schedTasks = new ArrayList(tasks.size()); + List schedTasks = new ArrayList<>(tasks.size()); for (TaskWithLocationHint task : tasks) { schedTasks.add(ScheduleTaskRequest.create( task.getTaskIndex(), task.getTaskLocationHint())); @@ -292,7 +292,7 @@ public void sendEventToProcessor(Collection events, int ta // attempt id won't be used anyway EventMetaData destinationMeta = new EventMetaData(EventProducerConsumerType.PROCESSOR, managedVertex.getName(), managedVertex.getName(), - TezTaskAttemptID.getInstance(managedVertex.getTask(taskId).getTaskId(), -1)); + TezTaskAttemptID.getInstance(managedVertex.getTask(taskId).getTaskID(), -1)); tezEvent.setDestinationInfo(destinationMeta); tezEvents.add(tezEvent); } @@ -571,7 +571,7 @@ private void sendInternalError(Exception e) { // state change must be triggered via an event transition LOG.error("Error after vertex manager callback " + managedVertex.getLogIdentifier(), e); appContext.getEventHandler().handle( - (new DAGEventInternalError(managedVertex.getVertexId().getDAGId(), + (new DAGEventInternalError(managedVertex.getVertexId().getDAGID(), "Error in VertexManager for vertex: " + managedVertex.getLogIdentifier() + ", error=" + ExceptionUtils.getStackTrace(e)))); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/speculation/legacy/LegacySpeculator.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/speculation/legacy/LegacySpeculator.java index 7aad8f642b..940bb23025 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/speculation/legacy/LegacySpeculator.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/speculation/legacy/LegacySpeculator.java @@ -271,7 +271,7 @@ public void notifyAttemptStatusUpdate(TezTaskAttemptID taId, /** * Absorbs one TaskAttemptStatus * - * @param reportedStatus the status report that we got from a task attempt + * @param reportedState the status report that we got from a task attempt * that we want to fold into the speculation data for this job * @param timestamp the time this status corresponds to. This matters * because statuses contain progress. @@ -331,7 +331,7 @@ public void handle(SpeculatorEvent event) { // with private long speculationValue(Task task, long now, boolean shouldUseTimeout) { Map attempts = task.getAttempts(); - TezTaskID taskID = task.getTaskId(); + TezTaskID taskID = task.getTaskID(); long acceptableRuntime = Long.MIN_VALUE; long result = Long.MIN_VALUE; @@ -359,7 +359,7 @@ private long speculationValue(Task task, long now, boolean shouldUseTimeout) { if (++numberRunningAttempts > 1) { return ALREADY_SPECULATING; } - runningTaskAttemptID = taskAttempt.getID(); + runningTaskAttemptID = taskAttempt.getTaskAttemptID(); long taskAttemptStartTime = estimator.attemptEnrolledTime(runningTaskAttemptID); @@ -404,7 +404,7 @@ private long speculationValue(Task task, long now, boolean shouldUseTimeout) { .hasStagnatedProgress(runningTaskAttemptID, now)) { // Stats have stagnated for a while, simulate heart-beat. // Now simulate the heart-beat - statusUpdate(taskAttempt.getID(), taskAttempt.getState(), + statusUpdate(taskAttempt.getTaskAttemptID(), taskAttempt.getState(), clock.getTime()); } } else { diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/TaskAttemptFailedDeleteRunnable.java b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/TaskAttemptFailedDeleteRunnable.java index d2587b5190..22c5b26a7c 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/TaskAttemptFailedDeleteRunnable.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/TaskAttemptFailedDeleteRunnable.java @@ -53,8 +53,8 @@ public void run() { BaseHttpConnection httpConnection = null; try { URL baseURL = TezRuntimeUtils.constructBaseURIForShuffleHandlerTaskAttemptFailed( - nodeId.getHost(), shufflePort, taskAttemptID.getTaskID().getVertexID().getDAGId(). - getApplicationId().toString(), taskAttemptID.getTaskID().getVertexID().getDAGId().getId(), + nodeId.getHost(), shufflePort, taskAttemptID.getTaskID().getVertexID().getDAGID(). + getApplicationId().toString(), taskAttemptID.getTaskID().getVertexID().getDAGID().getId(), taskAttemptID.toString(), false); httpConnection = TezRuntimeUtils.getHttpConnection(true, baseURL, httpConnectionParams, "FailedTaskAttemptDelete", jobTokenSecretManager); 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 f7fee3acfe..107fbf62c7 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 @@ -42,7 +42,7 @@ public AMSchedulerEventTAEnded(TaskAttempt attempt, ContainerId containerId, } public TezTaskAttemptID getAttemptID() { - return this.attempt.getID(); + return this.attempt.getTaskAttemptID(); } public TaskAttempt getAttempt() { diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerManager.java index 8e6bfe79a5..8a24ad3426 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerManager.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerManager.java @@ -389,7 +389,7 @@ private void handleTAUnsuccessfulEnd(AMSchedulerEventTAEnded event) { String msg = "Error in TaskScheduler for handling Task De-allocation" + ", eventType=" + event.getType() + ", scheduler=" + Utils.getTaskSchedulerIdentifierString(event.getSchedulerId(), appContext) - + ", taskAttemptId=" + attempt.getID(); + + ", taskAttemptId=" + attempt.getTaskAttemptID(); LOG.error(msg, e); sendEvent( new DAGAppMasterEventUserServiceFatalError( @@ -403,10 +403,10 @@ private void handleTAUnsuccessfulEnd(AMSchedulerEventTAEnded event) { ContainerId attemptContainerId = attempt.getAssignedContainerID(); if(!wasContainerAllocated) { - LOG.info("Task: " + attempt.getID() + + LOG.info("Task: " + attempt.getTaskAttemptID() + " has no container assignment in the scheduler"); if (attemptContainerId != null) { - LOG.error("No container allocated to task: " + attempt.getID() + LOG.error("No container allocated to task: " + attempt.getTaskAttemptID() + " according to scheduler. Task reported container id: " + attemptContainerId); } @@ -425,7 +425,7 @@ private void handleTAUnsuccessfulEnd(AMSchedulerEventTAEnded event) { Container container = amContainer.getContainer(); sendEvent(new AMNodeEventTaskAttemptEnded(container.getNodeId(), event.getSchedulerId(), attemptContainerId, - attempt.getID(), event.getState() == TaskAttemptState.FAILED)); + attempt.getTaskAttemptID(), event.getState() == TaskAttemptState.FAILED)); } } } @@ -458,7 +458,7 @@ private void handleTASucceeded(AMSchedulerEventTAEnded event) { String msg = "Error in TaskScheduler for handling Task De-allocation" + ", eventType=" + event.getType() + ", scheduler=" + Utils.getTaskSchedulerIdentifierString(event.getSchedulerId(), appContext) - + ", taskAttemptId=" + attempt.getID(); + + ", taskAttemptId=" + attempt.getTaskAttemptID(); LOG.error(msg, e); sendEvent( new DAGAppMasterEventUserServiceFatalError( @@ -468,7 +468,7 @@ private void handleTASucceeded(AMSchedulerEventTAEnded event) { } if (!wasContainerAllocated) { - LOG.error("De-allocated successful task: " + attempt.getID() + LOG.error("De-allocated successful task: " + attempt.getTaskAttemptID() + ", but TaskScheduler reported no container assigned to task"); } } @@ -483,15 +483,15 @@ private void handleTaLaunchRequest(AMSchedulerEventTALaunchRequest event) { if (taskAffinity != null) { Vertex vertex = appContext.getCurrentDAG().getVertex(taskAffinity.getVertexName()); Objects.requireNonNull(vertex, "Invalid vertex in task based affinity " + taskAffinity - + " for attempt: " + taskAttempt.getID()); + + " for attempt: " + taskAttempt.getTaskAttemptID()); int taskIndex = taskAffinity.getTaskIndex(); Preconditions.checkState(taskIndex >=0 && taskIndex < vertex.getTotalTasks(), "Invalid taskIndex in task based affinity " + taskAffinity - + " for attempt: " + taskAttempt.getID()); + + " for attempt: " + taskAttempt.getTaskAttemptID()); TaskAttempt affinityAttempt = vertex.getTask(taskIndex).getSuccessfulAttempt(); if (affinityAttempt != null) { Objects.requireNonNull(affinityAttempt.getAssignedContainerID(), - affinityAttempt.getID() == null ? null : affinityAttempt.getID().toString()); + affinityAttempt.getTaskAttemptID() == null ? null : affinityAttempt.getTaskAttemptID().toString()); try { taskSchedulers[event.getSchedulerId()].allocateTask(taskAttempt, event.getCapability(), @@ -503,7 +503,7 @@ private void handleTaLaunchRequest(AMSchedulerEventTALaunchRequest event) { String msg = "Error in TaskScheduler for handling Task Allocation" + ", eventType=" + event.getType() + ", scheduler=" + Utils.getTaskSchedulerIdentifierString(event.getSchedulerId(), appContext) - + ", taskAttemptId=" + taskAttempt.getID(); + + ", taskAttemptId=" + taskAttempt.getTaskAttemptID(); LOG.error(msg, e); sendEvent( new DAGAppMasterEventUserServiceFatalError( @@ -513,7 +513,7 @@ private void handleTaLaunchRequest(AMSchedulerEventTALaunchRequest event) { return; } LOG.info("No attempt for task affinity to " + taskAffinity + " for attempt " - + taskAttempt.getID() + " Ignoring."); + + taskAttempt.getTaskAttemptID() + " Ignoring."); // fall through with null hosts/racks } else { hosts = (locationHint.getHosts() != null) ? locationHint @@ -536,7 +536,7 @@ private void handleTaLaunchRequest(AMSchedulerEventTALaunchRequest event) { String msg = "Error in TaskScheduler for handling Task Allocation" + ", eventType=" + event.getType() + ", scheduler=" + Utils.getTaskSchedulerIdentifierString(event.getSchedulerId(), appContext) - + ", taskAttemptId=" + taskAttempt.getID(); + + ", taskAttemptId=" + taskAttempt.getTaskAttemptID(); LOG.error(msg, e); sendEvent( new DAGAppMasterEventUserServiceFatalError( @@ -552,7 +552,7 @@ private void handleTAStateUpdated(AMSchedulerEventTAStateUpdated event) { String msg = "Error in TaskScheduler for handling Task State Update" + ", eventType=" + event.getType() + ", scheduler=" + Utils.getTaskSchedulerIdentifierString(event.getSchedulerId(), appContext) - + ", taskAttemptId=" + event.getTaskAttempt().getID() + + ", taskAttemptId=" + event.getTaskAttempt().getTaskAttemptID() + ", state=" + event.getState(); LOG.error(msg, e); sendEvent( @@ -763,7 +763,7 @@ public synchronized void taskAllocated(int schedulerId, Object task, event.getContainerContext(), event.getLauncherId(), event.getTaskCommId())); } } - sendEvent(new AMContainerEventAssignTA(containerId, taskAttempt.getID(), + sendEvent(new AMContainerEventAssignTA(containerId, taskAttempt.getTaskAttemptID(), event.getRemoteTaskSpec(), event.getContainerContext().getLocalResources(), event .getContainerContext().getCredentials(), event.getPriority())); } 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 92e5817243..c3aae35910 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 @@ -41,7 +41,7 @@ public AMContainerEventLaunchRequest(ContainerId containerId, } public TezDAGID getDAGId() { - return this.vertexId.getDAGId(); + return this.vertexId.getDAGID(); } public TezVertexID getVertexId() { diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/web/AMWebController.java b/tez-dag/src/main/java/org/apache/tez/dag/app/web/AMWebController.java index 08d754d8a0..00cd26e2ce 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/web/AMWebController.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/web/AMWebController.java @@ -760,7 +760,7 @@ public void getTasksInfo() { ArrayList> tasksInfo = new ArrayList>(); for(Task t : tasks) { Map taskInfo = new HashMap(); - taskInfo.put("id", t.getTaskId().toString()); + taskInfo.put("id", t.getTaskID().toString()); taskInfo.put("progress", Float.toString(t.getProgress())); taskInfo.put("status", t.getState().toString()); @@ -810,7 +810,7 @@ else if(!attemptIDs.isEmpty()) { } TaskAttempt attempt = task. - getAttempt(TezTaskAttemptID.getInstance(task.getTaskId(), indexes.get(2))); + getAttempt(TezTaskAttemptID.getInstance(task.getTaskID(), indexes.get(2))); if(attempt == null) { continue; } @@ -858,7 +858,7 @@ public void getAttemptsInfo() { ArrayList> attemptsInfo = new ArrayList>(); for(TaskAttempt a : attempts) { Map attemptInfo = new HashMap(); - attemptInfo.put("id", a.getID().toString()); + attemptInfo.put("id", a.getTaskAttemptID().toString()); attemptInfo.put("progress", Float.toString(a.getProgress())); attemptInfo.put("status", a.getState().toString()); diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/DAGHistoryEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/DAGHistoryEvent.java index dfa6bbdf91..240d8bd96d 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/DAGHistoryEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/DAGHistoryEvent.java @@ -18,9 +18,10 @@ package org.apache.tez.dag.history; +import org.apache.tez.dag.records.DAGIDAware; import org.apache.tez.dag.records.TezDAGID; -public class DAGHistoryEvent { +public class DAGHistoryEvent implements DAGIDAware { private final HistoryEvent historyEvent; private final TezDAGID dagID; @@ -39,7 +40,8 @@ public HistoryEvent getHistoryEvent() { return historyEvent; } - public TezDAGID getDagID() { + @Override + public TezDAGID getDAGID() { return this.dagID; } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/HistoryEventHandler.java b/tez-dag/src/main/java/org/apache/tez/dag/history/HistoryEventHandler.java index 4fa1926ae9..f4dd789fd9 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/HistoryEventHandler.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/HistoryEventHandler.java @@ -124,7 +124,7 @@ public void serviceStop() throws Exception { * @throws IOException */ public void handleCriticalEvent(DAGHistoryEvent event) throws IOException { - TezDAGID dagId = event.getDagID(); + TezDAGID dagId = event.getDAGID(); String dagIdStr = "N/A"; if(dagId != null) { dagIdStr = dagId.toString(); @@ -161,7 +161,7 @@ public void handleCriticalEvent(DAGHistoryEvent event) throws IOException { } private boolean shouldLogEvent(DAGHistoryEvent event) { - TezDAGID dagId = event.getDagID(); + TezDAGID dagId = event.getDAGID(); HistoryLogLevel dagLogLevel = null; if (dagId != null) { @@ -207,7 +207,7 @@ private boolean shouldLogTaskAttemptEvents(DAGHistoryEvent event, HistoryLogLeve if (dagLogLevel == HistoryLogLevel.TASK_ATTEMPT && (eventType == HistoryEventType.TASK_ATTEMPT_STARTED || eventType == HistoryEventType.TASK_ATTEMPT_FINISHED)) { - TezDAGID dagId = event.getDagID(); + TezDAGID dagId = event.getDAGID(); Set filters = null; if (dagId != null) { filters = dagIdToTaskAttemptFilters.get(dagId); diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGFinishedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGFinishedEvent.java index 0a7ef561f7..6d1d6cd3ff 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGFinishedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGFinishedEvent.java @@ -32,6 +32,7 @@ import org.apache.tez.dag.history.HistoryEvent; import org.apache.tez.dag.history.HistoryEventType; import org.apache.tez.dag.history.SummaryEvent; +import org.apache.tez.dag.records.DAGIDAware; import org.apache.tez.dag.records.TezDAGID; import org.apache.tez.dag.recovery.records.RecoveryProtos; import org.apache.tez.dag.recovery.records.RecoveryProtos.DAGFinishedProto; @@ -40,7 +41,7 @@ import com.google.common.primitives.Ints; import com.google.protobuf.ByteString; -public class DAGFinishedEvent implements HistoryEvent, SummaryEvent { +public class DAGFinishedEvent implements HistoryEvent, SummaryEvent, DAGIDAware { private TezDAGID dagID; private long startTime; @@ -179,7 +180,8 @@ public DAGState getState() { return state; } - public TezDAGID getDagID() { + @Override + public TezDAGID getDAGID() { return dagID; } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGInitializedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGInitializedEvent.java index 9e6c8b282b..4df116ad37 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGInitializedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGInitializedEvent.java @@ -25,12 +25,13 @@ import com.google.protobuf.CodedOutputStream; import org.apache.tez.dag.history.HistoryEvent; import org.apache.tez.dag.history.HistoryEventType; +import org.apache.tez.dag.records.DAGIDAware; import org.apache.tez.dag.records.TezDAGID; import org.apache.tez.dag.records.TezVertexID; import org.apache.tez.dag.recovery.records.RecoveryProtos; import org.apache.tez.dag.recovery.records.RecoveryProtos.DAGInitializedProto; -public class DAGInitializedEvent implements HistoryEvent { +public class DAGInitializedEvent implements HistoryEvent, DAGIDAware { private TezDAGID dagID; private long initTime; @@ -101,7 +102,8 @@ public long getInitTime() { return this.initTime; } - public TezDAGID getDagID() { + @Override + public TezDAGID getDAGID() { return dagID; } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGStartedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGStartedEvent.java index f1fdcac809..9e1f6c4a50 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGStartedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGStartedEvent.java @@ -25,10 +25,11 @@ import org.apache.tez.dag.app.dag.DAGState; import org.apache.tez.dag.history.HistoryEvent; import org.apache.tez.dag.history.HistoryEventType; +import org.apache.tez.dag.records.DAGIDAware; import org.apache.tez.dag.records.TezDAGID; import org.apache.tez.dag.recovery.records.RecoveryProtos.DAGStartedProto; -public class DAGStartedEvent implements HistoryEvent { +public class DAGStartedEvent implements HistoryEvent, DAGIDAware { private TezDAGID dagID; private long startTime; @@ -97,7 +98,8 @@ public long getStartTime() { return this.startTime; } - public TezDAGID getDagID() { + @Override + public TezDAGID getDAGID() { return dagID; } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGSubmittedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGSubmittedEvent.java index e04ee80188..bed0f6f25a 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGSubmittedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGSubmittedEvent.java @@ -24,6 +24,7 @@ import com.google.protobuf.CodedInputStream; import com.google.protobuf.CodedOutputStream; +import org.apache.tez.dag.records.DAGIDAware; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.conf.Configuration; @@ -42,7 +43,7 @@ import org.apache.tez.dag.utils.ProtoUtils; -public class DAGSubmittedEvent implements HistoryEvent, SummaryEvent { +public class DAGSubmittedEvent implements HistoryEvent, SummaryEvent, DAGIDAware { private static final Logger LOG = LoggerFactory.getLogger(DAGSubmittedEvent.class); @@ -174,7 +175,8 @@ public DAGProtos.DAGPlan getDAGPlan() { return this.dagPlan; } - public TezDAGID getDagID() { + @Override + public TezDAGID getDAGID() { return dagID; } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskAttemptFinishedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskAttemptFinishedEvent.java index 18596495be..ce2eb729f6 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskAttemptFinishedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskAttemptFinishedEvent.java @@ -27,6 +27,7 @@ import org.apache.tez.common.TezConverterUtils; import org.apache.tez.common.counters.CounterGroup; import org.apache.tez.common.counters.TezCounter; +import org.apache.tez.dag.records.TaskAttemptIDAware; import org.apache.tez.runtime.api.TaskFailureType; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -50,7 +51,7 @@ import org.apache.tez.dag.recovery.records.RecoveryProtos.TezEventProto; import org.apache.tez.runtime.api.impl.TezEvent; -public class TaskAttemptFinishedEvent implements HistoryEvent { +public class TaskAttemptFinishedEvent implements HistoryEvent, TaskAttemptIDAware { private static final Logger LOG = LoggerFactory.getLogger(TaskAttemptFinishedEvent.class); @@ -307,6 +308,7 @@ public String toString() { return sb.toString(); } + @Override public TezTaskAttemptID getTaskAttemptID() { return taskAttemptId; } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskAttemptStartedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskAttemptStartedEvent.java index a49e47c97f..10ba439e1c 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskAttemptStartedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskAttemptStartedEvent.java @@ -27,10 +27,11 @@ import org.apache.hadoop.yarn.util.ConverterUtils; import org.apache.tez.dag.history.HistoryEvent; import org.apache.tez.dag.history.HistoryEventType; +import org.apache.tez.dag.records.TaskAttemptIDAware; import org.apache.tez.dag.records.TezTaskAttemptID; import org.apache.tez.dag.recovery.records.RecoveryProtos.TaskAttemptStartedProto; -public class TaskAttemptStartedEvent implements HistoryEvent { +public class TaskAttemptStartedEvent implements HistoryEvent, TaskAttemptIDAware { private TezTaskAttemptID taskAttemptId; private String inProgressLogsUrl; @@ -113,6 +114,7 @@ public String toString() { + ", nodeId=" + nodeId; } + @Override public TezTaskAttemptID getTaskAttemptID() { return this.taskAttemptId; } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskFinishedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskFinishedEvent.java index dd7afdfc7c..215968274c 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskFinishedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskFinishedEvent.java @@ -24,6 +24,7 @@ import com.google.protobuf.CodedOutputStream; import org.apache.tez.common.counters.CounterGroup; import org.apache.tez.common.counters.TezCounter; +import org.apache.tez.dag.records.TaskIDAware; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.tez.common.counters.TezCounters; @@ -34,7 +35,7 @@ import org.apache.tez.dag.records.TezTaskID; import org.apache.tez.dag.recovery.records.RecoveryProtos.TaskFinishedProto; -public class TaskFinishedEvent implements HistoryEvent { +public class TaskFinishedEvent implements HistoryEvent, TaskIDAware { private static final Logger LOG = LoggerFactory.getLogger(TaskFinishedEvent.class); @@ -160,6 +161,7 @@ public String toString() { return sb.toString(); } + @Override public TezTaskID getTaskID() { return taskID; } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskStartedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskStartedEvent.java index cc629699a4..577aef963c 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskStartedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskStartedEvent.java @@ -25,10 +25,11 @@ import org.apache.tez.dag.api.oldrecords.TaskState; import org.apache.tez.dag.history.HistoryEvent; import org.apache.tez.dag.history.HistoryEventType; +import org.apache.tez.dag.records.TaskIDAware; import org.apache.tez.dag.records.TezTaskID; import org.apache.tez.dag.recovery.records.RecoveryProtos.TaskStartedProto; -public class TaskStartedEvent implements HistoryEvent { +public class TaskStartedEvent implements HistoryEvent, TaskIDAware { private TezTaskID taskID; private String vertexName; @@ -104,6 +105,7 @@ public String toString() { + ", launchTime=" + startTime; } + @Override public TezTaskID getTaskID() { return taskID; } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexCommitStartedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexCommitStartedEvent.java index 8ff86b8af1..ca2cafc834 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexCommitStartedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexCommitStartedEvent.java @@ -99,7 +99,7 @@ public TezVertexID getVertexID() { @Override public void toSummaryProtoStream(OutputStream outputStream) throws IOException { SummaryEventProto.Builder builder = RecoveryProtos.SummaryEventProto.newBuilder() - .setDagId(vertexID.getDAGId().toString()) + .setDagId(vertexID.getDAGID().toString()) .setTimestamp(commitStartTime) .setEventType(getEventType().ordinal()) .setEventPayload( diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexConfigurationDoneEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexConfigurationDoneEvent.java index a2e20397cf..2f4fac079f 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexConfigurationDoneEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexConfigurationDoneEvent.java @@ -30,6 +30,7 @@ import org.apache.tez.dag.history.HistoryEvent; import org.apache.tez.dag.history.HistoryEventType; import org.apache.tez.dag.records.TezVertexID; +import org.apache.tez.dag.records.VertexIDAware; import org.apache.tez.dag.recovery.records.RecoveryProtos.EdgeManagerDescriptorProto; import org.apache.tez.dag.recovery.records.RecoveryProtos.RootInputSpecUpdateProto; import org.apache.tez.dag.recovery.records.RecoveryProtos.VertexConfigurationDoneProto; @@ -37,7 +38,7 @@ import com.google.common.collect.Maps; -public class VertexConfigurationDoneEvent implements HistoryEvent { +public class VertexConfigurationDoneEvent implements HistoryEvent, VertexIDAware { private TezVertexID vertexID; private long reconfigureDoneTime; @@ -182,6 +183,7 @@ public String toString() { + ", setParallelismCalledFlag=" + setParallelismCalledFlag; } + @Override public TezVertexID getVertexID() { return this.vertexID; } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexFinishedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexFinishedEvent.java index 58cb628b28..24612a6a2e 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexFinishedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexFinishedEvent.java @@ -25,6 +25,7 @@ import com.google.protobuf.CodedInputStream; import com.google.protobuf.CodedOutputStream; import org.apache.tez.dag.app.dag.impl.ServicePluginInfo; +import org.apache.tez.dag.records.VertexIDAware; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.tez.common.counters.TezCounters; @@ -39,7 +40,7 @@ import org.apache.tez.dag.recovery.records.RecoveryProtos.VertexFinishStateProto; import org.apache.tez.dag.recovery.records.RecoveryProtos.VertexFinishedProto; -public class VertexFinishedEvent implements HistoryEvent, SummaryEvent { +public class VertexFinishedEvent implements HistoryEvent, SummaryEvent, VertexIDAware { private static final Logger LOG = LoggerFactory.getLogger(VertexFinishedEvent.class); @@ -157,6 +158,7 @@ public String toString() { + (servicePluginInfo != null ? servicePluginInfo : "null"); } + @Override public TezVertexID getVertexID() { return this.vertexID; } @@ -207,7 +209,7 @@ public void toSummaryProtoStream(OutputStream outputStream) throws IOException { .build(); SummaryEventProto.Builder builder = RecoveryProtos.SummaryEventProto.newBuilder() - .setDagId(vertexID.getDAGId().toString()) + .setDagId(vertexID.getDAGID().toString()) .setTimestamp(finishTime) .setEventType(getEventType().ordinal()) .setEventPayload(finishStateProto.toByteString()); diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexInitializedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexInitializedEvent.java index e7452e6770..a019fe3784 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexInitializedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexInitializedEvent.java @@ -35,6 +35,7 @@ import org.apache.tez.dag.history.HistoryEventType; import org.apache.tez.dag.history.utils.TezEventUtils; import org.apache.tez.dag.records.TezVertexID; +import org.apache.tez.dag.records.VertexIDAware; import org.apache.tez.dag.recovery.records.RecoveryProtos; import org.apache.tez.dag.recovery.records.RecoveryProtos.TezEventProto; import org.apache.tez.dag.recovery.records.RecoveryProtos.VertexInitializedProto; @@ -42,7 +43,7 @@ import com.google.common.collect.Lists; -public class VertexInitializedEvent implements HistoryEvent { +public class VertexInitializedEvent implements HistoryEvent, VertexIDAware { private TezVertexID vertexID; private String vertexName; @@ -180,8 +181,9 @@ public String toString() { + (servicePluginInfo != null ? servicePluginInfo : "null"); } + @Override public TezVertexID getVertexID() { - return this.vertexID; + return vertexID; } public long getInitRequestedTime() { diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexStartedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexStartedEvent.java index 4a3e05f1f4..cdd73b4483 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexStartedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexStartedEvent.java @@ -26,9 +26,10 @@ import org.apache.tez.dag.history.HistoryEvent; import org.apache.tez.dag.history.HistoryEventType; import org.apache.tez.dag.records.TezVertexID; +import org.apache.tez.dag.records.VertexIDAware; import org.apache.tez.dag.recovery.records.RecoveryProtos.VertexStartedProto; -public class VertexStartedEvent implements HistoryEvent { +public class VertexStartedEvent implements HistoryEvent, VertexIDAware { private TezVertexID vertexID; private long startRequestedTime; @@ -94,8 +95,9 @@ public String toString() { + ", startedTime=" + startTime; } + @Override public TezVertexID getVertexID() { - return this.vertexID; + return vertexID; } public long getStartRequestedTime() { diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/logging/impl/HistoryEventJsonConversion.java b/tez-dag/src/main/java/org/apache/tez/dag/history/logging/impl/HistoryEventJsonConversion.java index e60575f38e..8f4cd1fa6a 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/logging/impl/HistoryEventJsonConversion.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/logging/impl/HistoryEventJsonConversion.java @@ -341,7 +341,7 @@ private static JSONObject convertContainerStoppedEvent(ContainerStoppedEvent eve private static JSONObject convertDAGFinishedEvent(DAGFinishedEvent event) throws JSONException { JSONObject jsonObject = new JSONObject(); jsonObject.put(ATSConstants.ENTITY, - event.getDagID().toString()); + event.getDAGID().toString()); jsonObject.put(ATSConstants.ENTITY_TYPE, EntityTypes.TEZ_DAG_ID.name()); @@ -384,7 +384,7 @@ private static JSONObject convertDAGFinishedEvent(DAGFinishedEvent event) throws private static JSONObject convertDAGInitializedEvent(DAGInitializedEvent event) throws JSONException { JSONObject jsonObject = new JSONObject(); jsonObject.put(ATSConstants.ENTITY, - event.getDagID().toString()); + event.getDAGID().toString()); jsonObject.put(ATSConstants.ENTITY_TYPE, EntityTypes.TEZ_DAG_ID.name()); @@ -416,7 +416,7 @@ private static JSONObject convertDAGInitializedEvent(DAGInitializedEvent event) private static JSONObject convertDAGStartedEvent(DAGStartedEvent event) throws JSONException { JSONObject jsonObject = new JSONObject(); jsonObject.put(ATSConstants.ENTITY, - event.getDagID().toString()); + event.getDAGID().toString()); jsonObject.put(ATSConstants.ENTITY_TYPE, EntityTypes.TEZ_DAG_ID.name()); @@ -439,7 +439,7 @@ private static JSONObject convertDAGStartedEvent(DAGStartedEvent event) throws J private static JSONObject convertDAGSubmittedEvent(DAGSubmittedEvent event) throws JSONException { JSONObject jsonObject = new JSONObject(); jsonObject.put(ATSConstants.ENTITY, - event.getDagID().toString()); + event.getDAGID().toString()); jsonObject.put(ATSConstants.ENTITY_TYPE, EntityTypes.TEZ_DAG_ID.name()); @@ -602,7 +602,7 @@ private static JSONObject convertTaskAttemptStartedEvent(TaskAttemptStartedEvent containerEntity.put(ATSConstants.ENTITY_TYPE, ATSConstants.CONTAINER_ID); JSONObject taskEntity = new JSONObject(); - taskEntity.put(ATSConstants.ENTITY, event.getTaskAttemptID().getTaskID().toString()); + taskEntity.put(ATSConstants.ENTITY, event.getTaskID().toString()); taskEntity.put(ATSConstants.ENTITY_TYPE, EntityTypes.TEZ_TASK_ID.name()); relatedEntities.put(nodeEntity); @@ -667,7 +667,7 @@ private static JSONObject convertTaskStartedEvent(TaskStartedEvent event) throws // Related entities JSONArray relatedEntities = new JSONArray(); JSONObject vertexEntity = new JSONObject(); - vertexEntity.put(ATSConstants.ENTITY, event.getTaskID().getVertexID().toString()); + vertexEntity.put(ATSConstants.ENTITY, event.getVertexID().toString()); vertexEntity.put(ATSConstants.ENTITY_TYPE, EntityTypes.TEZ_VERTEX_ID.name()); relatedEntities.put(vertexEntity); jsonObject.put(ATSConstants.RELATED_ENTITIES, relatedEntities); @@ -775,7 +775,7 @@ private static JSONObject convertVertexInitializedEvent(VertexInitializedEvent e // Related entities JSONArray relatedEntities = new JSONArray(); JSONObject vertexEntity = new JSONObject(); - vertexEntity.put(ATSConstants.ENTITY, event.getVertexID().getDAGId().toString()); + vertexEntity.put(ATSConstants.ENTITY, event.getDAGID().toString()); vertexEntity.put(ATSConstants.ENTITY_TYPE, EntityTypes.TEZ_DAG_ID.name()); relatedEntities.put(vertexEntity); jsonObject.put(ATSConstants.RELATED_ENTITIES, relatedEntities); @@ -815,7 +815,7 @@ private static JSONObject convertVertexStartedEvent(VertexStartedEvent event) // Related entities JSONArray relatedEntities = new JSONArray(); JSONObject vertexEntity = new JSONObject(); - vertexEntity.put(ATSConstants.ENTITY, event.getVertexID().getDAGId().toString()); + vertexEntity.put(ATSConstants.ENTITY, event.getDAGID().toString()); vertexEntity.put(ATSConstants.ENTITY_TYPE, EntityTypes.TEZ_DAG_ID.name()); relatedEntities.put(vertexEntity); jsonObject.put(ATSConstants.RELATED_ENTITIES, relatedEntities); diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/recovery/RecoveryService.java b/tez-dag/src/main/java/org/apache/tez/dag/history/recovery/RecoveryService.java index 45e7d2fc9c..5b5a9c7261 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/recovery/RecoveryService.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/recovery/RecoveryService.java @@ -300,7 +300,7 @@ public void handle(DAGHistoryEvent event) throws IOException { return; } - TezDAGID dagId = event.getDagID(); + TezDAGID dagId = event.getDAGID(); if (eventType.equals(HistoryEventType.DAG_SUBMITTED)) { DAGSubmittedEvent dagSubmittedEvent = (DAGSubmittedEvent) event.getHistoryEvent(); @@ -337,8 +337,8 @@ public void handle(DAGHistoryEvent event) throws IOException { if (summaryEvent.writeToRecoveryImmediately()) { handleRecoveryEvent(event); // outputStream may already be closed and removed - if (outputStreamMap.containsKey(event.getDagID())) { - doFlush(outputStreamMap.get(event.getDagID()), + if (outputStreamMap.containsKey(event.getDAGID())) { + doFlush(outputStreamMap.get(event.getDAGID()), appContext.getClock().getTime()); } } else { @@ -350,7 +350,7 @@ public void handle(DAGHistoryEvent event) throws IOException { } if (eventType.equals(HistoryEventType.DAG_FINISHED)) { LOG.info("DAG completed" - + ", dagId=" + event.getDagID() + + ", dagId=" + event.getDAGID() + ", queueSize=" + eventQueue.size()); completedDAGs.add(dagId); if (outputStreamMap.containsKey(dagId)) { @@ -359,7 +359,7 @@ public void handle(DAGHistoryEvent event) throws IOException { outputStreamMap.remove(dagId); } catch (IOException ioe) { LOG.warn("Error when trying to flush/close recovery file for" - + " dag, dagId=" + event.getDagID()); + + " dag, dagId=" + event.getDAGID()); } } } @@ -436,7 +436,7 @@ protected void handleRecoveryEvent(DAGHistoryEvent event) throws IOException { LOG.debug("Handling recovery event of type " + event.getHistoryEvent().getEventType()); } - TezDAGID dagID = event.getDagID(); + TezDAGID dagID = event.getDAGID(); if (completedDAGs.contains(dagID)) { // no need to recover completed DAGs diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/TestMockDAGAppMaster.java b/tez-dag/src/test/java/org/apache/tez/dag/app/TestMockDAGAppMaster.java index aa5d3b9535..6c0231c9b0 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/TestMockDAGAppMaster.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/TestMockDAGAppMaster.java @@ -261,7 +261,7 @@ public void testBasicEvents() throws Exception { Assert.assertEquals(DAGStatus.State.SUCCEEDED, dagClient.getDAGStatus(null).getState()); VertexImpl vImpl = (VertexImpl) dagImpl.getVertex(vB.getName()); TaskImpl tImpl = (TaskImpl) vImpl.getTask(1); - TezTaskAttemptID taId = TezTaskAttemptID.getInstance(tImpl.getTaskId(), 0); + TezTaskAttemptID taId = TezTaskAttemptID.getInstance(tImpl.getTaskID(), 0); List tEvents = vImpl.getTaskAttemptTezEvents(taId, 0, 0, 1000).getEvents(); Assert.assertEquals(2, tEvents.size()); // 2 from vA Assert.assertEquals(vA.getName(), tEvents.get(0).getDestinationInfo().getEdgeVertexName()); @@ -275,7 +275,7 @@ public void testBasicEvents() throws Exception { (targetIndex1 == 0 && targetIndex2 == 1) || (targetIndex1 == 1 && targetIndex2 == 0)); vImpl = (VertexImpl) dagImpl.getVertex(vC.getName()); tImpl = (TaskImpl) vImpl.getTask(1); - taId = TezTaskAttemptID.getInstance(tImpl.getTaskId(), 0); + taId = TezTaskAttemptID.getInstance(tImpl.getTaskID(), 0); tEvents = vImpl.getTaskAttemptTezEvents(taId, 0, 0, 1000).getEvents(); Assert.assertEquals(2, tEvents.size()); // 2 from vA Assert.assertEquals(vA.getName(), tEvents.get(0).getDestinationInfo().getEdgeVertexName()); @@ -289,7 +289,7 @@ public void testBasicEvents() throws Exception { (targetIndex1 == 0 && targetIndex2 == 1) || (targetIndex1 == 1 && targetIndex2 == 0)); vImpl = (VertexImpl) dagImpl.getVertex(vD.getName()); tImpl = (TaskImpl) vImpl.getTask(1); - taId = TezTaskAttemptID.getInstance(tImpl.getTaskId(), 0); + taId = TezTaskAttemptID.getInstance(tImpl.getTaskID(), 0); tEvents = vImpl.getTaskAttemptTezEvents(taId, 0, 0, 1000).getEvents(); Assert.assertEquals(1, tEvents.size()); // 1 from vA Assert.assertEquals(vA.getName(), tEvents.get(0).getDestinationInfo().getEdgeVertexName()); @@ -398,17 +398,17 @@ public void testMixedEdgeRouting() throws Exception { // vC uses on demand routing and its task does not provide events VertexImpl vImpl = (VertexImpl) dagImpl.getVertex(vC.getName()); TaskImpl tImpl = (TaskImpl) vImpl.getTask(0); - TezTaskAttemptID taId = TezTaskAttemptID.getInstance(tImpl.getTaskId(), 0); + TezTaskAttemptID taId = TezTaskAttemptID.getInstance(tImpl.getTaskID(), 0); Assert.assertEquals(0, tImpl.getTaskAttemptTezEvents(taId, 0, 1000).size()); // vD is mixed mode and only 1 out of 2 edges does legacy routing with task providing events vImpl = (VertexImpl) dagImpl.getVertex(vD.getName()); tImpl = (TaskImpl) vImpl.getTask(0); - taId = TezTaskAttemptID.getInstance(tImpl.getTaskId(), 0); + taId = TezTaskAttemptID.getInstance(tImpl.getTaskID(), 0); Assert.assertEquals(1, tImpl.getTaskAttemptTezEvents(taId, 0, 1000).size()); // vE has single legacy edge and does not use on demand routing and its task provides events vImpl = (VertexImpl) dagImpl.getVertex(vE.getName()); tImpl = (TaskImpl) vImpl.getTask(0); - taId = TezTaskAttemptID.getInstance(tImpl.getTaskId(), 0); + taId = TezTaskAttemptID.getInstance(tImpl.getTaskID(), 0); Assert.assertEquals(1, tImpl.getTaskAttemptTezEvents(taId, 0, 1000).size()); tezClient.stop(); diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/TestSpeculation.java b/tez-dag/src/test/java/org/apache/tez/dag/app/TestSpeculation.java index b6f50301c7..302281af9f 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/TestSpeculation.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/TestSpeculation.java @@ -326,7 +326,7 @@ public void testSingleTaskSpeculation() throws Exception { Task task = dagImpl.getTask(killedTaId.getTaskID()); Assert.assertEquals(entry.getValue().intValue(), task.getAttempts().size()); if (entry.getValue() > 1) { - Assert.assertEquals(successTaId, task.getSuccessfulAttempt().getID()); + Assert.assertEquals(successTaId, task.getSuccessfulAttempt().getTaskAttemptID()); TaskAttempt killedAttempt = task.getAttempt(killedTaId); Joiner.on(",").join(killedAttempt.getDiagnostics()).contains("Killed as speculative attempt"); Assert.assertEquals(TaskAttemptTerminationCause.TERMINATED_EFFECTIVE_SPECULATION, @@ -369,7 +369,7 @@ public void testBasicSpeculation(boolean withProgress) throws Exception { Task task = dagImpl.getTask(killedTaId.getTaskID()); Assert.assertEquals(ASSERT_SPECULATIONS_COUNT_MSG, 2, task.getAttempts().size()); - Assert.assertEquals(successTaId, task.getSuccessfulAttempt().getID()); + Assert.assertEquals(successTaId, task.getSuccessfulAttempt().getTaskAttemptID()); TaskAttempt killedAttempt = task.getAttempt(killedTaId); Joiner.on(",").join(killedAttempt.getDiagnostics()).contains("Killed as speculative attempt"); Assert.assertEquals(TaskAttemptTerminationCause.TERMINATED_EFFECTIVE_SPECULATION, @@ -380,7 +380,7 @@ public void testBasicSpeculation(boolean withProgress) throws Exception { .getValue()); Assert.assertEquals(1, dagImpl.getAllCounters().findCounter(TaskCounter.NUM_SPECULATIONS) .getValue()); - org.apache.tez.dag.app.dag.Vertex v = dagImpl.getVertex(killedTaId.getTaskID().getVertexID()); + org.apache.tez.dag.app.dag.Vertex v = dagImpl.getVertex(killedTaId.getVertexID()); Assert.assertEquals(1, v.getAllCounters().findCounter(TaskCounter.NUM_SPECULATIONS) .getValue()); } @@ -508,7 +508,7 @@ public void testBasicSpeculationNotUseful() throws Exception { Assert.assertEquals(DAGStatus.State.SUCCEEDED, dagClient.getDAGStatus(null).getState()); Task task = dagImpl.getTask(killedTaId.getTaskID()); Assert.assertEquals(2, task.getAttempts().size()); - Assert.assertEquals(successTaId, task.getSuccessfulAttempt().getID()); + Assert.assertEquals(successTaId, task.getSuccessfulAttempt().getTaskAttemptID()); TaskAttempt killedAttempt = task.getAttempt(killedTaId); Joiner.on(",").join(killedAttempt.getDiagnostics()).contains("Killed speculative attempt as"); Assert.assertEquals(TaskAttemptTerminationCause.TERMINATED_INEFFECTIVE_SPECULATION, @@ -517,7 +517,7 @@ public void testBasicSpeculationNotUseful() throws Exception { .getValue()); Assert.assertEquals(1, dagImpl.getAllCounters().findCounter(TaskCounter.NUM_SPECULATIONS) .getValue()); - org.apache.tez.dag.app.dag.Vertex v = dagImpl.getVertex(killedTaId.getTaskID().getVertexID()); + org.apache.tez.dag.app.dag.Vertex v = dagImpl.getVertex(killedTaId.getVertexID()); Assert.assertEquals(1, v.getAllCounters().findCounter(TaskCounter.NUM_SPECULATIONS) .getValue()); tezClient.stop(); 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 7611f1c44e..436c522ef7 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 @@ -167,7 +167,7 @@ private class VertexEventDispatcher implements EventHandler { @SuppressWarnings("unchecked") @Override public void handle(VertexEvent event) { - Vertex vertex = dag.getVertex(event.getVertexId()); + Vertex vertex = dag.getVertex(event.getVertexID()); ((EventHandler) vertex).handle(event); } } @@ -176,7 +176,7 @@ private class TaskEventDispatcher implements EventHandler { @SuppressWarnings("unchecked") @Override public void handle(TaskEvent event) { - Vertex vertex = dag.getVertex(event.getTaskID().getVertexID()); + Vertex vertex = dag.getVertex(event.getVertexID()); Task task = vertex.getTask(event.getTaskID()); ((EventHandler) task).handle(event); } @@ -563,7 +563,7 @@ public void testVertexCommit_OnDAGSuccess() throws Exception { initDAG(dag); startDAG(dag); VertexImpl v1 = (VertexImpl) dag.getVertex("vertex1"); - v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskId(), + v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskID(), TaskState.SUCCEEDED)); Assert.assertEquals(VertexState.SUCCEEDED, v1.getState()); Assert.assertNull(v1.getTerminationCause()); @@ -594,7 +594,7 @@ public void testVertexCommit_OnVertexSuccess() throws Exception { initDAG(dag); startDAG(dag); VertexImpl v1 = (VertexImpl) dag.getVertex("vertex1"); - v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskId(), + v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskID(), TaskState.SUCCEEDED)); Assert.assertEquals(VertexState.COMMITTING, v1.getState()); CountingOutputCommitter v1OutputCommitter_1 = (CountingOutputCommitter) v1 @@ -633,7 +633,7 @@ public void testVertexCommitFail1_OnVertexSuccess() throws Exception { initDAG(dag); startDAG(dag); VertexImpl v1 = (VertexImpl) dag.getVertex("vertex1"); - v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskId(), + v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskID(), TaskState.SUCCEEDED)); Assert.assertEquals(VertexState.COMMITTING, v1.getState()); CountingOutputCommitter v1OutputCommitter_1 = (CountingOutputCommitter) v1 @@ -669,7 +669,7 @@ public void testVertexCommitFail2_OnVertexSuccess() throws Exception { initDAG(dag); startDAG(dag); VertexImpl v1 = (VertexImpl) dag.getVertex("vertex1"); - v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskId(), + v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskID(), TaskState.SUCCEEDED)); Assert.assertEquals(VertexState.COMMITTING, v1.getState()); CountingOutputCommitter v1OutputCommitter_1 = (CountingOutputCommitter) v1 @@ -707,7 +707,7 @@ public void testVertexKilledWhileCommitting() throws Exception { initDAG(dag); startDAG(dag); VertexImpl v1 = (VertexImpl) dag.getVertex("vertex1"); - v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskId(), + v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskID(), TaskState.SUCCEEDED)); Assert.assertEquals(VertexState.COMMITTING, v1.getState()); // kill dag which will trigger the vertex killed event @@ -746,7 +746,7 @@ public void testVertexRescheduleWhileCommitting() throws Exception { initDAG(dag); startDAG(dag); VertexImpl v1 = (VertexImpl) dag.getVertex("vertex1"); - v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskId(), + v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskID(), TaskState.SUCCEEDED)); Assert.assertEquals(VertexState.COMMITTING, v1.getState()); // reschedule task @@ -786,7 +786,7 @@ public void testVertexRouteEventErrorWhileCommitting() throws Exception { initDAG(dag); startDAG(dag); VertexImpl v1 = (VertexImpl) dag.getVertex("vertex1"); - v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskId(), + v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskID(), TaskState.SUCCEEDED)); Assert.assertEquals(VertexState.COMMITTING, v1.getState()); // reschedule task @@ -831,7 +831,7 @@ public void testVertexInternalErrorWhileCommiting() throws Exception { initDAG(dag); startDAG(dag); VertexImpl v1 = (VertexImpl) dag.getVertex("vertex1"); - v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskId(), + v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskID(), TaskState.SUCCEEDED)); Assert.assertEquals(VertexState.COMMITTING, v1.getState()); // internal error @@ -875,11 +875,11 @@ public void testDAGCommitSucceeded_OnDAGSuccess() throws Exception { VertexImpl v2 = (VertexImpl) dag.getVertex("vertex2"); VertexImpl v3 = (VertexImpl) dag.getVertex("vertex3"); // need to make vertices to go to SUCCEEDED - v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskId(), + v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskId(), + v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskId(), + v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskID(), TaskState.SUCCEEDED)); waitUntil(dag, DAGState.COMMITTING); CountingOutputCommitter v12OutputCommitter = (CountingOutputCommitter) v1 @@ -931,11 +931,11 @@ public void testDAGCommitFail1_OnDAGSuccess() throws Exception { VertexImpl v2 = (VertexImpl) dag.getVertex("vertex2"); VertexImpl v3 = (VertexImpl) dag.getVertex("vertex3"); // need to make vertices to go to SUCCEEDED - v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskId(), + v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskId(), + v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskId(), + v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskID(), TaskState.SUCCEEDED)); waitUntil(dag, DAGState.COMMITTING); @@ -992,11 +992,11 @@ public void testDAGCommitFail2_OnDAGSuccess() throws Exception { VertexImpl v2 = (VertexImpl) dag.getVertex("vertex2"); VertexImpl v3 = (VertexImpl) dag.getVertex("vertex3"); // need to make vertices to go to SUCCEEDED - v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskId(), + v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskId(), + v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskId(), + v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskID(), TaskState.SUCCEEDED)); waitUntil(dag, DAGState.COMMITTING); @@ -1052,11 +1052,11 @@ public void testDAGCommitSucceeded1_OnVertexSuccess() throws Exception { VertexImpl v2 = (VertexImpl) dag.getVertex("vertex2"); VertexImpl v3 = (VertexImpl) dag.getVertex("vertex3"); - v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskId(), + v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskId(), + v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskId(), + v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskID(), TaskState.SUCCEEDED)); Assert.assertEquals(VertexState.SUCCEEDED, v1.getState()); Assert.assertEquals(VertexState.SUCCEEDED, v2.getState()); @@ -1110,11 +1110,11 @@ public void testDAGCommitSucceeded2_OnVertexSuccess() throws Exception { VertexImpl v2 = (VertexImpl) dag.getVertex("vertex2"); VertexImpl v3 = (VertexImpl) dag.getVertex("vertex3"); - v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskId(), + v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskId(), + v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskId(), + v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskID(), TaskState.SUCCEEDED)); Assert.assertEquals(VertexState.SUCCEEDED, v1.getState()); Assert.assertEquals(VertexState.SUCCEEDED, v2.getState()); @@ -1170,11 +1170,11 @@ public void testDAGCommitSucceeded3_OnVertexSuccess() throws Exception { VertexImpl v2 = (VertexImpl) dag.getVertex("vertex2"); VertexImpl v3 = (VertexImpl) dag.getVertex("vertex3"); - v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskId(), + v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskId(), + v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskId(), + v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskID(), TaskState.SUCCEEDED)); Assert.assertEquals(VertexState.SUCCEEDED, v1.getState()); Assert.assertEquals(VertexState.SUCCEEDED, v2.getState()); @@ -1235,11 +1235,11 @@ public void testDAGCommitFail1_OnVertexSuccess() throws Exception { VertexImpl v2 = (VertexImpl) dag.getVertex("vertex2"); VertexImpl v3 = (VertexImpl) dag.getVertex("vertex3"); - v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskId(), + v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskId(), + v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskId(), + v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskID(), TaskState.SUCCEEDED)); Assert.assertEquals(VertexState.SUCCEEDED, v1.getState()); Assert.assertEquals(VertexState.SUCCEEDED, v2.getState()); @@ -1294,11 +1294,11 @@ public void testDAGCommitFail2_OnVertexSuccess() throws Exception { VertexImpl v2 = (VertexImpl) dag.getVertex("vertex2"); VertexImpl v3 = (VertexImpl) dag.getVertex("vertex3"); - v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskId(), + v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskId(), + v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskId(), + v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskID(), TaskState.SUCCEEDED)); Assert.assertEquals(VertexState.SUCCEEDED, v1.getState()); Assert.assertEquals(VertexState.SUCCEEDED, v2.getState()); @@ -1353,11 +1353,11 @@ public void testDAGCommitFail3_OnVertexSuccess() throws Exception { VertexImpl v2 = (VertexImpl) dag.getVertex("vertex2"); VertexImpl v3 = (VertexImpl) dag.getVertex("vertex3"); - v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskId(), + v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskId(), + v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskId(), + v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskID(), TaskState.SUCCEEDED)); Assert.assertEquals(VertexState.SUCCEEDED, v1.getState()); Assert.assertEquals(VertexState.SUCCEEDED, v2.getState()); @@ -1416,11 +1416,11 @@ public void testDAGCommitFail4_OnVertexSuccess() throws Exception { VertexImpl v2 = (VertexImpl) dag.getVertex("vertex2"); VertexImpl v3 = (VertexImpl) dag.getVertex("vertex3"); - v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskId(), + v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskId(), + v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskId(), + v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskID(), TaskState.SUCCEEDED)); Assert.assertEquals(VertexState.SUCCEEDED, v1.getState()); Assert.assertEquals(VertexState.SUCCEEDED, v2.getState()); @@ -1475,11 +1475,11 @@ public void testDAGInternalErrorWhileCommiting_OnDAGSuccess() throws Exception { VertexImpl v2 = (VertexImpl) dag.getVertex("vertex2"); VertexImpl v3 = (VertexImpl) dag.getVertex("vertex3"); // need to make vertices to go to SUCCEEDED - v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskId(), + v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskId(), + v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskId(), + v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskID(), TaskState.SUCCEEDED)); waitUntil(dag, DAGState.COMMITTING); dag.handle(new DAGEvent(dag.getID(), DAGEventType.INTERNAL_ERROR)); @@ -1537,11 +1537,11 @@ private void _testDAGTerminatedWhileCommitting1_OnDAGSuccess(DAGTerminationCause VertexImpl v2 = (VertexImpl) dag.getVertex("vertex2"); VertexImpl v3 = (VertexImpl) dag.getVertex("vertex3"); // need to make vertices to go to SUCCEEDED - v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskId(), + v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskId(), + v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskId(), + v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskID(), TaskState.SUCCEEDED)); waitUntil(dag, DAGState.COMMITTING); dag.handle(new DAGEventTerminateDag(dag.getID(), terminationCause, null)); @@ -1602,11 +1602,11 @@ private void _testDAGTerminatedWhileCommitting1_OnVertexSuccess(DAGTerminationCa VertexImpl v2 = (VertexImpl) dag.getVertex("vertex2"); VertexImpl v3 = (VertexImpl) dag.getVertex("vertex3"); // need to make vertices to go to SUCCEEDED - v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskId(), + v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskId(), + v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskId(), + v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskID(), TaskState.SUCCEEDED)); Assert.assertEquals(VertexState.COMMITTING, v3.getState()); // dag is still in RUNNING because v3 has not completed @@ -1672,11 +1672,11 @@ private void _testDAGKilledWhileRunning_OnVertexSuccess(DAGTerminationCause term VertexImpl v2 = (VertexImpl) dag.getVertex("vertex2"); VertexImpl v3 = (VertexImpl) dag.getVertex("vertex3"); // need to make vertices to go to SUCCEEDED - v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskId(), + v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskId(), + v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskId(), + v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskID(), TaskState.SUCCEEDED)); Assert.assertEquals(VertexState.COMMITTING, v3.getState()); // dag is still in RUNNING because v3 has not completed @@ -1731,11 +1731,11 @@ public void testDAGCommitVertexRerunWhileCommitting_OnDAGSuccess() throws Except VertexImpl v2 = (VertexImpl) dag.getVertex("vertex2"); VertexImpl v3 = (VertexImpl) dag.getVertex("vertex3"); // need to make vertices to go to SUCCEEDED - v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskId(), + v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskId(), + v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskId(), + v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskID(), TaskState.SUCCEEDED)); waitUntil(dag, DAGState.COMMITTING); TezTaskID newTaskId = TezTaskID.getInstance(v1.getVertexId(), 1); @@ -1790,11 +1790,11 @@ public void testDAGCommitInternalErrorWhileCommiting_OnDAGSuccess() throws Excep VertexImpl v2 = (VertexImpl) dag.getVertex("vertex2"); VertexImpl v3 = (VertexImpl) dag.getVertex("vertex3"); // need to make vertices to go to SUCCEEDED - v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskId(), + v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskId(), + v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskId(), + v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskID(), TaskState.SUCCEEDED)); waitUntil(dag, DAGState.COMMITTING); dag.handle(new DAGEvent(dag.getID(), DAGEventType.INTERNAL_ERROR)); @@ -1840,11 +1840,11 @@ public void testVertexGroupCommitFinishedEventFail_OnVertexSuccess() throws Exce VertexImpl v2 = (VertexImpl) dag.getVertex("vertex2"); VertexImpl v3 = (VertexImpl) dag.getVertex("vertex3"); // need to make vertices to go to SUCCEEDED - v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskId(), + v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskId(), + v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskId(), + v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskID(), TaskState.SUCCEEDED)); CountingOutputCommitter v12OutputCommitter = (CountingOutputCommitter) v1 .getOutputCommitter("v12Out"); @@ -1895,11 +1895,11 @@ public void testDAGCommitStartedEventFail_OnDAGSuccess() throws Exception { VertexImpl v2 = (VertexImpl) dag.getVertex("vertex2"); VertexImpl v3 = (VertexImpl) dag.getVertex("vertex3"); // need to make vertices to go to SUCCEEDED - v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskId(), + v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskId(), + v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskId(), + v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskID(), TaskState.SUCCEEDED)); waitUntil(dag, DAGState.FAILED); Assert.assertEquals(DAGTerminationCause.RECOVERY_FAILURE, dag.getTerminationCause()); @@ -1959,11 +1959,11 @@ private void _testCommitCanceled_OnDAGSuccess(DAGTerminationCause terminationCau VertexImpl v2 = (VertexImpl) dag.getVertex("vertex2"); VertexImpl v3 = (VertexImpl) dag.getVertex("vertex3"); // need to make vertices to go to SUCCEEDED - v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskId(), + v1.handle(new VertexEventTaskCompleted(v1.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskId(), + v2.handle(new VertexEventTaskCompleted(v2.getTask(0).getTaskID(), TaskState.SUCCEEDED)); - v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskId(), + v3.handle(new VertexEventTaskCompleted(v3.getTask(0).getTaskID(), TaskState.SUCCEEDED)); waitUntil(dag, DAGState.COMMITTING); // mean the commits have been submitted to ThreadPool @@ -2110,7 +2110,7 @@ public void verifyDAGFinishedEvent(TezDAGID dagId, int expectedTimes) { for (HistoryEvent event : historyEvents) { if (event.getEventType() == HistoryEventType.DAG_FINISHED) { DAGFinishedEvent startedEvent = (DAGFinishedEvent)event; - if (startedEvent.getDagID().equals(dagId)) { + if (startedEvent.getDAGID().equals(dagId)) { actualTimes ++; } } 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 2f2b3b8b8f..f2b16a9094 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 @@ -221,7 +221,7 @@ private DAGImpl chooseDAG(TezDAGID curDAGId) { private class DagEventDispatcher implements EventHandler { @Override public void handle(DAGEvent event) { - DAGImpl dag = chooseDAG(event.getDAGId()); + DAGImpl dag = chooseDAG(event.getDAGID()); dag.handle(event); } } @@ -230,9 +230,9 @@ private class TaskEventDispatcher implements EventHandler { @SuppressWarnings("unchecked") @Override public void handle(TaskEvent event) { - TezDAGID id = event.getTaskID().getVertexID().getDAGId(); + TezDAGID id = event.getDAGID(); DAGImpl handler = chooseDAG(id); - Vertex vertex = handler.getVertex(event.getTaskID().getVertexID()); + Vertex vertex = handler.getVertex(event.getVertexID()); Task task = vertex.getTask(event.getTaskID()); ((EventHandler)task).handle(event); } @@ -249,10 +249,10 @@ public void handle(TaskAttemptEvent event) { private class TaskAttemptEventDisptacher2 implements EventHandler { @Override public void handle(TaskAttemptEvent event) { - TezDAGID id = event.getTaskAttemptID().getTaskID().getVertexID().getDAGId(); + TezDAGID id = event.getDAGID(); DAGImpl handler = chooseDAG(id); - Vertex vertex = handler.getVertex(event.getTaskAttemptID().getTaskID().getVertexID()); - Task task = vertex.getTask(event.getTaskAttemptID().getTaskID()); + Vertex vertex = handler.getVertex(event.getVertexID()); + Task task = vertex.getTask(event.getTaskID()); TaskAttempt ta = task.getAttempt(event.getTaskAttemptID()); ((EventHandler)ta).handle(event); } @@ -264,9 +264,9 @@ private class VertexEventDispatcher @SuppressWarnings("unchecked") @Override public void handle(VertexEvent event) { - TezDAGID id = event.getVertexId().getDAGId(); + TezDAGID id = event.getDAGID(); DAGImpl handler = chooseDAG(id); - Vertex vertex = handler.getVertex(event.getVertexId()); + Vertex vertex = handler.getVertex(event.getVertexID()); ((EventHandler) vertex).handle(event); } } @@ -1174,14 +1174,14 @@ public void testEdgeManager_RouteDataMovementEventToDestination() { VertexImpl v2 = (VertexImpl)dagWithCustomEdge.getVertex("vertex2"); dispatcher.await(); Task t1= v2.getTask(0); - TaskAttemptImpl ta1= (TaskAttemptImpl)t1.getAttempt(TezTaskAttemptID.getInstance(t1.getTaskId(), 0)); + TaskAttemptImpl ta1= (TaskAttemptImpl)t1.getAttempt(TezTaskAttemptID.getInstance(t1.getTaskID(), 0)); DataMovementEvent daEvent = DataMovementEvent.create(ByteBuffer.wrap(new byte[0])); TezEvent tezEvent = new TezEvent(daEvent, - new EventMetaData(EventProducerConsumerType.INPUT, "vertex1", "vertex2", ta1.getID())); + new EventMetaData(EventProducerConsumerType.INPUT, "vertex1", "vertex2", ta1.getTaskAttemptID())); dispatcher.getEventHandler().handle(new VertexEventRouteEvent(v2.getVertexId(), Lists.newArrayList(tezEvent))); dispatcher.await(); - v2.getTaskAttemptTezEvents(ta1.getID(), 0, 0, 1000); + v2.getTaskAttemptTezEvents(ta1.getTaskAttemptID(), 0, 0, 1000); dispatcher.await(); Assert.assertEquals(VertexState.FAILED, v2.getState()); @@ -1207,11 +1207,11 @@ public void testEdgeManager_RouteDataMovementEventToDestinationWithLegacyRouting dispatcher.await(); Task t1= v2.getTask(0); - TaskAttemptImpl ta1= (TaskAttemptImpl)t1.getAttempt(TezTaskAttemptID.getInstance(t1.getTaskId(), 0)); + TaskAttemptImpl ta1= (TaskAttemptImpl)t1.getAttempt(TezTaskAttemptID.getInstance(t1.getTaskID(), 0)); DataMovementEvent daEvent = DataMovementEvent.create(ByteBuffer.wrap(new byte[0])); TezEvent tezEvent = new TezEvent(daEvent, - new EventMetaData(EventProducerConsumerType.INPUT, "vertex1", "vertex2", ta1.getID())); + new EventMetaData(EventProducerConsumerType.INPUT, "vertex1", "vertex2", ta1.getTaskAttemptID())); dispatcher.getEventHandler().handle( new VertexEventRouteEvent(v2.getVertexId(), Lists.newArrayList(tezEvent))); dispatcher.await(); @@ -1239,13 +1239,13 @@ public void testEdgeManager_RouteInputSourceTaskFailedEventToDestinationLegacyRo dispatcher.await(); Task t1= v2.getTask(0); - TaskAttemptImpl ta1= (TaskAttemptImpl)t1.getAttempt(TezTaskAttemptID.getInstance(t1.getTaskId(), 0)); + TaskAttemptImpl ta1= (TaskAttemptImpl)t1.getAttempt(TezTaskAttemptID.getInstance(t1.getTaskID(), 0)); InputFailedEvent ifEvent = InputFailedEvent.create(0, 1); TezEvent tezEvent = new TezEvent(ifEvent, - new EventMetaData(EventProducerConsumerType.INPUT,"vertex1", "vertex2", ta1.getID())); + new EventMetaData(EventProducerConsumerType.INPUT,"vertex1", "vertex2", ta1.getTaskAttemptID())); dispatcher.getEventHandler().handle(new VertexEventRouteEvent(v2.getVertexId(), Lists.newArrayList(tezEvent))); dispatcher.await(); - v2.getTaskAttemptTezEvents(ta1.getID(), 0, 0, 1000); + v2.getTaskAttemptTezEvents(ta1.getTaskAttemptID(), 0, 0, 1000); dispatcher.await(); Assert.assertEquals(VertexState.FAILED, v2.getState()); @@ -1270,11 +1270,11 @@ public void testEdgeManager_GetNumDestinationConsumerTasks() { dispatcher.await(); Task t1= v2.getTask(0); - TaskAttemptImpl ta1= (TaskAttemptImpl)t1.getAttempt(TezTaskAttemptID.getInstance(t1.getTaskId(), 0)); + TaskAttemptImpl ta1= (TaskAttemptImpl)t1.getAttempt(TezTaskAttemptID.getInstance(t1.getTaskID(), 0)); InputReadErrorEvent ireEvent = InputReadErrorEvent.create("", 0, 0); TezEvent tezEvent = new TezEvent(ireEvent, - new EventMetaData(EventProducerConsumerType.INPUT,"vertex2", "vertex1", ta1.getID())); + new EventMetaData(EventProducerConsumerType.INPUT,"vertex2", "vertex1", ta1.getTaskAttemptID())); dispatcher.getEventHandler().handle( new VertexEventRouteEvent(v2.getVertexId(), Lists.newArrayList(tezEvent))); dispatcher.await(); @@ -1301,10 +1301,10 @@ public void testEdgeManager_RouteInputErrorEventToSource() { dispatcher.await(); Task t1= v2.getTask(0); - TaskAttemptImpl ta1= (TaskAttemptImpl)t1.getAttempt(TezTaskAttemptID.getInstance(t1.getTaskId(), 0)); + TaskAttemptImpl ta1= (TaskAttemptImpl)t1.getAttempt(TezTaskAttemptID.getInstance(t1.getTaskID(), 0)); InputReadErrorEvent ireEvent = InputReadErrorEvent.create("", 0, 0); TezEvent tezEvent = new TezEvent(ireEvent, - new EventMetaData(EventProducerConsumerType.INPUT,"vertex2", "vertex1", ta1.getID())); + new EventMetaData(EventProducerConsumerType.INPUT,"vertex2", "vertex1", ta1.getTaskAttemptID())); dispatcher.getEventHandler().handle(new VertexEventRouteEvent(v2.getVertexId(), Lists.newArrayList(tezEvent))); dispatcher.await(); // 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 95ea8a030e..acbef67dc0 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 @@ -209,7 +209,7 @@ private class TaskEventDispatcher implements EventHandler { @SuppressWarnings("unchecked") @Override public void handle(TaskEvent event) { - TaskImpl task = (TaskImpl) dag.getVertex(event.getTaskID().getVertexID()) + TaskImpl task = (TaskImpl) dag.getVertex(event.getVertexID()) .getTask(event.getTaskID()); task.handle(event); } @@ -220,8 +220,7 @@ private class TaskAttemptEventDispatcher implements EventHandler { @Override public void handle(TaskAttemptEvent event) { - Vertex vertex = dag.getVertex(event.getTaskAttemptID().getTaskID() - .getVertexID()); + Vertex vertex = dag.getVertex(event.getVertexID()); Task task = vertex.getTask(event.getTaskAttemptID().getTaskID()); TaskAttempt ta = task.getAttempt(event.getTaskAttemptID()); ((EventHandler) ta).handle(event); @@ -233,7 +232,7 @@ private class VertexEventDispatcher implements EventHandler { @SuppressWarnings("unchecked") @Override public void handle(VertexEvent event) { - VertexImpl vertex = (VertexImpl) dag.getVertex(event.getVertexId()); + VertexImpl vertex = (VertexImpl) dag.getVertex(event.getVertexID()); vertex.handle(event); } } diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGScheduler.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGScheduler.java index 07c361aded..e2df050799 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGScheduler.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGScheduler.java @@ -39,7 +39,7 @@ public class TestDAGScheduler { - class MockEventHandler implements EventHandler { + static class MockEventHandler implements EventHandler { TaskAttemptEventSchedule event; List events = Lists.newLinkedList(); @Override @@ -114,17 +114,17 @@ public void testConcurrencyLimit() { // schedule beyond limit and it gets scheduled mockAttempt = mock(TaskAttempt.class); - when(mockAttempt.getID()).thenReturn(TezTaskAttemptID.getInstance(tId0, 0)); + when(mockAttempt.getTaskAttemptID()).thenReturn(TezTaskAttemptID.getInstance(tId0, 0)); scheduler.scheduleTask(new DAGEventSchedulerUpdate( DAGEventSchedulerUpdate.UpdateType.TA_SCHEDULE, mockAttempt)); Assert.assertEquals(1, mockEventHandler.events.size()); mockAttempt = mock(TaskAttempt.class); - when(mockAttempt.getID()).thenReturn(TezTaskAttemptID.getInstance(tId0, 1)); + when(mockAttempt.getTaskAttemptID()).thenReturn(TezTaskAttemptID.getInstance(tId0, 1)); scheduler.scheduleTask(new DAGEventSchedulerUpdate( DAGEventSchedulerUpdate.UpdateType.TA_SCHEDULE, mockAttempt)); Assert.assertEquals(2, mockEventHandler.events.size()); mockAttempt = mock(TaskAttempt.class); - when(mockAttempt.getID()).thenReturn(TezTaskAttemptID.getInstance(tId0, 2)); + when(mockAttempt.getTaskAttemptID()).thenReturn(TezTaskAttemptID.getInstance(tId0, 2)); scheduler.scheduleTask(new DAGEventSchedulerUpdate( DAGEventSchedulerUpdate.UpdateType.TA_SCHEDULE, mockAttempt)); Assert.assertEquals(3, mockEventHandler.events.size()); @@ -139,34 +139,34 @@ public void testConcurrencyLimit() { // schedule beyond limit and it gets buffered mockAttempt = mock(TaskAttempt.class); mockAttempts.add(mockAttempt); - when(mockAttempt.getID()).thenReturn(TezTaskAttemptID.getInstance(tId1, requested++)); + when(mockAttempt.getTaskAttemptID()).thenReturn(TezTaskAttemptID.getInstance(tId1, requested++)); scheduler.scheduleTask(new DAGEventSchedulerUpdate( DAGEventSchedulerUpdate.UpdateType.TA_SCHEDULE, mockAttempt)); Assert.assertEquals(scheduled+1, mockEventHandler.events.size()); // scheduled - Assert.assertEquals(mockAttempts.get(scheduled).getID(), + Assert.assertEquals(mockAttempts.get(scheduled).getTaskAttemptID(), mockEventHandler.events.get(scheduled).getTaskAttemptID()); // matches order scheduled++; mockAttempt = mock(TaskAttempt.class); mockAttempts.add(mockAttempt); - when(mockAttempt.getID()).thenReturn(TezTaskAttemptID.getInstance(tId1, requested++)); + when(mockAttempt.getTaskAttemptID()).thenReturn(TezTaskAttemptID.getInstance(tId1, requested++)); scheduler.scheduleTask(new DAGEventSchedulerUpdate( DAGEventSchedulerUpdate.UpdateType.TA_SCHEDULE, mockAttempt)); Assert.assertEquals(scheduled+1, mockEventHandler.events.size()); // scheduled - Assert.assertEquals(mockAttempts.get(scheduled).getID(), + Assert.assertEquals(mockAttempts.get(scheduled).getTaskAttemptID(), mockEventHandler.events.get(scheduled).getTaskAttemptID()); // matches order scheduled++; mockAttempt = mock(TaskAttempt.class); mockAttempts.add(mockAttempt); - when(mockAttempt.getID()).thenReturn(TezTaskAttemptID.getInstance(tId1, requested++)); + when(mockAttempt.getTaskAttemptID()).thenReturn(TezTaskAttemptID.getInstance(tId1, requested++)); scheduler.scheduleTask(new DAGEventSchedulerUpdate( DAGEventSchedulerUpdate.UpdateType.TA_SCHEDULE, mockAttempt)); Assert.assertEquals(scheduled, mockEventHandler.events.size()); // buffered mockAttempt = mock(TaskAttempt.class); mockAttempts.add(mockAttempt); - when(mockAttempt.getID()).thenReturn(TezTaskAttemptID.getInstance(tId1, requested++)); + when(mockAttempt.getTaskAttemptID()).thenReturn(TezTaskAttemptID.getInstance(tId1, requested++)); scheduler.scheduleTask(new DAGEventSchedulerUpdate( DAGEventSchedulerUpdate.UpdateType.TA_SCHEDULE, mockAttempt)); Assert.assertEquals(scheduled, mockEventHandler.events.size()); // buffered @@ -174,14 +174,14 @@ public void testConcurrencyLimit() { scheduler.taskCompleted(new DAGEventSchedulerUpdate( DAGEventSchedulerUpdate.UpdateType.TA_COMPLETED, mockAttempts.get(completed++))); Assert.assertEquals(scheduled+1, mockEventHandler.events.size()); // scheduled - Assert.assertEquals(mockAttempts.get(scheduled).getID(), + Assert.assertEquals(mockAttempts.get(scheduled).getTaskAttemptID(), mockEventHandler.events.get(scheduled).getTaskAttemptID()); // matches order scheduled++; scheduler.taskCompleted(new DAGEventSchedulerUpdate( DAGEventSchedulerUpdate.UpdateType.TA_COMPLETED, mockAttempts.get(completed++))); Assert.assertEquals(scheduled+1, mockEventHandler.events.size()); // scheduled - Assert.assertEquals(mockAttempts.get(scheduled).getID(), + Assert.assertEquals(mockAttempts.get(scheduled).getTaskAttemptID(), mockEventHandler.events.get(scheduled).getTaskAttemptID()); // matches order scheduled++; @@ -191,14 +191,12 @@ public void testConcurrencyLimit() { mockAttempt = mock(TaskAttempt.class); mockAttempts.add(mockAttempt); - when(mockAttempt.getID()).thenReturn(TezTaskAttemptID.getInstance(tId1, requested++)); + when(mockAttempt.getTaskAttemptID()).thenReturn(TezTaskAttemptID.getInstance(tId1, requested++)); scheduler.scheduleTask(new DAGEventSchedulerUpdate( DAGEventSchedulerUpdate.UpdateType.TA_SCHEDULE, mockAttempt)); Assert.assertEquals(scheduled+1, mockEventHandler.events.size()); // scheduled - Assert.assertEquals(mockAttempts.get(scheduled).getID(), + Assert.assertEquals(mockAttempts.get(scheduled).getTaskAttemptID(), mockEventHandler.events.get(scheduled).getTaskAttemptID()); // matches order - scheduled++; - } @Test(timeout=5000) @@ -220,7 +218,6 @@ public void testConcurrencyLimitWithKilledNonRunningTask() { mockEventHandler); List mockAttempts = Lists.newArrayList(); - int completed = 0; int requested = 0; int scheduled = 0; scheduler.addVertexConcurrencyLimit(vId0, 1); // effective @@ -228,24 +225,24 @@ public void testConcurrencyLimitWithKilledNonRunningTask() { // schedule beyond limit and it gets buffered mockAttempt = mock(TaskAttempt.class); mockAttempts.add(mockAttempt); - when(mockAttempt.getID()).thenReturn(TezTaskAttemptID.getInstance(tId0, requested++)); + when(mockAttempt.getTaskAttemptID()).thenReturn(TezTaskAttemptID.getInstance(tId0, requested++)); scheduler.scheduleTask(new DAGEventSchedulerUpdate( DAGEventSchedulerUpdate.UpdateType.TA_SCHEDULE, mockAttempt)); Assert.assertEquals(scheduled+1, mockEventHandler.events.size()); // scheduled - Assert.assertEquals(mockAttempts.get(scheduled).getID(), + Assert.assertEquals(mockAttempts.get(scheduled).getTaskAttemptID(), mockEventHandler.events.get(scheduled).getTaskAttemptID()); // matches order scheduled++; mockAttempt = mock(TaskAttempt.class); mockAttempts.add(mockAttempt); - when(mockAttempt.getID()).thenReturn(TezTaskAttemptID.getInstance(tId0, requested++)); + when(mockAttempt.getTaskAttemptID()).thenReturn(TezTaskAttemptID.getInstance(tId0, requested++)); scheduler.scheduleTask(new DAGEventSchedulerUpdate( DAGEventSchedulerUpdate.UpdateType.TA_SCHEDULE, mockAttempt)); Assert.assertEquals(scheduled, mockEventHandler.events.size()); // buffered mockAttempt = mock(TaskAttempt.class); mockAttempts.add(mockAttempt); - when(mockAttempt.getID()).thenReturn(TezTaskAttemptID.getInstance(tId0, requested++)); + when(mockAttempt.getTaskAttemptID()).thenReturn(TezTaskAttemptID.getInstance(tId0, requested++)); scheduler.scheduleTask(new DAGEventSchedulerUpdate( DAGEventSchedulerUpdate.UpdateType.TA_SCHEDULE, mockAttempt)); Assert.assertEquals(scheduled, mockEventHandler.events.size()); // buffered @@ -253,7 +250,7 @@ public void testConcurrencyLimitWithKilledNonRunningTask() { scheduler.taskCompleted(new DAGEventSchedulerUpdate( DAGEventSchedulerUpdate.UpdateType.TA_COMPLETED, mockAttempts.get(1))); Assert.assertEquals(scheduled, mockEventHandler.events.size()); // buffered - Assert.assertEquals(mockAttempts.get(0).getID(), + Assert.assertEquals(mockAttempts.get(0).getTaskAttemptID(), mockEventHandler.events.get(0).getTaskAttemptID()); // matches order } diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGSchedulerNaturalOrderControlled.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGSchedulerNaturalOrderControlled.java index 63137c716a..3b1db4c6d1 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGSchedulerNaturalOrderControlled.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGSchedulerNaturalOrderControlled.java @@ -164,9 +164,9 @@ public void testSourceRequestDelayed() { for (Event raw : args.getAllValues()) { TaskAttemptEventSchedule event = (TaskAttemptEventSchedule) raw; if (count < vertices[2].getTotalTasks() - 3) { - assertEquals(2, event.getTaskAttemptID().getTaskID().getVertexID().getId()); + assertEquals(2, event.getVertexID().getId()); } else { - assertEquals(4, event.getTaskAttemptID().getTaskID().getVertexID().getId()); + assertEquals(4, event.getVertexID().getId()); } count++; } @@ -364,7 +364,7 @@ private TaskAttempt createTaskAttempt(TezVertexID vertexId, int taskIdInt, int a TaskAttempt taskAttempt = mock(TaskAttempt.class); TezTaskID taskId = TezTaskID.getInstance(vertexId, taskIdInt); TezTaskAttemptID taskAttemptId = TezTaskAttemptID.getInstance(taskId, attemptIdInt); - doReturn(taskAttemptId).when(taskAttempt).getID(); + doReturn(taskAttemptId).when(taskAttempt).getTaskAttemptID(); doReturn(vertexId).when(taskAttempt).getVertexID(); return taskAttempt; } 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 1143395204..c9c99f5630 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 @@ -232,9 +232,9 @@ private void verifyEvents(TezTaskAttemptID srcTAID, LinkedHashMap tasks) { for (Task task : tasks) { - TezTaskID taskID = task.getTaskId(); + TezTaskID taskID = task.getTaskID(); reset(task); - doReturn(taskID).when(task).getTaskId(); + doReturn(taskID).when(task).getTaskID(); } } @@ -243,7 +243,7 @@ private LinkedHashMap mockTasks(TezVertexID vertexID, int numTa for (int i = 0 ; i < numTasks ; i++) { Task task = mock(Task.class); TezTaskID taskID = TezTaskID.getInstance(vertexID, i); - doReturn(taskID).when(task).getTaskId(); + doReturn(taskID).when(task).getTaskID(); tasks.put(taskID, task); } return tasks; 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 7a2a05fb26..dc59b4615c 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 @@ -425,7 +425,7 @@ taListener, taskConf, new SystemClock(), taImpl.handle(new TaskAttemptEventKillRequest(taskAttemptID, null, TaskAttemptTerminationCause.TERMINATED_BY_CLIENT)); assertEquals(TaskAttemptStateInternal.KILL_IN_PROGRESS, taImpl.getInternalState()); - taImpl.handle(new TaskAttemptEventTezEventUpdate(taImpl.getID(), Collections.EMPTY_LIST)); + taImpl.handle(new TaskAttemptEventTezEventUpdate(taImpl.getTaskAttemptID(), Collections.EMPTY_LIST)); assertFalse( "InternalError occurred trying to handle TA_TEZ_EVENT_UPDATE in KILL_IN_PROGRESS state", eventHandler.internalError); @@ -480,7 +480,7 @@ public void testContainerTerminationWhileRunning() throws Exception { taListener, taskConf, new SystemClock(), mockHeartbeatHandler, appCtx, false, resource, createFakeContainerContext(), false); - TezTaskAttemptID taskAttemptID = taImpl.getID(); + TezTaskAttemptID taskAttemptID = taImpl.getTaskAttemptID(); ArgumentCaptor arg = ArgumentCaptor.forClass(Event.class); taImpl.handle(new TaskAttemptEventSchedule(taskAttemptID, 0, 0)); @@ -585,7 +585,7 @@ public void testContainerTerminatedWhileRunning() throws Exception { taListener, taskConf, new SystemClock(), mockHeartbeatHandler, appCtx, false, resource, createFakeContainerContext(), false); - TezTaskAttemptID taskAttemptID = taImpl.getID(); + TezTaskAttemptID taskAttemptID = taImpl.getTaskAttemptID(); ArgumentCaptor arg = ArgumentCaptor.forClass(Event.class); taImpl.handle(new TaskAttemptEventSchedule(taskAttemptID, 0, 0)); taImpl.handle(new TaskAttemptEventSubmitted(taskAttemptID, contId)); @@ -674,7 +674,7 @@ public void testContainerTerminatedAfterSuccess() throws Exception { taListener, taskConf, new SystemClock(), mockHeartbeatHandler, appCtx, false, resource, createFakeContainerContext(), false); - TezTaskAttemptID taskAttemptID = taImpl.getID(); + TezTaskAttemptID taskAttemptID = taImpl.getTaskAttemptID(); ArgumentCaptor arg = ArgumentCaptor.forClass(Event.class); taImpl.handle(new TaskAttemptEventSchedule(taskAttemptID, 0, 0)); @@ -764,7 +764,7 @@ public void testLastDataEventRecording() throws Exception { taListener, taskConf, new SystemClock(), mockHeartbeatHandler, appCtx, false, resource, createFakeContainerContext(), false); - TezTaskAttemptID taskAttemptID = taImpl.getID(); + TezTaskAttemptID taskAttemptID = taImpl.getTaskAttemptID(); taImpl.handle(new TaskAttemptEventSchedule(taskAttemptID, 0, 0)); taImpl.handle(new TaskAttemptEventSubmitted(taskAttemptID, contId)); @@ -849,7 +849,7 @@ public void testFailure() throws Exception { taListener, taskConf, new SystemClock(), mockHeartbeatHandler, appCtx, false, resource, createFakeContainerContext(), false); - TezTaskAttemptID taskAttemptID = taImpl.getID(); + TezTaskAttemptID taskAttemptID = taImpl.getTaskAttemptID(); ArgumentCaptor arg = ArgumentCaptor.forClass(Event.class); taImpl.handle(new TaskAttemptEventSchedule(taskAttemptID, 0, 0)); @@ -879,7 +879,7 @@ taListener, taskConf, new SystemClock(), assertEquals(TaskAttemptTerminationCause.APPLICATION_ERROR, taImpl.getTerminationCause()); assertEquals(TaskAttemptStateInternal.FAIL_IN_PROGRESS, taImpl.getInternalState()); - taImpl.handle(new TaskAttemptEventTezEventUpdate(taImpl.getID(), Collections.EMPTY_LIST)); + taImpl.handle(new TaskAttemptEventTezEventUpdate(taImpl.getTaskAttemptID(), Collections.EMPTY_LIST)); assertFalse( "InternalError occurred trying to handle TA_TEZ_EVENT_UPDATE in FAIL_IN_PROGRESS state", eventHandler.internalError); @@ -956,7 +956,7 @@ public void testFailureFatalError() throws Exception { taListener, taskConf, new SystemClock(), mockHeartbeatHandler, appCtx, false, resource, createFakeContainerContext(), false); - TezTaskAttemptID taskAttemptID = taImpl.getID(); + TezTaskAttemptID taskAttemptID = taImpl.getTaskAttemptID(); ArgumentCaptor arg = ArgumentCaptor.forClass(Event.class); taImpl.handle(new TaskAttemptEventSchedule(taskAttemptID, 0, 0)); @@ -986,7 +986,7 @@ taListener, taskConf, new SystemClock(), assertEquals(TaskAttemptTerminationCause.APPLICATION_ERROR, taImpl.getTerminationCause()); assertEquals(TaskAttemptStateInternal.FAIL_IN_PROGRESS, taImpl.getInternalState()); - taImpl.handle(new TaskAttemptEventTezEventUpdate(taImpl.getID(), Collections.EMPTY_LIST)); + taImpl.handle(new TaskAttemptEventTezEventUpdate(taImpl.getTaskAttemptID(), Collections.EMPTY_LIST)); assertFalse( "InternalError occurred trying to handle TA_TEZ_EVENT_UPDATE in FAIL_IN_PROGRESS state", eventHandler.internalError); @@ -1064,7 +1064,7 @@ public void testProgressTimeStampUpdate() throws Exception { taListener, taskConf, mockClock, mockHeartbeatHandler, appCtx, false, resource, createFakeContainerContext(), false); - TezTaskAttemptID taskAttemptID = taImpl.getID(); + TezTaskAttemptID taskAttemptID = taImpl.getTaskAttemptID(); ArgumentCaptor arg = ArgumentCaptor.forClass(Event.class); taImpl.handle(new TaskAttemptEventSchedule(taskAttemptID, 0, 0)); @@ -1080,7 +1080,7 @@ public void testProgressTimeStampUpdate() throws Exception { verify(eventHandler, atLeast(1)).handle(arg.capture()); if (arg.getValue() instanceof TaskAttemptEventAttemptFailed) { TaskAttemptEventAttemptFailed fEvent = (TaskAttemptEventAttemptFailed) arg.getValue(); - assertEquals(taImpl.getID(), fEvent.getTaskAttemptID()); + assertEquals(taImpl.getTaskAttemptID(), fEvent.getTaskAttemptID()); assertEquals(TaskAttemptTerminationCause.NO_PROGRESS, fEvent.getTerminationCause()); taImpl.handle(fEvent); fail("Should not fail since the timestamps do not differ by progress interval config"); @@ -1135,7 +1135,7 @@ public void testStatusUpdateWithNullCounters() throws Exception { taListener, taskConf, new SystemClock(), mockHeartbeatHandler, appCtx, false, resource, createFakeContainerContext(), false); - TezTaskAttemptID taskAttemptID = taImpl.getID(); + TezTaskAttemptID taskAttemptID = taImpl.getTaskAttemptID(); taImpl.handle(new TaskAttemptEventSchedule(taskAttemptID, 0, 0)); taImpl.handle(new TaskAttemptEventSubmitted(taskAttemptID, contId)); @@ -1203,7 +1203,7 @@ public void testProgressAfterSubmit() throws Exception { taListener, taskConf, mockClock, mockHeartbeatHandler, appCtx, false, resource, createFakeContainerContext(), false); - TezTaskAttemptID taskAttemptID = taImpl.getID(); + TezTaskAttemptID taskAttemptID = taImpl.getTaskAttemptID(); ArgumentCaptor arg = ArgumentCaptor.forClass(Event.class); mockClock.incrementTime(20L); taImpl.handle(new TaskAttemptEventSchedule(taskAttemptID, 0, 0)); @@ -1262,7 +1262,7 @@ public void testNoProgressFail() throws Exception { taListener, taskConf, mockClock, mockHeartbeatHandler, appCtx, false, resource, createFakeContainerContext(), false); - TezTaskAttemptID taskAttemptID = taImpl.getID(); + TezTaskAttemptID taskAttemptID = taImpl.getTaskAttemptID(); ArgumentCaptor arg = ArgumentCaptor.forClass(Event.class); taImpl.handle(new TaskAttemptEventSchedule(taskAttemptID, 0, 0)); @@ -1295,7 +1295,7 @@ public void testNoProgressFail() throws Exception { // failed event sent to self verify(eventHandler, atLeast(1)).handle(arg.capture()); TaskAttemptEventAttemptFailed fEvent = (TaskAttemptEventAttemptFailed) arg.getValue(); - assertEquals(taImpl.getID(), fEvent.getTaskAttemptID()); + assertEquals(taImpl.getTaskAttemptID(), fEvent.getTaskAttemptID()); assertEquals(TaskAttemptTerminationCause.NO_PROGRESS, fEvent.getTerminationCause()); assertEquals(TaskFailureType.NON_FATAL, fEvent.getTaskFailureType()); taImpl.handle(fEvent); @@ -1377,7 +1377,7 @@ public void testCompletedAtSubmitted() throws ServicePluginException { taListener, taskConf, new SystemClock(), mockHeartbeatHandler, appCtx, false, resource, createFakeContainerContext(), false); - TezTaskAttemptID taskAttemptID = taImpl.getID(); + TezTaskAttemptID taskAttemptID = taImpl.getTaskAttemptID(); ArgumentCaptor arg = ArgumentCaptor.forClass(Event.class); taImpl.handle(new TaskAttemptEventSchedule(taskAttemptID, 0, 0)); @@ -1460,7 +1460,7 @@ public void testSuccess() throws Exception { taListener, taskConf, new SystemClock(), mockHeartbeatHandler, appCtx, false, resource, createFakeContainerContext(), false); - TezTaskAttemptID taskAttemptID = taImpl.getID(); + TezTaskAttemptID taskAttemptID = taImpl.getTaskAttemptID(); ArgumentCaptor arg = ArgumentCaptor.forClass(Event.class); taImpl.handle(new TaskAttemptEventSchedule(taskAttemptID, 0, 0)); @@ -1548,7 +1548,7 @@ public void testContainerPreemptedAfterSuccess() throws Exception { taListener, taskConf, new SystemClock(), mockHeartbeatHandler, appCtx, false, resource, createFakeContainerContext(), false); - TezTaskAttemptID taskAttemptID = taImpl.getID(); + TezTaskAttemptID taskAttemptID = taImpl.getTaskAttemptID(); ArgumentCaptor arg = ArgumentCaptor.forClass(Event.class); @@ -1640,7 +1640,7 @@ public void testNodeFailedNonLeafVertex() throws Exception { taListener, taskConf, new SystemClock(), mockHeartbeatHandler, appCtx, false, resource, createFakeContainerContext(), false); - TezTaskAttemptID taskAttemptID = taImpl.getID(); + TezTaskAttemptID taskAttemptID = taImpl.getTaskAttemptID(); ArgumentCaptor arg = ArgumentCaptor.forClass(Event.class); @@ -1699,7 +1699,7 @@ taListener, taskConf, new SystemClock(), assertEquals(TaskAttemptTerminationCause.NODE_FAILED, taImpl.getTerminationCause()); assertEquals(TaskAttemptStateInternal.KILLED, taImpl.getInternalState()); - taImpl.handle(new TaskAttemptEventTezEventUpdate(taImpl.getID(), Collections.EMPTY_LIST)); + taImpl.handle(new TaskAttemptEventTezEventUpdate(taImpl.getTaskAttemptID(), Collections.EMPTY_LIST)); assertFalse( "InternalError occurred trying to handle TA_TEZ_EVENT_UPDATE in KILLED state", eventHandler.internalError); @@ -1747,7 +1747,7 @@ public void testNodeFailedLeafVertex() throws Exception { taListener, taskConf, new SystemClock(), mockHeartbeatHandler, appCtx, false, resource, createFakeContainerContext(), true); - TezTaskAttemptID taskAttemptID = taImpl.getID(); + TezTaskAttemptID taskAttemptID = taImpl.getTaskAttemptID(); ArgumentCaptor arg = ArgumentCaptor.forClass(Event.class); @@ -1847,7 +1847,7 @@ public void testMultipleOutputFailed() throws Exception { taListener, taskConf, new SystemClock(), mockHeartbeatHandler, appCtx, false, resource, createFakeContainerContext(), false); - TezTaskAttemptID taskAttemptID = taImpl.getID(); + TezTaskAttemptID taskAttemptID = taImpl.getTaskAttemptID(); taImpl.handle(new TaskAttemptEventSchedule(taskAttemptID, 0, 0)); taImpl.handle(new TaskAttemptEventSubmitted(taskAttemptID, contId)); @@ -1876,6 +1876,7 @@ taListener, taskConf, new SystemClock(), TezTaskID destTaskID = mock(TezTaskID.class); TezVertexID destVertexID = mock(TezVertexID.class); when(mockDestId1.getTaskID()).thenReturn(destTaskID); + when(mockDestId1.getVertexID()).thenReturn(destVertexID); when(destTaskID.getVertexID()).thenReturn(destVertexID); Vertex destVertex = mock(VertexImpl.class); when(destVertex.getRunningTasks()).thenReturn(11); @@ -1901,14 +1902,14 @@ taListener, taskConf, new SystemClock(), destTaskID = mock(TezTaskID.class); destVertexID = mock(TezVertexID.class); when(mockDestId2.getTaskID()).thenReturn(destTaskID); + when(mockDestId2.getVertexID()).thenReturn(destVertexID); when(destTaskID.getVertexID()).thenReturn(destVertexID); destVertex = mock(VertexImpl.class); when(destVertex.getRunningTasks()).thenReturn(11); when(mockDAG.getVertex(destVertexID)).thenReturn(destVertex); taImpl.handle(new TaskAttemptEventOutputFailed(taskAttemptID, tzEvent, 11)); - assertEquals("Task attempt is not in FAILED state", taImpl.getState(), - TaskAttemptState.FAILED); + assertEquals("Task attempt is not in FAILED state", TaskAttemptState.FAILED, taImpl.getState()); assertEquals(TaskAttemptTerminationCause.OUTPUT_LOST, taImpl.getTerminationCause()); // verify unregister is not invoked again verify(mockHeartbeatHandler, times(1)).unregister(taskAttemptID); @@ -1949,7 +1950,7 @@ taListener, taskConf, new SystemClock(), taListener, taskConf, new SystemClock(), mockHeartbeatHandler, appCtx, false, resource, createFakeContainerContext(), false); - TezTaskAttemptID taskAttemptID2 = taImpl2.getID(); + TezTaskAttemptID taskAttemptID2 = taImpl2.getTaskAttemptID(); taImpl2.handle(new TaskAttemptEventSchedule(taskAttemptID2, 0, 0)); taImpl2.handle(new TaskAttemptEventSubmitted(taskAttemptID2, contId)); @@ -1964,6 +1965,7 @@ taListener, taskConf, new SystemClock(), mockMeta = mock(EventMetaData.class); mockDestId1 = mock(TezTaskAttemptID.class); when(mockDestId1.getTaskID()).thenReturn(destTaskID); + when(mockDestId1.getVertexID()).thenReturn(destVertexID); when(mockMeta.getTaskAttemptID()).thenReturn(mockDestId1); tzEvent = new TezEvent(mockReEvent, mockMeta); //This should fail even when MAX_ALLOWED_OUTPUT_FAILURES_FRACTION is within limits, as @@ -1991,7 +1993,7 @@ taListener, taskConf, new SystemClock(), taListener, taskConf, mockClock, mockHeartbeatHandler, appCtx, false, resource, createFakeContainerContext(), false); - TezTaskAttemptID taskAttemptID3 = taImpl3.getID(); + TezTaskAttemptID taskAttemptID3 = taImpl3.getTaskAttemptID(); taImpl3.handle(new TaskAttemptEventSchedule(taskAttemptID3, 0, 0)); taImpl3.handle(new TaskAttemptEventSubmitted(taskAttemptID3, contId)); @@ -2006,6 +2008,7 @@ taListener, taskConf, new SystemClock(), mockMeta = mock(EventMetaData.class); mockDestId1 = mock(TezTaskAttemptID.class); when(mockDestId1.getTaskID()).thenReturn(destTaskID); + when(mockDestId1.getVertexID()).thenReturn(destVertexID); when(mockMeta.getTaskAttemptID()).thenReturn(mockDestId1); tzEvent = new TezEvent(mockReEvent, mockMeta); when(mockClock.getTime()).thenReturn(1000L); @@ -2073,7 +2076,7 @@ public void testTAFailureBasedOnRunningTasks() throws Exception { taListener, taskConf, new SystemClock(), mockHeartbeatHandler, appCtx, false, resource, createFakeContainerContext(), false); - TezTaskAttemptID taskAttemptID = taImpl.getID(); + TezTaskAttemptID taskAttemptID = taImpl.getTaskAttemptID(); taImpl.handle(new TaskAttemptEventSchedule(taskAttemptID, 0, 0)); taImpl.handle(new TaskAttemptEventSubmitted(taskAttemptID, contId)); @@ -2102,6 +2105,7 @@ taListener, taskConf, new SystemClock(), TezTaskID destTaskID = mock(TezTaskID.class); TezVertexID destVertexID = mock(TezVertexID.class); when(mockDestId1.getTaskID()).thenReturn(destTaskID); + when(mockDestId1.getVertexID()).thenReturn(destVertexID); when(destTaskID.getVertexID()).thenReturn(destVertexID); Vertex destVertex = mock(VertexImpl.class); when(destVertex.getRunningTasks()).thenReturn(5); @@ -2111,8 +2115,7 @@ taListener, taskConf, new SystemClock(), taImpl.handle(new TaskAttemptEventOutputFailed(taskAttemptID, tzEvent, 11)); // failure threshold is met due to running tasks. state is FAILED - assertEquals("Task attempt is not in FAILED state", taImpl.getState(), - TaskAttemptState.FAILED); + assertEquals("Task attempt is not in FAILED state", TaskAttemptState.FAILED, taImpl.getState()); } @SuppressWarnings("deprecation") @@ -2157,7 +2160,7 @@ taListener, taskConf, new SystemClock(), mockHeartbeatHandler, appCtx, false, resource, createFakeContainerContext(), true); Assert.assertEquals(TaskAttemptStateInternal.NEW, taImpl.getInternalState()); - taImpl.handle(new TaskAttemptEventKillRequest(taImpl.getID(), "kill it", + taImpl.handle(new TaskAttemptEventKillRequest(taImpl.getTaskAttemptID(), "kill it", TaskAttemptTerminationCause.TERMINATED_BY_CLIENT)); Assert.assertEquals(TaskAttemptStateInternal.KILLED, taImpl.getInternalState()); @@ -2190,7 +2193,7 @@ private void testMapTaskFailingForFetchFailureType(boolean isLocalFetch, InputReadErrorEvent.create("", 0, 1, 1, isLocalFetch, isDiskErrorAtSource, null); TezTaskAttemptID destTaskAttemptId = mock(TezTaskAttemptID.class); when(destTaskAttemptId.getTaskID()).thenReturn(mock(TezTaskID.class)); - when(destTaskAttemptId.getTaskID().getVertexID()).thenReturn(mock(TezVertexID.class)); + when(destTaskAttemptId.getVertexID()).thenReturn(mock(TezVertexID.class)); when(appCtx.getCurrentDAG()).thenReturn(mock(DAG.class)); when(appCtx.getCurrentDAG().getVertex(Mockito.any(TezVertexID.class))) .thenReturn(mock(Vertex.class)); @@ -2203,7 +2206,7 @@ private void testMapTaskFailingForFetchFailureType(boolean isLocalFetch, // the event is propagated to map task's event handler TaskAttemptEventOutputFailed outputFailedEvent = - new TaskAttemptEventOutputFailed(sourceAttempt.getID(), tezEvent, 11); + new TaskAttemptEventOutputFailed(sourceAttempt.getTaskAttemptID(), tezEvent, 11); Assert.assertEquals(TaskAttemptStateInternal.NEW, sourceAttempt.getInternalState()); TaskAttemptStateInternal resultState = new TaskAttemptImpl.OutputReportedFailedTransition() @@ -2236,13 +2239,13 @@ public void testMapTaskIsBlamedByDownstreamAttemptsFromDifferentHosts() { when(mockMeta.getTaskAttemptID()).thenReturn(destTaskAttemptId); // mapper task succeeded earlier - sourceAttempt.handle(new TaskAttemptEvent(sourceAttempt.getID(), TaskAttemptEventType.TA_DONE)); + sourceAttempt.handle(new TaskAttemptEvent(sourceAttempt.getTaskAttemptID(), TaskAttemptEventType.TA_DONE)); Assert.assertEquals(TaskAttemptStateInternal.SUCCEEDED, sourceAttempt.getInternalState()); // the event is propagated to map task's event handler TezEvent tezEvent = new TezEvent(inputReadErrorEvent1, mockMeta); TaskAttemptEventOutputFailed outputFailedEvent = - new TaskAttemptEventOutputFailed(sourceAttempt.getID(), tezEvent, 11); + new TaskAttemptEventOutputFailed(sourceAttempt.getTaskAttemptID(), tezEvent, 11); TaskAttemptStateInternal resultState = new TaskAttemptImpl.OutputReportedFailedTransition().transition(sourceAttempt, outputFailedEvent); // SUCCEEDED, as we haven't reached the host limit fraction @@ -2252,7 +2255,7 @@ public void testMapTaskIsBlamedByDownstreamAttemptsFromDifferentHosts() { // the second event is propagated to map task's event handler TezEvent tezEvent2 = new TezEvent(inputReadErrorEvent2, mockMeta); TaskAttemptEventOutputFailed outputFailedEvent2 = - new TaskAttemptEventOutputFailed(sourceAttempt.getID(), tezEvent2, 11); + new TaskAttemptEventOutputFailed(sourceAttempt.getTaskAttemptID(), tezEvent2, 11); TaskAttemptStateInternal resultState2 = new TaskAttemptImpl.OutputReportedFailedTransition().transition(sourceAttempt, outputFailedEvent2); diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskImpl.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskImpl.java index a28e786c6e..f4e40f7333 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskImpl.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskImpl.java @@ -386,9 +386,9 @@ public void testKillRunningTask() { LOG.info("--- START: testKillRunningTask ---"); TezTaskID taskId = getNewTaskID(); scheduleTaskAttempt(taskId); - launchTaskAttempt(mockTask.getLastAttempt().getID()); + launchTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID()); killTask(taskId); - mockTask.handle(createTaskTAKilledEvent(mockTask.getLastAttempt().getID())); + mockTask.handle(createTaskTAKilledEvent(mockTask.getLastAttempt().getTaskAttemptID())); assertEquals(TaskStateInternal.KILLED, mockTask.getInternalState()); verifyOutgoingEvents(eventHandler.events, VertexEventType.V_TASK_COMPLETED); } @@ -398,20 +398,20 @@ public void testTooManyFailedAttempts() { LOG.info("--- START: testTooManyFailedAttempts ---"); TezTaskID taskId = getNewTaskID(); scheduleTaskAttempt(taskId, TaskState.SCHEDULED); - launchTaskAttempt(mockTask.getLastAttempt().getID()); - failRunningTaskAttempt(mockTask.getLastAttempt().getID()); + launchTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID()); + failRunningTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID()); scheduleTaskAttempt(taskId, TaskState.RUNNING); - launchTaskAttempt(mockTask.getLastAttempt().getID()); - failRunningTaskAttempt(mockTask.getLastAttempt().getID()); + launchTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID()); + failRunningTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID()); scheduleTaskAttempt(taskId, TaskState.RUNNING); - launchTaskAttempt(mockTask.getLastAttempt().getID()); - failRunningTaskAttempt(mockTask.getLastAttempt().getID()); + launchTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID()); + failRunningTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID()); scheduleTaskAttempt(taskId, TaskState.RUNNING); - launchTaskAttempt(mockTask.getLastAttempt().getID()); - failRunningTaskAttempt(mockTask.getLastAttempt().getID(), false); + launchTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID()); + failRunningTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID(), false); assertEquals(TaskStateInternal.FAILED, mockTask.getInternalState()); verifyOutgoingEvents(eventHandler.events, VertexEventType.V_TASK_COMPLETED); @@ -431,11 +431,11 @@ public void testTooManyAttempts() { TezTaskID taskId = getNewTaskID(); scheduleTaskAttempt(taskId, TaskState.SCHEDULED); // attempt_0 - launchTaskAttempt(mockTask.getLastAttempt().getID()); - killRunningTaskAttempt(mockTask.getLastAttempt().getID(), TaskState.RUNNING, 1); // attempt_1 + launchTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID()); + killRunningTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID(), TaskState.RUNNING, 1); // attempt_1 - launchTaskAttempt(mockTask.getLastAttempt().getID()); - killRunningTaskAttempt(mockTask.getLastAttempt().getID(), TaskState.FAILED, 2); // attempt_2 -> reached 3 + launchTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID()); + killRunningTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID(), TaskState.FAILED, 2); // attempt_2 -> reached 3 assertEquals(TaskStateInternal.FAILED, mockTask.getInternalState()); verifyOutgoingEvents(eventHandler.events, VertexEventType.V_TASK_COMPLETED); @@ -446,9 +446,9 @@ public void testFailedAttemptWithFatalError() { LOG.info("--- START: testFailedAttemptWithFatalError ---"); TezTaskID taskId = getNewTaskID(); scheduleTaskAttempt(taskId, TaskState.SCHEDULED); - launchTaskAttempt(mockTask.getLastAttempt().getID()); + launchTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID()); mockTask.handle( - createTaskTAFailedEvent(mockTask.getLastAttempt().getID(), TaskFailureType.FATAL, null)); + createTaskTAFailedEvent(mockTask.getLastAttempt().getTaskAttemptID(), TaskFailureType.FATAL, null)); assertEquals(TaskStateInternal.FAILED, mockTask.getInternalState()); assertEquals(1, mockTask.failedAttempts); @@ -460,11 +460,11 @@ public void testKillRunningTaskPreviousKilledAttempts() { LOG.info("--- START: testKillRunningTaskPreviousKilledAttempts ---"); TezTaskID taskId = getNewTaskID(); scheduleTaskAttempt(taskId); - launchTaskAttempt(mockTask.getLastAttempt().getID()); - killRunningTaskAttempt(mockTask.getLastAttempt().getID(), TaskState.RUNNING); + launchTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID()); + killRunningTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID(), TaskState.RUNNING); assertEquals(TaskStateInternal.RUNNING, mockTask.getInternalState()); killTask(taskId); - mockTask.handle(createTaskTAKilledEvent(mockTask.getLastAttempt().getID())); + mockTask.handle(createTaskTAKilledEvent(mockTask.getLastAttempt().getTaskAttemptID())); assertEquals(TaskStateInternal.KILLED, mockTask.getInternalState()); verifyOutgoingEvents(eventHandler.events, VertexEventType.V_TASK_COMPLETED); @@ -478,9 +478,9 @@ public void testKillRunningTaskButAttemptSucceeds() { LOG.info("--- START: testKillRunningTaskButAttemptSucceeds ---"); TezTaskID taskId = getNewTaskID(); scheduleTaskAttempt(taskId); - launchTaskAttempt(mockTask.getLastAttempt().getID()); + launchTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID()); killTask(taskId); - mockTask.handle(createTaskTASucceededEvent(mockTask.getLastAttempt().getID())); + mockTask.handle(createTaskTASucceededEvent(mockTask.getLastAttempt().getTaskAttemptID())); assertEquals(TaskStateInternal.KILLED, mockTask.getInternalState()); } @@ -492,9 +492,9 @@ public void testKillRunningTaskButAttemptFails() { LOG.info("--- START: testKillRunningTaskButAttemptFails ---"); TezTaskID taskId = getNewTaskID(); scheduleTaskAttempt(taskId); - launchTaskAttempt(mockTask.getLastAttempt().getID()); + launchTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID()); killTask(taskId); - mockTask.handle(createTaskTAFailedEvent(mockTask.getLastAttempt().getID())); + mockTask.handle(createTaskTAFailedEvent(mockTask.getLastAttempt().getTaskAttemptID())); assertEquals(TaskStateInternal.KILLED, mockTask.getInternalState()); } @@ -507,8 +507,8 @@ public void testKillScheduledTaskAttempt() { LOG.info("--- START: testKillScheduledTaskAttempt ---"); TezTaskID taskId = getNewTaskID(); scheduleTaskAttempt(taskId); - TezTaskAttemptID lastTAId = mockTask.getLastAttempt().getID(); - killScheduledTaskAttempt(mockTask.getLastAttempt().getID()); + TezTaskAttemptID lastTAId = mockTask.getLastAttempt().getTaskAttemptID(); + killScheduledTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID()); // last killed attempt should be causal TA of next attempt Assert.assertEquals(lastTAId, mockTask.getLastAttempt().getSchedulingCausalTA()); } @@ -522,7 +522,7 @@ public void testLaunchTaskAttempt() { LOG.info("--- START: testLaunchTaskAttempt ---"); TezTaskID taskId = getNewTaskID(); scheduleTaskAttempt(taskId); - launchTaskAttempt(mockTask.getLastAttempt().getID()); + launchTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID()); } @Test(timeout = 5000) @@ -534,9 +534,9 @@ public void testKillRunningTaskAttempt() { LOG.info("--- START: testKillRunningTaskAttempt ---"); TezTaskID taskId = getNewTaskID(); scheduleTaskAttempt(taskId); - TezTaskAttemptID lastTAId = mockTask.getLastAttempt().getID(); - launchTaskAttempt(mockTask.getLastAttempt().getID()); - killRunningTaskAttempt(mockTask.getLastAttempt().getID(), TaskState.RUNNING); + TezTaskAttemptID lastTAId = mockTask.getLastAttempt().getTaskAttemptID(); + launchTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID()); + killRunningTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID(), TaskState.RUNNING); // last killed attempt should be causal TA of next attempt Assert.assertEquals(lastTAId, mockTask.getLastAttempt().getSchedulingCausalTA()); } @@ -550,9 +550,9 @@ public void testKillTaskAttemptServiceBusy() { LOG.info("--- START: testKillTaskAttemptServiceBusy ---"); TezTaskID taskId = getNewTaskID(); scheduleTaskAttempt(taskId); - launchTaskAttempt(mockTask.getLastAttempt().getID()); + launchTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID()); mockTask.handle(createTaskTAKilledEvent( - mockTask.getLastAttempt().getID(), new ServiceBusyEvent())); + mockTask.getLastAttempt().getTaskAttemptID(), new ServiceBusyEvent())); assertTaskRunningState(); verify(mockTask.getVertex(), times(0)).incrementKilledTaskAttemptCount(); verify(mockTask.getVertex(), times(1)).incrementRejectedTaskAttemptCount(); @@ -566,15 +566,15 @@ public void testKilledAttemptAtTaskKilled() { LOG.info("--- START: testKilledAttemptAtTaskKilled ---"); TezTaskID taskId = getNewTaskID(); scheduleTaskAttempt(taskId); - launchTaskAttempt(mockTask.getLastAttempt().getID()); + launchTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID()); killTask(taskId); - mockTask.handle(createTaskTAKilledEvent(mockTask.getLastAttempt().getID())); + mockTask.handle(createTaskTAKilledEvent(mockTask.getLastAttempt().getTaskAttemptID())); assertEquals(TaskStateInternal.KILLED, mockTask.getInternalState()); // Send duplicate kill for same attempt // This will not happen in practice but this is to simulate handling // of killed attempts in killed state. - mockTask.handle(createTaskTAKilledEvent(mockTask.getLastAttempt().getID())); + mockTask.handle(createTaskTAKilledEvent(mockTask.getLastAttempt().getTaskAttemptID())); assertEquals(TaskStateInternal.KILLED, mockTask.getInternalState()); } @@ -588,12 +588,12 @@ public void testKilledAttemptAtTaskFailed() { TezTaskID taskId = getNewTaskID(); scheduleTaskAttempt(taskId); for (int i = 0; i < mockTask.maxFailedAttempts; ++i) { - mockTask.handle(createTaskTAFailedEvent(mockTask.getLastAttempt().getID())); + mockTask.handle(createTaskTAFailedEvent(mockTask.getLastAttempt().getTaskAttemptID())); } assertEquals(TaskStateInternal.FAILED, mockTask.getInternalState()); // Send kill for an attempt - mockTask.handle(createTaskTAKilledEvent(mockTask.getLastAttempt().getID())); + mockTask.handle(createTaskTAKilledEvent(mockTask.getLastAttempt().getTaskAttemptID())); assertEquals(TaskStateInternal.FAILED, mockTask.getInternalState()); } @@ -609,7 +609,7 @@ public void testFetchedEventsModifyUnderlyingList() { scheduleTaskAttempt(taskId); sendTezEventsToTask(taskId, 2); TezTaskAttemptID attemptID = mockTask.getAttemptList().iterator().next() - .getID(); + .getTaskAttemptID(); fetchedList = mockTask.getTaskAttemptTezEvents(attemptID, 0, 100); assertEquals(2, fetchedList.size()); @@ -631,7 +631,7 @@ public void testTaskProgress() { scheduleTaskAttempt(taskId); float progress = 0f; assert (mockTask.getProgress() == progress); - launchTaskAttempt(mockTask.getLastAttempt().getID()); + launchTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID()); // update attempt1 progress = 50f; @@ -649,13 +649,13 @@ public void testTaskProgress() { // kill first attempt // should trigger a new attempt // as no successful attempts - failRunningTaskAttempt(mockTask.getLastAttempt().getID()); + failRunningTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID()); assert (mockTask.getAttemptList().size() == 2); assertEquals(1, mockTask.failedAttempts); verify(mockTask.getVertex(), times(1)).incrementFailedTaskAttemptCount(); assert (mockTask.getProgress() == 0f); - launchTaskAttempt(mockTask.getLastAttempt().getID()); + launchTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID()); progress = 50f; updateAttemptProgress(mockTask.getLastAttempt(), progress); assert (mockTask.getProgress() == progress); @@ -665,17 +665,17 @@ public void testTaskProgress() { public void testFailureDuringTaskAttemptCommit() { TezTaskID taskId = getNewTaskID(); scheduleTaskAttempt(taskId); - launchTaskAttempt(mockTask.getLastAttempt().getID()); + launchTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID()); updateAttemptState(mockTask.getLastAttempt(), TaskAttemptState.RUNNING); assertTrue("First attempt should commit", - mockTask.canCommit(mockTask.getLastAttempt().getID())); + mockTask.canCommit(mockTask.getLastAttempt().getTaskAttemptID())); // During the task attempt commit there is an exception which causes // the attempt to fail - TezTaskAttemptID lastTAId = mockTask.getLastAttempt().getID(); + TezTaskAttemptID lastTAId = mockTask.getLastAttempt().getTaskAttemptID(); updateAttemptState(mockTask.getLastAttempt(), TaskAttemptState.FAILED); assertEquals(1, mockTask.getAttemptList().size()); - failRunningTaskAttempt(mockTask.getLastAttempt().getID()); + failRunningTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID()); assertEquals(2, mockTask.getAttemptList().size()); assertEquals(1, mockTask.failedAttempts); @@ -683,13 +683,13 @@ public void testFailureDuringTaskAttemptCommit() { Assert.assertEquals(lastTAId, mockTask.getLastAttempt().getSchedulingCausalTA()); assertFalse("First attempt should not commit", - mockTask.canCommit(mockTask.getAttemptList().get(0).getID())); + mockTask.canCommit(mockTask.getAttemptList().get(0).getTaskAttemptID())); updateAttemptState(mockTask.getLastAttempt(), TaskAttemptState.RUNNING); assertTrue("Second attempt should commit", - mockTask.canCommit(mockTask.getLastAttempt().getID())); + mockTask.canCommit(mockTask.getLastAttempt().getTaskAttemptID())); updateAttemptState(mockTask.getLastAttempt(), TaskAttemptState.SUCCEEDED); - mockTask.handle(createTaskTASucceededEvent(mockTask.getLastAttempt().getID())); + mockTask.handle(createTaskTASucceededEvent(mockTask.getLastAttempt().getTaskAttemptID())); assertTaskSucceededState(); } @@ -703,14 +703,14 @@ public void testEventBacklogDuringTaskAttemptCommit() { // simulate // task in scheduled state due to event backlog - real task done and calling canCommit assertFalse("Commit should return false to make running task wait", - mockTask.canCommit(mockTask.getLastAttempt().getID())); - launchTaskAttempt(mockTask.getLastAttempt().getID()); + mockTask.canCommit(mockTask.getLastAttempt().getTaskAttemptID())); + launchTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID()); updateAttemptState(mockTask.getLastAttempt(), TaskAttemptState.RUNNING); assertTrue("Task state in AM is running now. Can commit.", - mockTask.canCommit(mockTask.getLastAttempt().getID())); + mockTask.canCommit(mockTask.getLastAttempt().getTaskAttemptID())); updateAttemptState(mockTask.getLastAttempt(), TaskAttemptState.SUCCEEDED); - mockTask.handle(createTaskTASucceededEvent(mockTask.getLastAttempt().getID())); + mockTask.handle(createTaskTASucceededEvent(mockTask.getLastAttempt().getTaskAttemptID())); assertTaskSucceededState(); } @@ -720,13 +720,13 @@ public void testEventBacklogDuringTaskAttemptCommit() { public void testChangeCommitTaskAttempt() { TezTaskID taskId = getNewTaskID(); scheduleTaskAttempt(taskId); - launchTaskAttempt(mockTask.getLastAttempt().getID()); + launchTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID()); updateAttemptState(mockTask.getLastAttempt(), TaskAttemptState.RUNNING); - TezTaskAttemptID lastTAId = mockTask.getLastAttempt().getID(); + TezTaskAttemptID lastTAId = mockTask.getLastAttempt().getTaskAttemptID(); // Add a speculative task attempt that succeeds - mockTask.handle(createTaskTAAddSpecAttempt(mockTask.getLastAttempt().getID())); - launchTaskAttempt(mockTask.getLastAttempt().getID()); + mockTask.handle(createTaskTAAddSpecAttempt(mockTask.getLastAttempt().getTaskAttemptID())); + launchTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID()); updateAttemptState(mockTask.getLastAttempt(), TaskAttemptState.RUNNING); assertEquals(2, mockTask.getAttemptList().size()); @@ -735,24 +735,24 @@ public void testChangeCommitTaskAttempt() { Assert.assertEquals(lastTAId, mockTask.getLastAttempt().getSchedulingCausalTA()); assertTrue("Second attempt should commit", - mockTask.canCommit(mockTask.getAttemptList().get(1).getID())); + mockTask.canCommit(mockTask.getAttemptList().get(1).getTaskAttemptID())); assertFalse("First attempt should not commit", - mockTask.canCommit(mockTask.getAttemptList().get(0).getID())); + mockTask.canCommit(mockTask.getAttemptList().get(0).getTaskAttemptID())); // During the task attempt commit there is an exception which causes // the second attempt to fail updateAttemptState(mockTask.getLastAttempt(), TaskAttemptState.FAILED); - failRunningTaskAttempt(mockTask.getLastAttempt().getID()); + failRunningTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID()); assertEquals(2, mockTask.getAttemptList().size()); assertFalse("Second attempt should not commit", - mockTask.canCommit(mockTask.getAttemptList().get(1).getID())); + mockTask.canCommit(mockTask.getAttemptList().get(1).getTaskAttemptID())); assertTrue("First attempt should commit", - mockTask.canCommit(mockTask.getAttemptList().get(0).getID())); + mockTask.canCommit(mockTask.getAttemptList().get(0).getTaskAttemptID())); updateAttemptState(mockTask.getAttemptList().get(0), TaskAttemptState.SUCCEEDED); - mockTask.handle(createTaskTASucceededEvent(mockTask.getAttemptList().get(0).getID())); + mockTask.handle(createTaskTASucceededEvent(mockTask.getAttemptList().get(0).getTaskAttemptID())); assertTaskSucceededState(); } @@ -762,15 +762,15 @@ public void testChangeCommitTaskAttempt() { public void testTaskSucceedAndRetroActiveFailure() { TezTaskID taskId = getNewTaskID(); scheduleTaskAttempt(taskId); - launchTaskAttempt(mockTask.getLastAttempt().getID()); + launchTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID()); updateAttemptState(mockTask.getLastAttempt(), TaskAttemptState.RUNNING); - mockTask.handle(createTaskTASucceededEvent(mockTask.getLastAttempt().getID())); + mockTask.handle(createTaskTASucceededEvent(mockTask.getLastAttempt().getTaskAttemptID())); // The task should now have succeeded assertTaskSucceededState(); verify(mockTask.stateChangeNotifier).taskSucceeded(any(String.class), eq(taskId), - eq(mockTask.getLastAttempt().getID().getId())); + eq(mockTask.getLastAttempt().getTaskAttemptID().getId())); ArgumentCaptor argumentCaptor = ArgumentCaptor.forClass(DAGHistoryEvent.class); verify(mockHistoryHandler).handle(argumentCaptor.capture()); @@ -789,7 +789,7 @@ public void testTaskSucceedAndRetroActiveFailure() { TaskAttemptEventOutputFailed outputFailedEvent = new TaskAttemptEventOutputFailed(mockDestId, mockTezEvent, 1); mockTask.handle( - createTaskTAFailedEvent(mockTask.getLastAttempt().getID(), TaskFailureType.NON_FATAL, + createTaskTAFailedEvent(mockTask.getLastAttempt().getTaskAttemptID(), TaskFailureType.NON_FATAL, outputFailedEvent)); // The task should still be in the scheduled state @@ -811,19 +811,19 @@ public void testTaskSucceedAndRetroActiveFailure() { public void testTaskSucceedAndRetroActiveKilled() { TezTaskID taskId = getNewTaskID(); scheduleTaskAttempt(taskId); - launchTaskAttempt(mockTask.getLastAttempt().getID()); + launchTaskAttempt(mockTask.getLastAttempt().getTaskAttemptID()); updateAttemptState(mockTask.getLastAttempt(), TaskAttemptState.RUNNING); - mockTask.handle(createTaskTASucceededEvent(mockTask.getLastAttempt().getID())); + mockTask.handle(createTaskTASucceededEvent(mockTask.getLastAttempt().getTaskAttemptID())); // The task should now have succeeded assertTaskSucceededState(); verify(mockTask.stateChangeNotifier).taskSucceeded(any(String.class), eq(taskId), - eq(mockTask.getLastAttempt().getID().getId())); + eq(mockTask.getLastAttempt().getTaskAttemptID().getId())); eventHandler.events.clear(); // Now kill the attempt after it has succeeded - mockTask.handle(createTaskTAKilledEvent(mockTask.getLastAttempt().getID())); + mockTask.handle(createTaskTAKilledEvent(mockTask.getLastAttempt().getTaskAttemptID())); // The task should still be in the scheduled state assertTaskScheduledState(); @@ -862,25 +862,25 @@ public void testFailedThenSpeculativeFailed() { TezTaskID taskId = getNewTaskID(); scheduleTaskAttempt(taskId); MockTaskAttemptImpl firstAttempt = mockTask.getLastAttempt(); - launchTaskAttempt(firstAttempt.getID()); + launchTaskAttempt(firstAttempt.getTaskAttemptID()); updateAttemptState(firstAttempt, TaskAttemptState.RUNNING); // Add a speculative task attempt - mockTask.handle(createTaskTAAddSpecAttempt(mockTask.getLastAttempt().getID())); + mockTask.handle(createTaskTAAddSpecAttempt(mockTask.getLastAttempt().getTaskAttemptID())); MockTaskAttemptImpl specAttempt = mockTask.getLastAttempt(); - launchTaskAttempt(specAttempt.getID()); + launchTaskAttempt(specAttempt.getTaskAttemptID()); updateAttemptState(specAttempt, TaskAttemptState.RUNNING); assertEquals(2, mockTask.getAttemptList().size()); // Fail the first attempt updateAttemptState(firstAttempt, TaskAttemptState.FAILED); - mockTask.handle(createTaskTAFailedEvent(firstAttempt.getID())); + mockTask.handle(createTaskTAFailedEvent(firstAttempt.getTaskAttemptID())); assertEquals(TaskState.FAILED, mockTask.getState()); assertEquals(2, mockTask.getAttemptList().size()); // Now fail the speculative attempt updateAttemptState(specAttempt, TaskAttemptState.FAILED); - mockTask.handle(createTaskTAFailedEvent(specAttempt.getID())); + mockTask.handle(createTaskTAFailedEvent(specAttempt.getTaskAttemptID())); assertEquals(TaskState.FAILED, mockTask.getState()); assertEquals(2, mockTask.getAttemptList().size()); } @@ -897,25 +897,25 @@ public void testFailedThenSpeculativeSucceeded() { TezTaskID taskId = getNewTaskID(); scheduleTaskAttempt(taskId); MockTaskAttemptImpl firstAttempt = mockTask.getLastAttempt(); - launchTaskAttempt(firstAttempt.getID()); + launchTaskAttempt(firstAttempt.getTaskAttemptID()); updateAttemptState(firstAttempt, TaskAttemptState.RUNNING); // Add a speculative task attempt - mockTask.handle(createTaskTAAddSpecAttempt(mockTask.getLastAttempt().getID())); + mockTask.handle(createTaskTAAddSpecAttempt(mockTask.getLastAttempt().getTaskAttemptID())); MockTaskAttemptImpl specAttempt = mockTask.getLastAttempt(); - launchTaskAttempt(specAttempt.getID()); + launchTaskAttempt(specAttempt.getTaskAttemptID()); updateAttemptState(specAttempt, TaskAttemptState.RUNNING); assertEquals(2, mockTask.getAttemptList().size()); // Fail the first attempt updateAttemptState(firstAttempt, TaskAttemptState.FAILED); - mockTask.handle(createTaskTAFailedEvent(firstAttempt.getID())); + mockTask.handle(createTaskTAFailedEvent(firstAttempt.getTaskAttemptID())); assertEquals(TaskState.FAILED, mockTask.getState()); assertEquals(2, mockTask.getAttemptList().size()); // Now succeed the speculative attempt updateAttemptState(specAttempt, TaskAttemptState.SUCCEEDED); - mockTask.handle(createTaskTASucceededEvent(specAttempt.getID())); + mockTask.handle(createTaskTASucceededEvent(specAttempt.getTaskAttemptID())); assertEquals(TaskState.FAILED, mockTask.getState()); assertEquals(2, mockTask.getAttemptList().size()); } @@ -932,10 +932,10 @@ public void testKilledBeforeSpeculatedSucceeded() { TezTaskID taskId = getNewTaskID(); scheduleTaskAttempt(taskId); MockTaskAttemptImpl firstAttempt = mockTask.getLastAttempt(); - launchTaskAttempt(firstAttempt.getID()); + launchTaskAttempt(firstAttempt.getTaskAttemptID()); updateAttemptState(firstAttempt, TaskAttemptState.RUNNING); - mockTask.handle(createTaskTAKilledEvent(firstAttempt.getID())); + mockTask.handle(createTaskTAKilledEvent(firstAttempt.getTaskAttemptID())); assertEquals(TaskStateInternal.RUNNING, mockTask.getInternalState()); // We need to manually override the current node id @@ -945,16 +945,16 @@ public void testKilledBeforeSpeculatedSucceeded() { mockNodeId = null; // Add a speculative task attempt - mockTask.handle(createTaskTAAddSpecAttempt(mockTask.getLastAttempt().getID())); + mockTask.handle(createTaskTAAddSpecAttempt(mockTask.getLastAttempt().getTaskAttemptID())); mockNodeId = nodeId; MockTaskAttemptImpl specAttempt = mockTask.getLastAttempt(); - launchTaskAttempt(specAttempt.getID()); + launchTaskAttempt(specAttempt.getTaskAttemptID()); updateAttemptState(specAttempt, TaskAttemptState.RUNNING); assertEquals(3, mockTask.getAttemptList().size()); // Now succeed the speculative attempt updateAttemptState(specAttempt, TaskAttemptState.SUCCEEDED); - mockTask.handle(createTaskTASucceededEvent(specAttempt.getID())); + mockTask.handle(createTaskTASucceededEvent(specAttempt.getTaskAttemptID())); assertEquals(TaskState.SUCCEEDED, mockTask.getState()); assertEquals(3, mockTask.getAttemptList().size()); } @@ -964,35 +964,35 @@ public void testKilledAttemptUpdatesDAGScheduler() { TezTaskID taskId = getNewTaskID(); scheduleTaskAttempt(taskId); MockTaskAttemptImpl firstAttempt = mockTask.getLastAttempt(); - launchTaskAttempt(firstAttempt.getID()); + launchTaskAttempt(firstAttempt.getTaskAttemptID()); updateAttemptState(firstAttempt, TaskAttemptState.RUNNING); // Add a speculative task attempt - mockTask.handle(createTaskTAAddSpecAttempt(firstAttempt.getID())); + mockTask.handle(createTaskTAAddSpecAttempt(firstAttempt.getTaskAttemptID())); MockTaskAttemptImpl specAttempt = mockTask.getLastAttempt(); - launchTaskAttempt(specAttempt.getID()); + launchTaskAttempt(specAttempt.getTaskAttemptID()); updateAttemptState(specAttempt, TaskAttemptState.RUNNING); assertEquals(2, mockTask.getAttemptList().size()); // Have the first task succeed eventHandler.events.clear(); - mockTask.handle(createTaskTASucceededEvent(firstAttempt.getID())); + mockTask.handle(createTaskTASucceededEvent(firstAttempt.getTaskAttemptID())); verifyOutgoingEvents(eventHandler.events, DAGEventType.DAG_SCHEDULER_UPDATE, VertexEventType.V_TASK_COMPLETED, VertexEventType.V_TASK_ATTEMPT_COMPLETED); // The task should now have succeeded and sent kill to other attempt assertTaskSucceededState(); verify(mockTask.stateChangeNotifier).taskSucceeded(any(String.class), eq(taskId), - eq(firstAttempt.getID().getId())); + eq(firstAttempt.getTaskAttemptID().getId())); @SuppressWarnings("rawtypes") Event event = eventHandler.events.get(eventHandler.events.size()-1); assertEquals(TaskAttemptEventType.TA_KILL_REQUEST, event.getType()); - assertEquals(specAttempt.getID(), + assertEquals(specAttempt.getTaskAttemptID(), ((TaskAttemptEventKillRequest) event).getTaskAttemptID()); eventHandler.events.clear(); // Emulate the spec attempt being killed - mockTask.handle(createTaskTAKilledEvent(specAttempt.getID())); + mockTask.handle(createTaskTAKilledEvent(specAttempt.getTaskAttemptID())); assertTaskSucceededState(); verifyOutgoingEvents(eventHandler.events, DAGEventType.DAG_SCHEDULER_UPDATE, VertexEventType.V_TASK_ATTEMPT_COMPLETED); @@ -1003,32 +1003,32 @@ public void testSpeculatedThenRetroactiveFailure() { TezTaskID taskId = getNewTaskID(); scheduleTaskAttempt(taskId); MockTaskAttemptImpl firstAttempt = mockTask.getLastAttempt(); - launchTaskAttempt(firstAttempt.getID()); + launchTaskAttempt(firstAttempt.getTaskAttemptID()); updateAttemptState(firstAttempt, TaskAttemptState.RUNNING); // Add a speculative task attempt - mockTask.handle(createTaskTAAddSpecAttempt(firstAttempt.getID())); + mockTask.handle(createTaskTAAddSpecAttempt(firstAttempt.getTaskAttemptID())); MockTaskAttemptImpl specAttempt = mockTask.getLastAttempt(); - launchTaskAttempt(specAttempt.getID()); + launchTaskAttempt(specAttempt.getTaskAttemptID()); updateAttemptState(specAttempt, TaskAttemptState.RUNNING); assertEquals(2, mockTask.getAttemptList().size()); // Have the first task succeed eventHandler.events.clear(); - mockTask.handle(createTaskTASucceededEvent(firstAttempt.getID())); + mockTask.handle(createTaskTASucceededEvent(firstAttempt.getTaskAttemptID())); // The task should now have succeeded and sent kill to other attempt assertTaskSucceededState(); verify(mockTask.stateChangeNotifier).taskSucceeded(any(String.class), eq(taskId), - eq(firstAttempt.getID().getId())); + eq(firstAttempt.getTaskAttemptID().getId())); @SuppressWarnings("rawtypes") Event event = eventHandler.events.get(eventHandler.events.size()-1); assertEquals(TaskAttemptEventType.TA_KILL_REQUEST, event.getType()); - assertEquals(specAttempt.getID(), + assertEquals(specAttempt.getTaskAttemptID(), ((TaskAttemptEventKillRequest) event).getTaskAttemptID()); // Emulate the spec attempt being killed - mockTask.handle(createTaskTAKilledEvent(specAttempt.getID())); + mockTask.handle(createTaskTAKilledEvent(specAttempt.getTaskAttemptID())); assertTaskSucceededState(); // Now fail the attempt after it has succeeded @@ -1039,7 +1039,7 @@ public void testSpeculatedThenRetroactiveFailure() { TaskAttemptEventOutputFailed outputFailedEvent = new TaskAttemptEventOutputFailed(mockDestId, mockTezEvent, 1); eventHandler.events.clear(); - mockTask.handle(createTaskTAFailedEvent(firstAttempt.getID(), TaskFailureType.NON_FATAL, outputFailedEvent)); + mockTask.handle(createTaskTAFailedEvent(firstAttempt.getTaskAttemptID(), TaskFailureType.NON_FATAL, outputFailedEvent)); // The task should still be in the scheduled state assertTaskScheduledState(); @@ -1059,15 +1059,15 @@ public void testIgnoreSpeculationOnSuccessfulOriginalAttempt() { TezTaskID taskId = getNewTaskID(); scheduleTaskAttempt(taskId); MockTaskAttemptImpl firstAttempt = mockTask.getLastAttempt(); - launchTaskAttempt(firstAttempt.getID()); + launchTaskAttempt(firstAttempt.getTaskAttemptID()); // Mock success of the first task attempt updateAttemptState(firstAttempt, TaskAttemptState.SUCCEEDED); - firstAttempt.handle(new TaskAttemptEvent(firstAttempt.getID(), TaskAttemptEventType.TA_DONE)); + firstAttempt.handle(new TaskAttemptEvent(firstAttempt.getTaskAttemptID(), TaskAttemptEventType.TA_DONE)); // Verify the speculation scheduling is ignored and no speculative attempt was added to the task - mockTask.handle(createTaskTAAddSpecAttempt(firstAttempt.getID())); + mockTask.handle(createTaskTAAddSpecAttempt(firstAttempt.getTaskAttemptID())); MockTaskAttemptImpl specAttempt = mockTask.getLastAttempt(); - launchTaskAttempt(specAttempt.getID()); + launchTaskAttempt(specAttempt.getTaskAttemptID()); assertEquals(1, mockTask.getAttemptList().size()); } @@ -1076,15 +1076,15 @@ public void testIgnoreSpeculationAfterOriginalAttemptCommit() { TezTaskID taskId = getNewTaskID(); scheduleTaskAttempt(taskId); MockTaskAttemptImpl firstAttempt = mockTask.getLastAttempt(); - launchTaskAttempt(firstAttempt.getID()); + launchTaskAttempt(firstAttempt.getTaskAttemptID()); updateAttemptState(firstAttempt, TaskAttemptState.RUNNING); // Mock commit of the first task attempt - mockTask.canCommit(firstAttempt.getID()); + mockTask.canCommit(firstAttempt.getTaskAttemptID()); // Verify the speculation scheduling is ignored and no speculative attempt was added to the task - mockTask.handle(createTaskTAAddSpecAttempt(firstAttempt.getID())); + mockTask.handle(createTaskTAAddSpecAttempt(firstAttempt.getTaskAttemptID())); MockTaskAttemptImpl specAttempt = mockTask.getLastAttempt(); - launchTaskAttempt(specAttempt.getID()); + launchTaskAttempt(specAttempt.getTaskAttemptID()); assertEquals(1, mockTask.getAttemptList().size()); } @@ -1094,10 +1094,10 @@ public void testSucceededAttemptStatusWithRetroActiveFailures() throws Interrupt TezTaskID taskId = getNewTaskID(); scheduleTaskAttempt(taskId); MockTaskAttemptImpl firstMockTaskAttempt = mockTask.getAttemptList().get(0); - launchTaskAttempt(firstMockTaskAttempt.getID()); - mockTask.handle(createTaskTAAddSpecAttempt(mockTask.getLastAttempt().getID())); + launchTaskAttempt(firstMockTaskAttempt.getTaskAttemptID()); + mockTask.handle(createTaskTAAddSpecAttempt(mockTask.getLastAttempt().getTaskAttemptID())); MockTaskAttemptImpl secondMockTaskAttempt = mockTask.getAttemptList().get(1); - launchTaskAttempt(secondMockTaskAttempt.getID()); + launchTaskAttempt(secondMockTaskAttempt.getTaskAttemptID()); firstMockTaskAttempt.handle(new TaskAttemptEventSchedule( TezTaskAttemptID.fromString(firstMockTaskAttempt.toString()), 10, 10)); @@ -1119,8 +1119,8 @@ public void testSucceededAttemptStatusWithRetroActiveFailures() throws Interrupt new TaskAttemptEvent(TezTaskAttemptID.fromString(firstMockTaskAttempt.toString()), TaskAttemptEventType.TA_DONE)); - mockTask.handle(new TaskEventTASucceeded(secondMockTaskAttempt.getID())); - mockTask.handle(new TaskEventTASucceeded(firstMockTaskAttempt.getID())); + mockTask.handle(new TaskEventTASucceeded(secondMockTaskAttempt.getTaskAttemptID())); + mockTask.handle(new TaskEventTASucceeded(firstMockTaskAttempt.getTaskAttemptID())); assertTrue("Attempts should have succeeded!", firstMockTaskAttempt.getInternalState() == TaskAttemptStateInternal.SUCCEEDED && secondMockTaskAttempt.getInternalState() == TaskAttemptStateInternal.SUCCEEDED); @@ -1148,10 +1148,10 @@ public void testFailedAttemptStatus() throws InterruptedException { TezTaskID taskId = getNewTaskID(); scheduleTaskAttempt(taskId); MockTaskAttemptImpl firstMockTaskAttempt = mockTask.getAttemptList().get(0); - launchTaskAttempt(firstMockTaskAttempt.getID()); - mockTask.handle(createTaskTAAddSpecAttempt(mockTask.getLastAttempt().getID())); + launchTaskAttempt(firstMockTaskAttempt.getTaskAttemptID()); + mockTask.handle(createTaskTAAddSpecAttempt(mockTask.getLastAttempt().getTaskAttemptID())); MockTaskAttemptImpl secondMockTaskAttempt = mockTask.getAttemptList().get(1); - launchTaskAttempt(secondMockTaskAttempt.getID()); + launchTaskAttempt(secondMockTaskAttempt.getTaskAttemptID()); firstMockTaskAttempt.handle(new TaskAttemptEventSchedule( TezTaskAttemptID.fromString(firstMockTaskAttempt.toString()), 10, 10)); @@ -1176,12 +1176,12 @@ public void testFailedAttemptStatus() throws InterruptedException { TaskAttemptTerminationCause.NO_PROGRESS)); firstMockTaskAttempt.handle(new TaskAttemptEventContainerTerminated(mockContainerId, - firstMockTaskAttempt.getID(), "test", TaskAttemptTerminationCause.NO_PROGRESS)); + firstMockTaskAttempt.getTaskAttemptID(), "test", TaskAttemptTerminationCause.NO_PROGRESS)); secondMockTaskAttempt.handle(new TaskAttemptEventContainerTerminated(mockContainerId, - secondMockTaskAttempt.getID(), "test", TaskAttemptTerminationCause.NO_PROGRESS)); - mockTask.handle(new TaskEventTAFailed(secondMockTaskAttempt.getID(), TaskFailureType.NON_FATAL, + secondMockTaskAttempt.getTaskAttemptID(), "test", TaskAttemptTerminationCause.NO_PROGRESS)); + mockTask.handle(new TaskEventTAFailed(secondMockTaskAttempt.getTaskAttemptID(), TaskFailureType.NON_FATAL, mock(TaskAttemptEvent.class))); - mockTask.handle(new TaskEventTAFailed(firstMockTaskAttempt.getID(), TaskFailureType.NON_FATAL, + mockTask.handle(new TaskEventTAFailed(firstMockTaskAttempt.getTaskAttemptID(), TaskFailureType.NON_FATAL, mock(TaskAttemptEvent.class))); assertTrue("Attempts should have failed!", firstMockTaskAttempt.getInternalState() == TaskAttemptStateInternal.FAILED @@ -1204,10 +1204,10 @@ public void testSucceededLeafTaskWithRetroFailures() throws InterruptedException TezTaskID taskId = getNewTaskID(); scheduleTaskAttempt(taskId); MockTaskAttemptImpl firstMockTaskAttempt = mockTask.getAttemptList().get(0); - launchTaskAttempt(firstMockTaskAttempt.getID()); - mockTask.handle(createTaskTAAddSpecAttempt(mockTask.getLastAttempt().getID())); + launchTaskAttempt(firstMockTaskAttempt.getTaskAttemptID()); + mockTask.handle(createTaskTAAddSpecAttempt(mockTask.getLastAttempt().getTaskAttemptID())); MockTaskAttemptImpl secondMockTaskAttempt = mockTask.getAttemptList().get(1); - launchTaskAttempt(secondMockTaskAttempt.getID()); + launchTaskAttempt(secondMockTaskAttempt.getTaskAttemptID()); firstMockTaskAttempt.handle(new TaskAttemptEventSchedule( TezTaskAttemptID.fromString(firstMockTaskAttempt.toString()), 10, 10)); @@ -1230,18 +1230,18 @@ public void testSucceededLeafTaskWithRetroFailures() throws InterruptedException TaskAttemptEventType.TA_FAILED, TaskFailureType.NON_FATAL, "test", TaskAttemptTerminationCause.CONTAINER_EXITED)); - mockTask.handle(new TaskEventTASucceeded(secondMockTaskAttempt.getID())); + mockTask.handle(new TaskEventTASucceeded(secondMockTaskAttempt.getTaskAttemptID())); firstMockTaskAttempt.handle(new TaskAttemptEventContainerTerminated(mockContainerId, - firstMockTaskAttempt.getID(), "test", TaskAttemptTerminationCause.NO_PROGRESS)); + firstMockTaskAttempt.getTaskAttemptID(), "test", TaskAttemptTerminationCause.NO_PROGRESS)); InputReadErrorEvent mockReEvent = InputReadErrorEvent.create("", 0, 0); - TezTaskAttemptID mockDestId = firstMockTaskAttempt.getID(); + TezTaskAttemptID mockDestId = firstMockTaskAttempt.getTaskAttemptID(); EventMetaData meta = new EventMetaData(EventProducerConsumerType.INPUT, "Vertex", "Edge", mockDestId); TezEvent tzEvent = new TezEvent(mockReEvent, meta); TaskAttemptEventOutputFailed outputFailedEvent = new TaskAttemptEventOutputFailed(mockDestId, tzEvent, 1); firstMockTaskAttempt.handle(outputFailedEvent); - mockTask.handle(new TaskEventTAFailed(firstMockTaskAttempt.getID(), TaskFailureType.NON_FATAL, + mockTask.handle(new TaskEventTAFailed(firstMockTaskAttempt.getTaskAttemptID(), TaskFailureType.NON_FATAL, mock(TaskAttemptEvent.class))); Assert.assertEquals(mockTask.getInternalState(), TaskStateInternal.SUCCEEDED); } @@ -1255,7 +1255,7 @@ private void failAttempt(MockTaskAttemptImpl taskAttempt, int index, int expecte new TaskAttemptEventOutputFailed(mockDestId, tzEvent, 1); taskAttempt.handle( outputFailedEvent); - TaskEvent tEventFail1 = new TaskEventTAFailed(taskAttempt.getID(), TaskFailureType.NON_FATAL, outputFailedEvent); + TaskEvent tEventFail1 = new TaskEventTAFailed(taskAttempt.getTaskAttemptID(), TaskFailureType.NON_FATAL, outputFailedEvent); mockTask.handle(tEventFail1); assertEquals("Unexpected number of incomplete attempts!", expectedIncompleteAttempts, mockTask.getUncompletedAttemptsCount()); @@ -1274,10 +1274,10 @@ public void testFailedTaskTransitionWithLaunchedAttempt() throws InterruptedExce TezTaskID taskId = getNewTaskID(); scheduleTaskAttempt(taskId); MockTaskAttemptImpl firstMockTaskAttempt = mockTask.getLastAttempt(); - launchTaskAttempt(firstMockTaskAttempt.getID()); - mockTask.handle(createTaskTAAddSpecAttempt(mockTask.getLastAttempt().getID())); + launchTaskAttempt(firstMockTaskAttempt.getTaskAttemptID()); + mockTask.handle(createTaskTAAddSpecAttempt(mockTask.getLastAttempt().getTaskAttemptID())); MockTaskAttemptImpl secondMockTaskAttempt = mockTask.getLastAttempt(); - launchTaskAttempt(secondMockTaskAttempt.getID()); + launchTaskAttempt(secondMockTaskAttempt.getTaskAttemptID()); firstMockTaskAttempt.handle(new TaskAttemptEventSchedule( TezTaskAttemptID.fromString(firstMockTaskAttempt.toString()), 10, 10)); @@ -1302,21 +1302,21 @@ public void testFailedTaskTransitionWithLaunchedAttempt() throws InterruptedExce TaskAttemptTerminationCause.NO_PROGRESS)); firstMockTaskAttempt.handle(new TaskAttemptEventContainerTerminated(mockContainerId, - firstMockTaskAttempt.getID(), "test", TaskAttemptTerminationCause.NO_PROGRESS)); + firstMockTaskAttempt.getTaskAttemptID(), "test", TaskAttemptTerminationCause.NO_PROGRESS)); secondMockTaskAttempt.handle(new TaskAttemptEventContainerTerminated(mockContainerId, - secondMockTaskAttempt.getID(), "test", TaskAttemptTerminationCause.NO_PROGRESS)); - mockTask.handle(new TaskEventTAFailed(secondMockTaskAttempt.getID(), TaskFailureType.NON_FATAL, + secondMockTaskAttempt.getTaskAttemptID(), "test", TaskAttemptTerminationCause.NO_PROGRESS)); + mockTask.handle(new TaskEventTAFailed(secondMockTaskAttempt.getTaskAttemptID(), TaskFailureType.NON_FATAL, mock(TaskAttemptEvent.class))); - mockTask.handle(new TaskEventTAFailed(firstMockTaskAttempt.getID(), TaskFailureType.NON_FATAL, + mockTask.handle(new TaskEventTAFailed(firstMockTaskAttempt.getTaskAttemptID(), TaskFailureType.NON_FATAL, mock(TaskAttemptEvent.class))); assertTrue("Attempts should have failed!", firstMockTaskAttempt.getInternalState() == TaskAttemptStateInternal.FAILED && secondMockTaskAttempt.getInternalState() == TaskAttemptStateInternal.FAILED); assertEquals("Task should have no uncompleted attempts!", 0, mockTask.getUncompletedAttemptsCount()); assertTrue("Task should have failed!", mockTask.getState() == TaskState.FAILED); - mockTask.handle(createTaskTAAddSpecAttempt(mockTask.getLastAttempt().getID())); + mockTask.handle(createTaskTAAddSpecAttempt(mockTask.getLastAttempt().getTaskAttemptID())); MockTaskAttemptImpl thirdMockTaskAttempt = mockTask.getLastAttempt(); - mockTask.handle(createTaskTALauncherEvent(thirdMockTaskAttempt.getID())); + mockTask.handle(createTaskTALauncherEvent(thirdMockTaskAttempt.getTaskAttemptID())); } @Test (timeout = 30000) @@ -1324,10 +1324,10 @@ public void testKilledTaskTransitionWithLaunchedAttempt() throws InterruptedExce TezTaskID taskId = getNewTaskID(); scheduleTaskAttempt(taskId); MockTaskAttemptImpl firstMockTaskAttempt = mockTask.getLastAttempt(); - launchTaskAttempt(firstMockTaskAttempt.getID()); - mockTask.handle(createTaskTAAddSpecAttempt(mockTask.getLastAttempt().getID())); + launchTaskAttempt(firstMockTaskAttempt.getTaskAttemptID()); + mockTask.handle(createTaskTAAddSpecAttempt(mockTask.getLastAttempt().getTaskAttemptID())); MockTaskAttemptImpl secondMockTaskAttempt = mockTask.getLastAttempt(); - launchTaskAttempt(secondMockTaskAttempt.getID()); + launchTaskAttempt(secondMockTaskAttempt.getTaskAttemptID()); firstMockTaskAttempt.handle(new TaskAttemptEventSchedule( TezTaskAttemptID.fromString(firstMockTaskAttempt.toString()), 10, 10)); @@ -1342,30 +1342,30 @@ public void testKilledTaskTransitionWithLaunchedAttempt() throws InterruptedExce new TaskAttemptEventStartedRemotely(TezTaskAttemptID.fromString(secondMockTaskAttempt.toString()))); firstMockTaskAttempt.handle( new TaskAttemptEventStartedRemotely(TezTaskAttemptID.fromString(firstMockTaskAttempt.toString()))); - mockTask.handle(new TaskEventTermination(mockTask.getTaskId(), + mockTask.handle(new TaskEventTermination(mockTask.getTaskID(), TaskAttemptTerminationCause.FRAMEWORK_ERROR, "test")); secondMockTaskAttempt.handle( new TaskAttemptEventAttemptKilled(TezTaskAttemptID.fromString(secondMockTaskAttempt.toString()),"test", TaskAttemptTerminationCause.FRAMEWORK_ERROR)); - mockTask.handle(new TaskEventTAKilled(secondMockTaskAttempt.getID(), - new TaskAttemptEvent(secondMockTaskAttempt.getID(), TaskAttemptEventType.TA_KILLED))); + mockTask.handle(new TaskEventTAKilled(secondMockTaskAttempt.getTaskAttemptID(), + new TaskAttemptEvent(secondMockTaskAttempt.getTaskAttemptID(), TaskAttemptEventType.TA_KILLED))); firstMockTaskAttempt.handle( new TaskAttemptEventAttemptKilled(TezTaskAttemptID.fromString(firstMockTaskAttempt.toString()),"test", TaskAttemptTerminationCause.FRAMEWORK_ERROR)); - mockTask.handle(new TaskEventTAKilled(firstMockTaskAttempt.getID(), - new TaskAttemptEvent(firstMockTaskAttempt.getID(), TaskAttemptEventType.TA_KILLED))); + mockTask.handle(new TaskEventTAKilled(firstMockTaskAttempt.getTaskAttemptID(), + new TaskAttemptEvent(firstMockTaskAttempt.getTaskAttemptID(), TaskAttemptEventType.TA_KILLED))); firstMockTaskAttempt.handle( new TaskAttemptEventAttemptKilled(TezTaskAttemptID.fromString(firstMockTaskAttempt.toString()),"test", TaskAttemptTerminationCause.FRAMEWORK_ERROR)); assertEquals("Task should have been killed!", mockTask.getInternalState(), TaskStateInternal.KILLED); - mockTask.handle(createTaskTAAddSpecAttempt(mockTask.getLastAttempt().getID())); + mockTask.handle(createTaskTAAddSpecAttempt(mockTask.getLastAttempt().getTaskAttemptID())); MockTaskAttemptImpl thirdMockTaskAttempt = mockTask.getLastAttempt(); - mockTask.handle(createTaskTALauncherEvent(thirdMockTaskAttempt.getID())); - mockTask.handle(createTaskTAAddSpecAttempt(mockTask.getLastAttempt().getID())); + mockTask.handle(createTaskTALauncherEvent(thirdMockTaskAttempt.getTaskAttemptID())); + mockTask.handle(createTaskTAAddSpecAttempt(mockTask.getLastAttempt().getTaskAttemptID())); MockTaskAttemptImpl fourthMockTaskAttempt = mockTask.getLastAttempt(); - mockTask.handle(createTaskTASucceededEvent(fourthMockTaskAttempt.getID())); + mockTask.handle(createTaskTASucceededEvent(fourthMockTaskAttempt.getTaskAttemptID())); MockTaskAttemptImpl fifthMockTaskAttempt = mockTask.getLastAttempt(); - mockTask.handle(createTaskTAFailedEvent(fifthMockTaskAttempt.getID())); + mockTask.handle(createTaskTAFailedEvent(fifthMockTaskAttempt.getTaskAttemptID())); } // TODO Add test to validate the correct commit attempt. @@ -1418,7 +1418,7 @@ public MockTaskImpl(TezVertexID vertexId, int partition, @Override protected TaskAttemptImpl createAttempt(int attemptNumber, TezTaskAttemptID schedCausalTA) { MockTaskAttemptImpl attempt = new MockTaskAttemptImpl( - TezBuilderUtils.newTaskAttemptId(getTaskId(), attemptNumber), + TezBuilderUtils.newTaskAttemptId(getTaskID(), attemptNumber), eventHandler, taskCommunicatorManagerInterface, conf, clock, taskHeartbeatHandler, appContext, true, taskResource, containerContext, schedCausalTA); 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 fb7872ff9a..5cdcf49206 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 @@ -386,7 +386,7 @@ private class TaskAttemptEventDispatcher implements EventHandler)task.getAttempt( event.getTaskAttemptID())).handle(event); @@ -399,7 +399,7 @@ private class TaskEventDispatcher implements EventHandler { @Override public void handle(TaskEvent event) { events.add(event); - VertexImpl vertex = vertexIdMap.get(event.getTaskID().getVertexID()); + VertexImpl vertex = vertexIdMap.get(event.getVertexID()); Task task = vertex.getTask(event.getTaskID()); if (task != null) { ((EventHandler)task).handle(event); @@ -437,7 +437,7 @@ private class VertexEventDispatcher @Override public void handle(VertexEvent event) { - VertexImpl vertex = vertexIdMap.get(event.getVertexId()); + VertexImpl vertex = vertexIdMap.get(event.getVertexID()); ((EventHandler) vertex).handle(event); } } @@ -3155,7 +3155,7 @@ private void checkTasks(Vertex v, int numTasks) { int i = 0; // iteration maintains order due to linked hash map for(Task task : tasks.values()) { - Assert.assertEquals(i, task.getTaskId().getId()); + Assert.assertEquals(i, task.getTaskID().getId()); i++; } } @@ -3638,7 +3638,7 @@ public void testVertexTaskAttemptProcessorFailure() throws Exception { startVertex(v); dispatcher.await(); TaskAttemptImpl ta = (TaskAttemptImpl) v.getTask(0).getAttempts().values().iterator().next(); - ta.handle(new TaskAttemptEventSchedule(ta.getID(), 2, 2)); + ta.handle(new TaskAttemptEventSchedule(ta.getTaskAttemptID(), 2, 2)); NodeId nid = NodeId.newInstance("127.0.0.1", 0); ContainerId contId = ContainerId.newInstance(appAttemptId, 3); @@ -3652,10 +3652,10 @@ public void testVertexTaskAttemptProcessorFailure() throws Exception { containers.addContainerIfNew(container, 0, 0, 0); doReturn(containers).when(appContext).getAllContainers(); - ta.handle(new TaskAttemptEventSubmitted(ta.getID(), contId)); - ta.handle(new TaskAttemptEventStartedRemotely(ta.getID())); + ta.handle(new TaskAttemptEventSubmitted(ta.getTaskAttemptID(), contId)); + ta.handle(new TaskAttemptEventStartedRemotely(ta.getTaskAttemptID())); Assert.assertEquals(TaskAttemptStateInternal.RUNNING, ta.getInternalState()); - ta.handle(new TaskAttemptEventAttemptFailed(ta.getID(), TaskAttemptEventType.TA_FAILED, + ta.handle(new TaskAttemptEventAttemptFailed(ta.getTaskAttemptID(), TaskAttemptEventType.TA_FAILED, TaskFailureType.NON_FATAL, "diag", TaskAttemptTerminationCause.APPLICATION_ERROR)); dispatcher.await(); @@ -3673,7 +3673,7 @@ public void testVertexTaskAttemptInputFailure() throws Exception { startVertex(v); dispatcher.await(); TaskAttemptImpl ta = (TaskAttemptImpl) v.getTask(0).getAttempts().values().iterator().next(); - ta.handle(new TaskAttemptEventSchedule(ta.getID(), 2, 2)); + ta.handle(new TaskAttemptEventSchedule(ta.getTaskAttemptID(), 2, 2)); NodeId nid = NodeId.newInstance("127.0.0.1", 0); ContainerId contId = ContainerId.newInstance(appAttemptId, 3); @@ -3687,11 +3687,11 @@ public void testVertexTaskAttemptInputFailure() throws Exception { containers.addContainerIfNew(container, 0, 0, 0); doReturn(containers).when(appContext).getAllContainers(); - ta.handle(new TaskAttemptEventSubmitted(ta.getID(), contId)); - ta.handle(new TaskAttemptEventStartedRemotely(ta.getID())); + ta.handle(new TaskAttemptEventSubmitted(ta.getTaskAttemptID(), contId)); + ta.handle(new TaskAttemptEventStartedRemotely(ta.getTaskAttemptID())); Assert.assertEquals(TaskAttemptStateInternal.RUNNING, ta.getInternalState()); - ta.handle(new TaskAttemptEventAttemptFailed(ta.getID(), TaskAttemptEventType.TA_FAILED, + ta.handle(new TaskAttemptEventAttemptFailed(ta.getTaskAttemptID(), TaskAttemptEventType.TA_FAILED, TaskFailureType.NON_FATAL, "diag", TaskAttemptTerminationCause.INPUT_READ_ERROR)); dispatcher.await(); @@ -3710,7 +3710,7 @@ public void testVertexTaskAttemptOutputFailure() throws Exception { startVertex(v); dispatcher.await(); TaskAttemptImpl ta = (TaskAttemptImpl) v.getTask(0).getAttempts().values().iterator().next(); - ta.handle(new TaskAttemptEventSchedule(ta.getID(), 2, 2)); + ta.handle(new TaskAttemptEventSchedule(ta.getTaskAttemptID(), 2, 2)); NodeId nid = NodeId.newInstance("127.0.0.1", 0); ContainerId contId = ContainerId.newInstance(appAttemptId, 3); @@ -3724,11 +3724,11 @@ public void testVertexTaskAttemptOutputFailure() throws Exception { containers.addContainerIfNew(container, 0, 0, 0); doReturn(containers).when(appContext).getAllContainers(); - ta.handle(new TaskAttemptEventSubmitted(ta.getID(), contId)); - ta.handle(new TaskAttemptEventStartedRemotely(ta.getID())); + ta.handle(new TaskAttemptEventSubmitted(ta.getTaskAttemptID(), contId)); + ta.handle(new TaskAttemptEventStartedRemotely(ta.getTaskAttemptID())); Assert.assertEquals(TaskAttemptStateInternal.RUNNING, ta.getInternalState()); - ta.handle(new TaskAttemptEventAttemptFailed(ta.getID(), TaskAttemptEventType.TA_FAILED, + ta.handle(new TaskAttemptEventAttemptFailed(ta.getTaskAttemptID(), TaskAttemptEventType.TA_FAILED, TaskFailureType.NON_FATAL, "diag", TaskAttemptTerminationCause.OUTPUT_WRITE_ERROR)); dispatcher.await(); @@ -4800,7 +4800,7 @@ public void testInputInitializerEventsMultipleSources() throws Exception { Assert.assertEquals(2, v2.getTotalTasks()); // Generate events from v2 to v3's initializer. 1 from task 0, 2 from task 1 for (Task task : v2.getTasks().values()) { - TezTaskID taskId = task.getTaskId(); + TezTaskID taskId = task.getTaskID(); TezTaskAttemptID attemptId = TezTaskAttemptID.getInstance(taskId, 0); int numEventsFromTask = taskId.getId() + 1; for (int i = 0; i < numEventsFromTask; i++) { @@ -5499,7 +5499,7 @@ public void testVertexWithInitializerSuccess() throws Exception { for (int i=0; i()); //Vertex 1, Task 4, Attempt 1, host1 TezTaskAttemptID taID114 = TezTaskAttemptID.getInstance(TezTaskID.getInstance(vertexID11, 4), 1); TaskAttempt ta114 = mock(TaskAttempt.class); - doReturn(taID114).when(ta114).getID(); + doReturn(taID114).when(ta114).getTaskAttemptID(); doReturn("Mock for TA " + taID114.toString()).when(ta114).toString(); AMSchedulerEventTALaunchRequest lrEvent14 = createLaunchRequestEvent( taID114, ta114, resource1, host1, racks, priority1, new HashMap()); @@ -1261,7 +1261,7 @@ public void testReuseConflictLocalResources() throws IOException, InterruptedExc //Vertex 2, Task 1, Attempt 1, host1, lr2 TezTaskAttemptID taID211 = TezTaskAttemptID.getInstance(TezTaskID.getInstance(vertexID21, 1), 1); TaskAttempt ta211 = mock(TaskAttempt.class); - doReturn(taID211).when(ta211).getID(); + doReturn(taID211).when(ta211).getTaskAttemptID(); doReturn("Mock for TA " + taID211.toString()).when(ta211).toString(); AMSchedulerEventTALaunchRequest lrEvent21 = createLaunchRequestEvent(taID211, ta211, resource1, host1, racks, priority1, v21LR); @@ -1297,7 +1297,7 @@ public void testReuseConflictLocalResources() throws IOException, InterruptedExc TezTaskAttemptID taID311 = TezTaskAttemptID.getInstance( TezTaskID.getInstance(TezVertexID.getInstance(dagID3, 1), 1), 1); TaskAttempt ta311 = mock(TaskAttempt.class); - doReturn(taID311).when(ta311).getID(); + doReturn(taID311).when(ta311).getTaskAttemptID(); doReturn("Mock for TA " + taID311).when(ta311).toString(); AMSchedulerEventTALaunchRequest lrEvent31 = createLaunchRequestEvent(taID311, ta311, resource1, host1, racks, priority1, v31LR); diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerManager.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerManager.java index dcf9a5dd69..3ddd1861d4 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerManager.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerManager.java @@ -189,7 +189,7 @@ public void testSimpleAllocate() throws Exception { TaskAttemptImpl mockTaskAttempt = mock(TaskAttemptImpl.class); TezTaskAttemptID mockAttemptId = mock(TezTaskAttemptID.class); when(mockAttemptId.getId()).thenReturn(0); - when(mockTaskAttempt.getID()).thenReturn(mockAttemptId); + when(mockTaskAttempt.getTaskAttemptID()).thenReturn(mockAttemptId); Resource resource = Resource.newInstance(1024, 1); ContainerContext containerContext = new ContainerContext(new HashMap(), new Credentials(), @@ -228,7 +228,7 @@ public void testTASucceededAfterContainerCleanup() throws Exception { TaskAttemptImpl mockTaskAttempt = mock(TaskAttemptImpl.class); TezTaskAttemptID mockAttemptId = mock(TezTaskAttemptID.class); when(mockAttemptId.getId()).thenReturn(0); - when(mockTaskAttempt.getID()).thenReturn(mockAttemptId); + when(mockTaskAttempt.getTaskAttemptID()).thenReturn(mockAttemptId); Resource resource = Resource.newInstance(1024, 1); ContainerContext containerContext = new ContainerContext(new HashMap(), new Credentials(), @@ -268,7 +268,7 @@ public void testTAUnsuccessfulAfterContainerCleanup() throws Exception { TaskAttemptImpl mockTaskAttempt = mock(TaskAttemptImpl.class); TezTaskAttemptID mockAttemptId = mock(TezTaskAttemptID.class); when(mockAttemptId.getId()).thenReturn(0); - when(mockTaskAttempt.getID()).thenReturn(mockAttemptId); + when(mockTaskAttempt.getTaskAttemptID()).thenReturn(mockAttemptId); ContainerId mockCId = mock(ContainerId.class); Container container = mock(Container.class); diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/web/TestAMWebController.java b/tez-dag/src/test/java/org/apache/tez/dag/app/web/TestAMWebController.java index 16b391b55f..94c8df6233 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/web/TestAMWebController.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/web/TestAMWebController.java @@ -640,7 +640,7 @@ Map getTasksTestHelper(List tasks, List > t //Creating mock tasks and attaching to mock vertex Map taskMap = Maps.newHashMap(); for(Task task : tasks) { - TezTaskID taskId = task.getTaskId(); + TezTaskID taskId = task.getTaskID(); int taskIndex = taskId.getId(); doReturn(task).when(mockVertex).getTask(taskIndex); taskMap.put(taskId, task); @@ -688,7 +688,7 @@ private List createMockTasks() { private Task createMockTask(String taskIDStr, TaskState status, float progress) { Task mockTask = mock(Task.class); - doReturn(TezTaskID.fromString(taskIDStr)).when(mockTask).getTaskId(); + doReturn(TezTaskID.fromString(taskIDStr)).when(mockTask).getTaskID(); doReturn(status).when(mockTask).getState(); doReturn(progress).when(mockTask).getProgress(); @@ -711,7 +711,7 @@ private Task createMockTask(String taskIDStr, TaskState status, float progress) private void verifySingleTaskResult(Task mockTask, Map taskResult) { Assert.assertEquals(3, taskResult.size()); - Assert.assertEquals(mockTask.getTaskId().toString(), taskResult.get("id")); + Assert.assertEquals(mockTask.getTaskID().toString(), taskResult.get("id")); Assert.assertEquals(mockTask.getState().toString(), taskResult.get("status")); Assert.assertEquals(Float.toString(mockTask.getProgress()), taskResult.get("progress")); } @@ -777,7 +777,7 @@ Map getAttemptsTestHelper(List attempts, List getAttemptsTestHelper(List attempts, List attemptsMap = Maps.newHashMap(); for(TaskAttempt attempt : attempts) { - TezTaskAttemptID attemptId = attempt.getID(); + TezTaskAttemptID attemptId = attempt.getTaskAttemptID(); doReturn(attempt).when(mockTask).getAttempt(attemptId); attemptsMap.put(attemptId, attempt); } @@ -835,7 +835,7 @@ private List createMockAttempts() { private TaskAttempt createMockAttempt(String attemptIDStr, TaskAttemptState status, float progress) { TaskAttempt mockAttempt = mock(TaskAttempt.class); - doReturn(TezTaskAttemptID.fromString(attemptIDStr)).when(mockAttempt).getID(); + doReturn(TezTaskAttemptID.fromString(attemptIDStr)).when(mockAttempt).getTaskAttemptID(); doReturn(status).when(mockAttempt).getState(); doReturn(progress).when(mockAttempt).getProgress(); @@ -858,7 +858,7 @@ private TaskAttempt createMockAttempt(String attemptIDStr, TaskAttemptState stat private void verifySingleAttemptResult(TaskAttempt mockTask, Map taskResult) { Assert.assertEquals(3, taskResult.size()); - Assert.assertEquals(mockTask.getID().toString(), taskResult.get("id")); + Assert.assertEquals(mockTask.getTaskAttemptID().toString(), taskResult.get("id")); Assert.assertEquals(mockTask.getState().toString(), taskResult.get("status")); Assert.assertEquals(Float.toString(mockTask.getProgress()), taskResult.get("progress")); } diff --git a/tez-dag/src/test/java/org/apache/tez/dag/history/events/TestHistoryEventsProtoConversion.java b/tez-dag/src/test/java/org/apache/tez/dag/history/events/TestHistoryEventsProtoConversion.java index 50a80cb5f5..51b4bf9f92 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/history/events/TestHistoryEventsProtoConversion.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/history/events/TestHistoryEventsProtoConversion.java @@ -186,8 +186,8 @@ private void testDAGSubmittedEvent() throws Exception { testProtoConversion(event); Assert.assertEquals(event.getApplicationAttemptId(), deserializedEvent.getApplicationAttemptId()); - Assert.assertEquals(event.getDagID(), - deserializedEvent.getDagID()); + Assert.assertEquals(event.getDAGID(), + deserializedEvent.getDAGID()); Assert.assertEquals(event.getDAGName(), deserializedEvent.getDAGName()); Assert.assertEquals(event.getSubmitTime(), @@ -204,8 +204,8 @@ private void testDAGInitializedEvent() throws Exception { "user", "dagName", null); DAGInitializedEvent deserializedEvent = (DAGInitializedEvent) testProtoConversion(event); - Assert.assertEquals(event.getDagID(), - deserializedEvent.getDagID()); + Assert.assertEquals(event.getDAGID(), + deserializedEvent.getDAGID()); Assert.assertEquals(event.getInitTime(), deserializedEvent.getInitTime()); logEvents(event, deserializedEvent); } @@ -216,8 +216,8 @@ private void testDAGStartedEvent() throws Exception { "user", "dagName"); DAGStartedEvent deserializedEvent = (DAGStartedEvent) testProtoConversion(event); - Assert.assertEquals(event.getDagID(), - deserializedEvent.getDagID()); + Assert.assertEquals(event.getDAGID(), + deserializedEvent.getDAGID()); Assert.assertEquals(event.getStartTime(), deserializedEvent.getStartTime()); logEvents(event, deserializedEvent); } @@ -243,8 +243,8 @@ private void testDAGFinishedEvent() throws Exception { DAGFinishedEvent deserializedEvent = (DAGFinishedEvent) testProtoConversion(event); Assert.assertEquals( - event.getDagID(), - deserializedEvent.getDagID()); + event.getDAGID(), + deserializedEvent.getDAGID()); Assert.assertEquals(event.getState(), deserializedEvent.getState()); Assert.assertNotEquals(event.getStartTime(), deserializedEvent.getStartTime()); Assert.assertEquals(event.getFinishTime(), deserializedEvent.getFinishTime()); @@ -264,8 +264,8 @@ private void testDAGFinishedEvent() throws Exception { DAGFinishedEvent deserializedEvent = (DAGFinishedEvent) testProtoConversion(event); Assert.assertEquals( - event.getDagID(), - deserializedEvent.getDagID()); + event.getDAGID(), + deserializedEvent.getDAGID()); Assert.assertEquals(event.getState(), deserializedEvent.getState()); Assert.assertNotEquals(event.getStartTime(), deserializedEvent.getStartTime()); Assert.assertEquals(event.getFinishTime(), deserializedEvent.getFinishTime()); 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 eef73a0d0a..7864e1c852 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 @@ -556,12 +556,12 @@ private void checkAndThrowExceptionForTests(SubmitWorkRequestProto request) thro } TaskSpec taskSpec = ProtoConverters.getTaskSpecfromProto(request.getTaskSpec()); - if (taskSpec.getTaskAttemptID().getTaskID().getId() == 0 && + if (taskSpec.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 && + } else if (taskSpec.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=" + 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 0f1b56d3d2..2fddd00522 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 @@ -39,7 +39,7 @@ public static JobID toMRJobId(TezDAGID dagId) { public static TaskID toMRTaskId(TezTaskID taskid) { return new TaskID( - toMRJobId(taskid.getVertexID().getDAGId()), + toMRJobId(taskid.getDAGID()), taskid.getVertexID().getId() == 0 ? TaskType.MAP : TaskType.REDUCE, taskid.getId()); } diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapreduce/TaskAttemptContextImpl.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapreduce/TaskAttemptContextImpl.java index e5e7022064..941c8732bd 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapreduce/TaskAttemptContextImpl.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapreduce/TaskAttemptContextImpl.java @@ -53,8 +53,8 @@ public static org.apache.hadoop.mapred.TaskAttemptID createMockTaskAttemptID( public static org.apache.hadoop.mapred.TaskAttemptID createMockTaskAttemptIDFromTezTaskAttemptId(TezTaskAttemptID tezTaId, boolean isMap) { - TezVertexID vId = tezTaId.getTaskID().getVertexID(); - ApplicationId appId = vId.getDAGId().getApplicationId(); + TezVertexID vId = tezTaId.getVertexID(); + ApplicationId appId = vId.getApplicationId(); return new org.apache.hadoop.mapred.TaskAttemptID( new org.apache.hadoop.mapred.TaskID(String.valueOf(appId.getClusterTimestamp()) + String.valueOf(vId.getId()), appId.getId(), @@ -65,7 +65,7 @@ public static org.apache.hadoop.mapred.TaskAttemptID createMockTaskAttemptID( public static org.apache.hadoop.mapred.TaskID createMockTaskAttemptIDFromTezTaskId(TezTaskID tezTaId, boolean isMap) { TezVertexID vId = tezTaId.getVertexID(); - ApplicationId appId = vId.getDAGId().getApplicationId(); + ApplicationId appId = vId.getApplicationId(); return new org.apache.hadoop.mapred.TaskID(String.valueOf(appId.getClusterTimestamp()) + String.valueOf(vId.getId()), appId.getId(), isMap ? TaskType.MAP : TaskType.REDUCE, tezTaId.getId()); diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/MRTask.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/MRTask.java index 1a13168cb7..55e5709c3b 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/MRTask.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/MRTask.java @@ -36,7 +36,6 @@ import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.FileSystem.Statistics; import org.apache.hadoop.fs.LocalDirAllocator; import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.DataInputBuffer; @@ -321,8 +320,7 @@ public void initTask(LogicalOutput output) throws IOException, } this.mrReporter = new MRTaskReporter(processorContext); this.useNewApi = jobConf.getUseNewMapper(); - TezDAGID dagId = IDConverter.fromMRTaskAttemptId(taskAttemptId).getTaskID() - .getVertexID().getDAGId(); + TezDAGID dagId = IDConverter.fromMRTaskAttemptId(taskAttemptId).getDAGID(); this.jobContext = new JobContextImpl(jobConf, dagId, mrReporter); this.taskAttemptContext = diff --git a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/SimpleHistoryParser.java b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/SimpleHistoryParser.java index c1711ce2cb..b57f4f0e65 100644 --- a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/SimpleHistoryParser.java +++ b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/SimpleHistoryParser.java @@ -264,8 +264,8 @@ protected void readEventsFromSource(String dagId, JSONObjectSource source, case Constants.TEZ_VERTEX_ID: String vertexName = entity; TezVertexID tezVertexID = TezVertexID.fromString(vertexName); - if (!tezDAGID.equals(tezVertexID.getDAGId())) { - LOG.warn("{} does not belong to {} ('{}' != '{}')}", vertexName, tezDAGID, tezDAGID, tezVertexID.getDAGId()); + if (!tezDAGID.equals(tezVertexID.getDAGID())) { + LOG.warn("{} does not belong to {} ('{}' != '{}')}", vertexName, tezDAGID, tezDAGID, tezVertexID.getDAGID()); continue; } if (!vertexJsonMap.containsKey(vertexName)) { @@ -278,9 +278,9 @@ protected void readEventsFromSource(String dagId, JSONObjectSource source, case Constants.TEZ_TASK_ID: String taskName = entity; TezTaskID tezTaskID = TezTaskID.fromString(taskName); - if (!tezDAGID.equals(tezTaskID.getVertexID().getDAGId())) { + if (!tezDAGID.equals(tezTaskID.getDAGID())) { LOG.warn("{} does not belong to {} ('{}' != '{}')}", taskName, tezDAGID, tezDAGID, - tezTaskID.getVertexID().getDAGId()); + tezTaskID.getDAGID()); continue; } if (!taskJsonMap.containsKey(taskName)) { @@ -293,9 +293,9 @@ protected void readEventsFromSource(String dagId, JSONObjectSource source, case Constants.TEZ_TASK_ATTEMPT_ID: String taskAttemptName = entity; TezTaskAttemptID tezAttemptId = TezTaskAttemptID.fromString(taskAttemptName); - if (!tezDAGID.equals(tezAttemptId.getTaskID().getVertexID().getDAGId())) { + if (!tezDAGID.equals(tezAttemptId.getDAGID())) { LOG.warn("{} does not belong to {} ('{}' != '{}')}", taskAttemptName, tezDAGID, tezDAGID, - tezAttemptId.getTaskID().getVertexID().getDAGId()); + tezAttemptId.getDAGID()); continue; } if (!attemptJsonMap.containsKey(taskAttemptName)) { diff --git a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/BaseParser.java b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/BaseParser.java index af8e292b24..9f3881c8b2 100644 --- a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/BaseParser.java +++ b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/BaseParser.java @@ -90,9 +90,8 @@ protected void linkParsedContents() { //Link task to task attempt TezTaskAttemptID taskAttemptId = TezTaskAttemptID.fromString(attemptInfo .getTaskAttemptId()); - VertexInfo vertexInfo = dagInfo.getVertexFromId(taskAttemptId.getTaskID() - .getVertexID().toString()); - Preconditions.checkState(vertexInfo != null, "Vertex " + taskAttemptId.getTaskID() + VertexInfo vertexInfo = dagInfo.getVertexFromId(taskAttemptId.getVertexID().toString()); + Preconditions.checkState(vertexInfo != null, "Vertex " + taskAttemptId .getVertexID().toString() + " is not present in DAG"); TaskInfo taskInfo = vertexInfo.getTask(taskAttemptId.getTaskID().toString()); attemptInfo.setTaskInfo(taskInfo); diff --git a/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/HistoryEventProtoConverter.java b/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/HistoryEventProtoConverter.java index 09079bdf1c..f5f436eb9d 100644 --- a/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/HistoryEventProtoConverter.java +++ b/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/HistoryEventProtoConverter.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -150,7 +150,7 @@ private HistoryEventProto.Builder makeBuilderForEvent(HistoryEvent event, long t } if (vertexId != null) { builder.setVertexId(vertexId.toString()); - dagId = vertexId.getDAGId(); + dagId = vertexId.getDAGID(); } if (dagId != null) { builder.setDagId(dagId.toString()); @@ -240,7 +240,7 @@ private HistoryEventProto convertContainerStoppedEvent(ContainerStoppedEvent eve private HistoryEventProto convertDAGSubmittedEvent(DAGSubmittedEvent event) { HistoryEventProto.Builder builder = makeBuilderForEvent(event, event.getSubmitTime(), - event.getDagID(), null, event.getApplicationAttemptId(), null, null, null, + event.getDAGID(), null, event.getApplicationAttemptId(), null, null, null, event.getUser()); addEventData(builder, ATSConstants.DAG_NAME, event.getDAGName()); if (event.getDAGPlan().hasCallerContext() && @@ -267,11 +267,11 @@ private HistoryEventProto convertDAGSubmittedEvent(DAGSubmittedEvent event) { private HistoryEventProto convertDAGInitializedEvent(DAGInitializedEvent event) { HistoryEventProto.Builder builder = makeBuilderForEvent(event, event.getInitTime(), - event.getDagID(), null, null, null, null, null, event.getUser()); + event.getDAGID(), null, null, null, null, null, event.getUser()); addEventData(builder, ATSConstants.DAG_NAME, event.getDagName()); if (event.getVertexNameIDMap() != null) { - Map nameIdStrMap = new TreeMap(); + Map nameIdStrMap = new TreeMap<>(); for (Entry entry : event.getVertexNameIDMap().entrySet()) { nameIdStrMap.put(entry.getKey(), entry.getValue().toString()); } @@ -282,7 +282,7 @@ private HistoryEventProto convertDAGInitializedEvent(DAGInitializedEvent event) private HistoryEventProto convertDAGStartedEvent(DAGStartedEvent event) { HistoryEventProto.Builder builder = makeBuilderForEvent(event, event.getStartTime(), - event.getDagID(), null, null, null, null, null, event.getUser()); + event.getDAGID(), null, null, null, null, null, event.getUser()); addEventData(builder, ATSConstants.DAG_NAME, event.getDagName()); addEventData(builder, ATSConstants.STATUS, event.getDagState().name()); @@ -292,7 +292,7 @@ private HistoryEventProto convertDAGStartedEvent(DAGStartedEvent event) { private HistoryEventProto convertDAGFinishedEvent(DAGFinishedEvent event) { HistoryEventProto.Builder builder = makeBuilderForEvent(event, event.getFinishTime(), - event.getDagID(), null, event.getApplicationAttemptId(), null, null, null, + event.getDAGID(), null, event.getApplicationAttemptId(), null, null, null, event.getUser()); addEventData(builder, ATSConstants.DAG_NAME, event.getDagName()); if (event.getDAGPlan().hasCallerContext()) { @@ -473,7 +473,7 @@ private HistoryEventProto convertVertexReconfigureDoneEvent(VertexConfigurationD HistoryEventProto.Builder builder = makeBuilderForEvent(event, event.getReconfigureDoneTime(), null, null, null, event.getVertexID(), null, null, null); if (event.getSourceEdgeProperties() != null && !event.getSourceEdgeProperties().isEmpty()) { - Map updatedEdgeManagers = new HashMap(); + Map updatedEdgeManagers = new HashMap<>(); for (Entry entry : event.getSourceEdgeProperties().entrySet()) { updatedEdgeManagers.put(entry.getKey(), DAGUtils.convertEdgeProperty(entry.getValue())); } diff --git a/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/ProtoHistoryLoggingService.java b/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/ProtoHistoryLoggingService.java index 008b05d616..d675a659a0 100644 --- a/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/ProtoHistoryLoggingService.java +++ b/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/ProtoHistoryLoggingService.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -73,7 +73,7 @@ public ProtoHistoryLoggingService() { } @Override - protected void serviceInit(Configuration conf) throws Exception { + protected void serviceInit(Configuration conf) { LOG.info("Initing ProtoHistoryLoggingService"); setConfig(conf); loggingDisabled = !conf.getBoolean(TezConfiguration.TEZ_AM_HISTORY_LOGGING_ENABLED, @@ -146,7 +146,7 @@ private void loop() { } catch (InterruptedException e) { LOG.info("EventQueue poll interrupted, ignoring it.", e); } catch (IOException e) { - TezDAGID dagid = evt.getDagID(); + TezDAGID dagid = evt.getDAGID(); HistoryEventType type = evt.getHistoryEvent().getEventType(); // Retry is hard, because there are several places where this exception can happen // the state will get messed up a lot. @@ -160,7 +160,7 @@ private void handleEvent(DAGHistoryEvent event) throws IOException { return; } HistoryEvent historyEvent = event.getHistoryEvent(); - if (event.getDagID() == null) { + if (event.getDAGID() == null) { if (historyEvent.getEventType() == HistoryEventType.APP_LAUNCHED) { appEventsFile = appEventsWriter.getPath().toString(); appLaunchedEventOffset = appEventsWriter.getOffset(); @@ -168,7 +168,7 @@ private void handleEvent(DAGHistoryEvent event) throws IOException { appEventsWriter.writeProto(converter.convert(historyEvent)); } else { HistoryEventType type = historyEvent.getEventType(); - TezDAGID dagId = event.getDagID(); + TezDAGID dagId = event.getDAGID(); if (type == HistoryEventType.DAG_FINISHED) { finishCurrentDag((DAGFinishedEvent)historyEvent); } else if (type == HistoryEventType.DAG_SUBMITTED) { @@ -220,7 +220,7 @@ private void finishCurrentDag(DAGFinishedEvent event) throws IOException { .setAppLaunchedEventOffset(appLaunchedEventOffset) .setWriteTime(System.currentTimeMillis()); if (event != null) { - entry.setDagId(event.getDagID().toString()); + entry.setDagId(event.getDAGID().toString()); } manifestEventsWriter.writeProto(entry.build()); manifestEventsWriter.hflush(); diff --git a/tez-plugins/tez-yarn-timeline-cache-plugin/src/main/java/org/apache/tez/dag/history/logging/ats/TimelineCachePluginImpl.java b/tez-plugins/tez-yarn-timeline-cache-plugin/src/main/java/org/apache/tez/dag/history/logging/ats/TimelineCachePluginImpl.java index d211feb0e3..2ee2c21b79 100644 --- a/tez-plugins/tez-yarn-timeline-cache-plugin/src/main/java/org/apache/tez/dag/history/logging/ats/TimelineCachePluginImpl.java +++ b/tez-plugins/tez-yarn-timeline-cache-plugin/src/main/java/org/apache/tez/dag/history/logging/ats/TimelineCachePluginImpl.java @@ -94,18 +94,18 @@ private Set convertToTimelineEntityGroupIds(String entity } else if (entityType.equals(EntityTypes.TEZ_VERTEX_ID.name())) { TezVertexID vertexID = TezVertexID.fromString(entityId); if (vertexID != null) { - return createTimelineEntityGroupIds(vertexID.getDAGId()); + return createTimelineEntityGroupIds(vertexID.getDAGID()); } } else if (entityType.equals(EntityTypes.TEZ_TASK_ID.name())) { TezTaskID taskID = TezTaskID.fromString(entityId); if (taskID != null) { - return createTimelineEntityGroupIds(taskID.getVertexID().getDAGId()); + return createTimelineEntityGroupIds(taskID.getDAGID()); } } else if (entityType.equals(EntityTypes.TEZ_TASK_ATTEMPT_ID.name())) { TezTaskAttemptID taskAttemptID = TezTaskAttemptID.fromString(entityId); if (taskAttemptID != null) { - return createTimelineEntityGroupIds(taskAttemptID.getTaskID().getVertexID().getDAGId()); + return createTimelineEntityGroupIds(taskAttemptID.getDAGID()); } } else if (entityType.equals(EntityTypes.TEZ_CONTAINER_ID.name())) { String cId = entityId; diff --git a/tez-plugins/tez-yarn-timeline-history-with-acls/src/test/java/org/apache/tez/dag/history/ats/acls/TestATSHistoryWithACLs.java b/tez-plugins/tez-yarn-timeline-history-with-acls/src/test/java/org/apache/tez/dag/history/ats/acls/TestATSHistoryWithACLs.java index f79a78b9fa..f284fe4049 100644 --- a/tez-plugins/tez-yarn-timeline-history-with-acls/src/test/java/org/apache/tez/dag/history/ats/acls/TestATSHistoryWithACLs.java +++ b/tez-plugins/tez-yarn-timeline-history-with-acls/src/test/java/org/apache/tez/dag/history/ats/acls/TestATSHistoryWithACLs.java @@ -451,7 +451,7 @@ public void testDagLoggingDisabled() throws Exception { DAGHistoryEvent event = new DAGHistoryEvent(tezDAGID, submittedEvent); historyLoggingService.handle(new DAGHistoryEvent(tezDAGID, submittedEvent)); Thread.sleep(1000l); - String url = "http://" + timelineAddress + "/ws/v1/timeline/TEZ_DAG_ID/"+event.getDagID(); + String url = "http://" + timelineAddress + "/ws/v1/timeline/TEZ_DAG_ID/"+event.getDAGID(); Client client = new Client(); WebResource resource = client.resource(url); @@ -496,7 +496,7 @@ public void testDagLoggingEnabled() throws Exception { DAGHistoryEvent event = new DAGHistoryEvent(tezDAGID, submittedEvent); historyLoggingService.handle(new DAGHistoryEvent(tezDAGID, submittedEvent)); Thread.sleep(1000l); - String url = "http://" + timelineAddress + "/ws/v1/timeline/TEZ_DAG_ID/"+event.getDagID(); + String url = "http://" + timelineAddress + "/ws/v1/timeline/TEZ_DAG_ID/"+event.getDAGID(); Client client = new Client(); WebResource resource = client.resource(url); diff --git a/tez-plugins/tez-yarn-timeline-history-with-fs/src/main/java/org/apache/tez/dag/history/logging/ats/ATSV15HistoryLoggingService.java b/tez-plugins/tez-yarn-timeline-history-with-fs/src/main/java/org/apache/tez/dag/history/logging/ats/ATSV15HistoryLoggingService.java index a71f0d8db0..5068fb77db 100644 --- a/tez-plugins/tez-yarn-timeline-history-with-fs/src/main/java/org/apache/tez/dag/history/logging/ats/ATSV15HistoryLoggingService.java +++ b/tez-plugins/tez-yarn-timeline-history-with-fs/src/main/java/org/apache/tez/dag/history/logging/ats/ATSV15HistoryLoggingService.java @@ -311,9 +311,9 @@ public TimelineEntityGroupId getGroupId(DAGHistoryEvent event) { case VERTEX_GROUP_COMMIT_FINISHED: case DAG_RECOVERED: String entityGroupId = numDagsPerGroup > 1 - ? event.getDagID().getGroupId(numDagsPerGroup) - : event.getDagID().toString(); - return TimelineEntityGroupId.newInstance(event.getDagID().getApplicationId(), entityGroupId); + ? event.getDAGID().getGroupId(numDagsPerGroup) + : event.getDAGID().toString(); + return TimelineEntityGroupId.newInstance(event.getApplicationId(), entityGroupId); case APP_LAUNCHED: case AM_LAUNCHED: case AM_STARTED: @@ -333,7 +333,7 @@ public void handle(DAGHistoryEvent event) { private boolean isValidEvent(DAGHistoryEvent event) { HistoryEventType eventType = event.getHistoryEvent().getEventType(); - TezDAGID dagId = event.getDagID(); + TezDAGID dagId = event.getDAGID(); if (eventType.equals(HistoryEventType.DAG_SUBMITTED)) { DAGSubmittedEvent dagSubmittedEvent = @@ -373,7 +373,7 @@ private boolean isValidEvent(DAGHistoryEvent event) { private void handleEvents(DAGHistoryEvent event) { String domainId = getDomainForEvent(event); // skippedDags is updated in the above call so check again. - if (event.getDagID() != null && skippedDAGs.contains(event.getDagID())) { + if (event.getDAGID() != null && skippedDAGs.contains(event.getDAGID())) { return; } TimelineEntityGroupId groupId = getGroupId(event); @@ -417,7 +417,7 @@ private String getDomainForEvent(DAGHistoryEvent event) { return domainId; } - TezDAGID dagId = event.getDagID(); + TezDAGID dagId = event.getDAGID(); HistoryEvent historyEvent = event.getHistoryEvent(); if (dagId == null || !HistoryEventType.isDAGSpecificEvent(historyEvent.getEventType())) { return domainId; diff --git a/tez-plugins/tez-yarn-timeline-history/src/main/java/org/apache/tez/dag/history/logging/ats/ATSHistoryLoggingService.java b/tez-plugins/tez-yarn-timeline-history/src/main/java/org/apache/tez/dag/history/logging/ats/ATSHistoryLoggingService.java index 6d035cce2c..c1883a9216 100644 --- a/tez-plugins/tez-yarn-timeline-history/src/main/java/org/apache/tez/dag/history/logging/ats/ATSHistoryLoggingService.java +++ b/tez-plugins/tez-yarn-timeline-history/src/main/java/org/apache/tez/dag/history/logging/ats/ATSHistoryLoggingService.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -291,7 +291,7 @@ public void handle(DAGHistoryEvent event) { private boolean isValidEvent(DAGHistoryEvent event) { HistoryEventType eventType = event.getHistoryEvent().getEventType(); - TezDAGID dagId = event.getDagID(); + TezDAGID dagId = event.getDAGID(); if (eventType.equals(HistoryEventType.DAG_SUBMITTED)) { DAGSubmittedEvent dagSubmittedEvent = @@ -333,7 +333,7 @@ private void handleEvents(List events) { for (DAGHistoryEvent event : events) { String domainId = getDomainForEvent(event); // skippedDags is updated in the above call so check again. - if (event.getDagID() != null && skippedDAGs.contains(event.getDagID())) { + if (event.getDAGID() != null && skippedDAGs.contains(event.getDAGID())) { continue; } List eventEntities = HistoryEventTimelineConversion.convertToTimelineEntities( @@ -377,7 +377,7 @@ private String getDomainForEvent(DAGHistoryEvent event) { return domainId; } - TezDAGID dagId = event.getDagID(); + TezDAGID dagId = event.getDAGID(); HistoryEvent historyEvent = event.getHistoryEvent(); if (dagId == null || !HistoryEventType.isDAGSpecificEvent(historyEvent.getEventType())) { return domainId; diff --git a/tez-plugins/tez-yarn-timeline-history/src/main/java/org/apache/tez/dag/history/logging/ats/HistoryEventTimelineConversion.java b/tez-plugins/tez-yarn-timeline-history/src/main/java/org/apache/tez/dag/history/logging/ats/HistoryEventTimelineConversion.java index c40d3a8f78..b9457a0761 100644 --- a/tez-plugins/tez-yarn-timeline-history/src/main/java/org/apache/tez/dag/history/logging/ats/HistoryEventTimelineConversion.java +++ b/tez-plugins/tez-yarn-timeline-history/src/main/java/org/apache/tez/dag/history/logging/ats/HistoryEventTimelineConversion.java @@ -32,7 +32,6 @@ import org.apache.tez.dag.api.EdgeProperty; import org.apache.tez.dag.api.TezUncheckedException; import org.apache.tez.dag.api.oldrecords.TaskAttemptState; -import org.apache.tez.dag.api.oldrecords.TaskState; import org.apache.tez.dag.api.records.DAGProtos.CallerContextProto; import org.apache.tez.dag.app.web.AMWebController; import org.apache.tez.dag.history.HistoryEvent; @@ -288,7 +287,7 @@ private static TimelineEntity convertContainerStoppedEvent(ContainerStoppedEvent private static TimelineEntity convertDAGFinishedEvent(DAGFinishedEvent event) { TimelineEntity atsEntity = new TimelineEntity(); - atsEntity.setEntityId(event.getDagID().toString()); + atsEntity.setEntityId(event.getDAGID().toString()); atsEntity.setEntityType(EntityTypes.TEZ_DAG_ID.name()); TimelineEvent finishEvt = new TimelineEvent(); @@ -298,7 +297,7 @@ private static TimelineEntity convertDAGFinishedEvent(DAGFinishedEvent event) { atsEntity.addPrimaryFilter(ATSConstants.USER, event.getUser()); atsEntity.addPrimaryFilter(ATSConstants.APPLICATION_ID, - event.getDagID().getApplicationId().toString()); + event.getApplicationId().toString()); atsEntity.addPrimaryFilter(ATSConstants.DAG_NAME, event.getDagName()); atsEntity.addPrimaryFilter(ATSConstants.STATUS, event.getState().name()); if (event.getDAGPlan().hasCallerContext() @@ -327,10 +326,10 @@ private static TimelineEntity convertDAGFinishedEvent(DAGFinishedEvent event) { private static TimelineEntity convertDAGFinishedToDAGExtraInfoEntity(DAGFinishedEvent event) { TimelineEntity atsEntity = new TimelineEntity(); - atsEntity.setEntityId(event.getDagID().toString()); + atsEntity.setEntityId(event.getDAGID().toString()); atsEntity.setEntityType(EntityTypes.TEZ_DAG_EXTRA_INFO.name()); - atsEntity.addRelatedEntity(EntityTypes.TEZ_DAG_ID.name(), event.getDagID().toString()); + atsEntity.addRelatedEntity(EntityTypes.TEZ_DAG_ID.name(), event.getDAGID().toString()); TimelineEvent submitEvt = new TimelineEvent(); submitEvt.setEventType(HistoryEventType.DAG_FINISHED.name()); @@ -345,7 +344,7 @@ private static TimelineEntity convertDAGFinishedToDAGExtraInfoEntity(DAGFinished private static TimelineEntity convertDAGInitializedEvent(DAGInitializedEvent event) { TimelineEntity atsEntity = new TimelineEntity(); - atsEntity.setEntityId(event.getDagID().toString()); + atsEntity.setEntityId(event.getDAGID().toString()); atsEntity.setEntityType(EntityTypes.TEZ_DAG_ID.name()); TimelineEvent initEvt = new TimelineEvent(); @@ -355,13 +354,13 @@ private static TimelineEntity convertDAGInitializedEvent(DAGInitializedEvent eve atsEntity.addPrimaryFilter(ATSConstants.USER, event.getUser()); atsEntity.addPrimaryFilter(ATSConstants.APPLICATION_ID, - event.getDagID().getApplicationId().toString()); + event.getApplicationId().toString()); atsEntity.addPrimaryFilter(ATSConstants.DAG_NAME, event.getDagName()); atsEntity.addOtherInfo(ATSConstants.INIT_TIME, event.getInitTime()); if (event.getVertexNameIDMap() != null) { - Map nameIdStrMap = new TreeMap(); + Map nameIdStrMap = new TreeMap<>(); for (Entry entry : event.getVertexNameIDMap().entrySet()) { nameIdStrMap.put(entry.getKey(), entry.getValue().toString()); } @@ -373,7 +372,7 @@ private static TimelineEntity convertDAGInitializedEvent(DAGInitializedEvent eve private static TimelineEntity convertDAGStartedEvent(DAGStartedEvent event) { TimelineEntity atsEntity = new TimelineEntity(); - atsEntity.setEntityId(event.getDagID().toString()); + atsEntity.setEntityId(event.getDAGID().toString()); atsEntity.setEntityType(EntityTypes.TEZ_DAG_ID.name()); TimelineEvent startEvt = new TimelineEvent(); @@ -383,7 +382,7 @@ private static TimelineEntity convertDAGStartedEvent(DAGStartedEvent event) { atsEntity.addPrimaryFilter(ATSConstants.USER, event.getUser()); atsEntity.addPrimaryFilter(ATSConstants.APPLICATION_ID, - event.getDagID().getApplicationId().toString()); + event.getApplicationId().toString()); atsEntity.addPrimaryFilter(ATSConstants.DAG_NAME, event.getDagName()); atsEntity.addOtherInfo(ATSConstants.START_TIME, event.getStartTime()); @@ -394,7 +393,7 @@ private static TimelineEntity convertDAGStartedEvent(DAGStartedEvent event) { private static TimelineEntity convertDAGSubmittedEvent(DAGSubmittedEvent event) { TimelineEntity atsEntity = new TimelineEntity(); - atsEntity.setEntityId(event.getDagID().toString()); + atsEntity.setEntityId(event.getDAGID().toString()); atsEntity.setEntityType(EntityTypes.TEZ_DAG_ID.name()); atsEntity.addRelatedEntity(EntityTypes.TEZ_APPLICATION.name(), @@ -412,7 +411,7 @@ private static TimelineEntity convertDAGSubmittedEvent(DAGSubmittedEvent event) atsEntity.addPrimaryFilter(ATSConstants.USER, event.getUser()); atsEntity.addPrimaryFilter(ATSConstants.DAG_NAME, event.getDAGName()); atsEntity.addPrimaryFilter(ATSConstants.APPLICATION_ID, - event.getDagID().getApplicationId().toString()); + event.getApplicationId().toString()); if (event.getDAGPlan().hasCallerContext() && event.getDAGPlan().getCallerContext().hasCallerId()) { @@ -450,10 +449,10 @@ private static TimelineEntity convertDAGSubmittedEvent(DAGSubmittedEvent event) private static TimelineEntity convertDAGSubmittedToDAGExtraInfoEntity(DAGSubmittedEvent event) { TimelineEntity atsEntity = new TimelineEntity(); - atsEntity.setEntityId(event.getDagID().toString()); + atsEntity.setEntityId(event.getDAGID().toString()); atsEntity.setEntityType(EntityTypes.TEZ_DAG_EXTRA_INFO.name()); - atsEntity.addRelatedEntity(EntityTypes.TEZ_DAG_ID.name(), event.getDagID().toString()); + atsEntity.addRelatedEntity(EntityTypes.TEZ_DAG_ID.name(), event.getDAGID().toString()); TimelineEvent submitEvt = new TimelineEvent(); submitEvt.setEventType(HistoryEventType.DAG_SUBMITTED.name()); @@ -477,13 +476,13 @@ private static TimelineEntity convertTaskAttemptFinishedEvent(TaskAttemptFinishe atsEntity.setEntityType(EntityTypes.TEZ_TASK_ATTEMPT_ID.name()); atsEntity.addPrimaryFilter(ATSConstants.APPLICATION_ID, - event.getTaskAttemptID().getTaskID().getVertexID().getDAGId().getApplicationId().toString()); + event.getApplicationId().toString()); atsEntity.addPrimaryFilter(EntityTypes.TEZ_DAG_ID.name(), - event.getTaskAttemptID().getTaskID().getVertexID().getDAGId().toString()); + event.getDAGID().toString()); atsEntity.addPrimaryFilter(EntityTypes.TEZ_VERTEX_ID.name(), - event.getTaskAttemptID().getTaskID().getVertexID().toString()); + event.getVertexID().toString()); atsEntity.addPrimaryFilter(EntityTypes.TEZ_TASK_ID.name(), - event.getTaskAttemptID().getTaskID().toString()); + event.getTaskID().toString()); TimelineEvent finishEvt = new TimelineEvent(); finishEvt.setEventType(HistoryEventType.TASK_ATTEMPT_FINISHED.name()); @@ -542,16 +541,16 @@ private static TimelineEntity convertTaskAttemptStartedEvent(TaskAttemptStartedE atsEntity.setStartTime(event.getStartTime()); atsEntity.addRelatedEntity(EntityTypes.TEZ_TASK_ID.name(), - event.getTaskAttemptID().getTaskID().toString()); + event.getTaskID().toString()); atsEntity.addPrimaryFilter(ATSConstants.APPLICATION_ID, - event.getTaskAttemptID().getTaskID().getVertexID().getDAGId().getApplicationId().toString()); + event.getApplicationId().toString()); atsEntity.addPrimaryFilter(EntityTypes.TEZ_DAG_ID.name(), - event.getTaskAttemptID().getTaskID().getVertexID().getDAGId().toString()); + event.getDAGID().toString()); atsEntity.addPrimaryFilter(EntityTypes.TEZ_VERTEX_ID.name(), - event.getTaskAttemptID().getTaskID().getVertexID().toString()); + event.getVertexID().toString()); atsEntity.addPrimaryFilter(EntityTypes.TEZ_TASK_ID.name(), - event.getTaskAttemptID().getTaskID().toString()); + event.getTaskID().toString()); TimelineEvent startEvt = new TimelineEvent(); startEvt.setEventType(HistoryEventType.TASK_ATTEMPT_STARTED.name()); @@ -579,11 +578,11 @@ private static TimelineEntity convertTaskFinishedEvent(TaskFinishedEvent event) atsEntity.setEntityType(EntityTypes.TEZ_TASK_ID.name()); atsEntity.addPrimaryFilter(ATSConstants.APPLICATION_ID, - event.getTaskID().getVertexID().getDAGId().getApplicationId().toString()); + event.getApplicationId().toString()); atsEntity.addPrimaryFilter(EntityTypes.TEZ_DAG_ID.name(), - event.getTaskID().getVertexID().getDAGId().toString()); + event.getDAGID().toString()); atsEntity.addPrimaryFilter(EntityTypes.TEZ_VERTEX_ID.name(), - event.getTaskID().getVertexID().toString()); + event.getVertexID().toString()); TimelineEvent finishEvt = new TimelineEvent(); finishEvt.setEventType(HistoryEventType.TASK_FINISHED.name()); @@ -614,14 +613,14 @@ private static TimelineEntity convertTaskStartedEvent(TaskStartedEvent event) { atsEntity.setEntityType(EntityTypes.TEZ_TASK_ID.name()); atsEntity.addRelatedEntity(EntityTypes.TEZ_VERTEX_ID.name(), - event.getTaskID().getVertexID().toString()); + event.getVertexID().toString()); atsEntity.addPrimaryFilter(ATSConstants.APPLICATION_ID, - event.getTaskID().getVertexID().getDAGId().getApplicationId().toString()); + event.getApplicationId().toString()); atsEntity.addPrimaryFilter(EntityTypes.TEZ_DAG_ID.name(), - event.getTaskID().getVertexID().getDAGId().toString()); + event.getDAGID().toString()); atsEntity.addPrimaryFilter(EntityTypes.TEZ_VERTEX_ID.name(), - event.getTaskID().getVertexID().toString()); + event.getVertexID().toString()); TimelineEvent startEvt = new TimelineEvent(); startEvt.setEventType(HistoryEventType.TASK_STARTED.name()); @@ -643,9 +642,9 @@ private static TimelineEntity convertVertexFinishedEvent(VertexFinishedEvent eve atsEntity.setEntityType(EntityTypes.TEZ_VERTEX_ID.name()); atsEntity.addPrimaryFilter(ATSConstants.APPLICATION_ID, - event.getVertexID().getDAGId().getApplicationId().toString()); + event.getApplicationId().toString()); atsEntity.addPrimaryFilter(EntityTypes.TEZ_DAG_ID.name(), - event.getVertexID().getDAGId().toString()); + event.getDAGID().toString()); TimelineEvent finishEvt = new TimelineEvent(); finishEvt.setEventType(HistoryEventType.VERTEX_FINISHED.name()); @@ -685,12 +684,12 @@ private static TimelineEntity convertVertexInitializedEvent(VertexInitializedEve atsEntity.setEntityType(EntityTypes.TEZ_VERTEX_ID.name()); atsEntity.addRelatedEntity(EntityTypes.TEZ_DAG_ID.name(), - event.getVertexID().getDAGId().toString()); + event.getDAGID().toString()); atsEntity.addPrimaryFilter(ATSConstants.APPLICATION_ID, - event.getVertexID().getDAGId().getApplicationId().toString()); + event.getApplicationId().toString()); atsEntity.addPrimaryFilter(EntityTypes.TEZ_DAG_ID.name(), - event.getVertexID().getDAGId().toString()); + event.getDAGID().toString()); TimelineEvent initEvt = new TimelineEvent(); initEvt.setEventType(HistoryEventType.VERTEX_INITIALIZED.name()); @@ -718,9 +717,9 @@ private static TimelineEntity convertVertexStartedEvent(VertexStartedEvent event atsEntity.setEntityType(EntityTypes.TEZ_VERTEX_ID.name()); atsEntity.addPrimaryFilter(ATSConstants.APPLICATION_ID, - event.getVertexID().getDAGId().getApplicationId().toString()); + event.getApplicationId().toString()); atsEntity.addPrimaryFilter(EntityTypes.TEZ_DAG_ID.name(), - event.getVertexID().getDAGId().toString()); + event.getDAGID().toString()); TimelineEvent startEvt = new TimelineEvent(); startEvt.setEventType(HistoryEventType.VERTEX_STARTED.name()); @@ -741,17 +740,17 @@ private static TimelineEntity convertVertexReconfigureDoneEvent( atsEntity.setEntityType(EntityTypes.TEZ_VERTEX_ID.name()); atsEntity.addPrimaryFilter(ATSConstants.APPLICATION_ID, - event.getVertexID().getDAGId().getApplicationId().toString()); + event.getApplicationId().toString()); atsEntity.addPrimaryFilter(EntityTypes.TEZ_DAG_ID.name(), - event.getVertexID().getDAGId().toString()); + event.getDAGID().toString()); TimelineEvent updateEvt = new TimelineEvent(); updateEvt.setEventType(HistoryEventType.VERTEX_CONFIGURE_DONE.name()); updateEvt.setTimestamp(event.getReconfigureDoneTime()); - Map eventInfo = new HashMap(); + Map eventInfo = new HashMap<>(); if (event.getSourceEdgeProperties() != null && !event.getSourceEdgeProperties().isEmpty()) { - Map updatedEdgeManagers = new HashMap(); + Map updatedEdgeManagers = new HashMap<>(); for (Entry entry : event.getSourceEdgeProperties().entrySet()) { updatedEdgeManagers.put(entry.getKey(), diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/EventMetaData.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/EventMetaData.java index 0542b33d03..abe8c9dd53 100644 --- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/EventMetaData.java +++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/EventMetaData.java @@ -28,6 +28,7 @@ import javax.annotation.Nullable; import org.apache.hadoop.io.Writable; +import org.apache.tez.dag.records.TaskAttemptIDAware; import org.apache.tez.dag.records.TezTaskAttemptID; import org.apache.tez.util.StringInterner; @@ -35,7 +36,7 @@ * Class that encapsulates all the information to identify the unique * object that either generated an Event or is the recipient of an Event. */ -public class EventMetaData implements Writable { +public class EventMetaData implements Writable, TaskAttemptIDAware { public static enum EventProducerConsumerType { INPUT, @@ -83,6 +84,7 @@ public EventProducerConsumerType getEventGenerator() { return producerConsumerType; } + @Override public TezTaskAttemptID getTaskAttemptID() { return taskAttemptID; } diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TaskSpec.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TaskSpec.java index 63c251c054..15e756b853 100644 --- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TaskSpec.java +++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TaskSpec.java @@ -30,12 +30,13 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.io.Writable; import org.apache.tez.dag.api.ProcessorDescriptor; +import org.apache.tez.dag.records.TaskAttemptIDAware; import org.apache.tez.dag.records.TezTaskAttemptID; import org.apache.tez.util.StringInterner; import com.google.common.collect.Lists; -public class TaskSpec implements Writable { +public class TaskSpec implements Writable, TaskAttemptIDAware { private TezTaskAttemptID taskAttemptId; private String dagName; @@ -128,7 +129,7 @@ public String getDAGName() { } public int getDagIdentifier() { - return taskAttemptId.getTaskID().getVertexID().getDAGId().getId(); + return taskAttemptId.getDAGID().getId(); } public int getVertexParallelism() { @@ -139,6 +140,7 @@ public String getVertexName() { return vertexName; } + @Override public TezTaskAttemptID getTaskAttemptID() { return taskAttemptId; } @@ -266,5 +268,4 @@ public String toString() { } return sb.toString(); } - } diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezTaskContextImpl.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezTaskContextImpl.java index a47dac1e0a..b645346654 100644 --- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezTaskContextImpl.java +++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezTaskContextImpl.java @@ -118,8 +118,7 @@ public TezTaskContextImpl(Configuration conf, String[] workDirs, int appAttemptN @Override public ApplicationId getApplicationId() { - return taskAttemptID.getTaskID().getVertexID().getDAGId() - .getApplicationId(); + return taskAttemptID.getApplicationId(); } @Override @@ -154,12 +153,12 @@ public String getTaskVertexName() { @Override public int getTaskVertexIndex() { - return taskAttemptID.getTaskID().getVertexID().getId(); + return taskAttemptID.getVertexID().getId(); } @Override public int getDagIdentifier() { - return taskAttemptID.getTaskID().getVertexID().getDAGId().getId(); + return taskAttemptID.getDAGID().getId(); } @Override 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 c82355a9fa..bc8c2d8391 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 @@ -61,7 +61,6 @@ 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.TezConstants; import org.apache.tez.dag.api.TezException; import org.apache.tez.dag.api.records.DAGProtos; import org.apache.tez.dag.records.TezVertexID; @@ -372,13 +371,12 @@ public URI apply(TezLocalResource input) { private void cleanupOnTaskChanged(ContainerTask containerTask) { Preconditions.checkState(!containerTask.shouldDie()); Preconditions.checkState(containerTask.getTaskSpec() != null); - TezVertexID newVertexID = containerTask.getTaskSpec().getTaskAttemptID().getTaskID() - .getVertexID(); + TezVertexID newVertexID = containerTask.getTaskSpec().getTaskAttemptID().getVertexID(); if (lastVertexID != null) { if (!lastVertexID.equals(newVertexID)) { objectRegistry.clearCache(ObjectRegistryImpl.ObjectLifeCycle.VERTEX); } - if (!lastVertexID.getDAGId().equals(newVertexID.getDAGId())) { + if (!lastVertexID.getDAGID().equals(newVertexID.getDAGID())) { objectRegistry.clearCache(ObjectRegistryImpl.ObjectLifeCycle.DAG); startedInputsMap = HashMultimap.create(); } diff --git a/tez-tests/src/test/java/org/apache/tez/test/RecoveryServiceWithEventHandlingHook.java b/tez-tests/src/test/java/org/apache/tez/test/RecoveryServiceWithEventHandlingHook.java index 3ed386e346..186ab7e659 100644 --- a/tez-tests/src/test/java/org/apache/tez/test/RecoveryServiceWithEventHandlingHook.java +++ b/tez-tests/src/test/java/org/apache/tez/test/RecoveryServiceWithEventHandlingHook.java @@ -334,7 +334,7 @@ public boolean match(HistoryEvent incomingEvent) { TaskStartedEvent otherEvent = (TaskStartedEvent) incomingEvent; TaskStartedEvent conditionEvent = (TaskStartedEvent) event; // compare vertexId and taskId - return otherEvent.getTaskID().getVertexID().getId() == conditionEvent.getTaskID().getVertexID().getId() + return otherEvent.getVertexID().getId() == conditionEvent.getVertexID().getId() && otherEvent.getTaskID().getId() == conditionEvent.getTaskID().getId(); } break; @@ -344,7 +344,7 @@ public boolean match(HistoryEvent incomingEvent) { TaskFinishedEvent otherEvent = (TaskFinishedEvent) incomingEvent; TaskFinishedEvent conditionEvent = (TaskFinishedEvent) event; // compare vertexId and taskId - return otherEvent.getTaskID().getVertexID().getId() == conditionEvent.getTaskID().getVertexID().getId() + return otherEvent.getVertexID().getId() == conditionEvent.getVertexID().getId() && otherEvent.getTaskID().getId() == conditionEvent.getTaskID().getId(); } break; @@ -354,9 +354,9 @@ public boolean match(HistoryEvent incomingEvent) { TaskAttemptStartedEvent otherEvent = (TaskAttemptStartedEvent) incomingEvent; TaskAttemptStartedEvent conditionEvent = (TaskAttemptStartedEvent) event; // compare vertexId, taskId & taskAttemptId - return otherEvent.getTaskAttemptID().getTaskID().getVertexID().getId() - == conditionEvent.getTaskAttemptID().getTaskID().getVertexID().getId() - && otherEvent.getTaskAttemptID().getTaskID().getId() == conditionEvent.getTaskAttemptID().getTaskID().getId() + return otherEvent.getVertexID().getId() + == conditionEvent.getVertexID().getId() + && otherEvent.getTaskID().getId() == conditionEvent.getTaskID().getId() && otherEvent.getTaskAttemptID().getId() == conditionEvent.getTaskAttemptID().getId(); } break; @@ -366,9 +366,9 @@ public boolean match(HistoryEvent incomingEvent) { TaskAttemptFinishedEvent otherEvent = (TaskAttemptFinishedEvent) incomingEvent; TaskAttemptFinishedEvent conditionEvent = (TaskAttemptFinishedEvent) event; // compare vertexId, taskId & taskAttemptId - return otherEvent.getTaskAttemptID().getTaskID().getVertexID().getId() - == conditionEvent.getTaskAttemptID().getTaskID().getVertexID().getId() - && otherEvent.getTaskAttemptID().getTaskID().getId() == conditionEvent.getTaskAttemptID().getTaskID().getId() + return otherEvent.getVertexID().getId() + == conditionEvent.getVertexID().getId() + && otherEvent.getTaskID().getId() == conditionEvent.getTaskID().getId() && otherEvent.getTaskAttemptID().getId() == conditionEvent.getTaskAttemptID().getId(); } break; diff --git a/tez-tests/src/test/java/org/apache/tez/test/TestAMRecovery.java b/tez-tests/src/test/java/org/apache/tez/test/TestAMRecovery.java index 6d3ab1c711..7fe3b3acc1 100644 --- a/tez-tests/src/test/java/org/apache/tez/test/TestAMRecovery.java +++ b/tez-tests/src/test/java/org/apache/tez/test/TestAMRecovery.java @@ -463,9 +463,9 @@ private List findTaskAttemptFinishedEvent( if (historyEvent.getEventType() == HistoryEventType.TASK_ATTEMPT_FINISHED) { TaskAttemptFinishedEvent taFinishedEvent = (TaskAttemptFinishedEvent) historyEvent; - if (taFinishedEvent.getTaskAttemptID().getTaskID().getVertexID() + if (taFinishedEvent.getVertexID() .getId() == vertexId - && taFinishedEvent.getTaskAttemptID().getTaskID().getId() == taskId) { + && taFinishedEvent.getTaskID().getId() == taskId) { resultEvents.add(taFinishedEvent); } } From 9f4cb313980d91e0661fc3bdc568815f470b4bba Mon Sep 17 00:00:00 2001 From: ghanko <54805928+ghanko@users.noreply.github.com> Date: Mon, 14 Feb 2022 10:23:54 +0100 Subject: [PATCH 094/137] TEZ-4379 Tez Hive Queries query page rows does not go over (#183) (Csaba Juhasz, Gergely Hanko reviewed by Laszlo Bodor) --- tez-ui/src/main/webapp/app/routes/server-side-ops.js | 2 +- .../src/main/webapp/tests/unit/routes/server-side-ops-test.js | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/tez-ui/src/main/webapp/app/routes/server-side-ops.js b/tez-ui/src/main/webapp/app/routes/server-side-ops.js index 85b4d033c7..161b7b7498 100644 --- a/tez-ui/src/main/webapp/app/routes/server-side-ops.js +++ b/tez-ui/src/main/webapp/app/routes/server-side-ops.js @@ -74,7 +74,7 @@ export default AbstractRoute.extend({ return this.load(null, query).then(function (data) { if(that.get("controller.loadingMore")) { that.set("controller.loadingMore", false); - that.get("loadedValue").pushObjects(data); + that.get("loadedValue").pushObjects(data.content); return data; } }); diff --git a/tez-ui/src/main/webapp/tests/unit/routes/server-side-ops-test.js b/tez-ui/src/main/webapp/tests/unit/routes/server-side-ops-test.js index 59e04f75ba..055da433eb 100644 --- a/tez-ui/src/main/webapp/tests/unit/routes/server-side-ops-test.js +++ b/tez-ui/src/main/webapp/tests/unit/routes/server-side-ops-test.js @@ -136,7 +136,7 @@ test('loadNewPage test', function(assert) { let currentQuery = { val: {} }, - data = [], + data = {content: []}, fromId = "id1", route = this.subject({ controller: Ember.Object.create(), @@ -144,7 +144,7 @@ test('loadNewPage test', function(assert) { fromId: fromId, loadedValue: { pushObjects: function (objs) { - assert.equal(data, objs); + assert.equal(data.content, objs); } }, load: function (value, query) { From f62bf12cea38c9db0e56bd527a2b2a294aa10f66 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?L=C3=A1szl=C3=B3=20Attila=20T=C3=B3th?= Date: Tue, 15 Feb 2022 10:42:03 +0100 Subject: [PATCH 095/137] TEZ-4384: Remove unused EnvironmentUpdateUtils remaining from TEZ-1837 (#186) (Laszlo Attila Toth reviewed by Laszlo Bodor) --- .../tez/common/EnvironmentUpdateUtils.java | 127 ------------------ .../common/TestEnvironmentUpdateUtils.java | 96 ------------- 2 files changed, 223 deletions(-) delete mode 100644 tez-common/src/main/java/org/apache/tez/common/EnvironmentUpdateUtils.java delete mode 100644 tez-common/src/test/java/org/apache/tez/common/TestEnvironmentUpdateUtils.java diff --git a/tez-common/src/main/java/org/apache/tez/common/EnvironmentUpdateUtils.java b/tez-common/src/main/java/org/apache/tez/common/EnvironmentUpdateUtils.java deleted file mode 100644 index 0e597b3779..0000000000 --- a/tez-common/src/main/java/org/apache/tez/common/EnvironmentUpdateUtils.java +++ /dev/null @@ -1,127 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.tez.common; - -import org.apache.hadoop.classification.InterfaceAudience; -import org.apache.hadoop.util.Shell; - -import java.lang.reflect.Field; -import java.util.HashMap; -import java.util.Map; - -/** - * A utility class which allows one to dynamically update/change Environment variables - */ -@InterfaceAudience.Private -public class EnvironmentUpdateUtils { - - /** - * Allows dynamic update to the environment variables. After calling put, - * System.getenv(key) will then return value. - * - * @param key System environment variable - * @param value Value to assign to system environment variable - */ - public synchronized static void put(String key, String value){ - Map environment = new HashMap(System.getenv()); - environment.put(key, value); - if (!Shell.WINDOWS) { - updateEnvironment(environment); - } else { - updateEnvironmentOnWindows(environment); - } - } - - /** - * Allows dynamic update to a collection of environment variables. After - * calling putAll, System.getenv(key) will then return value for each entry - * in the map - * - * @param additionalEnvironment Collection where the key is the System - * environment variable and the value is the value to assign the system - * environment variable - */ - public synchronized static void putAll(Map additionalEnvironment) { - Map environment = new HashMap(System.getenv()); - environment.putAll(additionalEnvironment); - if (!Shell.WINDOWS) { - updateEnvironment(environment); - } else { - updateEnvironmentOnWindows(environment); - } - } - - /** - * Finds and modifies internal storage for system environment variables using - * reflection - * - * @param environment Collection where the key is the System - * environment variable and the value is the value to assign the system - * environment variable - */ - @SuppressWarnings("unchecked") - private static void updateEnvironment(Map environment) { - final Map currentEnv = System.getenv(); - copyMapValuesToPrivateField(currentEnv.getClass(), currentEnv, "m", environment); - } - - /** - * Finds and modifies internal storage for system environment variables using reflection. This - * method works only on windows. Note that the actual env is not modified, rather the copy of env - * which the JVM creates at the beginning of execution is. - * - * @param environment Collection where the key is the System - * environment variable and the value is the value to assign the system - * environment variable - */ - @SuppressWarnings("unchecked") - private static void updateEnvironmentOnWindows(Map environment) { - try { - Class processEnvironmentClass = Class.forName("java.lang.ProcessEnvironment"); - copyMapValuesToPrivateField(processEnvironmentClass, null, "theEnvironment", environment); - copyMapValuesToPrivateField(processEnvironmentClass, null, "theCaseInsensitiveEnvironment", - environment); - } catch (ClassNotFoundException e) { - throw new IllegalStateException("Failed to update Environment variables", e); - } - } - - /** - * Copies the given map values to the field specified by {@code fieldName} - * @param klass The {@code Class} of the object - * @param object The object to modify or null if the field is static - * @param fieldName The name of the field to set - * @param newMapValues The values to replace the current map. - */ - @SuppressWarnings("unchecked") - private static void copyMapValuesToPrivateField(Class klass, Object object, String fieldName, - Map newMapValues) { - try { - Field field = klass.getDeclaredField(fieldName); - field.setAccessible(true); - Map currentMap = (Map) field.get(object); - currentMap.clear(); - currentMap.putAll(newMapValues); - } catch (NoSuchFieldException e) { - throw new IllegalStateException("Failed to update Environment variables", e); - } catch (IllegalAccessException e) { - throw new IllegalStateException("Failed to update Environment variables", e); - } - } -} diff --git a/tez-common/src/test/java/org/apache/tez/common/TestEnvironmentUpdateUtils.java b/tez-common/src/test/java/org/apache/tez/common/TestEnvironmentUpdateUtils.java deleted file mode 100644 index a9cecc216a..0000000000 --- a/tez-common/src/test/java/org/apache/tez/common/TestEnvironmentUpdateUtils.java +++ /dev/null @@ -1,96 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.tez.common; - -import static org.junit.Assert.assertEquals; - -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.junit.Test; - -import java.util.ArrayList; -import java.util.List; -import java.util.concurrent.Callable; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.TimeUnit; - -public class TestEnvironmentUpdateUtils { - - @Test(timeout = 5000) - public void testMultipleUpdateEnvironment() { - EnvironmentUpdateUtils.put("test.environment1", "test.value1"); - EnvironmentUpdateUtils.put("test.environment2", "test.value2"); - assertEquals("Environment was not set propertly", "test.value1", System.getenv("test.environment1")); - assertEquals("Environment was not set propertly", "test.value2", System.getenv("test.environment2")); - } - - @Test(timeout = 5000) - public void testConcurrentRequests() throws InterruptedException { - int timeoutSecond = 5; - int concurThread = 10; - int exceptionCount = 0; - List> tasks = new ArrayList>(); - List> pendingTasks = new ArrayList>(); - final ExecutorService callbackExecutor = Executors.newFixedThreadPool(concurThread, - new ThreadFactoryBuilder().setDaemon(false).setNameFormat("CallbackExecutor").build()); - ListeningExecutorService taskExecutorService = - MoreExecutors.listeningDecorator(callbackExecutor); - while(concurThread > 0){ - ListenableFuture runningTaskFuture = - taskExecutorService.submit(new EnvironmentRequest()); - pendingTasks.add(runningTaskFuture); - concurThread--; - } - - //waiting for all threads submitted to thread pool - for (ListenableFuture future : pendingTasks) { - try { - future.get(); - } catch (ExecutionException e) { - exceptionCount++; - } - } - - //stop accepting new threads and shutdown threadpool - taskExecutorService.shutdown(); - try { - if(!taskExecutorService.awaitTermination(timeoutSecond, TimeUnit.SECONDS)) { - taskExecutorService.shutdownNow(); - } - } catch (InterruptedException ie) { - taskExecutorService.shutdownNow(); - } - - assertEquals(0, exceptionCount); - } - - private class EnvironmentRequest implements Callable { - - @Override - public Object call() throws Exception { - EnvironmentUpdateUtils.put("test.environment.concurrent" - +Thread.currentThread().getId(), "test.evironment.concurrent"); - return null; - } - } - } From a495fb69bb809007e5be507d321b9e5a876a952b Mon Sep 17 00:00:00 2001 From: Mark Bathori <57758037+mbathori-cloudera@users.noreply.github.com> Date: Fri, 18 Feb 2022 09:25:36 +0100 Subject: [PATCH 096/137] TEZ-4389: TestSecureShuffle fails if localhost name is different than "localhost" (#191) (Mark Bathori reviewed by Laszlo Bodor) --- .../test/java/org/apache/tez/test/TestSecureShuffle.java | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) 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 7857c1ca2f..f9d4469779 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 @@ -302,10 +302,15 @@ public static X509Certificate generateCertificate(String dn, KeyPair pair, int d KeyPair keyPair = pair; X509V3CertificateGenerator certGen = new X509V3CertificateGenerator(); + String hostName = InetAddress.getLocalHost().getHostName(); String hostAddress = InetAddress.getLocalHost().getHostAddress(); certGen.addExtension(X509Extensions.SubjectAlternativeName, false, - new GeneralNames(new GeneralName[] { new GeneralName(GeneralName.iPAddress, hostAddress), - new GeneralName(GeneralName.dNSName, "localhost") })); + new GeneralNames(new GeneralName[] { + new GeneralName(GeneralName.iPAddress, hostAddress), + new GeneralName(GeneralName.dNSName, hostName), + new GeneralName(GeneralName.dNSName, "localhost") + }) + ); X500Principal dnName = new X500Principal(dn); certGen.setSerialNumber(sn); From 5241f58a5945dfce87a5b4e7ec81caf0c6f021ac Mon Sep 17 00:00:00 2001 From: Bodor Laszlo Date: Fri, 18 Feb 2022 20:12:32 +0100 Subject: [PATCH 097/137] TEZ-4300: Download protoc automatically compile/development time (#115) --- BUILDING.txt | 6 +- build-tools/.gitignore | 2 + build-tools/install-protobuf.sh | 76 +++++++++++++++++-- build-tools/protocw | 32 ++++++++ pom.xml | 22 ++++++ tez-api/pom.xml | 22 ++++++ .../tez-protobuf-history-plugin/pom.xml | 15 +++- 7 files changed, 167 insertions(+), 8 deletions(-) create mode 100644 build-tools/.gitignore create mode 100755 build-tools/protocw diff --git a/BUILDING.txt b/BUILDING.txt index 875bf3e097..ae81d9c6c8 100644 --- a/BUILDING.txt +++ b/BUILDING.txt @@ -117,8 +117,10 @@ It's important to note that maven will still include tez-ui project, but all of ---------------------------------------------------------------------------------- Protocol Buffer compiler: -The version of Protocol Buffer compiler, protoc, must be 2.5.0 and match the -version of the protobuf JAR. +The version of Protocol Buffer compiler, protoc, can be defined on-the-fly as: + $ mvn clean install -DskipTests -pl ./tez-api -Dprotobuf.version=3.7.1 + +The default version is defined in the root pom.xml. If you have multiple versions of protoc in your system, you can set in your build shell the PROTOC_PATH environment variable to point to the one you diff --git a/build-tools/.gitignore b/build-tools/.gitignore new file mode 100644 index 0000000000..adfc42ea48 --- /dev/null +++ b/build-tools/.gitignore @@ -0,0 +1,2 @@ +protobuf + diff --git a/build-tools/install-protobuf.sh b/build-tools/install-protobuf.sh index 902049dab1..c28729a8a7 100755 --- a/build-tools/install-protobuf.sh +++ b/build-tools/install-protobuf.sh @@ -1,4 +1,4 @@ -#!/bin/sh +#!/bin/bash # Licensed to the Apache Software Foundation (ASF) under one # or more contributor license agreements. See the NOTICE file @@ -16,7 +16,73 @@ # See the License for the specific language governing permissions and # limitations under the License. -set -ex -wget https://github.com/google/protobuf/releases/download/v2.5.0/protobuf-2.5.0.tar.gz -tar -xzvf protobuf-2.5.0.tar.gz -cd protobuf-2.5.0 && ./configure --prefix=/usr && make && sudo make install +# This script attempts to install an arbitrary version of protobuf if needed. +# The desired version should be the first parameter: $1. +# Typical usage: the script is automatically called from tez-api (by maven) during the build process. + +# This script runs from build-tools folder. The user can remove +# the dynamically installed protobuf anytime like: +# rm -rf ./build-tools/protobuf/ #from root folder + +set -x +PROTOBUF_VERSION=${1:-2.5.0} +PROTOBUF_MAJOR_VERSION=$(echo "$PROTOBUF_VERSION" | cut -d. -f1) +if [ -n "$ZSH_VERSION" ]; then + SCRIPT_DIR="${0:a:h}" +else + SCRIPT_DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )" +fi + +function install_protobuf { + # before protobuf 3, there is no pre-compiled executables are host on github, let's try to build and make it + if (( PROTOBUF_MAJOR_VERSION < 3 )); then + wget "https://github.com/google/protobuf/releases/download/v$PROTOBUF_VERSION/protobuf-$PROTOBUF_VERSION.tar.gz" + tar -xzvf "protobuf-$PROTOBUF_VERSION.tar.gz" + rm "protobuf-$PROTOBUF_VERSION.tar.gz" + cd "protobuf-$PROTOBUF_VERSION" && ./configure --prefix=/usr && make && sudo make install + # since protobuf 3, there are precompiled protoc executables on github, let's quickly download and use it + else + ARCH=`uname -m` + case "$(uname -s)" in + Darwin) + FILE_NAME="protoc-$PROTOBUF_VERSION-osx-$ARCH" + ;; + Linux) + if test $ARCH = "aarch64"; then + ARCH="aarch_64" + fi + FILE_NAME="protoc-$PROTOBUF_VERSION-linux-$ARCH" + ;; + *) + echo "Unsupported OS returned by uname -s, you'll have to install protobuf 3.x manually" + exit 1 + ;; + esac + rm -f "$FILE_NAME.zip" #cleanup unfinished file if any + wget "https://github.com/google/protobuf/releases/download/v$PROTOBUF_VERSION/$FILE_NAME.zip" + mkdir "$SCRIPT_DIR/protobuf" + unzip -o "$FILE_NAME.zip" -d "$SCRIPT_DIR/protobuf" + rm "$FILE_NAME.zip" + fi +} + +if test -f "$SCRIPT_DIR/protobuf/bin/protoc"; then + PROTOBUF_INSTALLED_VERSION=$("$SCRIPT_DIR/protobuf/bin/protoc" --version) +else + PROTOBUF_INSTALLED_VERSION=$(protoc --version) +fi + +PROTOC_EXIT_CODE=$? + +if [ $PROTOC_EXIT_CODE -eq 0 ]; then + PROTOBUF_INSTALLED_VERSION=$(echo "$PROTOBUF_INSTALLED_VERSION" | tr -s ' ' | cut -d ' ' -f 2) + if [ "$PROTOBUF_INSTALLED_VERSION" == "$PROTOBUF_VERSION" ]; then + echo "Current protobuf version is equal to the requested ($PROTOBUF_INSTALLED_VERSION), exiting..." + else + echo "Current protobuf version ($PROTOBUF_INSTALLED_VERSION) is not equal to the requested ($PROTOBUF_VERSION), installing $PROTOBUF_VERSION" + install_protobuf + fi +else + echo "protoc --version command had non-zero return value, need to install probuf" + install_protobuf +fi diff --git a/build-tools/protocw b/build-tools/protocw new file mode 100755 index 0000000000..6196071815 --- /dev/null +++ b/build-tools/protocw @@ -0,0 +1,32 @@ +#!/bin/bash + +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +### This is a protoc wrapper for tez, which can dinamically call protoc from a downloaded protobuf. + +if [ -n "$ZSH_VERSION" ]; then + SCRIPT_DIR="${0:a:h}" +else + SCRIPT_DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )" +fi + +if test -f "$SCRIPT_DIR/protobuf/bin/protoc"; then + "$SCRIPT_DIR/protobuf/bin/protoc" "$@" +else + protoc "$@" +fi +exit $? diff --git a/pom.xml b/pom.xml index aff586bad9..9499fe62b6 100644 --- a/pom.xml +++ b/pom.xml @@ -1310,6 +1310,28 @@ + + + protoc-path-env-variable-not-defined + + + !env.PROTOC_PATH + + + + ${basedir}/../build-tools/protocw + + diff --git a/tez-api/pom.xml b/tez-api/pom.xml index 8cab40c172..4e1825b09d 100644 --- a/tez-api/pom.xml +++ b/tez-api/pom.xml @@ -145,6 +145,28 @@ org.apache.rat apache-rat-plugin + + + exec-maven-plugin + org.codehaus.mojo + 1.6.0 + + + Install protobuf + initialize + + exec + + + ${basedir}/../build-tools/install-protobuf.sh + ${protobuf.version} + + + + org.apache.hadoop hadoop-maven-plugins diff --git a/tez-plugins/tez-protobuf-history-plugin/pom.xml b/tez-plugins/tez-protobuf-history-plugin/pom.xml index ab06981f3d..9d82ed3965 100644 --- a/tez-plugins/tez-protobuf-history-plugin/pom.xml +++ b/tez-plugins/tez-protobuf-history-plugin/pom.xml @@ -85,8 +85,21 @@ - + + + + protoc-path-env-variable-not-defined + + + !env.PROTOC_PATH + + + + ${basedir}/../../build-tools/protocw + + + From fa74b95876e1074b5ba2eec6ddb0eea2699c9149 Mon Sep 17 00:00:00 2001 From: Bodor Laszlo Date: Tue, 22 Feb 2022 10:32:30 +0100 Subject: [PATCH 098/137] TEZ-4388: Asynchttpclient can cause stuck TezChild processes (#189) (Laszlo Bodor reviewed by Rajesh Balamohan) --- .../tez/util/TezRuntimeShutdownHandler.java | 44 +++++++++++++++++++ .../org/apache/tez/runtime/task/TezChild.java | 10 ++++- .../http/async/netty/AsyncHttpConnection.java | 11 +++++ 3 files changed, 63 insertions(+), 2 deletions(-) create mode 100644 tez-common/src/main/java/org/apache/tez/util/TezRuntimeShutdownHandler.java diff --git a/tez-common/src/main/java/org/apache/tez/util/TezRuntimeShutdownHandler.java b/tez-common/src/main/java/org/apache/tez/util/TezRuntimeShutdownHandler.java new file mode 100644 index 0000000000..4881e08ab9 --- /dev/null +++ b/tez-common/src/main/java/org/apache/tez/util/TezRuntimeShutdownHandler.java @@ -0,0 +1,44 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.tez.util; + +import java.util.ArrayList; +import java.util.List; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public final class TezRuntimeShutdownHandler { + private static final Logger LOG = LoggerFactory.getLogger(TezRuntimeShutdownHandler.class); + + private static final List shutdownTasks = new ArrayList<>(); + + private TezRuntimeShutdownHandler() { + } + + public static void addShutdownTask(Runnable r) { + shutdownTasks.add(r); + } + + public static synchronized void shutdown() { + LOG.info("Handling {} shutdown tasks", shutdownTasks.size()); + for (Runnable shutdownTask : shutdownTasks) { + shutdownTask.run(); + } + } +} 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 bc8c2d8391..b89b12db2b 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 @@ -71,6 +71,7 @@ import org.apache.tez.runtime.api.impl.ExecutionContextImpl; import org.apache.tez.runtime.common.objectregistry.ObjectRegistryImpl; import org.apache.tez.runtime.internals.api.TaskReporterInterface; +import org.apache.tez.util.TezRuntimeShutdownHandler; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -390,8 +391,10 @@ public void shutdown() { LOG.info("Shutting down container {}", containerIdString); // It's possible that there's pending tasks on the executor. Those should be cancelled. List pendingRunnables = executor.shutdownNow(); + LOG.info("There are {} runnables in shared executor, cancelling those...", pendingRunnables.size()); for (Runnable r : pendingRunnables) { - LOG.info("Cancelling pending runnables during TezChild shutdown for containerId={}", containerIdString); + LOG.info("Cancelling pending runnable ({}) during TezChild shutdown for containerId={}", r.hashCode(), + containerIdString); ((FutureTask)r).cancel(false); } if (taskReporter != null) { @@ -401,6 +404,8 @@ public void shutdown() { RPC.stopProxy(umbilical); } } + TezRuntimeShutdownHandler.shutdown(); + LOG.info("TezChild shutdown finished"); } public static class ContainerExecutionResult { @@ -522,7 +527,8 @@ public static void main(String[] args) throws IOException, InterruptedException, System.getenv(), pid, new ExecutionContextImpl(System.getenv(Environment.NM_HOST.name())), credentials, Runtime.getRuntime().maxMemory(), System .getenv(ApplicationConstants.Environment.USER.toString()), null, true, hadoopShim); - tezChild.run(); + ContainerExecutionResult result = tezChild.run(); + LOG.info("TezChild is about to exit from main(), run() returned result: {}", result.toString()); } private void handleError(Throwable t) { 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 index 63b8934821..215e63af58 100644 --- 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 @@ -35,6 +35,7 @@ import org.apache.tez.runtime.library.common.security.SecureShuffleUtils; import org.apache.tez.runtime.library.common.shuffle.orderedgrouped.ShuffleHeader; import org.apache.tez.util.StopWatch; +import org.apache.tez.util.TezRuntimeShutdownHandler; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -102,6 +103,16 @@ private void initClient(HttpConnectionParams httpConnParams) throws IOException .build(); DefaultAsyncHttpClientConfig config = builder.build(); httpAsyncClient = new DefaultAsyncHttpClient(config); + TezRuntimeShutdownHandler.addShutdownTask(() -> { + try { + if (httpAsyncClient != null) { + httpAsyncClient.close(); + httpAsyncClient = null; + } + } catch (IOException e) { + LOG.warn("Error while closing async client (this won't block shutdown)", e); + } + }); } } } From 132ea4c1b3505a066d300173bd967d1bc04b142f Mon Sep 17 00:00:00 2001 From: Mark Bathori <57758037+mbathori-cloudera@users.noreply.github.com> Date: Thu, 10 Mar 2022 20:54:24 +0100 Subject: [PATCH 099/137] TEZ-4359: ShuffleHandler: Make sure of properly releasing netty reference counted objects (#194) (Mark Bathori reviewed by Laszlo Bodor) --- .../java/org/apache/tez/auxservices/ShuffleHandler.java | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java b/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java index 9730be1869..7e6fd750f8 100644 --- a/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java +++ b/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java @@ -1499,7 +1499,6 @@ protected void sendError(ChannelHandlerContext ctx, HttpResponseStatus status) { protected void sendError(ChannelHandlerContext ctx, String message, HttpResponseStatus status) { FullHttpResponse response = new DefaultFullHttpResponse(HTTP_1_1, status); sendError(ctx, message, response); - response.release(); } protected void sendError(ChannelHandlerContext ctx, String message, FullHttpResponse response) { @@ -1517,7 +1516,6 @@ private void sendFakeShuffleHeaderWithError(ChannelHandlerContext ctx, String me header.write(out); sendError(ctx, wrappedBuffer(out.getData(), 0, out.getLength()), fullResponse); - fullResponse.release(); } protected void sendError(ChannelHandlerContext ctx, ByteBuf content, @@ -1532,6 +1530,11 @@ protected void sendError(ChannelHandlerContext ctx, ByteBuf content, // Close the connection as soon as the error message is sent. ctx.channel().writeAndFlush(response).addListener(ChannelFutureListener.CLOSE); + /* + * The general rule of thumb is that the party that accesses a reference-counted object last + * is also responsible for the destruction of that reference-counted object. + */ + content.release(); } @Override From 20873a3a1008d3572f0d0cfefe1c146f8f42449e Mon Sep 17 00:00:00 2001 From: Syed Shameerur Rahman Date: Wed, 16 Mar 2022 23:48:39 +0530 Subject: [PATCH 100/137] TEZ-3363: Delete intermediate data at the vertex level for Shuffle Handler (#60) (Syed Shameerur Rahman reviewed by Laszlo Bodor) --- .../apache/tez/dag/api/TezConfiguration.java | 20 +- .../tez/common/DagContainerLauncher.java | 6 + .../org/apache/tez/dag/app/DAGAppMaster.java | 4 + .../dag/app/dag/event/VertexEventType.java | 1 + .../dag/event/VertexShuffleDataDeletion.java | 43 ++ .../apache/tez/dag/app/dag/impl/DAGImpl.java | 8 + .../tez/dag/app/dag/impl/VertexImpl.java | 72 ++- .../VertexShuffleDataDeletionContext.java | 96 ++++ .../launcher/ContainerLauncherManager.java | 8 + .../launcher/ContainerLauncherWrapper.java | 9 + .../tez/dag/app/launcher/DeletionTracker.java | 7 + .../dag/app/launcher/DeletionTrackerImpl.java | 23 + .../app/launcher/LocalContainerLauncher.java | 15 +- .../launcher/TezContainerLauncherImpl.java | 16 +- .../app/launcher/VertexDeleteRunnable.java | 82 +++ .../tez/dag/app/dag/impl/TestVertexImpl.java | 529 ++++++++++++++++-- .../TestContainerLauncherWrapper.java | 2 +- .../tez/auxservices/ShuffleHandler.java | 48 ++ .../tez/auxservices/TestShuffleHandler.java | 78 +++ .../library/common/TezRuntimeUtils.java | 19 + 20 files changed, 1020 insertions(+), 66 deletions(-) create mode 100644 tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexShuffleDataDeletion.java create mode 100644 tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexShuffleDataDeletionContext.java create mode 100644 tez-dag/src/main/java/org/apache/tez/dag/app/launcher/VertexDeleteRunnable.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 16d1dfcb70..71ebfee10b 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 @@ -883,6 +883,22 @@ public TezConfiguration(boolean loadDefaults) { + "dag.cleanup.on.completion"; public static final boolean TEZ_AM_DAG_CLEANUP_ON_COMPLETION_DEFAULT = false; + /** + * Integer value. Instructs AM to delete vertex shuffle data if a vertex and all its + * child vertices at a certain depth are completed. Value less than or equal to 0 indicates the feature + * is disabled. + * Let's say we have a dag Map1 - Reduce2 - Reduce3 - Reduce4. + * case:1 height = 1 + * when Reduce 2 completes all the shuffle data of Map1 will be deleted and so on for other vertex. + * case: 2 height = 2 + * when Reduce 3 completes all the shuffle data of Map1 will be deleted and so on for other vertex. + */ + @ConfigurationScope(Scope.AM) + @ConfigurationProperty(type="integer") + public static final String TEZ_AM_VERTEX_CLEANUP_HEIGHT = TEZ_AM_PREFIX + + "vertex.cleanup.height"; + public static final int TEZ_AM_VERTEX_CLEANUP_HEIGHT_DEFAULT = 0; + /** * Boolean value. Instructs AM to delete intermediate attempt data for failed task attempts. */ @@ -893,8 +909,8 @@ public TezConfiguration(boolean loadDefaults) { public static final boolean TEZ_AM_TASK_ATTEMPT_CLEANUP_ON_FAILURE_DEFAULT = false; /** - * Int value. Upper limit on the number of threads used to delete DAG directories and failed task attempts - * directories on nodes. + * Int value. Upper limit on the number of threads used to delete DAG directories, + * Vertex directories and failed task attempts directories on nodes. */ @ConfigurationScope(Scope.AM) @ConfigurationProperty(type="integer") diff --git a/tez-common/src/main/java/org/apache/tez/common/DagContainerLauncher.java b/tez-common/src/main/java/org/apache/tez/common/DagContainerLauncher.java index 6bda0a8da3..c2337af77e 100644 --- a/tez-common/src/main/java/org/apache/tez/common/DagContainerLauncher.java +++ b/tez-common/src/main/java/org/apache/tez/common/DagContainerLauncher.java @@ -24,9 +24,12 @@ import org.apache.tez.common.security.JobTokenSecretManager; import org.apache.tez.dag.records.TezDAGID; import org.apache.tez.dag.records.TezTaskAttemptID; +import org.apache.tez.dag.records.TezVertexID; import org.apache.tez.serviceplugins.api.ContainerLauncher; import org.apache.tez.serviceplugins.api.ContainerLauncherContext; +import java.util.Set; + /** * Plugin to allow custom container launchers to be written to launch containers that want to * support cleanup of DAG level directories upon DAG completion in session mode. The directories are created by @@ -43,6 +46,9 @@ public DagContainerLauncher(ContainerLauncherContext containerLauncherContext) { public abstract void dagComplete(TezDAGID dag, JobTokenSecretManager jobTokenSecretManager); + public abstract void vertexComplete(TezVertexID vertex, JobTokenSecretManager jobTokenSecretManager, + Set nodeIdList); + public abstract void taskAttemptFailed(TezTaskAttemptID taskAttemptID, JobTokenSecretManager jobTokenSecretManager, NodeId nodeId); } 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 972fadf854..58288612bb 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 @@ -2739,6 +2739,10 @@ String buildPluginComponentLog(List namedEntityDescriptor return sb.toString(); } + public void vertexComplete(TezVertexID completedVertexID, Set nodesList) { + getContainerLauncherManager().vertexComplete(completedVertexID, jobTokenSecretManager, nodesList); + } + public void taskAttemptFailed(TezTaskAttemptID attemptID, NodeId nodeId) { getContainerLauncherManager().taskAttemptFailed(attemptID, jobTokenSecretManager, nodeId); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventType.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventType.java index 15be94dfa1..ed325290cb 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventType.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventType.java @@ -34,6 +34,7 @@ public enum VertexEventType { V_START, V_SOURCE_TASK_ATTEMPT_COMPLETED, V_SOURCE_VERTEX_STARTED, + V_DELETE_SHUFFLE_DATA, //Producer:Task V_TASK_COMPLETED, diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexShuffleDataDeletion.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexShuffleDataDeletion.java new file mode 100644 index 0000000000..8ea3a154b3 --- /dev/null +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexShuffleDataDeletion.java @@ -0,0 +1,43 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tez.dag.app.dag.event; + +import org.apache.tez.dag.app.dag.Vertex; + + +public class VertexShuffleDataDeletion extends VertexEvent { + // child vertex + private Vertex sourceVertex; + // parent vertex + private Vertex targetVertex; + + public VertexShuffleDataDeletion(Vertex sourceVertex, Vertex targetVertex) { + super(targetVertex.getVertexId(), VertexEventType.V_DELETE_SHUFFLE_DATA); + this.sourceVertex = sourceVertex; + this.targetVertex = targetVertex; + } + + public Vertex getSourceVertex() { + return sourceVertex; + } + + public Vertex getTargetVertex() { + return targetVertex; + } +} \ No newline at end of file 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 c9337e473a..aa28e02441 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 @@ -51,6 +51,7 @@ import org.apache.tez.common.counters.LimitExceededException; import org.apache.tez.dag.app.dag.event.DAGEventTerminateDag; import org.apache.tez.dag.app.dag.event.DiagnosableEvent; +import org.apache.tez.runtime.library.common.shuffle.ShuffleUtils; import org.apache.tez.state.OnStateChangedCallback; import org.apache.tez.state.StateMachineTez; import org.slf4j.Logger; @@ -1772,6 +1773,13 @@ private static void parseVertexEdges(DAGImpl dag, Map edgePlan vertex.setInputVertices(inVertices); vertex.setOutputVertices(outVertices); + boolean cleanupShuffleDataAtVertexLevel = dag.dagConf.getInt(TezConfiguration.TEZ_AM_VERTEX_CLEANUP_HEIGHT, + TezConfiguration.TEZ_AM_VERTEX_CLEANUP_HEIGHT_DEFAULT) > 0 && ShuffleUtils.isTezShuffleHandler(dag.dagConf); + if (cleanupShuffleDataAtVertexLevel) { + int deletionHeight = dag.dagConf.getInt(TezConfiguration.TEZ_AM_VERTEX_CLEANUP_HEIGHT, + TezConfiguration.TEZ_AM_VERTEX_CLEANUP_HEIGHT_DEFAULT); + ((VertexImpl) vertex).initShuffleDeletionContext(deletionHeight); + } } /** 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 934dd4e76e..e55b10a659 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 @@ -114,6 +114,8 @@ import org.apache.tez.dag.app.dag.RootInputInitializerManager; import org.apache.tez.dag.app.dag.StateChangeNotifier; import org.apache.tez.dag.app.dag.Task; +import org.apache.tez.dag.app.dag.TaskAttempt; +import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.tez.dag.app.dag.TaskAttemptStateInternal; import org.apache.tez.dag.app.dag.TaskTerminationCause; import org.apache.tez.dag.app.dag.Vertex; @@ -130,6 +132,7 @@ import org.apache.tez.dag.app.dag.event.TaskEventScheduleTask; import org.apache.tez.dag.app.dag.event.TaskEventTermination; import org.apache.tez.dag.app.dag.event.TaskEventType; +import org.apache.tez.dag.app.dag.event.VertexShuffleDataDeletion; import org.apache.tez.dag.app.dag.event.VertexEvent; import org.apache.tez.dag.app.dag.event.VertexEventCommitCompleted; import org.apache.tez.dag.app.dag.event.VertexEventInputDataInformation; @@ -187,6 +190,7 @@ 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.library.common.shuffle.ShuffleUtils; import org.apache.tez.state.OnStateChangedCallback; import org.apache.tez.state.StateMachineTez; import org.apache.tez.util.StringInterner; @@ -556,7 +560,8 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex, EventHandl VertexEventType.V_ROUTE_EVENT, VertexEventType.V_SOURCE_TASK_ATTEMPT_COMPLETED, VertexEventType.V_TASK_ATTEMPT_COMPLETED, - VertexEventType.V_TASK_RESCHEDULED)) + VertexEventType.V_TASK_RESCHEDULED, + VertexEventType.V_DELETE_SHUFFLE_DATA)) // Transitions from SUCCEEDED state .addTransition( @@ -592,6 +597,9 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex, EventHandl .addTransition(VertexState.SUCCEEDED, VertexState.SUCCEEDED, VertexEventType.V_TASK_ATTEMPT_COMPLETED, new TaskAttemptCompletedEventTransition()) + .addTransition(VertexState.SUCCEEDED, VertexState.SUCCEEDED, + VertexEventType.V_DELETE_SHUFFLE_DATA, + new VertexShuffleDeleteTransition()) // Transitions from FAILED state @@ -613,7 +621,8 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex, EventHandl VertexEventType.V_ROOT_INPUT_INITIALIZED, VertexEventType.V_SOURCE_TASK_ATTEMPT_COMPLETED, VertexEventType.V_NULL_EDGE_INITIALIZED, - VertexEventType.V_INPUT_DATA_INFORMATION)) + VertexEventType.V_INPUT_DATA_INFORMATION, + VertexEventType.V_DELETE_SHUFFLE_DATA)) // Transitions from KILLED state .addTransition( @@ -635,7 +644,8 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex, EventHandl VertexEventType.V_TASK_COMPLETED, VertexEventType.V_ROOT_INPUT_INITIALIZED, VertexEventType.V_NULL_EDGE_INITIALIZED, - VertexEventType.V_INPUT_DATA_INFORMATION)) + VertexEventType.V_INPUT_DATA_INFORMATION, + VertexEventType.V_DELETE_SHUFFLE_DATA)) // No transitions from INTERNAL_ERROR state. Ignore all. .addTransition( @@ -655,7 +665,8 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex, EventHandl VertexEventType.V_INTERNAL_ERROR, VertexEventType.V_ROOT_INPUT_INITIALIZED, VertexEventType.V_NULL_EDGE_INITIALIZED, - VertexEventType.V_INPUT_DATA_INFORMATION)) + VertexEventType.V_INPUT_DATA_INFORMATION, + VertexEventType.V_DELETE_SHUFFLE_DATA)) // create the topology tables .installTopology(); @@ -729,6 +740,9 @@ private void augmentStateMachine() { @VisibleForTesting Map sourceVertices; private Map targetVertices; + private boolean cleanupShuffleDataAtVertexLevel; + @VisibleForTesting + VertexShuffleDataDeletionContext vShuffleDeletionContext; Set uninitializedEdges = Sets.newHashSet(); // using a linked hash map to conveniently map edge names to a contiguous index LinkedHashMap ioIndices = Maps.newLinkedHashMap(); @@ -1151,7 +1165,9 @@ public VertexImpl(TezVertexID vertexId, VertexPlan vertexPlan, .append(", ContainerLauncher=").append(containerLauncherIdentifier).append(":").append(containerLauncherName) .append(", TaskCommunicator=").append(taskCommunicatorIdentifier).append(":").append(taskCommName); LOG.info(sb.toString()); - + cleanupShuffleDataAtVertexLevel = vertexConf.getInt(TezConfiguration.TEZ_AM_VERTEX_CLEANUP_HEIGHT, + TezConfiguration.TEZ_AM_VERTEX_CLEANUP_HEIGHT_DEFAULT) > 0 && + ShuffleUtils.isTezShuffleHandler(vertexConf); stateMachine = new StateMachineTez( stateMachineFactory.make(this), this); augmentStateMachine(); @@ -2306,6 +2322,12 @@ static VertexState checkTasksForCompletion(final VertexImpl vertex) { if((vertexSucceeded || vertexFailuresBelowThreshold) && vertex.terminationCause == null) { if(vertexSucceeded) { LOG.info("All tasks have succeeded, vertex:" + vertex.logIdentifier); + if (vertex.cleanupShuffleDataAtVertexLevel) { + + for (Vertex v : vertex.vShuffleDeletionContext.getAncestors()) { + vertex.eventHandler.handle(new VertexShuffleDataDeletion(vertex, v)); + } + } } else { LOG.info("All tasks in the vertex " + vertex.logIdentifier + " have completed and the percentage of failed tasks (failed/total) (" + vertex.failedTaskCount + "/" + vertex.numTasks + ") is less that the threshold of " + vertex.maxFailuresPercent); vertex.addDiagnostic("Vertex succeeded as percentage of failed tasks (failed/total) (" + vertex.failedTaskCount + "/" + vertex.numTasks + ") is less that the threshold of " + vertex.maxFailuresPercent); @@ -3758,6 +3780,36 @@ public VertexState transition(VertexImpl vertex, VertexEvent event) { } } + private static class VertexShuffleDeleteTransition implements + SingleArcTransition { + + @Override + public void transition(VertexImpl vertex, VertexEvent event) { + int incompleteChildrenVertices = vertex.vShuffleDeletionContext.getIncompleteChildrenVertices(); + incompleteChildrenVertices = incompleteChildrenVertices - 1; + vertex.vShuffleDeletionContext.setIncompleteChildrenVertices(incompleteChildrenVertices); + // check if all the child vertices are completed + if (incompleteChildrenVertices == 0) { + LOG.info("Vertex shuffle data deletion for vertex name: " + + vertex.getName() + " with vertex id: " + vertex.getVertexId()); + // Get nodes of all the task attempts in vertex + Set nodes = Sets.newHashSet(); + Map tasksMap = vertex.getTasks(); + tasksMap.keySet().forEach(taskId -> { + Map taskAttemptMap = tasksMap.get(taskId).getAttempts(); + taskAttemptMap.keySet().forEach(attemptId -> { + nodes.add(taskAttemptMap.get(attemptId).getNodeId()); + }); + }); + vertex.appContext.getAppMaster().vertexComplete( + vertex.vertexId, nodes); + } else { + LOG.debug("The number of incomplete child vertex are {} for the vertex {}", + incompleteChildrenVertices, vertex.vertexId); + } + } + } + private static class TaskCompletedAfterVertexSuccessTransition implements MultipleArcTransition { @Override @@ -4930,4 +4982,14 @@ public boolean getTaskRescheduleRelaxedLocality() { public Map> getDownstreamBlamingHosts(){ return downstreamBlamingHosts; } + + /** + * Initialize context from vertex shuffle deletion. + * @param deletionHeight + */ + public void initShuffleDeletionContext(int deletionHeight) { + VertexShuffleDataDeletionContext vShuffleDeletionContext = new VertexShuffleDataDeletionContext(deletionHeight); + vShuffleDeletionContext.setSpannedVertices(this); + this.vShuffleDeletionContext = vShuffleDeletionContext; + } } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexShuffleDataDeletionContext.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexShuffleDataDeletionContext.java new file mode 100644 index 0000000000..4ffdf11231 --- /dev/null +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexShuffleDataDeletionContext.java @@ -0,0 +1,96 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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 org.apache.tez.dag.app.dag.Vertex; + +import java.util.HashSet; +import java.util.Set; + +public class VertexShuffleDataDeletionContext { + private int deletionHeight; + private int incompleteChildrenVertices; + private Set ancestors; + private Set children; + + public VertexShuffleDataDeletionContext(int deletionHeight) { + this.deletionHeight = deletionHeight; + this.incompleteChildrenVertices = 0; + this.ancestors = new HashSet<>(); + this.children = new HashSet<>(); + } + + public void setSpannedVertices(Vertex vertex) { + getSpannedVerticesAncestors(vertex, ancestors, deletionHeight); + getSpannedVerticesChildren(vertex, children, deletionHeight); + setIncompleteChildrenVertices(children.size()); + } + + /** + * get all the ancestor vertices at a particular depth. + */ + private static void getSpannedVerticesAncestors(Vertex vertex, Set ancestorVertices, int level) { + if (level == 0) { + ancestorVertices.add(vertex); + return; + } + + if (level == 1) { + ancestorVertices.addAll(vertex.getInputVertices().keySet()); + return; + } + + vertex.getInputVertices().forEach((inVertex, edge) -> getSpannedVerticesAncestors(inVertex, ancestorVertices, + level - 1)); + } + + /** + * get all the child vertices at a particular depth. + */ + private static void getSpannedVerticesChildren(Vertex vertex, Set childVertices, int level) { + if (level == 0) { + childVertices.add(vertex); + return; + } + + if (level == 1) { + childVertices.addAll(vertex.getOutputVertices().keySet()); + return; + } + + vertex.getOutputVertices().forEach((outVertex, edge) -> getSpannedVerticesChildren(outVertex, childVertices, + level - 1)); + } + + public void setIncompleteChildrenVertices(int incompleteChildrenVertices) { + this.incompleteChildrenVertices = incompleteChildrenVertices; + } + + public int getIncompleteChildrenVertices() { + return this.incompleteChildrenVertices; + } + + public Set getAncestors() { + return this.ancestors; + } + + public Set getChildren() { + return this.children; + } +} diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherManager.java index b0e0f0cf0d..65360d6d01 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherManager.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherManager.java @@ -16,6 +16,7 @@ import java.net.UnknownHostException; import java.util.List; +import java.util.Set; import com.google.common.annotations.VisibleForTesting; import org.apache.tez.common.Preconditions; @@ -37,6 +38,7 @@ import org.apache.tez.dag.app.dag.event.DAGAppMasterEventUserServiceFatalError; import org.apache.tez.dag.records.TezDAGID; import org.apache.tez.dag.records.TezTaskAttemptID; +import org.apache.tez.dag.records.TezVertexID; import org.apache.tez.serviceplugins.api.ContainerLaunchRequest; import org.apache.tez.serviceplugins.api.ContainerLauncher; import org.apache.tez.serviceplugins.api.ContainerLauncherContext; @@ -202,6 +204,12 @@ public void dagComplete(TezDAGID dag, JobTokenSecretManager secretManager) { } } + public void vertexComplete(TezVertexID vertex, JobTokenSecretManager secretManager, Set nodeIdList) { + for (int i = 0; i < containerLaunchers.length; i++) { + containerLaunchers[i].vertexComplete(vertex, secretManager, nodeIdList); + } + } + public void taskAttemptFailed(TezTaskAttemptID taskAttemptID, JobTokenSecretManager secretManager, NodeId nodeId) { for (int i = 0; i < containerLaunchers.length; i++) { containerLaunchers[i].taskAttemptFailed(taskAttemptID, secretManager, nodeId); diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherWrapper.java b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherWrapper.java index 5d262bdab4..4703abe100 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherWrapper.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/ContainerLauncherWrapper.java @@ -14,11 +14,14 @@ package org.apache.tez.dag.app.launcher; +import java.util.Set; + import org.apache.tez.common.DagContainerLauncher; import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.tez.common.security.JobTokenSecretManager; import org.apache.tez.dag.records.TezDAGID; import org.apache.tez.dag.records.TezTaskAttemptID; +import org.apache.tez.dag.records.TezVertexID; import org.apache.tez.serviceplugins.api.ContainerLaunchRequest; import org.apache.tez.serviceplugins.api.ContainerLauncher; import org.apache.tez.serviceplugins.api.ContainerStopRequest; @@ -49,6 +52,12 @@ public void dagComplete(TezDAGID dag, JobTokenSecretManager jobTokenSecretManage } } + public void vertexComplete(TezVertexID vertex, JobTokenSecretManager jobTokenSecretManager, Set nodeIdList) { + if (real instanceof DagContainerLauncher) { + ((DagContainerLauncher) real).vertexComplete(vertex, jobTokenSecretManager, nodeIdList); + } + } + public void taskAttemptFailed(TezTaskAttemptID taskAttemptID, JobTokenSecretManager jobTokenSecretManager, NodeId nodeId) { if (real instanceof DagContainerLauncher) { diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/DeletionTracker.java b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/DeletionTracker.java index 87b7366bfc..56760c86be 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/DeletionTracker.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/DeletionTracker.java @@ -18,11 +18,14 @@ package org.apache.tez.dag.app.launcher; +import java.util.Set; + import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.tez.common.security.JobTokenSecretManager; import org.apache.tez.dag.records.TezDAGID; import org.apache.tez.dag.records.TezTaskAttemptID; +import org.apache.tez.dag.records.TezVertexID; public abstract class DeletionTracker { @@ -36,6 +39,10 @@ public void dagComplete(TezDAGID dag, JobTokenSecretManager jobTokenSecretManage //do nothing } + public void vertexComplete(TezVertexID vertex, JobTokenSecretManager jobTokenSecretManager, Set nodeIdList) { + //do nothing + } + public void taskAttemptFailed(TezTaskAttemptID taskAttemptID, JobTokenSecretManager jobTokenSecretManager, NodeId nodeId) { //do nothing diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/DeletionTrackerImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/DeletionTrackerImpl.java index e4204bfc71..73eaa68e72 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/DeletionTrackerImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/DeletionTrackerImpl.java @@ -21,6 +21,7 @@ import java.util.HashMap; import java.util.Map; +import java.util.Set; import java.util.concurrent.ExecutorService; import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.RejectedExecutionException; @@ -35,6 +36,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.tez.dag.api.TezConfiguration; import org.apache.tez.dag.records.TezTaskAttemptID; +import org.apache.tez.dag.records.TezVertexID; import org.apache.tez.runtime.library.common.TezRuntimeUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -113,4 +115,25 @@ public void shutdown() { } nodeIdShufflePortMap = null; } + + @Override + public void vertexComplete(TezVertexID vertex, JobTokenSecretManager jobTokenSecretManager, Set nodeIdList) { + super.vertexComplete(vertex, jobTokenSecretManager, nodeIdList); + String vertexId = String.format("%02d", vertex.getId()); + for (NodeId nodeId : nodeIdList) { + Integer shufflePort = null; + if (nodeIdShufflePortMap != null) { + shufflePort = nodeIdShufflePortMap.get(nodeId); + } + if (shufflePort != null) { + VertexDeleteRunnable vertexDeleteRunnable = new VertexDeleteRunnable(vertex, jobTokenSecretManager, nodeId, + shufflePort, vertexId, TezRuntimeUtils.getHttpConnectionParams(conf)); + try { + dagCleanupService.submit(vertexDeleteRunnable); + } catch (RejectedExecutionException rejectedException) { + LOG.info("Ignoring deletion request for " + vertexDeleteRunnable); + } + } + } + } } 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 ebc8f95566..47cc9f1325 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 @@ -24,6 +24,7 @@ import java.nio.ByteBuffer; import java.util.HashMap; import java.util.Map; +import java.util.Set; import java.util.concurrent.BlockingQueue; import java.util.concurrent.Callable; import java.util.concurrent.CancellationException; @@ -50,6 +51,7 @@ import org.apache.tez.dag.records.TezDAGID; import org.apache.tez.common.security.JobTokenSecretManager; import org.apache.tez.dag.records.TezTaskAttemptID; +import org.apache.tez.dag.records.TezVertexID; import org.apache.tez.runtime.library.common.TezRuntimeUtils; import org.apache.tez.runtime.library.common.shuffle.ShuffleUtils; import org.apache.tez.serviceplugins.api.ContainerLaunchRequest; @@ -96,6 +98,7 @@ public class LocalContainerLauncher extends DagContainerLauncher { int shufflePort = TezRuntimeUtils.INVALID_PORT; private DeletionTracker deletionTracker; private boolean dagDelete; + private boolean vertexDelete; private boolean failedTaskAttemptDelete; private final ConcurrentHashMap> @@ -162,11 +165,14 @@ public LocalContainerLauncher(ContainerLauncherContext containerLauncherContext, dagDelete = ShuffleUtils.isTezShuffleHandler(conf) && conf.getBoolean(TezConfiguration.TEZ_AM_DAG_CLEANUP_ON_COMPLETION, TezConfiguration.TEZ_AM_DAG_CLEANUP_ON_COMPLETION_DEFAULT); + vertexDelete = ShuffleUtils.isTezShuffleHandler(conf) && + conf.getInt(TezConfiguration.TEZ_AM_VERTEX_CLEANUP_HEIGHT, + TezConfiguration.TEZ_AM_VERTEX_CLEANUP_HEIGHT_DEFAULT) > 0; failedTaskAttemptDelete = ShuffleUtils.isTezShuffleHandler(conf) && conf.getBoolean(TezConfiguration.TEZ_AM_TASK_ATTEMPT_CLEANUP_ON_FAILURE, TezConfiguration.TEZ_AM_TASK_ATTEMPT_CLEANUP_ON_FAILURE_DEFAULT); - if (dagDelete || failedTaskAttemptDelete) { + if (dagDelete || vertexDelete || failedTaskAttemptDelete) { String deletionTrackerClassName = conf.get(TezConfiguration.TEZ_AM_DELETION_TRACKER_CLASS, TezConfiguration.TEZ_AM_DELETION_TRACKER_CLASS_DEFAULT); deletionTracker = ReflectionUtils.createClazzInstance( @@ -454,6 +460,13 @@ public void dagComplete(TezDAGID dag, JobTokenSecretManager jobTokenSecretManage } } + @Override + public void vertexComplete(TezVertexID dag, JobTokenSecretManager jobTokenSecretManager, Set nodeIdList) { + if (vertexDelete && deletionTracker != null) { + deletionTracker.vertexComplete(dag, jobTokenSecretManager, nodeIdList); + } + } + @Override public void taskAttemptFailed(TezTaskAttemptID taskAttemptID, JobTokenSecretManager jobTokenSecretManager, NodeId nodeId) { diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/TezContainerLauncherImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/TezContainerLauncherImpl.java index 88ed4f7b89..654224adb5 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/TezContainerLauncherImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/TezContainerLauncherImpl.java @@ -21,8 +21,8 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.util.Collections; -import java.util.HashMap; import java.util.Map; +import java.util.Set; import java.util.concurrent.BlockingQueue; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.LinkedBlockingQueue; @@ -43,6 +43,7 @@ import org.apache.tez.dag.api.TezException; import org.apache.tez.dag.records.TezDAGID; import org.apache.tez.dag.records.TezTaskAttemptID; +import org.apache.tez.dag.records.TezVertexID; import org.apache.tez.runtime.library.common.TezRuntimeUtils; import org.apache.tez.runtime.library.common.shuffle.ShuffleUtils; import org.apache.tez.serviceplugins.api.ContainerLaunchRequest; @@ -93,6 +94,7 @@ public class TezContainerLauncherImpl extends DagContainerLauncher { private AtomicBoolean serviceStopped = new AtomicBoolean(false); private DeletionTracker deletionTracker = null; private boolean dagDelete; + private boolean vertexDelete; private boolean failedTaskAttemptDelete; private Container getContainer(ContainerOp event) { @@ -339,11 +341,14 @@ public void run() { dagDelete = ShuffleUtils.isTezShuffleHandler(conf) && conf.getBoolean(TezConfiguration.TEZ_AM_DAG_CLEANUP_ON_COMPLETION, TezConfiguration.TEZ_AM_DAG_CLEANUP_ON_COMPLETION_DEFAULT); + vertexDelete = ShuffleUtils.isTezShuffleHandler(conf) && + conf.getInt(TezConfiguration.TEZ_AM_VERTEX_CLEANUP_HEIGHT, + TezConfiguration.TEZ_AM_VERTEX_CLEANUP_HEIGHT_DEFAULT) > 0; failedTaskAttemptDelete = ShuffleUtils.isTezShuffleHandler(conf) && conf.getBoolean(TezConfiguration.TEZ_AM_TASK_ATTEMPT_CLEANUP_ON_FAILURE, TezConfiguration.TEZ_AM_TASK_ATTEMPT_CLEANUP_ON_FAILURE_DEFAULT); - if (dagDelete || failedTaskAttemptDelete) { + if (dagDelete || vertexDelete || failedTaskAttemptDelete) { String deletionTrackerClassName = conf.get(TezConfiguration.TEZ_AM_DELETION_TRACKER_CLASS, TezConfiguration.TEZ_AM_DELETION_TRACKER_CLASS_DEFAULT); deletionTracker = ReflectionUtils.createClazzInstance( @@ -454,6 +459,13 @@ public void dagComplete(TezDAGID dag, JobTokenSecretManager jobTokenSecretManage } } + @Override + public void vertexComplete(TezVertexID vertex, JobTokenSecretManager jobTokenSecretManager, Set nodeIdList) { + if (vertexDelete && deletionTracker != null) { + deletionTracker.vertexComplete(vertex, jobTokenSecretManager, nodeIdList); + } + } + @Override public void taskAttemptFailed(TezTaskAttemptID taskAttemptID, JobTokenSecretManager jobTokenSecretManager, NodeId nodeId) { diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/VertexDeleteRunnable.java b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/VertexDeleteRunnable.java new file mode 100644 index 0000000000..a8d25379a5 --- /dev/null +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/VertexDeleteRunnable.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 + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tez.dag.app.launcher; + +import org.apache.hadoop.yarn.api.records.NodeId; +import org.apache.tez.common.security.JobTokenSecretManager; +import org.apache.tez.dag.records.TezVertexID; +import org.apache.tez.http.BaseHttpConnection; +import org.apache.tez.http.HttpConnectionParams; +import org.apache.tez.runtime.library.common.TezRuntimeUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.net.URL; + +public class VertexDeleteRunnable implements Runnable { + private static final Logger LOG = LoggerFactory.getLogger(VertexDeleteRunnable.class); + final private TezVertexID vertex; + final private JobTokenSecretManager jobTokenSecretManager; + final private NodeId nodeId; + final private int shufflePort; + final private String vertexId; + final private HttpConnectionParams httpConnectionParams; + + VertexDeleteRunnable(TezVertexID vertex, JobTokenSecretManager jobTokenSecretManager, + NodeId nodeId, int shufflePort, String vertexId, + HttpConnectionParams httpConnectionParams) { + this.vertex = vertex; + this.jobTokenSecretManager = jobTokenSecretManager; + this.nodeId = nodeId; + this.shufflePort = shufflePort; + this.vertexId = vertexId; + this.httpConnectionParams = httpConnectionParams; + } + + @Override + public void run() { + BaseHttpConnection httpConnection = null; + try { + URL baseURL = TezRuntimeUtils.constructBaseURIForShuffleHandlerVertexComplete( + nodeId.getHost(), shufflePort, + vertex.getDAGID().getApplicationId().toString(), vertex.getDAGID().getId(), vertexId, false); + httpConnection = TezRuntimeUtils.getHttpConnection(true, baseURL, httpConnectionParams, + "VertexDelete", jobTokenSecretManager); + httpConnection.connect(); + httpConnection.getInputStream(); + } catch (Exception e) { + LOG.warn("Could not setup HTTP Connection to the node %s " + nodeId.getHost() + + " for vertex shuffle delete. ", e); + } finally { + try { + if (httpConnection != null) { + httpConnection.cleanup(true); + } + } catch (IOException e) { + LOG.warn("Encountered IOException for " + nodeId.getHost() + " during close. ", e); + } + } + } + + @Override + public String toString() { + return "VertexDeleteRunnable nodeId=" + nodeId + ", shufflePort=" + shufflePort + ", vertexId=" + vertexId; + } +} 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 5cdcf49206..c118110948 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 @@ -20,6 +20,15 @@ import java.nio.ByteBuffer; +import org.apache.tez.common.TezUtils; +import org.apache.tez.common.security.JobTokenSecretManager; +import org.apache.tez.dag.api.NamedEntityDescriptor; +import org.apache.tez.dag.app.DAGAppMaster; +import org.apache.tez.dag.app.dag.TaskAttempt; +import org.apache.tez.dag.app.launcher.ContainerLauncherManager; +import org.apache.tez.dag.app.launcher.TezContainerLauncherImpl; +import org.apache.tez.dag.app.rm.container.AMContainer; +import org.apache.tez.serviceplugins.api.ContainerLauncherDescriptor; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -2395,12 +2404,254 @@ private DAGPlan createDAGPlanForGraceParallelism() throws IOException { .build(); } - private void setupVertices() { + /** + * The dag is of the following structure. + * vertex1 vertex2 + * \ / + * vertex 3 + * / \ + * vertex4 vertex5 + * \ / + * vertex6 + * @return dagPlan + */ + + public DAGPlan createDAGPlanVertexShuffleDelete() { + LOG.info("Setting up dag plan"); + DAGPlan dag = DAGPlan.newBuilder() + .setName("testverteximpl") + .setDagConf(DAGProtos.ConfigurationProto.newBuilder() + .addConfKeyValues(DAGProtos.PlanKeyValuePair.newBuilder() + .setKey(TezConfiguration.TEZ_AM_TASK_MAX_FAILED_ATTEMPTS) + .setValue(3 + ""))) + .addVertex( + VertexPlan.newBuilder() + .setName("vertex1") + .setType(PlanVertexType.NORMAL) + .addTaskLocationHint( + PlanTaskLocationHint.newBuilder() + .addHost("host1") + .addRack("rack1") + .build() + ) + .setTaskConfig( + PlanTaskConfiguration.newBuilder() + .setNumTasks(1) + .setVirtualCores(4) + .setMemoryMb(1024) + .setJavaOpts("") + .setTaskModule("x1.y1") + .build() + ) + .setVertexConf(DAGProtos.ConfigurationProto.newBuilder() + .addConfKeyValues(DAGProtos.PlanKeyValuePair.newBuilder() + .setKey(TezConfiguration.TEZ_AM_TASK_MAX_FAILED_ATTEMPTS) + .setValue(2+""))) + .addOutEdgeId("e1") + .build() + ) + .addVertex( + VertexPlan.newBuilder() + .setName("vertex2") + .setType(PlanVertexType.NORMAL) + .addTaskLocationHint( + PlanTaskLocationHint.newBuilder() + .addHost("host2") + .addRack("rack2") + .build() + ) + .setTaskConfig( + PlanTaskConfiguration.newBuilder() + .setNumTasks(2) + .setVirtualCores(4) + .setMemoryMb(1024) + .setJavaOpts("") + .setTaskModule("x2.y2") + .build() + ) + .addOutEdgeId("e2") + .build() + ) + .addVertex( + VertexPlan.newBuilder() + .setName("vertex3") + .setType(PlanVertexType.NORMAL) + .setProcessorDescriptor(TezEntityDescriptorProto.newBuilder().setClassName("x3.y3")) + .addTaskLocationHint( + PlanTaskLocationHint.newBuilder() + .addHost("host3") + .addRack("rack3") + .build() + ) + .setTaskConfig( + PlanTaskConfiguration.newBuilder() + .setNumTasks(2) + .setVirtualCores(4) + .setMemoryMb(1024) + .setJavaOpts("foo") + .setTaskModule("x3.y3") + .build() + ) + .addInEdgeId("e1") + .addInEdgeId("e2") + .addOutEdgeId("e3") + .addOutEdgeId("e4") + .build() + ) + .addVertex( + VertexPlan.newBuilder() + .setName("vertex4") + .setType(PlanVertexType.NORMAL) + .addTaskLocationHint( + PlanTaskLocationHint.newBuilder() + .addHost("host4") + .addRack("rack4") + .build() + ) + .setTaskConfig( + PlanTaskConfiguration.newBuilder() + .setNumTasks(2) + .setVirtualCores(4) + .setMemoryMb(1024) + .setJavaOpts("") + .setTaskModule("x4.y4") + .build() + ) + .addInEdgeId("e3") + .addOutEdgeId("e5") + .build() + ) + .addVertex( + VertexPlan.newBuilder() + .setName("vertex5") + .setType(PlanVertexType.NORMAL) + .addTaskLocationHint( + PlanTaskLocationHint.newBuilder() + .addHost("host5") + .addRack("rack5") + .build() + ) + .setTaskConfig( + PlanTaskConfiguration.newBuilder() + .setNumTasks(2) + .setVirtualCores(4) + .setMemoryMb(1024) + .setJavaOpts("") + .setTaskModule("x5.y5") + .build() + ) + .addInEdgeId("e4") + .addOutEdgeId("e6") + .build() + ) + .addVertex( + VertexPlan.newBuilder() + .setName("vertex6") + .setType(PlanVertexType.NORMAL) + .addTaskLocationHint( + PlanTaskLocationHint.newBuilder() + .addHost("host6") + .addRack("rack6") + .build() + ) + .setTaskConfig( + PlanTaskConfiguration.newBuilder() + .setNumTasks(2) + .setVirtualCores(4) + .setMemoryMb(1024) + .setJavaOpts("") + .setTaskModule("x6.y6") + .build() + ) + .addInEdgeId("e5") + .addInEdgeId("e6") + .build() + ) + .addEdge( + EdgePlan.newBuilder() + .setEdgeDestination(TezEntityDescriptorProto.newBuilder().setClassName("i3_v1")) + .setInputVertexName("vertex1") + .setEdgeSource(TezEntityDescriptorProto.newBuilder().setClassName("o1")) + .setOutputVertexName("vertex3") + .setDataMovementType(PlanEdgeDataMovementType.SCATTER_GATHER) + .setId("e1") + .setDataSourceType(PlanEdgeDataSourceType.PERSISTED) + .setSchedulingType(PlanEdgeSchedulingType.SEQUENTIAL) + .build() + ) + .addEdge( + EdgePlan.newBuilder() + .setEdgeDestination(TezEntityDescriptorProto.newBuilder().setClassName("i3_v2")) + .setInputVertexName("vertex2") + .setEdgeSource(TezEntityDescriptorProto.newBuilder().setClassName("o2")) + .setOutputVertexName("vertex3") + .setDataMovementType(PlanEdgeDataMovementType.SCATTER_GATHER) + .setId("e2") + .setDataSourceType(PlanEdgeDataSourceType.PERSISTED) + .setSchedulingType(PlanEdgeSchedulingType.SEQUENTIAL) + .build() + ) + .addEdge( + EdgePlan.newBuilder() + .setEdgeDestination(TezEntityDescriptorProto.newBuilder().setClassName("i4_v3")) + .setInputVertexName("vertex3") + .setEdgeSource(TezEntityDescriptorProto.newBuilder().setClassName("o3_v4")) + .setOutputVertexName("vertex4") + .setDataMovementType(PlanEdgeDataMovementType.SCATTER_GATHER) + .setId("e3") + .setDataSourceType(PlanEdgeDataSourceType.PERSISTED) + .setSchedulingType(PlanEdgeSchedulingType.SEQUENTIAL) + .build() + ) + .addEdge( + EdgePlan.newBuilder() + .setEdgeDestination(TezEntityDescriptorProto.newBuilder().setClassName("i5_v3")) + .setInputVertexName("vertex3") + .setEdgeSource(TezEntityDescriptorProto.newBuilder().setClassName("o3_v5")) + .setOutputVertexName("vertex5") + .setDataMovementType(PlanEdgeDataMovementType.SCATTER_GATHER) + .setId("e4") + .setDataSourceType(PlanEdgeDataSourceType.PERSISTED) + .setSchedulingType(PlanEdgeSchedulingType.SEQUENTIAL) + .build() + ) + .addEdge( + EdgePlan.newBuilder() + .setEdgeDestination(TezEntityDescriptorProto.newBuilder().setClassName("i6_v4")) + .setInputVertexName("vertex4") + .setEdgeSource(TezEntityDescriptorProto.newBuilder().setClassName("o4")) + .setOutputVertexName("vertex6") + .setDataMovementType(PlanEdgeDataMovementType.SCATTER_GATHER) + .setId("e5") + .setDataSourceType(PlanEdgeDataSourceType.PERSISTED) + .setSchedulingType(PlanEdgeSchedulingType.SEQUENTIAL) + .build() + ) + .addEdge( + EdgePlan.newBuilder() + .setEdgeDestination(TezEntityDescriptorProto.newBuilder().setClassName("i6_v5")) + .setInputVertexName("vertex5") + .setEdgeSource(TezEntityDescriptorProto.newBuilder().setClassName("o5")) + .setOutputVertexName("vertex6") + .setDataMovementType(PlanEdgeDataMovementType.SCATTER_GATHER) + .setId("e6") + .setDataSourceType(PlanEdgeDataSourceType.PERSISTED) + .setSchedulingType(PlanEdgeSchedulingType.SEQUENTIAL) + .build() + ) + .build(); + + return dag; + } + + private void setupVertices(boolean cleanupShuffleDataAtVertexLevel) { int vCnt = dagPlan.getVertexCount(); LOG.info("Setting up vertices from dag plan, verticesCnt=" + vCnt); vertices = new HashMap(); vertexIdMap = new HashMap(); Configuration dagConf = new Configuration(false); + dagConf.setBoolean(TezConfiguration.TEZ_AM_DAG_CLEANUP_ON_COMPLETION, true); + conf.setInt(TezConfiguration.TEZ_AM_VERTEX_CLEANUP_HEIGHT, cleanupShuffleDataAtVertexLevel ? 1 : 0); dagConf.set("abc", "foobar"); for (int i = 0; i < vCnt; ++i) { VertexPlan vPlan = dagPlan.getVertex(i); @@ -2447,7 +2698,6 @@ private void parseVertexEdges() { Map outVertices = new HashMap(); - for(String inEdgeId : vertexPlan.getInEdgeIdList()){ EdgePlan edgePlan = edgePlans.get(inEdgeId); Vertex inVertex = this.vertices.get(edgePlan.getInputVertexName()); @@ -2472,8 +2722,14 @@ private void parseVertexEdges() { + ", outputVerticesCnt=" + outVertices.size()); vertex.setOutputVertices(outVertices); } + + for (Map.Entry vertex : vertices.entrySet()) { + VertexImpl vertexImpl = vertex.getValue(); + vertexImpl.initShuffleDeletionContext(2); + } } + public void setupPreDagCreation() { LOG.info("____________ RESETTING CURRENT DAG ____________"); conf = new Configuration(); @@ -2488,8 +2744,9 @@ public void setupPreDagCreation() { } @SuppressWarnings({ "unchecked", "rawtypes" }) - public void setupPostDagCreation() throws TezException { + public void setupPostDagCreation(boolean cleanupShuffleDataAtVertexLevel) throws TezException { String dagName = "dag0"; + taskCommunicatorManagerInterface = mock(TaskCommunicatorManagerInterface.class); // dispatcher may be created multiple times (setupPostDagCreation may be called multiples) if (dispatcher != null) { dispatcher.stop(); @@ -2499,6 +2756,40 @@ public void setupPostDagCreation() throws TezException { when(appContext.getHadoopShim()).thenReturn(new DefaultHadoopShim()); when(appContext.getContainerLauncherName(anyInt())).thenReturn( TezConstants.getTezYarnServicePluginName()); + DAGAppMaster mockDagAppMaster = mock(DAGAppMaster.class); + when(appContext.getAppMaster()).thenReturn(mockDagAppMaster); + doCallRealMethod().when(mockDagAppMaster).vertexComplete(any(TezVertexID.class), any(Set.class)); + List containerDescriptors = new ArrayList<>(); + ContainerLauncherDescriptor containerLaunchers = + ContainerLauncherDescriptor.create("ContainerLaunchers", + TezContainerLauncherImpl.class.getName()); + conf.setBoolean(TezConfiguration.TEZ_AM_DAG_CLEANUP_ON_COMPLETION, true); + conf.set(TezConfiguration.TEZ_AM_SHUFFLE_AUXILIARY_SERVICE_ID, "tez_shuffle"); + conf.setInt(TezConfiguration.TEZ_AM_VERTEX_CLEANUP_HEIGHT, 0); + try { + containerLaunchers.setUserPayload(UserPayload.create( + TezUtils.createByteStringFromConf(conf).asReadOnlyByteBuffer())); + } catch (IOException e) { + e.printStackTrace(); + } + containerDescriptors.add(containerLaunchers); + ContainerLauncherManager mockContainerLauncherManager = spy(new ContainerLauncherManager(appContext, + taskCommunicatorManagerInterface, "test", containerDescriptors, false)); + doCallRealMethod().when(mockContainerLauncherManager).vertexComplete(any( + TezVertexID.class), any(JobTokenSecretManager.class + ), any(Set.class)); + when(appContext.getAppMaster().getContainerLauncherManager()).thenReturn( + mockContainerLauncherManager); + mockContainerLauncherManager.init(conf); + mockContainerLauncherManager.start(); + AMContainerMap amContainerMap = mock(AMContainerMap.class); + AMContainer amContainer = mock(AMContainer.class); + Container mockContainer = mock(Container.class); + when(amContainer.getContainer()).thenReturn(mockContainer); + when(mockContainer.getNodeId()).thenReturn(mock(NodeId.class)); + when(mockContainer.getNodeHttpAddress()).thenReturn("localhost:12345"); + when(amContainerMap.get(any(ContainerId.class))).thenReturn(amContainer); + when(appContext.getAllContainers()).thenReturn(amContainerMap); thh = mock(TaskHeartbeatHandler.class); historyEventHandler = mock(HistoryEventHandler.class); @@ -2557,7 +2848,7 @@ public ListenableFuture answer(InvocationOnMock invocation) { updateTracker.stop(); } updateTracker = new StateChangeNotifierForTest(appContext.getCurrentDAG()); - setupVertices(); + setupVertices(cleanupShuffleDataAtVertexLevel); when(dag.getVertex(any(TezVertexID.class))).thenAnswer(new Answer() { @Override public Vertex answer(InvocationOnMock invocation) throws Throwable { @@ -2622,7 +2913,7 @@ public void setup() throws TezException { setupPreDagCreation(); dagPlan = createTestDAGPlan(); invalidDagPlan = createInvalidDAGPlan(); - setupPostDagCreation(); + setupPostDagCreation(false); } @After @@ -2750,7 +3041,7 @@ public void testVertexInit() throws AMUserCodeException { public void testNonExistVertexManager() throws TezException { setupPreDagCreation(); dagPlan = createDAGPlanWithNonExistVertexManager(); - setupPostDagCreation(); + setupPostDagCreation(false); VertexImpl v1 = vertices.get("vertex1"); v1.handle(new VertexEvent(v1.getVertexId(), VertexEventType.V_INIT)); Assert.assertEquals(VertexState.FAILED, v1.getState()); @@ -2763,7 +3054,7 @@ public void testNonExistVertexManager() throws TezException { public void testNonExistInputInitializer() throws TezException { setupPreDagCreation(); dagPlan = createDAGPlanWithNonExistInputInitializer(); - setupPostDagCreation(); + setupPostDagCreation(false); VertexImpl v1 = vertices.get("vertex1"); v1.handle(new VertexEvent(v1.getVertexId(), VertexEventType.V_INIT)); Assert.assertEquals(VertexState.FAILED, v1.getState()); @@ -2776,7 +3067,7 @@ public void testNonExistInputInitializer() throws TezException { public void testNonExistOutputCommitter() throws TezException { setupPreDagCreation(); dagPlan = createDAGPlanWithNonExistOutputCommitter(); - setupPostDagCreation(); + setupPostDagCreation(false); VertexImpl v1 = vertices.get("vertex1"); v1.handle(new VertexEvent(v1.getVertexId(), VertexEventType.V_INIT)); Assert.assertEquals(VertexState.FAILED, v1.getState()); @@ -2815,7 +3106,7 @@ public void testVertexConfigureEventWithReconfigure() throws Exception { setupPreDagCreation(); // initialize() will make VM call planned() and started() will make VM call done() dagPlan = createDAGPlanWithVMException("TestVMStateUpdate", VMExceptionLocation.NoExceptionDoReconfigure); - setupPostDagCreation(); + setupPostDagCreation(false); TestUpdateListener listener = new TestUpdateListener(); updateTracker @@ -3824,7 +4115,7 @@ public void testFailuresMaxPercentSourceTaskAttemptCompletionEvents() throws Tez conf.setFloat(TezConfiguration.TEZ_VERTEX_FAILURES_MAXPERCENT, 50.0f); conf.setInt(TezConfiguration.TEZ_AM_TASK_MAX_FAILED_ATTEMPTS, 1); dagPlan = createTestDAGPlan(); - setupPostDagCreation(); + setupPostDagCreation(false); initAllVertices(VertexState.INITED); VertexImpl v4 = vertices.get("vertex4"); @@ -3879,7 +4170,7 @@ public void testFailuresMaxPercentExceededSourceTaskAttemptCompletionEvents() th conf.setFloat(TezConfiguration.TEZ_VERTEX_FAILURES_MAXPERCENT, 50.0f); conf.setInt(TezConfiguration.TEZ_AM_TASK_MAX_FAILED_ATTEMPTS, 1); dagPlan = createTestDAGPlan(); - setupPostDagCreation(); + setupPostDagCreation(false); initAllVertices(VertexState.INITED); VertexImpl v4 = vertices.get("vertex4"); @@ -3978,7 +4269,7 @@ public void testTaskReschedule() { public void testTerminatingVertexForTaskComplete() throws Exception { setupPreDagCreation(); dagPlan = createSamplerDAGPlan(false); - setupPostDagCreation(); + setupPostDagCreation(false); VertexImpl vertex = spy(vertices.get("A")); initVertex(vertex); startVertex(vertex); @@ -3996,7 +4287,7 @@ public void testTerminatingVertexForTaskComplete() throws Exception { public void testTerminatingVertexForVComplete() throws Exception { setupPreDagCreation(); dagPlan = createSamplerDAGPlan(false); - setupPostDagCreation(); + setupPostDagCreation(false); VertexImpl vertex = spy(vertices.get("A")); initVertex(vertex); startVertex(vertex); @@ -4251,7 +4542,7 @@ public void testBadCommitter2() throws Exception { public void testVertexInitWithCustomVertexManager() throws Exception { setupPreDagCreation(); dagPlan = createDAGWithCustomVertexManager(); - setupPostDagCreation(); + setupPostDagCreation(false); int numTasks = 3; VertexImpl v1 = vertices.get("v1"); @@ -4305,7 +4596,7 @@ public void testVertexInitWithCustomVertexManager() throws Exception { public void testVertexManagerHeuristic() throws TezException { setupPreDagCreation(); dagPlan = createDAGPlanWithMixedEdges(); - setupPostDagCreation(); + setupPostDagCreation(false); initAllVertices(VertexState.INITED); Assert.assertEquals(ImmediateStartVertexManager.class, vertices.get("vertex1").getVertexManager().getPlugin().getClass()); @@ -4330,7 +4621,7 @@ public void testVertexWithOneToOneSplit() throws Exception { useCustomInitializer = true; setupPreDagCreation(); dagPlan = createDAGPlanForOneToOneSplit("TestInputInitializer", -1, true); - setupPostDagCreation(); + setupPostDagCreation(false); int numTasks = 5; VertexImplWithControlledInitializerManager v1 = (VertexImplWithControlledInitializerManager) vertices @@ -4397,7 +4688,7 @@ public void testVertexWithOneToOneSplitWhileRunning() throws Exception { // create a diamond shaped dag with 1-1 edges. setupPreDagCreation(); dagPlan = createDAGPlanForOneToOneSplit(null, numTasks, false); - setupPostDagCreation(); + setupPostDagCreation(false); VertexImpl v1 = vertices.get("vertex1"); v1.vertexReconfigurationPlanned(); initAllVertices(VertexState.INITED); @@ -4436,7 +4727,7 @@ public void testVertexWithOneToOneSplitWhileInited() throws Exception { // create a diamond shaped dag with 1-1 edges. setupPreDagCreation(); dagPlan = createDAGPlanForOneToOneSplit(null, numTasks, false); - setupPostDagCreation(); + setupPostDagCreation(false); VertexImpl v1 = vertices.get("vertex1"); v1.vertexReconfigurationPlanned(); initAllVertices(VertexState.INITED); @@ -4478,7 +4769,7 @@ public void testVertexVMErrorReport() throws Exception { // create a diamond shaped dag with 1-1 edges. setupPreDagCreation(); dagPlan = createDAGPlanForOneToOneSplit(null, numTasks, false); - setupPostDagCreation(); + setupPostDagCreation(false); VertexImpl v1 = vertices.get("vertex1"); initAllVertices(VertexState.INITED); @@ -4522,7 +4813,7 @@ public void testVertexWithInitializerFailure() throws Exception { useCustomInitializer = true; setupPreDagCreation(); dagPlan = createDAGPlanWithInputInitializer("TestInputInitializer"); - setupPostDagCreation(); + setupPostDagCreation(false); VertexImplWithControlledInitializerManager v1 = (VertexImplWithControlledInitializerManager) vertices .get("vertex1"); @@ -4567,7 +4858,7 @@ public void testVertexWithInitializerParallelismSetTo0() throws InterruptedExcep setupPreDagCreation(); dagPlan = createDAGPlanWithInitializer0Tasks(RootInitializerSettingParallelismTo0.class.getName()); - setupPostDagCreation(); + setupPostDagCreation(false); VertexImpl v1 = vertices.get("vertex1"); VertexImpl v2 = vertices.get("vertex2"); @@ -4615,7 +4906,7 @@ public void testInputInitializerVertexStateUpdates() throws Exception { initializer.setNumVertexStateUpdateEvents(3); setupPreDagCreation(); dagPlan = createDAGPlanWithRunningInitializer(); - setupPostDagCreation(); + setupPostDagCreation(false); VertexImplWithRunningInputInitializer v1 = (VertexImplWithRunningInputInitializer) vertices.get("vertex1"); @@ -4650,7 +4941,7 @@ public void testInputInitializerEventMultipleAttempts() throws Exception { (EventHandlingRootInputInitializer) customInitializer; setupPreDagCreation(); dagPlan = createDAGPlanWithRunningInitializer4(); - setupPostDagCreation(); + setupPostDagCreation(false); VertexImplWithRunningInputInitializer v1 = (VertexImplWithRunningInputInitializer) vertices.get("vertex1"); @@ -4738,7 +5029,7 @@ public void testInputInitializerEventsMultipleSources() throws Exception { initializer.setNumExpectedEvents(4); setupPreDagCreation(); dagPlan = createDAGPlanWithRunningInitializer4(); - setupPostDagCreation(); + setupPostDagCreation(false); VertexImplWithRunningInputInitializer v1 = (VertexImplWithRunningInputInitializer) vertices.get("vertex1"); @@ -4861,7 +5152,7 @@ public void testInputInitializerEventNoDirectConnection() throws Exception { (EventHandlingRootInputInitializer) customInitializer; setupPreDagCreation(); dagPlan = createDAGPlanWithRunningInitializer4(); - setupPostDagCreation(); + setupPostDagCreation(false); VertexImplWithRunningInputInitializer v1 = (VertexImplWithRunningInputInitializer) vertices.get("vertex1"); @@ -4941,7 +5232,7 @@ public void testInputInitializerEventsAtNew() throws Exception { (EventHandlingRootInputInitializer) customInitializer; setupPreDagCreation(); dagPlan = createDAGPlanWithRunningInitializer3(); - setupPostDagCreation(); + setupPostDagCreation(false); VertexImplWithRunningInputInitializer v1 = (VertexImplWithRunningInputInitializer) vertices.get("vertex1"); @@ -5027,7 +5318,7 @@ public void testInputInitializerEvents() throws Exception { (EventHandlingRootInputInitializer) customInitializer; setupPreDagCreation(); dagPlan = createDAGPlanWithRunningInitializer(); - setupPostDagCreation(); + setupPostDagCreation(false); VertexImplWithRunningInputInitializer v1 = (VertexImplWithRunningInputInitializer) vertices.get("vertex1"); @@ -5104,7 +5395,7 @@ public void testInputInitializerEvents() throws Exception { public void testTaskSchedulingWithCustomEdges() throws TezException { setupPreDagCreation(); dagPlan = createCustomDAGWithCustomEdges(); - setupPostDagCreation(); + setupPostDagCreation(false); /** * @@ -5402,7 +5693,7 @@ public void testVertexWithMultipleInitializers1() throws Exception { useCustomInitializer = true; setupPreDagCreation(); dagPlan = createDAGPlanWithMultipleInitializers("TestInputInitializer"); - setupPostDagCreation(); + setupPostDagCreation(false); VertexImplWithControlledInitializerManager v1 = (VertexImplWithControlledInitializerManager) vertices .get("vertex1"); @@ -5432,7 +5723,7 @@ public void testVertexWithMultipleInitializers2() throws Exception { useCustomInitializer = true; setupPreDagCreation(); dagPlan = createDAGPlanWithMultipleInitializers("TestInputInitializer"); - setupPostDagCreation(); + setupPostDagCreation(false); VertexImplWithControlledInitializerManager v1 = (VertexImplWithControlledInitializerManager) vertices .get("vertex1"); @@ -5462,7 +5753,7 @@ public void testVertexWithInitializerSuccess() throws Exception { useCustomInitializer = true; setupPreDagCreation(); dagPlan = createDAGPlanWithInputInitializer("TestInputInitializer"); - setupPostDagCreation(); + setupPostDagCreation(false); VertexImplWithControlledInitializerManager v1 = (VertexImplWithControlledInitializerManager) vertices .get("vertex1"); @@ -5563,7 +5854,7 @@ public void testVertexWithInitializerSuccessLegacyRouting() throws Exception { useCustomInitializer = true; setupPreDagCreation(); dagPlan = createDAGPlanWithInputInitializer("TestInputInitializer"); - setupPostDagCreation(); + setupPostDagCreation(false); VertexImplWithControlledInitializerManager v1 = (VertexImplWithControlledInitializerManager) vertices .get("vertex1"); @@ -5638,7 +5929,7 @@ public void testVertexWithInputDistributor() throws Exception { useCustomInitializer = true; setupPreDagCreation(); dagPlan = createDAGPlanWithInputDistributor("TestInputInitializer"); - setupPostDagCreation(); + setupPostDagCreation(false); VertexImplWithControlledInitializerManager v1 = (VertexImplWithControlledInitializerManager) vertices .get("vertex1"); @@ -5673,7 +5964,7 @@ public void testVertexRootInputSpecUpdateAll() throws Exception { useCustomInitializer = true; setupPreDagCreation(); dagPlan = createDAGPlanWithInputInitializer("TestInputInitializer"); - setupPostDagCreation(); + setupPostDagCreation(false); int expectedNumTasks = RootInputSpecUpdaterVertexManager.NUM_TASKS; VertexImplWithControlledInitializerManager v3 = (VertexImplWithControlledInitializerManager) vertices @@ -5703,7 +5994,7 @@ public void testVertexRootInputSpecUpdatePerTask() throws Exception { useCustomInitializer = true; setupPreDagCreation(); dagPlan = createDAGPlanWithInputInitializer("TestInputInitializer"); - setupPostDagCreation(); + setupPostDagCreation(false); int expectedNumTasks = RootInputSpecUpdaterVertexManager.NUM_TASKS; VertexImplWithControlledInitializerManager v4 = (VertexImplWithControlledInitializerManager) vertices @@ -6015,7 +6306,7 @@ public void completeInputInitialization(int initializerIndex, int targetTasks, public void testVertexGroupInput() throws TezException { setupPreDagCreation(); dagPlan = createVertexGroupDAGPlan(); - setupPostDagCreation(); + setupPostDagCreation(false); VertexImpl vA = vertices.get("A"); VertexImpl vB = vertices.get("B"); @@ -6044,7 +6335,7 @@ public void testStartWithUninitializedCustomEdge() throws Exception { // been initialized setupPreDagCreation(); dagPlan = createSamplerDAGPlan(true); - setupPostDagCreation(); + setupPostDagCreation(false); VertexImpl vA = vertices.get("A"); VertexImpl vB = vertices.get("B"); @@ -6093,7 +6384,7 @@ public void testVertexConfiguredDoneByVMBeforeEdgeDefined() throws Exception { // been initialized setupPreDagCreation(); dagPlan = createSamplerDAGPlan(true); - setupPostDagCreation(); + setupPostDagCreation(false); VertexImpl vA = vertices.get("A"); VertexImpl vB = vertices.get("B"); @@ -6167,7 +6458,7 @@ public void testInitStartRace() throws TezException { // been initialized setupPreDagCreation(); dagPlan = createSamplerDAGPlan(false); - setupPostDagCreation(); + setupPostDagCreation(false); VertexImpl vA = vertices.get("A"); VertexImpl vB = vertices.get("B"); @@ -6190,7 +6481,7 @@ public void testInitStartRace2() throws TezException { // been initialized setupPreDagCreation(); dagPlan = createSamplerDAGPlan2(); - setupPostDagCreation(); + setupPostDagCreation(false); VertexImpl vA = vertices.get("A"); VertexImpl vB = vertices.get("B"); @@ -6215,7 +6506,7 @@ public void testInitStartRace2() throws TezException { public void testTez2684() throws IOException, TezException { setupPreDagCreation(); dagPlan = createSamplerDAGPlan2(); - setupPostDagCreation(); + setupPostDagCreation(false); VertexImpl vA = vertices.get("A"); VertexImpl vB = vertices.get("B"); @@ -6255,7 +6546,7 @@ public void testTez2684() throws IOException, TezException { public void testVertexGraceParallelism() throws IOException, TezException { setupPreDagCreation(); dagPlan = createDAGPlanForGraceParallelism(); - setupPostDagCreation(); + setupPostDagCreation(false); VertexImpl vA = vertices.get("A"); VertexImpl vB = vertices.get("B"); @@ -6323,7 +6614,7 @@ public void testVMEventBeforeVertexInitialized() throws Exception { useCustomInitializer = true; setupPreDagCreation(); dagPlan = createDAGPlanWithCountingVM(); - setupPostDagCreation(); + setupPostDagCreation(false); VertexImpl v1 = vertices.get("vertex1"); VertexImpl v2 = vertices.get("vertex2"); @@ -6380,7 +6671,7 @@ public void testExceptionFromVM_Initialize() throws TezException { useCustomInitializer = true; setupPreDagCreation(); dagPlan = createDAGPlanWithVMException("TestInputInitializer", VMExceptionLocation.Initialize); - setupPostDagCreation(); + setupPostDagCreation(false); VertexImplWithControlledInitializerManager v1 = (VertexImplWithControlledInitializerManager) vertices .get("vertex1"); @@ -6399,7 +6690,7 @@ public void testExceptionFromVM_OnRootVertexInitialized() throws Exception { useCustomInitializer = true; setupPreDagCreation(); dagPlan = createDAGPlanWithVMException("TestInputInitializer", VMExceptionLocation.OnRootVertexInitialized); - setupPostDagCreation(); + setupPostDagCreation(false); VertexImplWithControlledInitializerManager v1 = (VertexImplWithControlledInitializerManager) vertices .get("vertex1"); @@ -6423,7 +6714,7 @@ public void testExceptionFromVM_OnVertexStarted() throws Exception { useCustomInitializer = true; setupPreDagCreation(); dagPlan = createDAGPlanWithVMException("TestInputInitializer", VMExceptionLocation.OnVertexStarted); - setupPostDagCreation(); + setupPostDagCreation(false); VertexImplWithControlledInitializerManager v1 = (VertexImplWithControlledInitializerManager) vertices .get("vertex1"); @@ -6450,7 +6741,7 @@ public void testExceptionFromVM_OnSourceTaskCompleted() throws Exception { useCustomInitializer = true; setupPreDagCreation(); dagPlan = createDAGPlanWithVMException("TestInputInitializer", VMExceptionLocation.OnSourceTaskCompleted); - setupPostDagCreation(); + setupPostDagCreation(false); VertexImplWithControlledInitializerManager v1 = (VertexImplWithControlledInitializerManager) vertices .get("vertex1"); @@ -6486,7 +6777,7 @@ public void testExceptionFromVM_OnVertexManagerEventReceived() throws Exception useCustomInitializer = true; setupPreDagCreation(); dagPlan = createDAGPlanWithVMException("TestInputInitializer", VMExceptionLocation.OnVertexManagerEventReceived); - setupPostDagCreation(); + setupPostDagCreation(false); VertexImplWithControlledInitializerManager v1 = (VertexImplWithControlledInitializerManager) vertices .get("vertex1"); @@ -6514,7 +6805,7 @@ public void testExceptionFromVM_OnVertexManagerVertexStateUpdated() throws Excep useCustomInitializer = true; setupPreDagCreation(); dagPlan = createDAGPlanWithVMException("TestVMStateUpdate", VMExceptionLocation.OnVertexManagerVertexStateUpdated); - setupPostDagCreation(); + setupPostDagCreation(false); VertexImplWithControlledInitializerManager v1 = (VertexImplWithControlledInitializerManager) vertices .get("vertex1"); @@ -6543,7 +6834,7 @@ public void testExceptionFromII_Initialize() throws InterruptedException, TezExc (EventHandlingRootInputInitializer) customInitializer; setupPreDagCreation(); dagPlan = createDAGPlanWithIIException(); - setupPostDagCreation(); + setupPostDagCreation(false); VertexImplWithRunningInputInitializer v1 = (VertexImplWithRunningInputInitializer) vertices.get("vertex1"); @@ -6564,7 +6855,7 @@ public void testExceptionFromII_InitFailedAfterInitialized() throws Exception { useCustomInitializer = true; setupPreDagCreation(); dagPlan = createDAGPlanWithIIException(); - setupPostDagCreation(); + setupPostDagCreation(false); VertexImplWithControlledInitializerManager v1 = (VertexImplWithControlledInitializerManager)vertices.get("vertex1"); @@ -6588,7 +6879,7 @@ public void testExceptionFromII_InitFailedAfterRunning() throws Exception { useCustomInitializer = true; setupPreDagCreation(); dagPlan = createDAGPlanWithIIException(); - setupPostDagCreation(); + setupPostDagCreation(false); VertexImplWithControlledInitializerManager v1 = (VertexImplWithControlledInitializerManager)vertices.get("vertex1"); @@ -6616,7 +6907,7 @@ public void testExceptionFromII_HandleInputInitializerEvent() throws Exception { (EventHandlingRootInputInitializer) customInitializer; setupPreDagCreation(); dagPlan = createDAGPlanWithRunningInitializer(); - setupPostDagCreation(); + setupPostDagCreation(false); VertexImplWithRunningInputInitializer v1 = (VertexImplWithRunningInputInitializer) vertices.get("vertex1"); @@ -6666,7 +6957,7 @@ public void testExceptionFromII_OnVertexStateUpdated() throws InterruptedExcepti (EventHandlingRootInputInitializer) customInitializer; setupPreDagCreation(); dagPlan = createDAGPlanWithRunningInitializer(); - setupPostDagCreation(); + setupPostDagCreation(false); VertexImplWithRunningInputInitializer v1 = (VertexImplWithRunningInputInitializer) vertices.get("vertex1"); @@ -6695,7 +6986,7 @@ public void testExceptionFromII_InitSucceededAfterInitFailure() throws Interrupt (EventHandlingRootInputInitializer) customInitializer; setupPreDagCreation(); dagPlan = createDAGPlanWithRunningInitializer(); - setupPostDagCreation(); + setupPostDagCreation(false); VertexImplWithRunningInputInitializer v1 = (VertexImplWithRunningInputInitializer) vertices.get("vertex1"); @@ -7212,4 +7503,132 @@ public void testPickupDagLocalResourceOnScheduleTask() { Assert.assertTrue(localResourceMap.containsKey("dag lr")); Assert.assertTrue(localResourceMap.containsKey("vertex lr")); } + + @Test + public void testVertexShuffleDelete() throws Exception { + setupPreDagCreation(); + dagPlan = createDAGPlanVertexShuffleDelete(); + setupPostDagCreation(true); + checkSpannedVertices(); + runVertices(); + Mockito.verify(appContext.getAppMaster().getContainerLauncherManager(), + times(3)).vertexComplete(any(TezVertexID.class), + any(JobTokenSecretManager.class), any(Set.class)); + } + + private void checkSpannedVertices() { + // vertex1 should have 0 ancestor and 2 children at height = 2 + VertexImpl v1 = vertices.get("vertex1"); + checkResults(v1.vShuffleDeletionContext.getAncestors(), new ArrayList<>()); + checkResults(v1.vShuffleDeletionContext.getChildren(), Arrays.asList("vertex5", "vertex4")); + + // vertex2 should have 0 ancestor and 2 children at height = 2 + VertexImpl v2 = vertices.get("vertex2"); + checkResults(v2.vShuffleDeletionContext.getAncestors(), new ArrayList<>()); + checkResults(v2.vShuffleDeletionContext.getChildren(), Arrays.asList("vertex5", "vertex4")); + + // vertex3 should have 0 ancestor and 1 children at height = 2 + VertexImpl v3 = vertices.get("vertex3"); + checkResults(v3.vShuffleDeletionContext.getAncestors(), new ArrayList<>()); + checkResults(v3.vShuffleDeletionContext.getChildren(), Arrays.asList("vertex6")); + + // vertex4 should have 2 ancestor and 0 children at height = 2 + VertexImpl v4 = vertices.get("vertex4"); + checkResults(v4.vShuffleDeletionContext.getAncestors(), Arrays.asList("vertex1", "vertex2")); + checkResults(v4.vShuffleDeletionContext.getChildren(), new ArrayList<>()); + + // vertex5 should have 2 ancestor and 0 children at height = 2 + VertexImpl v5 = vertices.get("vertex5"); + checkResults(v5.vShuffleDeletionContext.getAncestors(), Arrays.asList("vertex1", "vertex2")); + checkResults(v5.vShuffleDeletionContext.getChildren(), new ArrayList<>()); + + // vertex6 should have 1 ancestor and 0 children at height = 2 + VertexImpl v6 = vertices.get("vertex6"); + checkResults(v6.vShuffleDeletionContext.getAncestors(), Arrays.asList("vertex3")); + checkResults(v6.vShuffleDeletionContext.getChildren(), new ArrayList<>()); + } + + private void checkResults(Set actual, List expected) { + assertEquals(actual.size(), expected.size()); + for (Vertex vertex : actual) { + assertTrue(expected.contains(vertex.getName())); + } + } + + private void runVertices() { + VertexImpl v1 = vertices.get("vertex1"); + VertexImpl v2 = vertices.get("vertex2"); + VertexImpl v3 = vertices.get("vertex3"); + VertexImpl v4 = vertices.get("vertex4"); + VertexImpl v5 = vertices.get("vertex5"); + VertexImpl v6 = vertices.get("vertex6"); + dispatcher.getEventHandler().handle(new VertexEvent(v1.getVertexId(), VertexEventType.V_INIT)); + dispatcher.getEventHandler().handle(new VertexEvent(v2.getVertexId(), VertexEventType.V_INIT)); + dispatcher.await(); + dispatcher.getEventHandler().handle(new VertexEvent(v1.getVertexId(), VertexEventType.V_START)); + dispatcher.getEventHandler().handle(new VertexEvent(v2.getVertexId(), VertexEventType.V_START)); + dispatcher.await(); + + TezTaskID v1t1 = TezTaskID.getInstance(v1.getVertexId(), 0); + Map attempts = v1.getTask(v1t1).getAttempts(); + startAttempts(attempts); + v1.handle(new VertexEventTaskCompleted(v1t1, TaskState.SUCCEEDED)); + TezTaskID v2t1 = TezTaskID.getInstance(v2.getVertexId(), 0); + attempts = v2.getTask(v2t1).getAttempts(); + startAttempts(attempts); + v2.handle(new VertexEventTaskCompleted(v2t1, TaskState.SUCCEEDED)); + TezTaskID v2t2 = TezTaskID.getInstance(v2.getVertexId(), 1); + attempts = v2.getTask(v2t2).getAttempts(); + startAttempts(attempts); + v2.handle(new VertexEventTaskCompleted(v2t2, TaskState.SUCCEEDED)); + TezTaskID v3t1 = TezTaskID.getInstance(v3.getVertexId(), 0); + v3.scheduleTasks(Lists.newArrayList(ScheduleTaskRequest.create(0, null))); + dispatcher.await(); + attempts = v3.getTask(v3t1).getAttempts(); + startAttempts(attempts); + v3.handle(new VertexEventTaskCompleted(v3t1, TaskState.SUCCEEDED)); + TezTaskID v3t2 = TezTaskID.getInstance(v3.getVertexId(), 1); + attempts = v3.getTask(v3t2).getAttempts(); + startAttempts(attempts); + v3.handle(new VertexEventTaskCompleted(v3t2, TaskState.SUCCEEDED)); + dispatcher.await(); + TezTaskID v4t1 = TezTaskID.getInstance(v4.getVertexId(), 0); + attempts = v4.getTask(v4t1).getAttempts(); + startAttempts(attempts); + v4.handle(new VertexEventTaskCompleted(v4t1, TaskState.SUCCEEDED)); + TezTaskID v4t2 = TezTaskID.getInstance(v4.getVertexId(), 1); + attempts = v4.getTask(v4t2).getAttempts(); + startAttempts(attempts); + v4.handle(new VertexEventTaskCompleted(v4t2, TaskState.SUCCEEDED)); + TezTaskID v5t1 = TezTaskID.getInstance(v5.getVertexId(), 0); + attempts = v5.getTask(v5t1).getAttempts(); + startAttempts(attempts); + v5.handle(new VertexEventTaskCompleted(v5t1, TaskState.SUCCEEDED)); + TezTaskID v5t2 = TezTaskID.getInstance(v5.getVertexId(), 1); + attempts = v5.getTask(v5t2).getAttempts(); + startAttempts(attempts); + v5.handle(new VertexEventTaskCompleted(v5t2, TaskState.SUCCEEDED)); + TezTaskID v6t1 = TezTaskID.getInstance(v6.getVertexId(), 0); + attempts = v6.getTask(v6t1).getAttempts(); + startAttempts(attempts); + v6.handle(new VertexEventTaskCompleted(v6t1, TaskState.SUCCEEDED)); + TezTaskID v6t2 = TezTaskID.getInstance(v6.getVertexId(), 1); + attempts = v6.getTask(v6t2).getAttempts(); + startAttempts(attempts); + v6.handle(new VertexEventTaskCompleted(v6t2, TaskState.SUCCEEDED)); + dispatcher.await(); + } + + private void startAttempts(Map attempts) { + for (Map.Entry entry : attempts.entrySet()) { + TezTaskAttemptID id = entry.getKey(); + TaskAttemptImpl taskAttempt = (TaskAttemptImpl)entry.getValue(); + taskAttempt.handle(new TaskAttemptEventSchedule(id, 10, 10)); + dispatcher.await(); + ContainerId mockContainer = mock(ContainerId.class, RETURNS_DEEP_STUBS); + taskAttempt.handle(new TaskAttemptEventSubmitted(id, mockContainer)); + taskAttempt.handle(new TaskAttemptEventStartedRemotely(id)); + dispatcher.await(); + } + } } diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/launcher/TestContainerLauncherWrapper.java b/tez-dag/src/test/java/org/apache/tez/dag/app/launcher/TestContainerLauncherWrapper.java index c4f4eff0cf..cb7d62dca0 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/launcher/TestContainerLauncherWrapper.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/launcher/TestContainerLauncherWrapper.java @@ -24,7 +24,7 @@ public class TestContainerLauncherWrapper { @Test(timeout = 5000) public void testDelegation() throws Exception { PluginWrapperTestHelpers.testDelegation(ContainerLauncherWrapper.class, ContainerLauncher.class, - Sets.newHashSet("getContainerLauncher", "dagComplete", "taskAttemptFailed")); + Sets.newHashSet("getContainerLauncher", "dagComplete", "vertexComplete", "taskAttemptFailed")); } } diff --git a/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java b/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java index 7e6fd750f8..0fa1c03a96 100644 --- a/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java +++ b/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java @@ -18,6 +18,7 @@ package org.apache.tez.auxservices; +import org.apache.hadoop.fs.RemoteIterator; import org.apache.hadoop.util.DiskChecker; import static org.fusesource.leveldbjni.JniDBFactory.asString; import static org.fusesource.leveldbjni.JniDBFactory.bytes; @@ -1009,6 +1010,7 @@ private void handleRequest(ChannelHandlerContext ctx, FullHttpRequest request) final Map> q = new QueryStringDecoder(request.getUri()).parameters(); final List keepAliveList = q.get("keepAlive"); final List dagCompletedQ = q.get("dagAction"); + final List vertexCompletedQ = q.get("vertexAction"); final List taskAttemptFailedQ = q.get("taskAttemptAction"); boolean keepAliveParam = false; if (keepAliveList != null && keepAliveList.size() == 1) { @@ -1019,6 +1021,7 @@ private void handleRequest(ChannelHandlerContext ctx, FullHttpRequest request) final Range reduceRange = splitReduces(q.get("reduce")); final List jobQ = q.get("job"); final List dagIdQ = q.get("dag"); + final List vertexIdQ = q.get("vertex"); if (LOG.isDebugEnabled()) { LOG.debug("RECV: " + request.getUri() + "\n mapId: " + mapIds + @@ -1031,6 +1034,9 @@ private void handleRequest(ChannelHandlerContext ctx, FullHttpRequest request) if (deleteDagDirectories(ctx.channel(), dagCompletedQ, jobQ, dagIdQ)) { return; } + if (deleteVertexDirectories(ctx.channel(), vertexCompletedQ, jobQ, dagIdQ, vertexIdQ)) { + return; + } if (deleteTaskAttemptDirectories(ctx.channel(), taskAttemptFailedQ, jobQ, dagIdQ, mapIds)) { return; } @@ -1155,6 +1161,25 @@ private boolean deleteDagDirectories(Channel channel, return false; } + private boolean deleteVertexDirectories(Channel channel, List vertexCompletedQ, + List jobQ, List dagIdQ, + List vertexIdQ) { + if (jobQ == null || jobQ.isEmpty()) { + return false; + } + if (notEmptyAndContains(vertexCompletedQ, "delete") && !isNullOrEmpty(vertexIdQ)) { + try { + deleteTaskDirsOfVertex(jobQ.get(0), dagIdQ.get(0), vertexIdQ.get(0), userRsrc.get(jobQ.get(0))); + } catch (IOException e) { + LOG.warn("Encountered exception during vertex delete " + e); + } + channel.writeAndFlush(new DefaultHttpResponse(HTTP_1_1, OK)) + .addListener(ChannelFutureListener.CLOSE); + return true; + } + return false; + } + private boolean deleteTaskAttemptDirectories(Channel channel, List taskAttemptFailedQ, List jobQ, List dagIdQ, List taskAttemptIdQ) { if (jobQ == null || jobQ.isEmpty()) { @@ -1256,6 +1281,29 @@ private String getBaseLocation(String jobId, String dagId, String user) { return baseStr; } + /** + * Delete shuffle data in task directories belonging to a vertex. + */ + private void deleteTaskDirsOfVertex(String jobId, String dagId, String vertexId, String user) throws IOException { + String baseStr = getBaseLocation(jobId, dagId, user); + FileContext lfc = FileContext.getLocalFSFileContext(); + for(Path dagPath : lDirAlloc.getAllLocalPathsToRead(baseStr, conf)) { + RemoteIterator status = lfc.listStatus(dagPath); + final JobID jobID = JobID.forName(jobId); + String taskDirPrefix = String.format("attempt%s_%s_%s_", + jobID.toString().replace("job", ""), dagId, vertexId); + while (status.hasNext()) { + FileStatus fileStatus = status.next(); + Path attemptPath = fileStatus.getPath(); + if (attemptPath.getName().startsWith(taskDirPrefix)) { + if(lfc.delete(attemptPath, true)) { + LOG.debug("deleted shuffle data in task directory: {}", attemptPath); + } + } + } + } + } + private String getDagLocation(String jobId, String dagId, String user) { final JobID jobID = JobID.forName(jobId); final ApplicationId appID = diff --git a/tez-plugins/tez-aux-services/src/test/java/org/apache/tez/auxservices/TestShuffleHandler.java b/tez-plugins/tez-aux-services/src/test/java/org/apache/tez/auxservices/TestShuffleHandler.java index 45dd0adb06..b91e0eb203 100644 --- a/tez-plugins/tez-aux-services/src/test/java/org/apache/tez/auxservices/TestShuffleHandler.java +++ b/tez-plugins/tez-aux-services/src/test/java/org/apache/tez/auxservices/TestShuffleHandler.java @@ -24,6 +24,7 @@ import static org.junit.Assert.assertTrue; import static io.netty.buffer.Unpooled.wrappedBuffer; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.fail; import static org.junit.Assume.assumeTrue; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -1312,6 +1313,83 @@ protected void sendError(ChannelHandlerContext ctx, String message, } } + @Test + public void testVertexShuffleDelete() throws Exception { + final ArrayList failures = new ArrayList(1); + Configuration conf = new Configuration(); + conf.setInt(ShuffleHandler.MAX_SHUFFLE_CONNECTIONS, 3); + conf.setInt(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY, 0); + conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION, + "simple"); + UserGroupInformation.setConfiguration(conf); + File absLogDir = new File("target", TestShuffleHandler.class. + getSimpleName() + "LocDir").getAbsoluteFile(); + conf.set(YarnConfiguration.NM_LOCAL_DIRS, absLogDir.getAbsolutePath()); + ApplicationId appId = ApplicationId.newInstance(12345L, 1); + String appAttemptId = "attempt_12345_0001_1_00_000000_0_10003_0"; + String user = "randomUser"; + List fileMap = new ArrayList(); + String vertexDirStr = StringUtils.join(Path.SEPARATOR, new String[] { absLogDir.getAbsolutePath(), + ShuffleHandler.USERCACHE, user, ShuffleHandler.APPCACHE, appId.toString(), "dag_1/output/" + appAttemptId}); + File vertexDir = new File(vertexDirStr); + Assert.assertFalse("vertex directory should not be present", vertexDir.exists()); + createShuffleHandlerFiles(absLogDir, user, appId.toString(), appAttemptId, + conf, fileMap); + ShuffleHandler shuffleHandler = new ShuffleHandler() { + @Override + protected Shuffle getShuffle(Configuration conf) { + // replace the shuffle handler with one stubbed for testing + return new Shuffle(conf) { + @Override + protected void sendError(ChannelHandlerContext ctx, String message, + HttpResponseStatus status) { + if (failures.size() == 0) { + failures.add(new Error(message)); + ctx.channel().close(); + } + } + }; + } + }; + shuffleHandler.init(conf); + try { + shuffleHandler.start(); + DataOutputBuffer outputBuffer = new DataOutputBuffer(); + outputBuffer.reset(); + Token jt = + new Token("identifier".getBytes(), + "password".getBytes(), new Text(user), new Text("shuffleService")); + jt.write(outputBuffer); + shuffleHandler + .initializeApplication(new ApplicationInitializationContext(user, + appId, ByteBuffer.wrap(outputBuffer.getData(), 0, + outputBuffer.getLength()))); + URL url = + new URL( + "http://127.0.0.1:" + + shuffleHandler.getConfig().get( + ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY) + + "/mapOutput?vertexAction=delete&job=job_12345_0001&dag=1&vertex=00"); + HttpURLConnection conn = (HttpURLConnection) url.openConnection(); + conn.setRequestProperty(ShuffleHeader.HTTP_HEADER_NAME, + ShuffleHeader.DEFAULT_HTTP_HEADER_NAME); + conn.setRequestProperty(ShuffleHeader.HTTP_HEADER_VERSION, + ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION); + Assert.assertTrue("Attempt Directory does not exist!", vertexDir.exists()); + conn.connect(); + try { + DataInputStream is = new DataInputStream(conn.getInputStream()); + is.close(); + Assert.assertFalse("Vertex Directory was not deleted", vertexDir.exists()); + } catch (EOFException e) { + fail("Encountered Exception!" + e.getMessage()); + } + } finally { + shuffleHandler.stop(); + FileUtil.fullyDelete(absLogDir); + } + } + @Test(timeout = 5000) public void testFailedTaskAttemptDelete() throws Exception { final ArrayList failures = new ArrayList(1); diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/TezRuntimeUtils.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/TezRuntimeUtils.java index 48b23bc694..a75925c5fe 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/TezRuntimeUtils.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/TezRuntimeUtils.java @@ -187,6 +187,25 @@ public static URL constructBaseURIForShuffleHandlerDagComplete( return new URL(sb.toString()); } + public static URL constructBaseURIForShuffleHandlerVertexComplete( + String host, int port, String appId, int dagIdentifier, String vertexIndentifier, boolean sslShuffle) + throws MalformedURLException { + String httpProtocol = (sslShuffle) ? "https://" : "http://"; + StringBuilder sb = new StringBuilder(httpProtocol); + sb.append(host); + sb.append(":"); + sb.append(port); + sb.append("/"); + sb.append("mapOutput?vertexAction=delete"); + sb.append("&job="); + sb.append(appId.replace("application", "job")); + sb.append("&dag="); + sb.append(String.valueOf(dagIdentifier)); + sb.append("&vertex="); + sb.append(String.valueOf(vertexIndentifier)); + return new URL(sb.toString()); + } + public static URL constructBaseURIForShuffleHandlerTaskAttemptFailed( String host, int port, String appId, int dagIdentifier, String taskAttemptIdentifier, boolean sslShuffle) throws MalformedURLException { From f724c546069885e29e6446813805bb63bf0d5d9d Mon Sep 17 00:00:00 2001 From: Ramesh Kumar Date: Wed, 23 Mar 2022 21:34:47 -0700 Subject: [PATCH 101/137] TEZ-4397 Open Tez Input splits asynchronously Contributed by Ramesh Kumar Thangarajan --- .../split/TezGroupedSplitsInputFormat.java | 72 +++++++++++++++++-- .../mapreduce/grouper/TezSplitGrouper.java | 11 +++ 2 files changed, 79 insertions(+), 4 deletions(-) diff --git a/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/split/TezGroupedSplitsInputFormat.java b/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/split/TezGroupedSplitsInputFormat.java index 61ba560300..6266ec1bcf 100644 --- a/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/split/TezGroupedSplitsInputFormat.java +++ b/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/split/TezGroupedSplitsInputFormat.java @@ -19,8 +19,16 @@ package org.apache.hadoop.mapred.split; import java.io.IOException; +import java.util.LinkedList; import java.util.Objects; +import java.util.Queue; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.atomic.AtomicInteger; +import com.google.common.util.concurrent.ThreadFactoryBuilder; +import org.apache.tez.mapreduce.grouper.TezSplitGrouper; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.classification.InterfaceAudience.Public; @@ -129,14 +137,58 @@ public class TezGroupedSplitsRecordReader implements RecordReader { int idx = 0; long progress; RecordReader curReader; - + final AtomicInteger initIndex; + final int numReaders; + final ExecutorService initReaderExecService; + final Queue>> initedReaders; + public TezGroupedSplitsRecordReader(TezGroupedSplit split, JobConf job, Reporter reporter) throws IOException { this.groupedSplit = split; this.job = job; this.reporter = reporter; + this.initIndex = new AtomicInteger(0); + int numThreads = conf.getInt(TezSplitGrouper.TEZ_GROUPING_SPLIT_INIT_THREADS, + TezSplitGrouper.TEZ_GROUPING_SPLIT_INIT_THREADS_DEFAULT); + this.numReaders = conf.getInt(TezSplitGrouper.TEZ_GROUPING_SPLIT_INIT_NUM_RECORDREADERS, + TezSplitGrouper.TEZ_GROUPING_SPLIT_INIT_NUM_RECORDREADERS_DEFAULT); + this.initReaderExecService = Executors.newFixedThreadPool(numThreads, + new ThreadFactoryBuilder() + .setDaemon(true) + .setPriority(Thread.MAX_PRIORITY) + .setNameFormat("TEZ-Split-Init-Thread-%d") + .build()); + this.initedReaders = new LinkedList<>(); + preInitReaders(); initNextRecordReader(); } + + private void preInitReaders() { + if (initReaderExecService == null) { + LOG.info("Init record reader threadpool is not initialized"); + return; + } + for (int i = 0; i < numReaders; i++) { + initedReaders.offer(this.initReaderExecService.submit(() -> { + try { + int index = initIndex.getAndIncrement(); + if (index >= groupedSplit.wrappedSplits.size()) { + return null; + } + InputSplit s = groupedSplit.wrappedSplits.get(index); + RecordReader reader = wrappedInputFormat.getRecordReader(s, job, reporter); + LOG.debug("Init Thread processed reader number {} initialization", index); + return reader; + } catch (Exception e) { + if (e instanceof InterruptedException) { + Thread.currentThread().interrupt(); + } + cancelsFutures(); + throw new RuntimeException(e); + } + })); + } + } @Override public boolean next(K key, V value) throws IOException { @@ -183,6 +235,8 @@ protected boolean initNextRecordReader() throws IOException { // if all chunks have been processed, nothing more to do. if (idx == groupedSplit.wrappedSplits.size()) { + LOG.info("Shutting down the init record reader threadpool"); + initReaderExecService.shutdownNow(); return false; } @@ -193,15 +247,25 @@ protected boolean initNextRecordReader() throws IOException { // get a record reader for the idx-th chunk try { - curReader = wrappedInputFormat.getRecordReader( - groupedSplit.wrappedSplits.get(idx), job, reporter); + curReader = initedReaders.poll().get(); + preInitReaders(); } catch (Exception e) { - throw new RuntimeException (e); + if (e instanceof InterruptedException) { + Thread.currentThread().interrupt(); + } + cancelsFutures(); + throw new RuntimeException(e); } idx++; return true; } + private void cancelsFutures() { + for (Future> f : initedReaders) { + f.cancel(true); + } + } + @Override public long getPos() throws IOException { long subprogress = 0; // bytes processed in current split diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/grouper/TezSplitGrouper.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/grouper/TezSplitGrouper.java index a1d6b6c806..3b2f17d1ff 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/grouper/TezSplitGrouper.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/grouper/TezSplitGrouper.java @@ -102,6 +102,17 @@ public abstract class TezSplitGrouper { public static final String TEZ_GROUPING_NODE_LOCAL_ONLY = "tez.grouping.node.local.only"; public static final boolean TEZ_GROUPING_NODE_LOCAL_ONLY_DEFAULT = false; + /** + * Number of threads used to initialize the grouped splits, to asynchronously open the readers. + */ + public static final String TEZ_GROUPING_SPLIT_INIT_THREADS = "tez.grouping.split.init-threads"; + public static final int TEZ_GROUPING_SPLIT_INIT_THREADS_DEFAULT = 4; + + /** + * Number of record readers to asynchronously and proactively init. + */ + public static final String TEZ_GROUPING_SPLIT_INIT_NUM_RECORDREADERS = "tez.grouping.split.init.num-recordreaders"; + public static final int TEZ_GROUPING_SPLIT_INIT_NUM_RECORDREADERS_DEFAULT = 10; static class LocationHolder { List splits; From 3e452e9859c507f089c6d6c01cbfabe24fd7b1d9 Mon Sep 17 00:00:00 2001 From: Himanshu Mishra Date: Fri, 1 Apr 2022 14:15:03 +0530 Subject: [PATCH 102/137] TEZ-4399: ShuffleHandler fails with SSLHandshakeException not found when SSL is enabled (#196) (Himanshu Mishra reviewed by Laszlo Bodor) --- .../org/apache/tez/dag/app/launcher/DagDeleteRunnable.java | 2 +- .../tez/dag/app/launcher/TaskAttemptFailedDeleteRunnable.java | 2 +- .../org/apache/tez/dag/app/launcher/VertexDeleteRunnable.java | 3 ++- tez-plugins/tez-aux-services/pom.xml | 1 + 4 files changed, 5 insertions(+), 3 deletions(-) diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/DagDeleteRunnable.java b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/DagDeleteRunnable.java index eac745e1ff..cbea36b86d 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/DagDeleteRunnable.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/DagDeleteRunnable.java @@ -54,7 +54,7 @@ public void run() { try { URL baseURL = TezRuntimeUtils.constructBaseURIForShuffleHandlerDagComplete( nodeId.getHost(), shufflePort, - dag.getApplicationId().toString(), dag.getId(), false); + dag.getApplicationId().toString(), dag.getId(), httpConnectionParams.isSslShuffle()); httpConnection = TezRuntimeUtils.getHttpConnection(true, baseURL, httpConnectionParams, "DAGDelete", jobTokenSecretManager); httpConnection.connect(); diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/TaskAttemptFailedDeleteRunnable.java b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/TaskAttemptFailedDeleteRunnable.java index 22c5b26a7c..3a9f5345a0 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/TaskAttemptFailedDeleteRunnable.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/TaskAttemptFailedDeleteRunnable.java @@ -55,7 +55,7 @@ public void run() { URL baseURL = TezRuntimeUtils.constructBaseURIForShuffleHandlerTaskAttemptFailed( nodeId.getHost(), shufflePort, taskAttemptID.getTaskID().getVertexID().getDAGID(). getApplicationId().toString(), taskAttemptID.getTaskID().getVertexID().getDAGID().getId(), - taskAttemptID.toString(), false); + taskAttemptID.toString(), httpConnectionParams.isSslShuffle()); httpConnection = TezRuntimeUtils.getHttpConnection(true, baseURL, httpConnectionParams, "FailedTaskAttemptDelete", jobTokenSecretManager); httpConnection.connect(); diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/VertexDeleteRunnable.java b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/VertexDeleteRunnable.java index a8d25379a5..3bfec9663a 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/VertexDeleteRunnable.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/launcher/VertexDeleteRunnable.java @@ -56,7 +56,8 @@ public void run() { try { URL baseURL = TezRuntimeUtils.constructBaseURIForShuffleHandlerVertexComplete( nodeId.getHost(), shufflePort, - vertex.getDAGID().getApplicationId().toString(), vertex.getDAGID().getId(), vertexId, false); + vertex.getDAGID().getApplicationId().toString(), vertex.getDAGID().getId(), vertexId, + httpConnectionParams.isSslShuffle()); httpConnection = TezRuntimeUtils.getHttpConnection(true, baseURL, httpConnectionParams, "VertexDelete", jobTokenSecretManager); httpConnection.connect(); diff --git a/tez-plugins/tez-aux-services/pom.xml b/tez-plugins/tez-aux-services/pom.xml index 76f431b3ac..9493bd51dd 100644 --- a/tez-plugins/tez-aux-services/pom.xml +++ b/tez-plugins/tez-aux-services/pom.xml @@ -247,6 +247,7 @@ javax.crypto.* javax.security.** + javax.net.** From 627f33077480afdcefc0611fbde87d6be0010176 Mon Sep 17 00:00:00 2001 From: csjuhasz-c Date: Thu, 14 Apr 2022 11:33:56 +0200 Subject: [PATCH 103/137] TEZ-4398: Gitignore pyc files (#199) (Csaba Juhasz reviewed by Laszlo Bodor) --- .gitignore | 1 + 1 file changed, 1 insertion(+) diff --git a/.gitignore b/.gitignore index 01c99576d6..85d660672c 100644 --- a/.gitignore +++ b/.gitignore @@ -2,6 +2,7 @@ *.ipr *.iws *.DS_Store +*.pyc .idea .svn .classpath From 9f8d6fbd34bc99756e9115373917a05fad2ab8fc Mon Sep 17 00:00:00 2001 From: guptanikhil007 Date: Tue, 26 Apr 2022 04:29:03 +0530 Subject: [PATCH 104/137] Create a FileSystem for given Path (#201) Co-authored-by: Nikhil Gupta --- tez-api/src/main/java/org/apache/tez/client/TezClient.java | 5 +++-- .../client/rpc/DAGClientAMProtocolBlockingPBServerImpl.java | 5 +++-- 2 files changed, 6 insertions(+), 4 deletions(-) diff --git a/tez-api/src/main/java/org/apache/tez/client/TezClient.java b/tez-api/src/main/java/org/apache/tez/client/TezClient.java index 378017b364..c37f0c181d 100644 --- a/tez-api/src/main/java/org/apache/tez/client/TezClient.java +++ b/tez-api/src/main/java/org/apache/tez/client/TezClient.java @@ -688,14 +688,15 @@ private DAGClient submitDAGSession(DAG dag) throws TezException, IOException { sessionAppId.toString()), TezConstants.TEZ_PB_PLAN_BINARY_NAME + serializedSubmitDAGPlanRequestCounter.incrementAndGet()); - try (FSDataOutputStream fsDataOutputStream = stagingFs.create(dagPlanPath, false)) { + FileSystem fs = dagPlanPath.getFileSystem(stagingFs.getConf()); + try (FSDataOutputStream fsDataOutputStream = fs.create(dagPlanPath, false)) { LOG.info("Send dag plan using YARN local resources since it's too large" + ", dag plan size=" + request.getSerializedSize() + ", max dag plan size through IPC=" + maxSubmitDAGRequestSizeThroughIPC + ", max IPC message size= " + amConfig.getTezConfiguration().getInt( CommonConfigurationKeys.IPC_MAXIMUM_DATA_LENGTH, CommonConfigurationKeys.IPC_MAXIMUM_DATA_LENGTH_DEFAULT)); request.writeTo(fsDataOutputStream); - request = requestBuilder.clear().setSerializedRequestPath(stagingFs.resolvePath(dagPlanPath).toString()).build(); + request = requestBuilder.clear().setSerializedRequestPath(fs.resolvePath(dagPlanPath).toString()).build(); } } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientAMProtocolBlockingPBServerImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientAMProtocolBlockingPBServerImpl.java index 72cf0d5642..4bdb468859 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientAMProtocolBlockingPBServerImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientAMProtocolBlockingPBServerImpl.java @@ -166,7 +166,8 @@ public SubmitDAGResponseProto submitDAG(RpcController controller, if (request.hasSerializedRequestPath()) { // need to deserialize large request from hdfs Path requestPath = new Path(request.getSerializedRequestPath()); - try (FSDataInputStream fsDataInputStream = stagingFs.open(requestPath)) { + FileSystem fs = requestPath.getFileSystem(stagingFs.getConf()); + try (FSDataInputStream fsDataInputStream = fs.open(requestPath)) { CodedInputStream in = CodedInputStream.newInstance(fsDataInputStream); in.setSizeLimit(Integer.MAX_VALUE); @@ -183,7 +184,7 @@ public SubmitDAGResponseProto submitDAG(RpcController controller, } String dagId = real.submitDAG(dagPlan, additionalResources); return SubmitDAGResponseProto.newBuilder().setDagId(dagId).build(); - } catch(TezException e) { + } catch(IOException | TezException e) { throw wrapException(e); } } From 2a9495afe1ef9080941cb903fba0189abb7d3c8f Mon Sep 17 00:00:00 2001 From: guptanikhil007 Date: Fri, 29 Apr 2022 12:26:19 +0530 Subject: [PATCH 105/137] TEZ-4411: Update FileSaver dependency (#206) (Nikhil Gupta reviewed by Laszlo Bodor, Deependra Patel) --- tez-ui/src/main/resources/META-INF/LICENSE.txt | 2 +- tez-ui/src/main/webapp/bower-shrinkwrap.json | 8 ++++---- tez-ui/src/main/webapp/bower.json | 4 ++-- tez-ui/src/main/webapp/ember-cli-build.js | 2 +- 4 files changed, 8 insertions(+), 8 deletions(-) diff --git a/tez-ui/src/main/resources/META-INF/LICENSE.txt b/tez-ui/src/main/resources/META-INF/LICENSE.txt index 608dc614c8..833a2b00c6 100644 --- a/tez-ui/src/main/resources/META-INF/LICENSE.txt +++ b/tez-ui/src/main/resources/META-INF/LICENSE.txt @@ -224,7 +224,7 @@ The Apache TEZ tez-ui bundles the following files under the MIT License: - jquery-ui v1.11.4 (http://jqueryui.com/) - Copyright 2014 jQuery Foundation and other contributors - jquery-mousewheel v3.1.13 (https://github.com/jquery/jquery-mousewheel) - Copyright 2006, 2014 jQuery Foundation and other contributors, https://jquery.org/ - CodeMirror 5.11.0 (https://codemirror.net/) - Copyright (C) 2015 by Marijn Haverbeke and others - - file-saver.js v1.20150507.2 (https://github.com/Teleborder/FileSaver.js) - Authored by Eli Grey + - file-saver v1.3.4 (https://github.com/eligrey/FileSaver.js) - Authored by Eli Grey - moment v2.12.0 (http://momentjs.com/) - Copyright (c) 2011-2015 Tim Wood, Iskren Chernev, Moment.js contributors - moment-timezone v0.5.0 (http://momentjs.com/timezone/) - Copyright (c) 2014 Tim Wood - font-awesome css/less files v4.5.0 (http://fontawesome.io/) - Created by Dave Gandy diff --git a/tez-ui/src/main/webapp/bower-shrinkwrap.json b/tez-ui/src/main/webapp/bower-shrinkwrap.json index 357d57691a..c8c506edce 100644 --- a/tez-ui/src/main/webapp/bower-shrinkwrap.json +++ b/tez-ui/src/main/webapp/bower-shrinkwrap.json @@ -2,9 +2,6 @@ "https://github.com/FortAwesome/Font-Awesome.git": { "4.5.0": "593ad563a987977f14102be935d0abc2a172903e" }, - "https://github.com/Teleborder/FileSaver.js.git": { - "1.20150507.2": "b7cf622909258086bc63ad764d08fcaed780ab42" - }, "https://github.com/adamwdraper/Numeral-js.git": { "1.5.3": "f97f14bb8bab988f28f1d854525b4cfeff8ec9e1" }, @@ -26,6 +23,9 @@ "https://github.com/dockyard/qunit-notifications.git": { "0.1.1": "7a13f6dba5a340e1cb9e0b64c1c711e4d7edaca1" }, + "https://github.com/eligrey/FileSaver.js.git": { + "1.2.0": "a6d11998e279e94f2926b2a897231355dfab48ab" + }, "https://github.com/ember-cli/ember-cli-shims.git": { "0.0.6": "dcab43b58d5698690050bb9a46ead5c8663c7da1" }, @@ -69,4 +69,4 @@ "https://github.com/twbs/bootstrap.git": { "3.3.6": "81df608a40bf0629a1dc08e584849bb1e43e0b7a" } -} \ No newline at end of file +} diff --git a/tez-ui/src/main/webapp/bower.json b/tez-ui/src/main/webapp/bower.json index cca56d817a..fb0498f11e 100644 --- a/tez-ui/src/main/webapp/bower.json +++ b/tez-ui/src/main/webapp/bower.json @@ -22,7 +22,7 @@ "snippet-ss": "1.11.0", "jquery-mousewheel": "3.1.13", "codemirror": "5.11.0", - "file-saver.js": "1.20150507.2", - "zip-js": "1.0.0" + "zip-js": "1.0.0", + "file-saver": "v1.2.0" } } diff --git a/tez-ui/src/main/webapp/ember-cli-build.js b/tez-ui/src/main/webapp/ember-cli-build.js index e4217e9591..f34092c52a 100644 --- a/tez-ui/src/main/webapp/ember-cli-build.js +++ b/tez-ui/src/main/webapp/ember-cli-build.js @@ -64,7 +64,7 @@ module.exports = function(defaults) { app.import('bower_components/more-js/dist/more.js'); - app.import('bower_components/file-saver.js/FileSaver.js'); + app.import('bower_components/file-saver/FileSaver.js'); app.import('bower_components/zip-js/WebContent/zip.js'); app.import('bower_components/codemirror/lib/codemirror.js'); From 4d1a86070316a1b85196d06df94787268084a5da Mon Sep 17 00:00:00 2001 From: Syed Shameerur Rahman Date: Sat, 30 Apr 2022 13:37:04 +0530 Subject: [PATCH 106/137] TEZ-4403: Upgrade SLF4J Version To 1.7.36 (#198) (Syed Shameerur Rahman reviewed by Laszlo Bodor) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 9499fe62b6..10c4200312 100644 --- a/pom.xml +++ b/pom.xml @@ -62,7 +62,7 @@ 4.1.72.Final 0.13.0 1.19 - 1.7.30 + 1.7.36 2.5.0 0.7.45 ${env.PROTOC_PATH} From 75876fcc76cff736a6321c2e85ca1a3fdcc9dd42 Mon Sep 17 00:00:00 2001 From: Murali Krishna Date: Mon, 2 May 2022 12:09:54 +0530 Subject: [PATCH 107/137] TEZ-4405: Replace log4j 1.x with reload4j (#200) (D M Murali Krishna Reddy reviewed by Laszlo Bodor) --- pom.xml | 2 +- tez-common/pom.xml | 2 +- tez-ext-service-tests/pom.xml | 2 +- tez-plugins/tez-aux-services/pom.xml | 2 +- tez-runtime-internals/pom.xml | 2 +- 5 files changed, 5 insertions(+), 5 deletions(-) diff --git a/pom.xml b/pom.xml index 10c4200312..9ed00c9974 100644 --- a/pom.xml +++ b/pom.xml @@ -253,7 +253,7 @@ org.slf4j - slf4j-log4j12 + slf4j-reload4j ${slf4j.version} diff --git a/tez-common/pom.xml b/tez-common/pom.xml index 5fd6910b18..0fd4255abe 100644 --- a/tez-common/pom.xml +++ b/tez-common/pom.xml @@ -27,7 +27,7 @@ org.slf4j - slf4j-log4j12 + slf4j-reload4j com.google.guava diff --git a/tez-ext-service-tests/pom.xml b/tez-ext-service-tests/pom.xml index 327d457187..8c2d84ee7c 100644 --- a/tez-ext-service-tests/pom.xml +++ b/tez-ext-service-tests/pom.xml @@ -36,7 +36,7 @@ org.slf4j - slf4j-log4j12 + slf4j-reload4j com.google.guava diff --git a/tez-plugins/tez-aux-services/pom.xml b/tez-plugins/tez-aux-services/pom.xml index 9493bd51dd..b9204d2a9d 100644 --- a/tez-plugins/tez-aux-services/pom.xml +++ b/tez-plugins/tez-aux-services/pom.xml @@ -207,7 +207,7 @@ true - log4j:log4j + ch.qos.reload4j:reload4j org.slf4j:* diff --git a/tez-runtime-internals/pom.xml b/tez-runtime-internals/pom.xml index f7e14dae59..65841cf666 100644 --- a/tez-runtime-internals/pom.xml +++ b/tez-runtime-internals/pom.xml @@ -76,7 +76,7 @@ org.slf4j - slf4j-log4j12 + slf4j-reload4j From 798ddda06dbd4353f1088b7b4f2200831ff28910 Mon Sep 17 00:00:00 2001 From: Bodor Laszlo Date: Wed, 4 May 2022 06:44:54 +0200 Subject: [PATCH 108/137] TEZ-4347: Add some diagnostic endpoints to TezAM's WebUIService (#160) (Laszlo Bodor reviewed by Rajesh Balamohan) --- .../apache/tez/dag/api/TezConfiguration.java | 12 ++ .../apache/tez/dag/api/client/DAGClient.java | 9 ++ .../tez/dag/api/client/DAGClientImpl.java | 7 +- .../tez/dag/api/client/DAGClientInternal.java | 2 + .../dag/api/client/DAGClientTimelineImpl.java | 5 + .../dag/api/client/rpc/DAGClientRPCImpl.java | 12 ++ .../src/main/proto/DAGClientAMProtocol.proto | 8 + .../AbstractServletToControllerAdapter.java | 104 +++++++++++++ .../web/ServletToControllerAdapters.java | 45 ++++++ .../apache/tez/common/web/package-info.java | 22 +++ .../tez/dag/api/client/DAGClientHandler.java | 3 + ...GClientAMProtocolBlockingPBServerImpl.java | 8 + .../org/apache/tez/dag/app/DAGAppMaster.java | 4 + .../apache/tez/dag/app/web/WebUIService.java | 23 ++- .../tez/dag/api/client/MRDAGClient.java | 6 + .../test/java/org/apache/tez/test/TestAM.java | 146 ++++++++++++++++++ 16 files changed, 414 insertions(+), 2 deletions(-) create mode 100644 tez-common/src/main/java/org/apache/tez/common/web/AbstractServletToControllerAdapter.java create mode 100644 tez-common/src/main/java/org/apache/tez/common/web/ServletToControllerAdapters.java create mode 100644 tez-common/src/main/java/org/apache/tez/common/web/package-info.java create mode 100644 tez-tests/src/test/java/org/apache/tez/test/TestAM.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 71ebfee10b..1ffd70a3ff 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 @@ -1980,6 +1980,18 @@ public TezConfiguration(boolean loadDefaults) { + "tez-ui.webservice.enable"; public static final boolean TEZ_AM_WEBSERVICE_ENABLE_DEFAULT = true; + /** + * String value. Range of ports that the AM can use for the WebUIService. Leave blank + * to use all possible ports. Expert level setting. It's hadoop standard range configuration. + * For example 50000-50050,50100-50200 + */ + @ConfigurationScope(Scope.AM) + @ConfigurationProperty(type="boolean") + public static final String TEZ_AM_WEBSERVICE_PORT_RANGE = TEZ_AM_PREFIX + + "tez-ui.webservice.port-range"; + + public static final String TEZ_AM_WEBSERVICE_PORT_RANGE_DEFAULT = "50000-50050"; + // 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); diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClient.java b/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClient.java index ec20ef1de4..944bff3fbd 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClient.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClient.java @@ -140,4 +140,13 @@ public DAGStatus waitForCompletion(long timeMs) throws IOException, TezException public abstract DAGStatus waitForCompletionWithStatusUpdates(@Nullable Set statusGetOpts) throws IOException, TezException, InterruptedException; + /** + * Returns the Tez AM's web ui address if any. + * + * @return The http web UI address + * @throws IOException + * @throws TezException + */ + public abstract String getWebUIAddress() throws IOException, TezException; + } 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 3c0de00819..bfea96b998 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 @@ -30,10 +30,10 @@ import java.util.concurrent.TimeUnit; import com.google.common.annotations.VisibleForTesting; + import org.apache.hadoop.security.UserGroupInformation; import org.apache.tez.common.CachedEntity; import org.apache.tez.common.Preconditions; - import org.apache.hadoop.yarn.exceptions.ApplicationNotFoundException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -670,6 +670,11 @@ public DAGClientInternal getRealClient() { return realClient; } + @Override + public String getWebUIAddress() throws IOException, TezException { + return realClient.getWebUIAddress(); + } + private double getProgress(Progress progress) { return (progress.getTotalTaskCount() == 0 ? 0.0 : (double) (progress.getSucceededTaskCount()) / progress.getTotalTaskCount()); diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClientInternal.java b/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClientInternal.java index a3c898a855..8346d53da7 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClientInternal.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClientInternal.java @@ -125,4 +125,6 @@ public abstract VertexStatus getVertexStatus(String vertexName, */ public abstract DAGStatus waitForCompletionWithStatusUpdates(@Nullable Set statusGetOpts) throws IOException, TezException, InterruptedException; + + public abstract String getWebUIAddress() throws IOException, TezException; } 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 17d2386860..4ec9c94354 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 @@ -523,4 +523,9 @@ public DAGStatus getDAGStatus(@Nullable Set statusOptions, return getDAGStatus(statusOptions); } + @Override + public String getWebUIAddress() throws IOException, TezException { + throw new TezException("DAGClientTimelineImpl.getWebUIAddress is not supported"); + } + } diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientRPCImpl.java b/tez-api/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientRPCImpl.java index 5d5752e6e2..798160a4ab 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientRPCImpl.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientRPCImpl.java @@ -48,6 +48,7 @@ import org.apache.tez.dag.api.client.VertexStatus; import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.GetDAGStatusRequestProto; import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.GetVertexStatusRequestProto; +import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.GetWebUIAddressRequestProto; import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.TryKillDAGRequestProto; import com.google.common.annotations.VisibleForTesting; @@ -303,4 +304,15 @@ public DAGStatus waitForCompletionWithStatusUpdates(@Nullable Set throw new TezException("not supported"); } + @Override + public String getWebUIAddress() throws IOException, TezException { + LOG.debug("getWebUIAddress via AM for app: {} dag: {}", appId, dagId); + GetWebUIAddressRequestProto.Builder requestProtoBuilder = GetWebUIAddressRequestProto.newBuilder(); + try { + return proxy.getWebUIAddress(null, requestProtoBuilder.build()).getWebUiAddress(); + } catch (ServiceException e) { + RPCUtil.unwrapAndThrowException(e); + throw new TezException(e); + } + } } diff --git a/tez-api/src/main/proto/DAGClientAMProtocol.proto b/tez-api/src/main/proto/DAGClientAMProtocol.proto index 113c9ccfce..f0ff3916ea 100644 --- a/tez-api/src/main/proto/DAGClientAMProtocol.proto +++ b/tez-api/src/main/proto/DAGClientAMProtocol.proto @@ -90,6 +90,13 @@ message GetAMStatusResponseProto { required TezAppMasterStatusProto status = 1; } +message GetWebUIAddressRequestProto { +} + +message GetWebUIAddressResponseProto { + required string web_ui_address = 1; +} + service DAGClientAMProtocol { rpc getAllDAGs (GetAllDAGsRequestProto) returns (GetAllDAGsResponseProto); rpc getDAGStatus (GetDAGStatusRequestProto) returns (GetDAGStatusResponseProto); @@ -98,4 +105,5 @@ service DAGClientAMProtocol { rpc submitDAG (SubmitDAGRequestProto) returns (SubmitDAGResponseProto); rpc shutdownSession (ShutdownSessionRequestProto) returns (ShutdownSessionResponseProto); rpc getAMStatus (GetAMStatusRequestProto) returns (GetAMStatusResponseProto); + rpc getWebUIAddress (GetWebUIAddressRequestProto) returns (GetWebUIAddressResponseProto); } diff --git a/tez-common/src/main/java/org/apache/tez/common/web/AbstractServletToControllerAdapter.java b/tez-common/src/main/java/org/apache/tez/common/web/AbstractServletToControllerAdapter.java new file mode 100644 index 0000000000..b79b5d5d9c --- /dev/null +++ b/tez-common/src/main/java/org/apache/tez/common/web/AbstractServletToControllerAdapter.java @@ -0,0 +1,104 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tez.common.web; + +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; +import java.util.Enumeration; +import java.util.concurrent.atomic.AtomicBoolean; + +import javax.servlet.ServletConfig; +import javax.servlet.ServletContext; +import javax.servlet.ServletException; +import javax.servlet.http.HttpServlet; +import javax.servlet.http.HttpServletRequest; +import javax.servlet.http.HttpServletResponse; + +import org.apache.hadoop.yarn.webapp.Controller; + +/** + * AbstractServletToControllerAdapter is a common ancestor for classes + * that wish to adapt servlets to yarn webapp controllers. + * The adapter is responsible for: + * 1. creating a servlet instance + * 2. creating a dummy ServletConfig + * 3. delegating calls to the servlet instance's doGet method + */ +public abstract class AbstractServletToControllerAdapter extends Controller { + private AtomicBoolean initialized = new AtomicBoolean(false); + protected HttpServlet servlet; + + @Override + public void index() { + if (initialized.compareAndSet(false, true)) { + initServlet(); + } + try { + /* + * This reflection workaround is needed because HttpServlet.doGet is protected + * (even if subclasses have it public). + */ + Method doGetMethod = + this.servlet.getClass().getMethod("doGet", HttpServletRequest.class, HttpServletResponse.class); + doGetMethod.setAccessible(true); + doGetMethod.invoke(this.servlet, request(), response()); + } catch (IllegalAccessException | IllegalArgumentException | InvocationTargetException | NoSuchMethodException + | SecurityException e) { + throw new RuntimeException(e); + } + } + + /** + * Creates a dummy servlet config which is suitable for initializing a servlet instance. + * @param servletName + * @return a ServletConfig instance initialized with a ServletContext + */ + private ServletConfig getDummyServletConfig(String servletName) { + return new ServletConfig() { + + @Override + public String getServletName() { + return servletName; + } + + @Override + public ServletContext getServletContext() { + return request().getServletContext(); + } + + @Override + public Enumeration getInitParameterNames() { + return null; + } + + @Override + public String getInitParameter(String name) { + return null; + } + }; + } + + private void initServlet() { + try { + servlet.init(getDummyServletConfig(this.servlet.getClass().getSimpleName())); + } catch (ServletException e) { + throw new RuntimeException(e); + } + } +} diff --git a/tez-common/src/main/java/org/apache/tez/common/web/ServletToControllerAdapters.java b/tez-common/src/main/java/org/apache/tez/common/web/ServletToControllerAdapters.java new file mode 100644 index 0000000000..35ca1b6408 --- /dev/null +++ b/tez-common/src/main/java/org/apache/tez/common/web/ServletToControllerAdapters.java @@ -0,0 +1,45 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tez.common.web; + +import javax.servlet.ServletException; + +import org.apache.hadoop.conf.ConfServlet; +import org.apache.hadoop.http.HttpServer2.StackServlet; +import org.apache.hadoop.jmx.JMXJsonServlet; + +public class ServletToControllerAdapters { + public static class JMXJsonServletController extends AbstractServletToControllerAdapter { + public JMXJsonServletController() throws ServletException { + this.servlet = new JMXJsonServlet(); + } + } + + public static class ConfServletController extends AbstractServletToControllerAdapter { + public ConfServletController() throws ServletException { + this.servlet = new ConfServlet(); + } + } + + public static class StackServletController extends AbstractServletToControllerAdapter { + public StackServletController() throws ServletException { + this.servlet = new StackServlet(); + } + } +} diff --git a/tez-common/src/main/java/org/apache/tez/common/web/package-info.java b/tez-common/src/main/java/org/apache/tez/common/web/package-info.java new file mode 100644 index 0000000000..2fbda31fda --- /dev/null +++ b/tez-common/src/main/java/org/apache/tez/common/web/package-info.java @@ -0,0 +1,22 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +@Private +package org.apache.tez.common.web; + +import org.apache.hadoop.classification.InterfaceAudience.Private; \ No newline at end of file diff --git a/tez-dag/src/main/java/org/apache/tez/dag/api/client/DAGClientHandler.java b/tez-dag/src/main/java/org/apache/tez/dag/api/client/DAGClientHandler.java index 4cdd1ec9d1..1de62012e7 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/api/client/DAGClientHandler.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/api/client/DAGClientHandler.java @@ -186,4 +186,7 @@ public long getLastHeartbeatTime() { return lastHeartbeatTime.get(); } + public String getWebUIAddress() { + return dagAppMaster.getWebUIAddress(); + } } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientAMProtocolBlockingPBServerImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientAMProtocolBlockingPBServerImpl.java index 4bdb468859..5c24a27908 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientAMProtocolBlockingPBServerImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientAMProtocolBlockingPBServerImpl.java @@ -45,6 +45,8 @@ import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.GetDAGStatusResponseProto; import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.GetVertexStatusRequestProto; import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.GetVertexStatusResponseProto; +import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.GetWebUIAddressRequestProto; +import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.GetWebUIAddressResponseProto; import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.ShutdownSessionRequestProto; import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.ShutdownSessionResponseProto; import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.SubmitDAGRequestProto; @@ -227,4 +229,10 @@ public GetAMStatusResponseProto getAMStatus(RpcController controller, } } + @Override + public GetWebUIAddressResponseProto getWebUIAddress(RpcController controller, GetWebUIAddressRequestProto request) + throws ServiceException { + String address = real.getWebUIAddress(); + return GetWebUIAddressResponseProto.newBuilder().setWebUiAddress(address).build(); + } } 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 58288612bb..ca50b2ac6b 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 @@ -2618,6 +2618,10 @@ private boolean enableWebUIService() { TezConfiguration.TEZ_AM_WEBSERVICE_ENABLE_DEFAULT); } + public String getWebUIAddress() { + return webUIService == null ? null : webUIService.getBaseUrl(); + } + @VisibleForTesting static void parseAllPlugins( List taskSchedulerDescriptors, BiMap taskSchedulerPluginMap, diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/web/WebUIService.java b/tez-dag/src/main/java/org/apache/tez/dag/app/web/WebUIService.java index 1670370187..bf94a73002 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/web/WebUIService.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/web/WebUIService.java @@ -23,6 +23,10 @@ import java.net.InetSocketAddress; import org.apache.tez.common.Preconditions; +import org.apache.tez.common.web.ServletToControllerAdapters.ConfServletController; +import org.apache.tez.common.web.ServletToControllerAdapters.JMXJsonServletController; +import org.apache.tez.common.web.ServletToControllerAdapters.StackServletController; + import com.google.inject.name.Names; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -32,6 +36,7 @@ import org.apache.hadoop.yarn.webapp.WebApp; import org.apache.hadoop.yarn.webapp.WebApps; import org.apache.hadoop.yarn.webapp.YarnWebParams; +import org.apache.tez.dag.api.TezConfiguration; import org.apache.tez.dag.api.TezUncheckedException; import org.apache.tez.dag.app.AppContext; @@ -51,6 +56,7 @@ public class WebUIService extends AbstractService { private final AppContext context; private TezAMWebApp tezAMWebApp; private WebApp webApp; + private String baseUrl = ""; //url without paths, like http://host:port private String trackingUrl = ""; private String historyUrl = ""; @@ -88,9 +94,16 @@ protected void serviceStart() throws Exception { // certificates, however AM user is not trusted. // ideally the withHttpPolicy should be used, however hadoop 2.2 does not have the api conf.set("yarn.http.policy", "HTTP_ONLY"); + if (conf.get(TezConfiguration.TEZ_AM_WEBSERVICE_PORT_RANGE) == null) { + conf.set(TezConfiguration.TEZ_AM_WEBSERVICE_PORT_RANGE, + TezConfiguration.TEZ_AM_WEBSERVICE_PORT_RANGE_DEFAULT); + LOG.info( + "Using default port range for WebUIService: " + conf.get(TezConfiguration.TEZ_AM_WEBSERVICE_PORT_RANGE)); + } this.webApp = WebApps .$for(this.tezAMWebApp) .with(conf) + .withPortRange(conf, TezConfiguration.TEZ_AM_WEBSERVICE_PORT_RANGE) .start(this.tezAMWebApp); InetSocketAddress address = webApp.getListenerAddress(); if (address != null) { @@ -105,7 +118,8 @@ protected void serviceStart() throws Exception { LOG.warn("Failed to resolve canonical hostname for " + context.getAppMaster().getAppNMHost()); } - trackingUrl = "http://" + hostname + ":" + port + "/ui/"; + baseUrl = "http://" + hostname + ":" + port; + trackingUrl = baseUrl + "/ui/"; LOG.info("Instantiated WebUIService at " + trackingUrl); } } catch (Exception e) { @@ -125,6 +139,10 @@ protected void serviceStop() throws Exception { super.serviceStop(); } + public String getBaseUrl() { + return baseUrl; + } + public String getTrackingURL() { return trackingUrl; } @@ -214,6 +232,9 @@ public void setup() { "getTasksInfo"); route(WS_PREFIX_V2 + pajoin("attemptsInfo", ATTEMPT_ID, DAG_ID), AMWebController.class, "getAttemptsInfo"); + route("/jmx", JMXJsonServletController.class); + route("/conf", ConfServletController.class); + route("/stacks", StackServletController.class); } } } diff --git a/tez-mapreduce/src/main/java/org/apache/tez/dag/api/client/MRDAGClient.java b/tez-mapreduce/src/main/java/org/apache/tez/dag/api/client/MRDAGClient.java index 16dc2f8c01..c2646bd819 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/dag/api/client/MRDAGClient.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/dag/api/client/MRDAGClient.java @@ -106,4 +106,10 @@ public DAGStatus getDAGStatus(@Nullable Set statusOptions, long timeout) throws IOException, TezException { return getDAGStatus(statusOptions); } + + @Override + public String getWebUIAddress() throws IOException, TezException { + throw new TezException("MRDAGClient.getWebUIAddress is not supported"); + } + } diff --git a/tez-tests/src/test/java/org/apache/tez/test/TestAM.java b/tez-tests/src/test/java/org/apache/tez/test/TestAM.java new file mode 100644 index 0000000000..3e8a58cf7a --- /dev/null +++ b/tez-tests/src/test/java/org/apache/tez/test/TestAM.java @@ -0,0 +1,146 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.tez.test; + +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; + +import java.io.IOException; +import java.net.HttpURLConnection; +import java.net.URL; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.conf.Configuration.IntegerRanges; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.MiniDFSCluster; +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +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.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.runtime.library.processor.SleepProcessor; +import org.apache.tez.runtime.library.processor.SleepProcessor.SleepProcessorConfig; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TestAM { + + private static final Logger LOG = LoggerFactory.getLogger(TestAM.class); + + private static MiniTezCluster tezCluster; + private static MiniDFSCluster dfsCluster; + + private static Configuration conf = new Configuration(); + private static FileSystem remoteFs; + + private static final String TEST_ROOT_DIR = "target" + Path.SEPARATOR + TestAM.class.getName() + "-tmpDir"; + + @BeforeClass + public static void setup() throws IOException { + try { + conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, TEST_ROOT_DIR); + dfsCluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).format(true).racks(null).build(); + remoteFs = dfsCluster.getFileSystem(); + } catch (IOException io) { + throw new RuntimeException("problem starting mini dfs cluster", io); + } + + if (tezCluster == null) { + tezCluster = new MiniTezCluster(TestAM.class.getName(), 1, 1, 1); + Configuration tezClusterConf = new Configuration(); + tezClusterConf.set("fs.defaultFS", remoteFs.getUri().toString()); // use HDFS + tezClusterConf.setInt("yarn.nodemanager.delete.debug-delay-sec", 20000); + tezClusterConf.setLong(TezConfiguration.TEZ_AM_SLEEP_TIME_BEFORE_EXIT_MILLIS, 1000); + tezClusterConf.set(YarnConfiguration.PROXY_ADDRESS, "localhost"); + tezCluster.init(tezClusterConf); + tezCluster.start(); + } + } + + @AfterClass + public static void tearDown() { + if (tezCluster != null) { + tezCluster.stop(); + tezCluster = null; + } + if (dfsCluster != null) { + dfsCluster.shutdown(); + dfsCluster = null; + } + } + + @Test(timeout = 60000) + public void testAMWebUIService() throws TezException, IOException, InterruptedException { + SleepProcessorConfig spConf = new SleepProcessorConfig(1); + + DAG dag = DAG.create("TezSleepProcessor"); + Vertex vertex = Vertex.create("SleepVertex", + ProcessorDescriptor.create(SleepProcessor.class.getName()).setUserPayload(spConf.toUserPayload()), 1, + Resource.newInstance(1024, 1)); + dag.addVertex(vertex); + + TezConfiguration tezConf = new TezConfiguration(tezCluster.getConfig()); + TezClient tezSession = TezClient.create("TezSleepProcessor", tezConf, false); + tezSession.start(); + + DAGClient dagClient = tezSession.submitDAG(dag); + + DAGStatus dagStatus = dagClient.getDAGStatus(null); + while (!dagStatus.isCompleted()) { + Thread.sleep(500L); + dagStatus = dagClient.getDAGStatus(null); + } + + String webUIAddress = dagClient.getWebUIAddress(); + assertNotNull("getWebUIAddress should return TezAM's web UI address", webUIAddress); + LOG.info("TezAM webUI address: " + webUIAddress); + + checkAddress(webUIAddress + "/jmx"); + checkAddress(webUIAddress + "/conf"); + checkAddress(webUIAddress + "/stacks"); + + URL url = new URL(webUIAddress); + IntegerRanges portRange = conf.getRange(TezConfiguration.TEZ_AM_WEBSERVICE_PORT_RANGE, + TezConfiguration.TEZ_AM_WEBSERVICE_PORT_RANGE_DEFAULT); + assertTrue("WebUIService port should be in the defined range (got: " + url.getPort() + ")", + portRange.getRangeStart() <= url.getPort()); + + tezSession.stop(); + } + + private void checkAddress(String url) { + boolean success = false; + try { + HttpURLConnection connection = (HttpURLConnection) new URL(url).openConnection(); + connection.connect(); + success = (connection.getResponseCode() == 200); + } catch (Exception e) { + LOG.error("Error while checking url: " + url, e); + } + assertTrue(url + " should be available", success); + } +} From 98a81b1b66ee86273865ff4a77699d728ebb4786 Mon Sep 17 00:00:00 2001 From: skysiders <64545691+skysiders@users.noreply.github.com> Date: Tue, 10 May 2022 13:30:59 +0800 Subject: [PATCH 109/137] TEZ-4412 ensure mkDirForAM create directory with special permissions (#209) (Zhang Dongsheng reviewed by Laszlo Bodor) --- .../java/org/apache/tez/common/TezCommonUtils.java | 9 ++++++++- .../org/apache/tez/common/TestTezCommonUtils.java | 14 ++++++++++++++ 2 files changed, 22 insertions(+), 1 deletion(-) diff --git a/tez-api/src/main/java/org/apache/tez/common/TezCommonUtils.java b/tez-api/src/main/java/org/apache/tez/common/TezCommonUtils.java index 3163968908..5c2876444c 100644 --- a/tez-api/src/main/java/org/apache/tez/common/TezCommonUtils.java +++ b/tez-api/src/main/java/org/apache/tez/common/TezCommonUtils.java @@ -291,7 +291,14 @@ public static Path getSummaryRecoveryPath(Path attemptRecoverPath) { * @throws IOException */ public static void mkDirForAM(FileSystem fs, Path dir) throws IOException { - fs.mkdirs(dir, new FsPermission(TEZ_AM_DIR_PERMISSION)); + FsPermission perm = new FsPermission(TEZ_AM_DIR_PERMISSION); + fs.mkdirs(dir, perm); + if (!fs.getFileStatus(dir).getPermission().equals(perm)) { + LOG.warn("Directory " + dir.toString() + " created with unexpected permissions : " + + fs.getFileStatus(dir).getPermission() + ". Fixing permissions to correct value : " + + perm.toString()); + fs.setPermission(dir, perm); + } } /** diff --git a/tez-api/src/test/java/org/apache/tez/common/TestTezCommonUtils.java b/tez-api/src/test/java/org/apache/tez/common/TestTezCommonUtils.java index d7bd397386..d5dc6fd6b5 100644 --- a/tez-api/src/test/java/org/apache/tez/common/TestTezCommonUtils.java +++ b/tez-api/src/test/java/org/apache/tez/common/TestTezCommonUtils.java @@ -26,6 +26,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.CommonConfigurationKeys; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hdfs.MiniDFSCluster; @@ -413,4 +414,17 @@ public void testGetDAGSessionTimeout() { } + @Test + public void testMkDirForAM() throws IOException { + Configuration remoteConf = new Configuration(); + remoteConf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, TEST_ROOT_DIR); + remoteConf.set(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY, "777"); + MiniDFSCluster miniDFS = new MiniDFSCluster.Builder(remoteConf).numDataNodes(3).format(true).racks(null) + .build(); + FileSystem remoteFileSystem = miniDFS.getFileSystem(); + Path path = new Path(TEST_ROOT_DIR + "/testMkDirForAM"); + TezCommonUtils.mkDirForAM(remoteFileSystem, path); + Assert.assertEquals(TezCommonUtils.TEZ_AM_DIR_PERMISSION, remoteFileSystem.getFileStatus(path).getPermission()); + miniDFS.shutdown(); + } } From ba851fa06e4f57584b938706276797db9807e1d8 Mon Sep 17 00:00:00 2001 From: Mark Bathori <57758037+mbathori-cloudera@users.noreply.github.com> Date: Sun, 29 May 2022 18:58:04 +0200 Subject: [PATCH 110/137] TEZ-4417: Missing wget command in docker image (#210) (Mark Bathori reviewed by Laszlo Bodor) --- build-tools/docker/Dockerfile | 1 + 1 file changed, 1 insertion(+) diff --git a/build-tools/docker/Dockerfile b/build-tools/docker/Dockerfile index f51cc3bd2b..158408c85e 100644 --- a/build-tools/docker/Dockerfile +++ b/build-tools/docker/Dockerfile @@ -55,6 +55,7 @@ RUN apt-get -q update && apt-get -q install --no-install-recommends -y \ rsync \ software-properties-common \ ssh-client \ + wget \ xz-utils \ && apt-get clean \ && rm -rf /var/lib/apt/lists/* From 24a77c93f2f3d7933fbd6a3fcbcb9cb5f9d58845 Mon Sep 17 00:00:00 2001 From: Murali Krishna Date: Sun, 29 May 2022 22:37:36 +0530 Subject: [PATCH 111/137] TEZ-4409: Upgrade async-http-client to 2.12.3 to resolve CVE (#204) (D M Murali Krishna Reddy reviewed by Laszlo Bodor) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 9ed00c9974..1a51c6a9a2 100644 --- a/pom.xml +++ b/pom.xml @@ -243,7 +243,7 @@ org.asynchttpclient async-http-client - 2.12.1 + 2.12.3 io.netty From cf9e3ff308402904cd0d4eb0ece313104270de8a Mon Sep 17 00:00:00 2001 From: Mark Bathori <57758037+mbathori-cloudera@users.noreply.github.com> Date: Wed, 1 Jun 2022 19:54:15 +0200 Subject: [PATCH 112/137] TEZ-4421: Missing sudo command in docker image (#214) (Mark Bathori reviewed by Laszlo Bodor) --- build-tools/docker/Dockerfile | 1 + 1 file changed, 1 insertion(+) diff --git a/build-tools/docker/Dockerfile b/build-tools/docker/Dockerfile index 158408c85e..3e792604ec 100644 --- a/build-tools/docker/Dockerfile +++ b/build-tools/docker/Dockerfile @@ -55,6 +55,7 @@ RUN apt-get -q update && apt-get -q install --no-install-recommends -y \ rsync \ software-properties-common \ ssh-client \ + sudo \ wget \ xz-utils \ && apt-get clean \ From e5a5578025409cbc18f5e727ea35ec03ec4bcdac Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?L=C3=A1szl=C3=B3=20Attila=20T=C3=B3th?= Date: Mon, 6 Jun 2022 21:50:39 +0200 Subject: [PATCH 113/137] TEZ-4383: upgrade to mockito 4.3.1 (#190) (Laszlo Attila Toth reviewed by Laszlo Bodor) --- pom.xml | 4 +- tez-api/pom.xml | 2 +- .../org/apache/tez/client/TestTezClient.java | 68 +++++----- .../tez/common/security/TestTokenCache.java | 4 +- .../tez/dag/api/TestEntityDescriptor.java | 13 +- .../tez/dag/api/client/rpc/TestDAGClient.java | 48 +++---- tez-common/pom.xml | 2 +- tez-dag/pom.xml | 6 +- .../dag/api/client/TestDAGClientHandler.java | 4 +- ...GClientAMProtocolBlockingPBServerImpl.java | 7 +- .../tez/dag/app/TestRecoveryParser.java | 8 +- .../dag/app/TestTaskCommunicatorManager.java | 8 +- .../dag/app/TestTaskCommunicatorManager1.java | 4 +- .../dag/app/TestTaskCommunicatorManager2.java | 10 +- .../dag/TestRootInputInitializerManager.java | 14 +- .../dag/app/dag/TestStateChangeNotifier.java | 6 +- .../tez/dag/app/dag/impl/TestDAGImpl.java | 19 +-- .../tez/dag/app/dag/impl/TestDAGRecovery.java | 13 +- ...estDAGSchedulerNaturalOrderControlled.java | 38 +++--- .../apache/tez/dag/app/dag/impl/TestEdge.java | 2 +- .../impl/TestImmediateStartVertexManager.java | 2 +- .../dag/impl/TestRootInputVertexManager.java | 12 +- .../tez/dag/app/dag/impl/TestTaskAttempt.java | 25 ++-- .../tez/dag/app/dag/impl/TestTaskImpl.java | 8 +- .../tez/dag/app/dag/impl/TestVertexImpl.java | 24 ++-- .../dag/app/dag/impl/TestVertexManager.java | 6 +- .../TestContainerLauncherManager.java | 6 +- .../tez/dag/app/rm/TestContainerReuse.java | 90 ++++++------- .../app/rm/TestDagAwareYarnTaskScheduler.java | 120 +++++++++--------- .../app/rm/TestLocalTaskSchedulerService.java | 4 +- .../tez/dag/app/rm/TestTaskScheduler.java | 41 +++--- .../dag/app/rm/TestTaskSchedulerManager.java | 8 +- .../dag/app/rm/container/TestAMContainer.java | 8 +- .../tez/dag/app/web/TestAMWebController.java | 6 +- .../history/recovery/TestRecoveryService.java | 4 +- tez-ext-service-tests/pom.xml | 6 +- tez-mapreduce/pom.xml | 6 +- .../mapred/split/TestGroupedSplits.java | 10 +- .../tez/mapreduce/combine/TestMRCombiner.java | 27 ++-- tez-plugins/tez-aux-services/pom.xml | 2 +- .../tez/auxservices/TestShuffleHandler.java | 20 +-- tez-plugins/tez-history-parser/pom.xml | 2 +- .../tez-protobuf-history-plugin/pom.xml | 2 +- .../pom.xml | 2 +- .../tez-yarn-timeline-history-with-fs/pom.xml | 7 +- .../ats/TestATSV15HistoryLoggingService.java | 15 +-- tez-plugins/tez-yarn-timeline-history/pom.xml | 2 +- .../ats/TestATSHistoryLoggingService.java | 110 +++++++--------- tez-runtime-internals/pom.xml | 6 +- .../TestLogicalIOProcessorRuntimeTask.java | 9 +- .../tez/runtime/task/TestTaskReporter.java | 8 +- tez-runtime-library/pom.xml | 2 +- .../TestFairShuffleVertexManager.java | 3 +- .../TestInputReadyVertexManager.java | 7 +- .../TestShuffleVertexManager.java | 6 +- .../TestShuffleVertexManagerBase.java | 37 +++--- .../TestShuffleVertexManagerUtils.java | 11 +- ...tesianProductVertexManagerPartitioned.java | 19 ++- ...TestFairCartesianProductVertexManager.java | 43 +++---- .../library/common/shuffle/TestFetcher.java | 11 +- .../common/shuffle/TestShuffleUtils.java | 44 +++---- .../TestShuffleInputEventHandlerImpl.java | 18 +-- .../shuffle/impl/TestShuffleManager.java | 10 +- .../shuffle/orderedgrouped/TestFetcher.java | 38 +++--- .../shuffle/orderedgrouped/TestShuffle.java | 10 +- ...huffleInputEventHandlerOrderedGrouped.java | 24 ++-- .../orderedgrouped/TestShuffleScheduler.java | 38 +++--- .../common/sort/impl/TestPipelinedSorter.java | 2 +- .../sort/impl/dflt/TestDefaultSorter.java | 9 +- .../TestUnorderedPartitionedKVWriter.java | 27 ++-- .../input/TestOrderedGroupedKVInput.java | 2 +- .../library/output/OutputTestHelpers.java | 7 +- .../output/TestOnFileSortedOutput.java | 7 +- .../output/TestOnFileUnorderedKVOutput.java | 7 +- .../runtime/library/utils/TestCodecUtils.java | 11 +- tez-tools/analyzers/job-analyzer/pom.xml | 10 +- 76 files changed, 638 insertions(+), 643 deletions(-) diff --git a/pom.xml b/pom.xml index 1a51c6a9a2..c496e0ce87 100644 --- a/pom.xml +++ b/pom.xml @@ -704,8 +704,8 @@ org.mockito - mockito-all - 1.10.8 + mockito-core + 4.3.1 org.apache.commons diff --git a/tez-api/pom.xml b/tez-api/pom.xml index 4e1825b09d..cd5abdb03a 100644 --- a/tez-api/pom.xml +++ b/tez-api/pom.xml @@ -107,7 +107,7 @@ org.mockito - mockito-all + mockito-core test diff --git a/tez-api/src/test/java/org/apache/tez/client/TestTezClient.java b/tez-api/src/test/java/org/apache/tez/client/TestTezClient.java index 67c4a604b6..4f33348f89 100644 --- a/tez-api/src/test/java/org/apache/tez/client/TestTezClient.java +++ b/tez-api/src/test/java/org/apache/tez/client/TestTezClient.java @@ -38,14 +38,14 @@ import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.isNull; import static org.mockito.Mockito.RETURNS_DEEP_STUBS; -import static org.mockito.Mockito.times; +import static org.mockito.Mockito.any; import static org.mockito.Mockito.atLeast; import static org.mockito.Mockito.doCallRealMethod; +import static org.mockito.Mockito.isNull; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; @@ -89,9 +89,7 @@ import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolBlockingPB; import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.GetAMStatusRequestProto; import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.GetAMStatusResponseProto; -import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.GetDAGStatusRequestProto; import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.GetDAGStatusResponseProto; -import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.ShutdownSessionRequestProto; import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.SubmitDAGRequestProto; import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.TezAppMasterStatusProto; import org.apache.tez.dag.api.records.DAGProtos.DAGStatusProto; @@ -188,10 +186,11 @@ TezClientForTest configureAndCreateTezClient(Map lrs, boo YarnClient yarnClient = mock(YarnClient.class, RETURNS_DEEP_STUBS); when(yarnClient.createApplication().getNewApplicationResponse().getApplicationId()).thenReturn(appId1); when(yarnClient.getApplicationReport(appId1).getYarnApplicationState()).thenReturn(YarnApplicationState.NEW); - when(yarnClient.submitApplication(any(ApplicationSubmissionContext.class))).thenReturn(appId1); + when(yarnClient.submitApplication(any())).thenReturn(appId1); DAGClientAMProtocolBlockingPB sessionAmProxy = mock(DAGClientAMProtocolBlockingPB.class, RETURNS_DEEP_STUBS); - when(sessionAmProxy.getAMStatus(any(RpcController.class), any(GetAMStatusRequestProto.class))) + when(sessionAmProxy.getAMStatus(any(), any())) + .thenReturn(GetAMStatusResponseProto.newBuilder().setStatus(TezAppMasterStatusProto.RUNNING).build()) .thenReturn(GetAMStatusResponseProto.newBuilder().setStatus(TezAppMasterStatusProto.RUNNING).build()); client.sessionAmProxy = sessionAmProxy; @@ -255,7 +254,7 @@ private void _testTezClientSessionLargeDAGPlan(int maxIPCMsgSize, int payloadSiz client.stop(); ArgumentCaptor captor = ArgumentCaptor.forClass(SubmitDAGRequestProto.class); - verify(client.sessionAmProxy).submitDAG((RpcController)any(), captor.capture()); + verify(client.sessionAmProxy).submitDAG(any(), captor.capture()); SubmitDAGRequestProto request = captor.getValue(); if (shouldSerialize) { @@ -308,12 +307,12 @@ public void testGetClient() throws Exception { verify(client2.mockYarnClient, times(0)).submitApplication(captor.capture()); // Validate dag submission from second TezClient as normal */ - verify(client2.sessionAmProxy, times(1)).submitDAG((RpcController)any(), (SubmitDAGRequestProto) any()); + verify(client2.sessionAmProxy, times(1)).submitDAG(any(), any()); // Validate stop from new TezClient as normal */ client2.stop(); - verify(client2.sessionAmProxy, times(1)).shutdownSession((RpcController) any(), - (ShutdownSessionRequestProto) any()); + verify(client2.sessionAmProxy, times(1)).shutdownSession(any(), + any()); verify(client2.mockYarnClient, times(1)).stop(); /* END reuse of AM from new TezClient */ } @@ -330,7 +329,7 @@ public TezClientForTest testTezClient(boolean isSession, boolean shouldStop) thr when(client.mockYarnClient.getApplicationReport(client.mockAppId).getYarnApplicationState()) .thenReturn(YarnApplicationState.RUNNING); client.start(); - verify(client.mockYarnClient, times(1)).init((Configuration)any()); + verify(client.mockYarnClient, times(1)).init(any()); verify(client.mockYarnClient, times(1)).start(); if (isSession) { verify(client.mockYarnClient, times(1)).submitApplication(captor.capture()); @@ -353,6 +352,10 @@ public TezClientForTest testTezClient(boolean isSession, boolean shouldStop) thr Vertex vertex = Vertex.create("Vertex", ProcessorDescriptor.create("P"), 1, Resource.newInstance(1, 1)); DAG dag = DAG.create("DAG").addVertex(vertex).addTaskLocalFiles(lrDAG); + if (!isSession) { + when(client.sessionAmProxy.getAMStatus(any(), any())) + .thenReturn(GetAMStatusResponseProto.newBuilder().setStatus(TezAppMasterStatusProto.SHUTDOWN).build()); + } DAGClient dagClient = client.submitDAG(dag); assertTrue(dagClient.getExecutionContext().contains(client.mockAppId.toString())); @@ -360,7 +363,7 @@ public TezClientForTest testTezClient(boolean isSession, boolean shouldStop) thr if (isSession) { verify(client.mockYarnClient, times(1)).submitApplication(captor.capture()); - verify(client.sessionAmProxy, times(1)).submitDAG((RpcController)any(), (SubmitDAGRequestProto) any()); + verify(client.sessionAmProxy, times(1)).submitDAG(any(), any()); } else { verify(client.mockYarnClient, times(1)).submitApplication(captor.capture()); ApplicationSubmissionContext context = captor.getValue(); @@ -399,7 +402,7 @@ public TezClientForTest testTezClient(boolean isSession, boolean shouldStop) thr assertEquals(dagClient.getSessionIdentifierString(), client.mockAppId.toString()); // additional resource is sent ArgumentCaptor captor1 = ArgumentCaptor.forClass(SubmitDAGRequestProto.class); - verify(client.sessionAmProxy, times(2)).submitDAG((RpcController)any(), captor1.capture()); + verify(client.sessionAmProxy, times(2)).submitDAG(any(), captor1.capture()); SubmitDAGRequestProto proto = captor1.getValue(); Assert.assertEquals(1, proto.getAdditionalAmResources().getLocalResourcesCount()); Assert.assertEquals(lrName2, proto.getAdditionalAmResources().getLocalResources(0).getName()); @@ -426,8 +429,8 @@ public TezClientForTest testTezClient(boolean isSession, boolean shouldStop) thr if(shouldStop) { client.stop(); if (isSession) { - verify(client.sessionAmProxy, times(1)).shutdownSession((RpcController) any(), - (ShutdownSessionRequestProto) any()); + verify(client.sessionAmProxy, times(1)).shutdownSession(any(), + any()); } verify(client.mockYarnClient, times(1)).stop(); } @@ -443,14 +446,14 @@ public void testPreWarm() throws Exception { .thenReturn(YarnApplicationState.RUNNING); when( - client.sessionAmProxy.getAMStatus((RpcController) any(), (GetAMStatusRequestProto) any())) + client.sessionAmProxy.getAMStatus(any(), any())) .thenReturn(GetAMStatusResponseProto.newBuilder().setStatus(TezAppMasterStatusProto.READY).build()); PreWarmVertex vertex = PreWarmVertex.create("PreWarm", 1, Resource.newInstance(1, 1)); client.preWarm(vertex); ArgumentCaptor captor1 = ArgumentCaptor.forClass(SubmitDAGRequestProto.class); - verify(client.sessionAmProxy, times(1)).submitDAG((RpcController)any(), captor1.capture()); + verify(client.sessionAmProxy, times(1)).submitDAG(any(), captor1.capture()); SubmitDAGRequestProto proto = captor1.getValue(); assertTrue(proto.getDAGPlan().getName().startsWith(TezConstants.TEZ_PREWARM_DAG_NAME_PREFIX)); @@ -467,7 +470,7 @@ public void testPreWarmCloseStuck() throws Exception { when(client.mockYarnClient.getApplicationReport(client.mockAppId).getYarnApplicationState()) .thenReturn(YarnApplicationState.RUNNING); - when(client.sessionAmProxy.getAMStatus((RpcController) any(), (GetAMStatusRequestProto) any())) + when(client.sessionAmProxy.getAMStatus(any(), any())) .thenReturn(GetAMStatusResponseProto.newBuilder().setStatus(TezAppMasterStatusProto.READY).build()); PreWarmVertex vertex = PreWarmVertex.create("PreWarm", 1, Resource.newInstance(1, 1)); @@ -480,7 +483,7 @@ public void testPreWarmCloseStuck() throws Exception { private void setClientToReportStoppedDags(TezClientForTest client) throws Exception { when(client.mockYarnClient.getApplicationReport(client.mockAppId).getYarnApplicationState()) .thenReturn(YarnApplicationState.FINISHED); - when(client.sessionAmProxy.getDAGStatus(isNull(RpcController.class), any(GetDAGStatusRequestProto.class))) + when(client.sessionAmProxy.getDAGStatus(isNull(), any())) .thenReturn(GetDAGStatusResponseProto.newBuilder().setDagStatus(DAGStatusProto.newBuilder() .addDiagnostics("Diagnostics_0").setState(DAGStatusStateProto.DAG_SUCCEEDED) .setDAGProgress(ProgressProto.newBuilder() @@ -502,8 +505,8 @@ public void testPreWarmWithTimeout() throws Exception { spyClient.mockAppId).getYarnApplicationState()) .thenReturn(YarnApplicationState.RUNNING); when( - spyClient.sessionAmProxy.getAMStatus((RpcController) any(), - (GetAMStatusRequestProto) any())) + spyClient.sessionAmProxy.getAMStatus(any(), + any())) .thenReturn( GetAMStatusResponseProto.newBuilder().setStatus( TezAppMasterStatusProto.INITIALIZING).build()); @@ -518,15 +521,15 @@ public void testPreWarmWithTimeout() throws Exception { endTime = Time.monotonicNow(); assertTrue("Time taken is not as expected", (endTime - startTime) > timeout); - verify(spyClient, times(0)).submitDAG(any(DAG.class)); + verify(spyClient, times(0)).submitDAG(any()); Assert.assertTrue("Unexpected Exception message", te.getMessage().contains("Tez AM not ready")); } when( - spyClient.sessionAmProxy.getAMStatus((RpcController) any(), - (GetAMStatusRequestProto) any())) + spyClient.sessionAmProxy.getAMStatus(any(), + any())) .thenReturn( GetAMStatusResponseProto.newBuilder().setStatus( TezAppMasterStatusProto.READY).build()); @@ -536,7 +539,7 @@ public void testPreWarmWithTimeout() throws Exception { endTime = Time.monotonicNow(); assertTrue("Time taken is not as expected", (endTime - startTime) <= timeout); - verify(spyClient, times(1)).submitDAG(any(DAG.class)); + verify(spyClient, times(1)).submitDAG(any()); } catch (TezException te) { fail("PreWarm should have succeeded!"); } @@ -571,7 +574,7 @@ public void run() { endTime = Time.monotonicNow(); assertTrue("Time taken is not as expected", (endTime - startTime) <= timeout); - verify(spyClient, times(2)).submitDAG(any(DAG.class)); + verify(spyClient, times(2)).submitDAG(any()); setClientToReportStoppedDags(client); spyClient.stop(); client.stop(); @@ -926,8 +929,7 @@ public void testAMClientHeartbeat() throws Exception { Thread.sleep(1000); } client.stop(); - verify(client.sessionAmProxy, atLeast(3)).getAMStatus(any(RpcController.class), - any(GetAMStatusRequestProto.class)); + verify(client.sessionAmProxy, atLeast(3)).getAMStatus(any(), any()); conf.setInt(TezConfiguration.TEZ_AM_CLIENT_HEARTBEAT_TIMEOUT_SECS, -1); final TezClientForTest client2 = configureAndCreateTezClient(conf); @@ -940,10 +942,7 @@ public void testAMClientHeartbeat() throws Exception { Thread.sleep(1000); } client2.stop(); - verify(client2.sessionAmProxy, times(0)).getAMStatus(any(RpcController.class), - any(GetAMStatusRequestProto.class)); - - + verify(client2.sessionAmProxy, times(0)).getAMStatus(any(), any()); } @Test(timeout = 20000) @@ -987,8 +986,7 @@ public void testAMHeartbeatFailOnGetAMStatus() throws Exception { final TezClientForTest client = configureAndCreateTezClient(conf); client.start(); - when(client.sessionAmProxy.getAMStatus(any(RpcController.class), - any(GetAMStatusRequestProto.class))).thenThrow(new ServiceException("error")); + when(client.sessionAmProxy.getAMStatus(any(), any())).thenThrow(new ServiceException("error")); client.callRealGetSessionAMProxy = true; when(client.mockYarnClient.getApplicationReport(client.mockAppId).getYarnApplicationState()) .thenReturn(YarnApplicationState.FAILED); diff --git a/tez-api/src/test/java/org/apache/tez/common/security/TestTokenCache.java b/tez-api/src/test/java/org/apache/tez/common/security/TestTokenCache.java index fcb1e98aee..4c144174c7 100644 --- a/tez-api/src/test/java/org/apache/tez/common/security/TestTokenCache.java +++ b/tez-api/src/test/java/org/apache/tez/common/security/TestTokenCache.java @@ -18,7 +18,7 @@ package org.apache.tez.common.security; -import static org.mockito.Matchers.any; +import static org.mockito.Mockito.any; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; @@ -153,7 +153,7 @@ private MockFileSystem createFileSystemForServiceName(final String service) throws IOException { MockFileSystem mockFs = new MockFileSystem(); when(mockFs.getCanonicalServiceName()).thenReturn(service); - when(mockFs.getDelegationToken(any(String.class))).thenAnswer( + when(mockFs.getDelegationToken(any())).thenAnswer( new Answer>() { int unique = 0; @Override diff --git a/tez-api/src/test/java/org/apache/tez/dag/api/TestEntityDescriptor.java b/tez-api/src/test/java/org/apache/tez/dag/api/TestEntityDescriptor.java index 606bf42717..82c22ad4ef 100644 --- a/tez-api/src/test/java/org/apache/tez/dag/api/TestEntityDescriptor.java +++ b/tez-api/src/test/java/org/apache/tez/dag/api/TestEntityDescriptor.java @@ -23,7 +23,6 @@ import java.io.DataInputStream; import java.io.DataOutputStream; import java.io.IOException; -import java.nio.ByteBuffer; import org.apache.commons.lang.RandomStringUtils; import org.apache.hadoop.conf.Configuration; @@ -31,11 +30,11 @@ import org.apache.tez.common.TezUtils; import org.junit.Assert; import org.junit.Test; -import org.mockito.Mockito; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.eq; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.eq; import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.verify; public class TestEntityDescriptor { @@ -57,7 +56,7 @@ public void testSingularWrite(InputDescriptor entityDescriptor, InputDescriptor ByteArrayOutputStream bos = new ByteArrayOutputStream(out.getData().length); bos.write(out.getData()); - Mockito.verify(entityDescriptor).writeSingular(eq(out), any(ByteBuffer.class)); + verify(entityDescriptor).writeSingular(eq(out), any()); deserialized.readFields(new DataInputStream(new ByteArrayInputStream(bos.toByteArray()))); verifyResults(entityDescriptor, deserialized, payload, confVal); } @@ -69,12 +68,12 @@ public void testSegmentedWrite(InputDescriptor entityDescriptor, InputDescriptor entityDescriptor.write(out); out.close(); - Mockito.verify(entityDescriptor).writeSegmented(eq(out), any(ByteBuffer.class)); + verify(entityDescriptor).writeSegmented(eq(out), any()); deserialized.readFields(new DataInputStream(new ByteArrayInputStream(bos.toByteArray()))); verifyResults(entityDescriptor, deserialized, payload, confVal); } - @Test (timeout=1000) + @Test (timeout=3000) public void testEntityDescriptorHadoopSerialization() throws IOException { /* This tests the alternate serialization code path * if the DataOutput is not DataOutputBuffer diff --git a/tez-api/src/test/java/org/apache/tez/dag/api/client/rpc/TestDAGClient.java b/tez-api/src/test/java/org/apache/tez/dag/api/client/rpc/TestDAGClient.java index f0cc20b062..0ea5d1a26d 100644 --- a/tez-api/src/test/java/org/apache/tez/dag/api/client/rpc/TestDAGClient.java +++ b/tez-api/src/test/java/org/apache/tez/dag/api/client/rpc/TestDAGClient.java @@ -20,8 +20,10 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; -import static org.mockito.Matchers.*; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.argThat; import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.isNull; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; @@ -164,30 +166,22 @@ private void setUpData(){ .setVertexCounters(vertexCountersProto) .build(); } - - private static class DAGCounterRequestMatcher extends ArgumentMatcher{ + + private static class DAGCounterRequestMatcher implements ArgumentMatcher{ @Override - public boolean matches(Object argument) { - if (argument instanceof GetDAGStatusRequestProto){ - GetDAGStatusRequestProto requestProto = (GetDAGStatusRequestProto)argument; - return requestProto.getStatusOptionsCount() != 0 - && requestProto.getStatusOptionsList().get(0) == StatusGetOptsProto.GET_COUNTERS; - } - return false; + public boolean matches(GetDAGStatusRequestProto requestProto) { + return requestProto != null && requestProto.getStatusOptionsCount() != 0 + && requestProto.getStatusOptionsList().get(0) == StatusGetOptsProto.GET_COUNTERS; } } - - private static class VertexCounterRequestMatcher extends ArgumentMatcher{ + + private static class VertexCounterRequestMatcher implements ArgumentMatcher{ @Override - public boolean matches(Object argument) { - if (argument instanceof GetVertexStatusRequestProto){ - GetVertexStatusRequestProto requestProto = (GetVertexStatusRequestProto)argument; - return requestProto.getStatusOptionsCount() != 0 - && requestProto.getStatusOptionsList().get(0) == StatusGetOptsProto.GET_COUNTERS; - } - return false; + public boolean matches(GetVertexStatusRequestProto requestProto) { + return requestProto != null && requestProto.getStatusOptionsCount() != 0 + && requestProto.getStatusOptionsList().get(0) == StatusGetOptsProto.GET_COUNTERS; } } @@ -202,14 +196,14 @@ public void setUp() throws YarnException, IOException, TezException, ServiceExce dagIdStr = "dag_9999_0001_1"; mockProxy = mock(DAGClientAMProtocolBlockingPB.class); // return the response with Counters is the request match the CounterMatcher - when(mockProxy.getDAGStatus(isNull(RpcController.class), any(GetDAGStatusRequestProto.class))) + when(mockProxy.getDAGStatus(isNull(), any())) .thenReturn(GetDAGStatusResponseProto.newBuilder().setDagStatus(dagStatusProtoWithoutCounters).build()); - when(mockProxy.getDAGStatus(isNull(RpcController.class), argThat(new DAGCounterRequestMatcher()))) + when(mockProxy.getDAGStatus(isNull(), argThat(new DAGCounterRequestMatcher()))) .thenReturn(GetDAGStatusResponseProto.newBuilder().setDagStatus(dagStatusProtoWithCounters).build()); - when(mockProxy.getVertexStatus(isNull(RpcController.class), any(GetVertexStatusRequestProto.class))) + when(mockProxy.getVertexStatus(isNull(), any())) .thenReturn(GetVertexStatusResponseProto.newBuilder().setVertexStatus(vertexStatusProtoWithoutCounters).build()); - when(mockProxy.getVertexStatus(isNull(RpcController.class), argThat(new VertexCounterRequestMatcher()))) + when(mockProxy.getVertexStatus(isNull(), argThat(new VertexCounterRequestMatcher()))) .thenReturn(GetVertexStatusResponseProto.newBuilder().setVertexStatus(vertexStatusProtoWithCounters).build()); TezConfiguration tezConf = new TezConfiguration(); @@ -270,7 +264,7 @@ public void testTryKillDAG() throws Exception{ @Test(timeout = 5000) public void testWaitForCompletion() throws Exception{ // first time return DAG_RUNNING, second time return DAG_SUCCEEDED - when(mockProxy.getDAGStatus(isNull(RpcController.class), any(GetDAGStatusRequestProto.class))) + when(mockProxy.getDAGStatus(isNull(), any())) .thenReturn(GetDAGStatusResponseProto.newBuilder().setDagStatus(dagStatusProtoWithoutCounters) .build()) .thenReturn(GetDAGStatusResponseProto.newBuilder().setDagStatus @@ -291,7 +285,7 @@ public void testWaitForCompletion() throws Exception{ public void testWaitForCompletionWithStatusUpdates() throws Exception{ // first time and second time return DAG_RUNNING, third time return DAG_SUCCEEDED - when(mockProxy.getDAGStatus(isNull(RpcController.class), any(GetDAGStatusRequestProto.class))) + when(mockProxy.getDAGStatus(isNull(), any())) .thenReturn(GetDAGStatusResponseProto.newBuilder().setDagStatus( DAGStatusProto.newBuilder(dagStatusProtoWithCounters) .setState(DAGStatusStateProto.DAG_RUNNING).build()).build()) @@ -317,7 +311,7 @@ public void testWaitForCompletionWithStatusUpdates() throws Exception{ verify(mockProxy, times(4)) .getDAGStatus(rpcControllerArgumentCaptor.capture(), argumentCaptor.capture()); - when(mockProxy.getDAGStatus(isNull(RpcController.class), any(GetDAGStatusRequestProto.class))) + when(mockProxy.getDAGStatus(isNull(), any())) .thenReturn(GetDAGStatusResponseProto.newBuilder().setDagStatus( DAGStatusProto.newBuilder(dagStatusProtoWithCounters) .setState(DAGStatusStateProto.DAG_RUNNING).build()).build()) @@ -555,7 +549,7 @@ public Object answer(InvocationOnMock invocation) throws Throwable { return GetDAGStatusResponseProto.newBuilder().setDagStatus(constructDagStatusProto( stateProto)).build(); } - }).when(mock).getDAGStatus(isNull(RpcController.class), any(GetDAGStatusRequestProto.class)); + }).when(mock).getDAGStatus(isNull(), any()); return mock; } diff --git a/tez-common/pom.xml b/tez-common/pom.xml index 0fd4255abe..9ccf02a38c 100644 --- a/tez-common/pom.xml +++ b/tez-common/pom.xml @@ -63,7 +63,7 @@ org.mockito - mockito-all + mockito-core test diff --git a/tez-dag/pom.xml b/tez-dag/pom.xml index 723082c54e..356ed2c2e9 100644 --- a/tez-dag/pom.xml +++ b/tez-dag/pom.xml @@ -87,6 +87,10 @@ org.apache.hadoop hadoop-yarn-server-web-proxy + + org.roaringbitmap + RoaringBitmap + org.slf4j slf4j-api @@ -123,7 +127,7 @@ org.mockito - mockito-all + mockito-core test diff --git a/tez-dag/src/test/java/org/apache/tez/dag/api/client/TestDAGClientHandler.java b/tez-dag/src/test/java/org/apache/tez/dag/api/client/TestDAGClientHandler.java index bf07838a45..c7daebf02d 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/api/client/TestDAGClientHandler.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/api/client/TestDAGClientHandler.java @@ -51,11 +51,11 @@ public void testDAGClientHandler() throws TezException { DAG mockDAG = mock(DAG.class); when(mockDAG.getID()).thenReturn(mockTezDAGId); DAGStatusBuilder mockDagStatusBuilder = mock(DAGStatusBuilder.class); - when(mockDAG.getDAGStatus(anySetOf(StatusGetOpts.class))).thenReturn( + when(mockDAG.getDAGStatus(anySet())).thenReturn( mockDagStatusBuilder); VertexStatusBuilder mockVertexStatusBuilder = mock(VertexStatusBuilder.class); - when(mockDAG.getVertexStatus(anyString(), anySetOf(StatusGetOpts.class))) + when(mockDAG.getVertexStatus(anyString(), anySet())) .thenReturn(mockVertexStatusBuilder); DAGAppMaster mockDagAM = mock(DAGAppMaster.class); diff --git a/tez-dag/src/test/java/org/apache/tez/dag/api/client/rpc/TestDAGClientAMProtocolBlockingPBServerImpl.java b/tez-dag/src/test/java/org/apache/tez/dag/api/client/rpc/TestDAGClientAMProtocolBlockingPBServerImpl.java index 040ca2fb7b..1aea8f793d 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/api/client/rpc/TestDAGClientAMProtocolBlockingPBServerImpl.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/api/client/rpc/TestDAGClientAMProtocolBlockingPBServerImpl.java @@ -28,8 +28,11 @@ import static junit.framework.TestCase.assertEquals; import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertTrue; -import static org.mockito.Matchers.any; -import static org.mockito.Mockito.*; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; import org.apache.hadoop.fs.CommonConfigurationKeys; import org.apache.hadoop.fs.FileSystem; diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/TestRecoveryParser.java b/tez-dag/src/test/java/org/apache/tez/dag/app/TestRecoveryParser.java index 1c09d5d9e8..57148ee7e9 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/TestRecoveryParser.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/TestRecoveryParser.java @@ -111,7 +111,7 @@ public void setUp() throws IllegalArgumentException, IOException { mockAppMaster.dagIDs = new HashSet(); when(mockAppMaster.getConfig()).thenReturn(new Configuration()); mockDAGImpl = mock(DAGImpl.class); - when(mockAppMaster.createDAG(any(DAGPlan.class), any(TezDAGID.class))).thenReturn(mockDAGImpl); + when(mockAppMaster.createDAG(any(), any())).thenReturn(mockDAGImpl); parser = new RecoveryParser(mockAppMaster, localFS, recoveryPath, 3); } @@ -198,7 +198,7 @@ public void testSkipAllOtherEvents_1() throws IOException { assertTrue(dagData.reason.contains("DAG Commit was in progress, not recoverable,")); // DAGSubmittedEvent is handled but DAGInitializedEvent and DAGStartedEvent in the next attempt are both skipped // due to the dag is not recoerable. - verify(mockAppMaster).createDAG(any(DAGPlan.class),any(TezDAGID.class)); + verify(mockAppMaster).createDAG(any(), any()); assertNull(dagData.getDAGInitializedEvent()); assertNull(dagData.getDAGStartedEvent()); } @@ -245,7 +245,7 @@ public void testSkipAllOtherEvents_2() throws IOException { assertEquals(DAGState.FAILED, dagData.dagState); assertEquals(true, dagData.isCompleted); // DAGSubmittedEvent, DAGInitializedEvent and DAGFinishedEvent is handled - verify(mockAppMaster).createDAG(any(DAGPlan.class),any(TezDAGID.class)); + verify(mockAppMaster).createDAG(any(), any()); // DAGInitializedEvent may not been handled before DAGFinishedEvent, // because DAGFinishedEvent's writeToRecoveryImmediately is true assertNotNull(dagData.getDAGFinishedEvent()); @@ -294,7 +294,7 @@ public void testLastCorruptedRecoveryRecord() throws IOException { assertEquals(null, dagData.reason); assertEquals(false, dagData.nonRecoverable); // verify DAGSubmitedEvent & DAGInititlizedEvent is handled. - verify(mockAppMaster).createDAG(any(DAGPlan.class),any(TezDAGID.class)); + verify(mockAppMaster).createDAG(any(), any()); assertNotNull(dagData.getDAGInitializedEvent()); } diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskCommunicatorManager.java b/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskCommunicatorManager.java index c7f97d3ea9..3935789860 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskCommunicatorManager.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskCommunicatorManager.java @@ -18,10 +18,10 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.eq; import static org.mockito.Mockito.RETURNS_DEEP_STUBS; +import static org.mockito.Mockito.any; import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.eq; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.reset; import static org.mockito.Mockito.spy; @@ -187,7 +187,7 @@ public void testEventRouting() throws Exception { AppContext appContext = mock(AppContext.class, RETURNS_DEEP_STUBS); NodeId nodeId = NodeId.newInstance("host1", 3131); - when(appContext.getAllContainers().get(any(ContainerId.class)).getContainer().getNodeId()) + when(appContext.getAllContainers().get(any()).getContainer().getNodeId()) .thenReturn(nodeId); TaskHeartbeatHandler thh = mock(TaskHeartbeatHandler.class); ContainerHeartbeatHandler chh = mock(ContainerHeartbeatHandler.class); @@ -348,7 +348,7 @@ public void testTaskCommunicatorUserError() { assertTrue(event.getDiagnosticInfo().contains(expectedId)); - when(appContext.getAllContainers().get(any(ContainerId.class)).getContainer().getNodeId()) + when(appContext.getAllContainers().get(any()).getContainer().getNodeId()) .thenReturn(mock(NodeId.class)); taskCommunicatorManager.registerRunningContainer(mock(ContainerId.class), 0); diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskCommunicatorManager1.java b/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskCommunicatorManager1.java index 0f8afaafe6..d7e62ee530 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskCommunicatorManager1.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskCommunicatorManager1.java @@ -19,7 +19,7 @@ import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; -import static org.mockito.Matchers.any; +import static org.mockito.Mockito.any; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.times; @@ -144,7 +144,7 @@ public void setUp() throws TezException { AMContainer amContainer = mock(AMContainer.class); Container container = mock(Container.class); doReturn(nodeId).when(container).getNodeId(); - doReturn(amContainer).when(amContainerMap).get(any(ContainerId.class)); + doReturn(amContainer).when(amContainerMap).get(any()); doReturn(container).when(amContainer).getContainer(); Configuration conf = new TezConfiguration(); diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskCommunicatorManager2.java b/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskCommunicatorManager2.java index bb7e94b5c7..ac60001a46 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskCommunicatorManager2.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskCommunicatorManager2.java @@ -16,10 +16,10 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.anyInt; -import static org.mockito.Matchers.eq; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.anyInt; import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.eq; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.reset; import static org.mockito.Mockito.times; @@ -256,7 +256,7 @@ private static class TaskCommunicatorManagerWrapperForTest { doReturn(dag).when(appContext).getCurrentDAG(); doReturn(vertex).when(dag).getVertex(eq(vertexId)); doReturn(new TaskAttemptEventInfo(0, new LinkedList(), 0)).when(vertex) - .getTaskAttemptTezEvents(any(TezTaskAttemptID.class), anyInt(), anyInt(), anyInt()); + .getTaskAttemptTezEvents(any(), anyInt(), anyInt(), anyInt()); doReturn(appAttemptId).when(appContext).getApplicationAttemptId(); doReturn(credentials).when(appContext).getAppCredentials(); doReturn(appAcls).when(appContext).getApplicationACLs(); @@ -267,7 +267,7 @@ private static class TaskCommunicatorManagerWrapperForTest { AMContainer amContainer = mock(AMContainer.class); Container container = mock(Container.class); doReturn(nodeId).when(container).getNodeId(); - doReturn(amContainer).when(amContainerMap).get(any(ContainerId.class)); + doReturn(amContainer).when(amContainerMap).get(any()); doReturn(container).when(amContainer).getContainer(); userPayload = TezUtils.createUserPayloadFromConf(conf); diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/TestRootInputInitializerManager.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/TestRootInputInitializerManager.java index ffa4309602..2c5aa337b7 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/TestRootInputInitializerManager.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/TestRootInputInitializerManager.java @@ -15,8 +15,8 @@ package org.apache.tez.dag.app.dag; import static org.junit.Assert.assertEquals; -import static org.mockito.Matchers.any; import static org.mockito.Mockito.RETURNS_DEEP_STUBS; +import static org.mockito.Mockito.any; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.never; @@ -101,7 +101,7 @@ public void testEventBeforeSuccess() throws Exception { eventList.add(te1); initializerWrapper.handleInputInitializerEvents(eventList); - verify(initializer, never()).handleInputInitializerEvent(any(List.class)); + verify(initializer, never()).handleInputInitializerEvent(any()); eventList.clear(); // First attempt, Task success notification @@ -123,7 +123,7 @@ public void testEventBeforeSuccess() throws Exception { eventList.add(te2); initializerWrapper.handleInputInitializerEvents(eventList); - verify(initializer, never()).handleInputInitializerEvent(any(List.class)); + verify(initializer, never()).handleInputInitializerEvent(any()); eventList.clear(); reset(initializer); @@ -176,7 +176,7 @@ public void testSuccessBeforeEvent() throws Exception { eventList.add(te1); initializerWrapper.handleInputInitializerEvents(eventList); - verify(initializer, never()).handleInputInitializerEvent(any(List.class)); + verify(initializer, never()).handleInputInitializerEvent(any()); eventList.clear(); // First attempt, Task success notification @@ -192,7 +192,7 @@ public void testSuccessBeforeEvent() throws Exception { TezTaskAttemptID srcTaskAttemptId12 = TezTaskAttemptID.getInstance(srcTaskId1, 2); // 2nd attempt succeeded initializerWrapper.onTaskSucceeded(srcVertexName, srcTaskId1, srcTaskAttemptId12.getId()); - verify(initializer, never()).handleInputInitializerEvent(any(List.class)); + verify(initializer, never()).handleInputInitializerEvent(any()); // 2nd attempt send event EventMetaData sourceInfo12 = @@ -203,7 +203,7 @@ public void testSuccessBeforeEvent() throws Exception { eventList.add(te2); initializerWrapper.handleInputInitializerEvents(eventList); - verify(initializer, never()).handleInputInitializerEvent(any(List.class)); + verify(initializer, never()).handleInputInitializerEvent(any()); } @@ -269,4 +269,4 @@ static void awaitInitialize() throws InterruptedException { } } } -} \ No newline at end of file +} diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/TestStateChangeNotifier.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/TestStateChangeNotifier.java index d20903d348..b43782e165 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/TestStateChangeNotifier.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/TestStateChangeNotifier.java @@ -20,7 +20,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.fail; -import static org.mockito.Matchers.any; +import static org.mockito.Mockito.any; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.never; @@ -139,14 +139,14 @@ public void testEventsOnRegistration() { verify(mockListener13, times(1)).onStateUpdated(argumentCaptor.capture()); assertEquals(VertexState.RUNNING, argumentCaptor.getValue().getVertexState()); - verify(mockListener14, never()).onStateUpdated(any(VertexStateUpdate.class)); + verify(mockListener14, never()).onStateUpdated(any()); // Vertex has not notified of state tracker.reset(); VertexStateUpdateListener mockListener2 = mock(VertexStateUpdateListener.class); tracker.registerForVertexUpdates(v2.getName(), null, mockListener2); Assert.assertEquals(0, tracker.totalCount.get()); // there should no be any event sent out - verify(mockListener2, never()).onStateUpdated(any(VertexStateUpdate.class)); + verify(mockListener2, never()).onStateUpdated(any()); // Vertex has notified about parallelism update only tracker.stateChanged(v3.getVertexId(), new VertexStateUpdateParallelismUpdated(v3.getName(), 23, -1)); 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 f2b16a9094..9a3292e8b8 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 @@ -18,11 +18,13 @@ package org.apache.tez.dag.app.dag.impl; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.anyInt; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.anyInt; +import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; import static org.mockito.Mockito.when; import java.io.IOException; @@ -149,7 +151,6 @@ import org.junit.BeforeClass; import org.junit.Ignore; import org.junit.Test; -import org.mockito.Mockito; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; @@ -856,7 +857,7 @@ public void setup() { when(appContext.getHadoopShim()).thenReturn(defaultShim); when(appContext.getApplicationID()).thenReturn(appAttemptId.getApplicationId()); - Mockito.doAnswer(new Answer() { + doAnswer(new Answer() { public ListenableFuture answer(InvocationOnMock invocation) { Object[] args = invocation.getArguments(); CallableEvent e = (CallableEvent) args[0]; @@ -1942,17 +1943,17 @@ public void testDAGHang() throws Exception { conf.setBoolean( TezConfiguration.TEZ_AM_COMMIT_ALL_OUTPUTS_ON_DAG_SUCCESS, false); - dag = Mockito.spy(new DAGImpl(dagId, conf, dagPlan, + dag = spy(new DAGImpl(dagId, conf, dagPlan, dispatcher.getEventHandler(), taskCommunicatorManagerInterface, fsTokens, clock, "user", thh, appContext)); StateMachineTez spyStateMachine = - Mockito.spy(new StateMachineTez( + spy(new StateMachineTez( dag.stateMachineFactory.make(dag), dag)); when(dag.getStateMachine()).thenReturn(spyStateMachine); dag.entityUpdateTracker = new StateChangeNotifierForTest(dag); doReturn(dag).when(appContext).getCurrentDAG(); - DAGImpl.OutputKey outputKey = Mockito.mock(DAGImpl.OutputKey.class); - ListenableFuture future = Mockito.mock(ListenableFuture.class); + DAGImpl.OutputKey outputKey = mock(DAGImpl.OutputKey.class); + ListenableFuture future = mock(ListenableFuture.class); dag.commitFutures.put(outputKey, future); initDAG(dag); startDAG(dag); @@ -1975,7 +1976,7 @@ public void testDAGHang() throws Exception { DAGEventCommitCompleted dagEvent = new DAGEventCommitCompleted( dagId, outputKey, false , new RuntimeException("test")); doThrow(new RuntimeException("test")).when( - dag).logJobHistoryUnsuccesfulEvent(any(DAGState.class), any(TezCounters.class)); + dag).logJobHistoryUnsuccesfulEvent(any(), any()); dag.handle(dagEvent); dispatcher.await(); Assert.assertTrue("DAG did not terminate!", dag.getInternalState() == DAGState.FAILED); 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 acbef67dc0..159dd9fb7f 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 @@ -17,11 +17,6 @@ */ package org.apache.tez.dag.app.dag.impl; -import static org.mockito.Matchers.any; -import static org.mockito.Mockito.doReturn; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; - import java.io.IOException; import java.nio.ByteBuffer; import java.util.ArrayList; @@ -132,13 +127,17 @@ import org.junit.Before; import org.junit.BeforeClass; import org.junit.Test; -import org.mockito.Mockito; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import static org.junit.Assert.*; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ListeningExecutorService; @@ -326,7 +325,7 @@ public void setup() { when(appContext.getApplicationID()).thenReturn(appAttemptId.getApplicationId()); when(appContext.getClock()).thenReturn(new SystemClock()); - Mockito.doAnswer(new Answer() { + doAnswer(new Answer() { public ListenableFuture answer(InvocationOnMock invocation) { Object[] args = invocation.getArguments(); CallableEvent e = (CallableEvent) args[0]; diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGSchedulerNaturalOrderControlled.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGSchedulerNaturalOrderControlled.java index 3b1db4c6d1..f79f359a7b 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGSchedulerNaturalOrderControlled.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGSchedulerNaturalOrderControlled.java @@ -19,7 +19,7 @@ package org.apache.tez.dag.app.dag.impl; import static org.junit.Assert.assertEquals; -import static org.mockito.Matchers.any; +import static org.mockito.Mockito.any; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.never; @@ -65,35 +65,35 @@ public void testSimpleFlow() { for (int i = 0; i < vertices[0].getTotalTasks(); i++) { dagScheduler.scheduleTaskEx(createScheduleRequest(vertices[0].getVertexId(), i, 0)); } - verify(eventHandler, times(vertices[0].getTotalTasks())).handle(any(Event.class)); + verify(eventHandler, times(vertices[0].getTotalTasks())).handle(any()); reset(eventHandler); // Schedule 3 tasks belonging to v2 for (int i = 0; i < 3; i++) { dagScheduler.scheduleTaskEx(createScheduleRequest(vertices[2].getVertexId(), i, 0)); } - verify(eventHandler, times(3)).handle(any(Event.class)); + verify(eventHandler, times(3)).handle(any()); reset(eventHandler); // Schedule 3 tasks belonging to v3 for (int i = 0; i < 3; i++) { dagScheduler.scheduleTaskEx(createScheduleRequest(vertices[3].getVertexId(), i, 0)); } - verify(eventHandler, times(3)).handle(any(Event.class)); + verify(eventHandler, times(3)).handle(any()); reset(eventHandler); // Schedule remaining tasks belonging to v2 for (int i = 3; i < vertices[2].getTotalTasks(); i++) { dagScheduler.scheduleTaskEx(createScheduleRequest(vertices[2].getVertexId(), i, 0)); } - verify(eventHandler, times(vertices[2].getTotalTasks() - 3)).handle(any(Event.class)); + verify(eventHandler, times(vertices[2].getTotalTasks() - 3)).handle(any()); reset(eventHandler); // Schedule remaining tasks belonging to v3 for (int i = 3; i < vertices[3].getTotalTasks(); i++) { dagScheduler.scheduleTaskEx(createScheduleRequest(vertices[3].getVertexId(), i, 0)); } - verify(eventHandler, times(vertices[3].getTotalTasks() - 3)).handle(any(Event.class)); + verify(eventHandler, times(vertices[3].getTotalTasks() - 3)).handle(any()); reset(eventHandler); @@ -101,7 +101,7 @@ public void testSimpleFlow() { for (int i = 0; i < vertices[4].getTotalTasks(); i++) { dagScheduler.scheduleTaskEx(createScheduleRequest(vertices[4].getVertexId(), i, 0)); } - verify(eventHandler, times(vertices[4].getTotalTasks())).handle(any(Event.class)); + verify(eventHandler, times(vertices[4].getTotalTasks())).handle(any()); reset(eventHandler); } @@ -124,7 +124,7 @@ public void testSourceRequestDelayed() { for (int i = 0; i < vertices[0].getTotalTasks(); i++) { dagScheduler.scheduleTaskEx(createScheduleRequest(vertices[0].getVertexId(), i, 0)); } - verify(eventHandler, times(vertices[0].getTotalTasks())).handle(any(Event.class)); + verify(eventHandler, times(vertices[0].getTotalTasks())).handle(any()); reset(eventHandler); // v2 behaving as if configured with slow-start. @@ -132,14 +132,14 @@ public void testSourceRequestDelayed() { for (int i = 0; i < vertices[3].getTotalTasks(); i++) { dagScheduler.scheduleTaskEx(createScheduleRequest(vertices[3].getVertexId(), i, 0)); } - verify(eventHandler, times(vertices[3].getTotalTasks())).handle(any(Event.class)); + verify(eventHandler, times(vertices[3].getTotalTasks())).handle(any()); reset(eventHandler); // Scheduling all tasks belonging to v4. None should get scheduled. for (int i = 0; i < vertices[4].getTotalTasks(); i++) { dagScheduler.scheduleTaskEx(createScheduleRequest(vertices[4].getVertexId(), i, 0)); } - verify(eventHandler, never()).handle(any(Event.class)); + verify(eventHandler, never()).handle(any()); reset(eventHandler); // v2 now starts scheduling ... @@ -147,7 +147,7 @@ public void testSourceRequestDelayed() { for (int i = 0; i < 3; i++) { dagScheduler.scheduleTaskEx(createScheduleRequest(vertices[2].getVertexId(), i, 0)); } - verify(eventHandler, times(3)).handle(any(Event.class)); + verify(eventHandler, times(3)).handle(any()); reset(eventHandler); // Schedule remaining tasks belonging to v2 @@ -192,7 +192,7 @@ public void testParallelismUpdated() { for (int i = 0; i < vertices[0].getTotalTasks(); i++) { dagScheduler.scheduleTaskEx(createScheduleRequest(vertices[0].getVertexId(), i, 0)); } - verify(eventHandler, times(vertices[0].getTotalTasks())).handle(any(Event.class)); + verify(eventHandler, times(vertices[0].getTotalTasks())).handle(any()); reset(eventHandler); assertEquals(10, vertices[2].getTotalTasks()); @@ -202,14 +202,14 @@ public void testParallelismUpdated() { for (int i = 0; i < vertices[3].getTotalTasks(); i++) { dagScheduler.scheduleTaskEx(createScheduleRequest(vertices[3].getVertexId(), i, 0)); } - verify(eventHandler, times(vertices[3].getTotalTasks())).handle(any(Event.class)); + verify(eventHandler, times(vertices[3].getTotalTasks())).handle(any()); reset(eventHandler); // Schedule all tasks belonging to v4 for (int i = 0; i < vertices[4].getTotalTasks(); i++) { dagScheduler.scheduleTaskEx(createScheduleRequest(vertices[4].getVertexId(), i, 0)); } - verify(eventHandler, never()).handle(any(Event.class)); + verify(eventHandler, never()).handle(any()); reset(eventHandler); // Reset the parallelism for v2. @@ -221,7 +221,7 @@ public void testParallelismUpdated() { dagScheduler.scheduleTaskEx(createScheduleRequest(vertices[2].getVertexId(), i, 0)); } verify(eventHandler, times(vertices[2].getTotalTasks() + vertices[4].getTotalTasks())) - .handle(any(Event.class)); + .handle(any()); reset(eventHandler); } @@ -243,7 +243,7 @@ public void testMultipleRequestsForSameTask() { for (int i = 0; i < vertices[0].getTotalTasks() - 1; i++) { dagScheduler.scheduleTaskEx(createScheduleRequest(vertices[0].getVertexId(), i, 0)); } - verify(eventHandler, times(vertices[0].getTotalTasks() - 1)).handle(any(Event.class)); + verify(eventHandler, times(vertices[0].getTotalTasks() - 1)).handle(any()); reset(eventHandler); @@ -252,7 +252,7 @@ public void testMultipleRequestsForSameTask() { dagScheduler.scheduleTaskEx(createScheduleRequest(vertices[2].getVertexId(), i, 0)); } // Nothing should be scheduled - verify(eventHandler, never()).handle(any(Event.class)); + verify(eventHandler, never()).handle(any()); reset(eventHandler); // Schedule an extra attempt for all but 1 task belonging to v0 @@ -260,14 +260,14 @@ public void testMultipleRequestsForSameTask() { dagScheduler.scheduleTaskEx(createScheduleRequest(vertices[0].getVertexId(), i, 1)); } // Only v0 requests should have gone out - verify(eventHandler, times(vertices[0].getTotalTasks() - 1)).handle(any(Event.class)); + verify(eventHandler, times(vertices[0].getTotalTasks() - 1)).handle(any()); reset(eventHandler); // Schedule last task of v0, with attempt 1 dagScheduler.scheduleTaskEx( createScheduleRequest(vertices[0].getVertexId(), vertices[0].getTotalTasks() - 1, 1)); // One v0 request and all of v2 should have gone out - verify(eventHandler, times(1 + vertices[2].getTotalTasks())).handle(any(Event.class)); + verify(eventHandler, times(1 + vertices[2].getTotalTasks())).handle(any()); } 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 c9c99f5630..c6578ffa31 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 @@ -21,7 +21,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; -import static org.mockito.Matchers.eq; +import static org.mockito.Mockito.eq; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.reset; diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestImmediateStartVertexManager.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestImmediateStartVertexManager.java index a17c7c5b45..67a9da52b1 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestImmediateStartVertexManager.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestImmediateStartVertexManager.java @@ -37,7 +37,7 @@ import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; -import static org.mockito.Matchers.anyList; +import static org.mockito.Mockito.anyList; import static org.mockito.Mockito.anySet; import static org.mockito.Mockito.anyString; import static org.mockito.Mockito.doAnswer; diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestRootInputVertexManager.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestRootInputVertexManager.java index 39a291ebbd..144d28d4a5 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestRootInputVertexManager.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestRootInputVertexManager.java @@ -23,11 +23,11 @@ import static org.apache.tez.dag.app.dag.impl.RootInputVertexManager.TEZ_ROOT_INPUT_VERTEX_MANAGER_MIN_SRC_FRACTION; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.anyListOf; -import static org.mockito.Matchers.eq; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.anyList; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.eq; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -161,7 +161,7 @@ public void testRootInputVertexManagerSlowStart() { VertexManagerPluginContext mockContext = mock(VertexManagerPluginContext.class); - when(mockContext.getVertexStatistics(any(String.class))) + when(mockContext.getVertexStatistics(any())) .thenReturn(mock(VertexStatistics.class)); when(mockContext.getInputVertexEdgeProperties()) .thenReturn(mockInputVertices); @@ -176,7 +176,7 @@ public void testRootInputVertexManagerSlowStart() { final List scheduledTasks = Lists.newLinkedList(); doAnswer(new ScheduledTasksAnswer(scheduledTasks)).when( - mockContext).scheduleTasks(anyListOf(VertexManagerPluginContext.ScheduleTaskRequest.class)); + mockContext).scheduleTasks(anyList()); // source vertices have 0 tasks. immediate start of all managed tasks when(mockContext.getVertexNumTasks(mockSrcVertexId1)).thenReturn(0); @@ -508,7 +508,7 @@ public void testTezDrainCompletionsOnVertexStart() throws IOException { VertexManagerPluginContext mockContext = mock(VertexManagerPluginContext.class); - when(mockContext.getVertexStatistics(any(String.class))) + when(mockContext.getVertexStatistics(any())) .thenReturn(mock(VertexStatistics.class)); when(mockContext.getInputVertexEdgeProperties()) .thenReturn(mockInputVertices); 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 dc59b4615c..d0088bfc54 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 @@ -22,7 +22,9 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.fail; -import static org.mockito.Matchers.anyInt; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.anyInt; +import static org.mockito.Mockito.anyString; import static org.mockito.Mockito.atLeast; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; @@ -128,7 +130,6 @@ import org.junit.BeforeClass; import org.junit.Test; import org.mockito.ArgumentCaptor; -import org.mockito.Mockito; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -181,11 +182,11 @@ private void createMockVertex(Configuration conf) { when(mockVertex.getServicePluginInfo()).thenReturn(servicePluginInfo); when(mockVertex.getVertexConfig()).thenReturn(new VertexImpl.VertexConfigImpl(conf)); AppContext appContext = mock(AppContext.class); - when(appContext.getTaskScheduerIdentifier(Mockito.anyString())).thenReturn(0); + when(appContext.getTaskScheduerIdentifier(anyString())).thenReturn(0); when(mockVertex.getAppContext()).thenReturn(appContext); AMNodeTracker nodeTracker = mock(AMNodeTracker.class); - when(nodeTracker.getNumNodes(Mockito.anyInt())).thenReturn(10); - when(nodeTracker.getNumActiveNodes(Mockito.anyInt())).thenReturn(8); + when(nodeTracker.getNumNodes(anyInt())).thenReturn(10); + when(nodeTracker.getNumActiveNodes(anyInt())).thenReturn(8); when(appContext.getNodeTracker()).thenReturn(nodeTracker); } @@ -1922,7 +1923,7 @@ taListener, taskConf, new SystemClock(), assertEquals(true, taImpl.inputFailedReported); int expectedEventsAfterFetchFailure = expectedEventsTillSucceeded + 2; - arg.getAllValues().clear(); + arg = ArgumentCaptor.forClass(Event.class); verify(eventHandler, times(expectedEventsAfterFetchFailure)).handle(arg.capture()); Event e = verifyEventType( arg.getAllValues().subList(expectedEventsTillSucceeded, @@ -2195,10 +2196,9 @@ private void testMapTaskFailingForFetchFailureType(boolean isLocalFetch, when(destTaskAttemptId.getTaskID()).thenReturn(mock(TezTaskID.class)); when(destTaskAttemptId.getVertexID()).thenReturn(mock(TezVertexID.class)); when(appCtx.getCurrentDAG()).thenReturn(mock(DAG.class)); - when(appCtx.getCurrentDAG().getVertex(Mockito.any(TezVertexID.class))) + when(appCtx.getCurrentDAG().getVertex(any(TezVertexID.class))) .thenReturn(mock(Vertex.class)); - when(appCtx.getCurrentDAG().getVertex(Mockito.any(TezVertexID.class)).getRunningTasks()) - .thenReturn(100); + when(mock(Vertex.class).getRunningTasks()).thenReturn(100); EventMetaData mockMeta = mock(EventMetaData.class); when(mockMeta.getTaskAttemptID()).thenReturn(destTaskAttemptId); @@ -2228,12 +2228,13 @@ public void testMapTaskIsBlamedByDownstreamAttemptsFromDifferentHosts() { InputReadErrorEvent inputReadErrorEvent2 = InputReadErrorEvent.create("", 1, 1, 1, false, false, "downstream_host_2"); - TezTaskAttemptID destTaskAttemptId = mock(TezTaskAttemptID.class); + TezTaskAttemptID destTaskAttemptId = mock(TezTaskAttemptID.class, RETURNS_DEEP_STUBS); when(destTaskAttemptId.getTaskID()).thenReturn(mock(TezTaskID.class)); when(destTaskAttemptId.getTaskID().getVertexID()).thenReturn(mock(TezVertexID.class)); when(appCtx.getCurrentDAG()).thenReturn(mock(DAG.class)); - when(appCtx.getCurrentDAG().getVertex(Mockito.any(TezVertexID.class))).thenReturn(mock(Vertex.class)); - when(appCtx.getCurrentDAG().getVertex(Mockito.any(TezVertexID.class)).getRunningTasks()).thenReturn(100); + when(appCtx.getCurrentDAG().getVertex(any(TezVertexID.class))) + .thenReturn(mock(Vertex.class)); + when(mock(Vertex.class).getRunningTasks()).thenReturn(100); EventMetaData mockMeta = mock(EventMetaData.class); when(mockMeta.getTaskAttemptID()).thenReturn(destTaskAttemptId); diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskImpl.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskImpl.java index f4e40f7333..3bf59977e5 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskImpl.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskImpl.java @@ -769,7 +769,7 @@ public void testTaskSucceedAndRetroActiveFailure() { // The task should now have succeeded assertTaskSucceededState(); - verify(mockTask.stateChangeNotifier).taskSucceeded(any(String.class), eq(taskId), + verify(mockTask.stateChangeNotifier).taskSucceeded(any(), eq(taskId), eq(mockTask.getLastAttempt().getTaskAttemptID().getId())); ArgumentCaptor argumentCaptor = ArgumentCaptor.forClass(DAGHistoryEvent.class); @@ -818,7 +818,7 @@ public void testTaskSucceedAndRetroActiveKilled() { // The task should now have succeeded assertTaskSucceededState(); - verify(mockTask.stateChangeNotifier).taskSucceeded(any(String.class), eq(taskId), + verify(mockTask.stateChangeNotifier).taskSucceeded(any(), eq(taskId), eq(mockTask.getLastAttempt().getTaskAttemptID().getId())); eventHandler.events.clear(); @@ -982,7 +982,7 @@ public void testKilledAttemptUpdatesDAGScheduler() { // The task should now have succeeded and sent kill to other attempt assertTaskSucceededState(); - verify(mockTask.stateChangeNotifier).taskSucceeded(any(String.class), eq(taskId), + verify(mockTask.stateChangeNotifier).taskSucceeded(any(), eq(taskId), eq(firstAttempt.getTaskAttemptID().getId())); @SuppressWarnings("rawtypes") Event event = eventHandler.events.get(eventHandler.events.size()-1); @@ -1019,7 +1019,7 @@ public void testSpeculatedThenRetroactiveFailure() { // The task should now have succeeded and sent kill to other attempt assertTaskSucceededState(); - verify(mockTask.stateChangeNotifier).taskSucceeded(any(String.class), eq(taskId), + verify(mockTask.stateChangeNotifier).taskSucceeded(any(), eq(taskId), eq(firstAttempt.getTaskAttemptID().getId())); @SuppressWarnings("rawtypes") Event event = eventHandler.events.get(eventHandler.events.size()-1); 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 c118110948..10ecfe503c 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 @@ -32,9 +32,16 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.anyInt; -import static org.mockito.Mockito.*; +import static org.mockito.Mockito.RETURNS_DEEP_STUBS; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.anyInt; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.doCallRealMethod; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.when; import java.io.ByteArrayOutputStream; import java.io.DataInput; @@ -85,6 +92,7 @@ import org.apache.tez.runtime.library.common.shuffle.ShuffleUtils; import org.apache.tez.runtime.library.shuffle.impl.ShuffleUserPayloads; import org.apache.tez.test.GraceShuffleVertexManagerForTest; +import org.mockito.Mockito; import org.roaringbitmap.RoaringBitmap; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -230,7 +238,6 @@ import org.junit.BeforeClass; import org.junit.Ignore; import org.junit.Test; -import org.mockito.Mockito; import org.mockito.internal.util.collections.Sets; import com.google.common.base.Joiner; @@ -2739,7 +2746,7 @@ public void setupPreDagCreation() { dagId = TezDAGID.getInstance(appAttemptId.getApplicationId(), 1); taskSpecificLaunchCmdOption = mock(TaskSpecificLaunchCmdOption.class); doReturn(false).when(taskSpecificLaunchCmdOption).addTaskSpecificLaunchCmdOption( - any(String.class), + any(), anyInt()); } @@ -2816,7 +2823,7 @@ public void setupPostDagCreation(boolean cleanupShuffleDataAtVertexLevel) throws execService = mock(ListeningExecutorService.class); final ListenableFuture mockFuture = mock(ListenableFuture.class); - Mockito.doAnswer(new Answer() { + doAnswer(new Answer() { public ListenableFuture answer(InvocationOnMock invocation) { Object[] args = invocation.getArguments(); CallableEvent e = (CallableEvent) args[0]; @@ -3133,7 +3140,7 @@ public void testVertexConfigureEventWithReconfigure() throws Exception { Assert.assertEquals("vertex2", listener.events.get(0).getVertexName()); Assert.assertEquals(org.apache.tez.dag.api.event.VertexState.CONFIGURED, listener.events.get(0).getVertexState()); - updateTracker.unregisterForVertexUpdates("vertex2", listener); + updateTracker.unregisterForVertexUpdates("vertex2", listener); } @Test (timeout=5000) @@ -7512,8 +7519,7 @@ public void testVertexShuffleDelete() throws Exception { checkSpannedVertices(); runVertices(); Mockito.verify(appContext.getAppMaster().getContainerLauncherManager(), - times(3)).vertexComplete(any(TezVertexID.class), - any(JobTokenSecretManager.class), any(Set.class)); + times(3)).vertexComplete(any(), any(), any()); } private void checkSpannedVertices() { diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexManager.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexManager.java index f7a60377c4..058dc76164 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexManager.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexManager.java @@ -24,8 +24,9 @@ import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; -import static org.mockito.Matchers.any; import static org.mockito.Mockito.RETURNS_DEEP_STUBS; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.never; @@ -70,7 +71,6 @@ import org.junit.Before; import org.junit.Test; import org.mockito.ArgumentCaptor; -import org.mockito.Mockito; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; @@ -92,7 +92,7 @@ public void setup() { mockAppContext = mock(AppContext.class, RETURNS_DEEP_STUBS); execService = mock(ListeningExecutorService.class); final ListenableFuture mockFuture = mock(ListenableFuture.class); - Mockito.doAnswer(new Answer() { + doAnswer(new Answer() { public ListenableFuture answer(InvocationOnMock invocation) { Object[] args = invocation.getArguments(); CallableEvent e = (CallableEvent) args[0]; diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/launcher/TestContainerLauncherManager.java b/tez-dag/src/test/java/org/apache/tez/dag/app/launcher/TestContainerLauncherManager.java index b3568eb20f..0bd41f4a60 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/launcher/TestContainerLauncherManager.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/launcher/TestContainerLauncherManager.java @@ -19,7 +19,7 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; -import static org.mockito.Matchers.any; +import static org.mockito.Mockito.any; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.mock; @@ -340,7 +340,7 @@ public void testContainerLauncherUserError() throws ServicePluginException { // launch container doThrow(new RuntimeException("testexception")).when(containerLauncher) - .launchContainer(any(ContainerLaunchRequest.class)); + .launchContainer(any()); ContainerLaunchContext clc1 = mock(ContainerLaunchContext.class); Container container1 = mock(Container.class); ContainerLauncherLaunchRequestEvent launchRequestEvent = @@ -365,7 +365,7 @@ public void testContainerLauncherUserError() throws ServicePluginException { // stop container doThrow(new RuntimeException("teststopexception")).when(containerLauncher) - .stopContainer(any(ContainerStopRequest.class)); + .stopContainer(any()); ContainerId containerId2 = mock(ContainerId.class); NodeId nodeId2 = mock(NodeId.class); ContainerLauncherStopRequestEvent stopRequestEvent = 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 ef61cad94e..fe5a888b56 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 @@ -21,10 +21,10 @@ import static org.junit.Assert.assertNull; 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.Mockito.doReturn; +import static org.mockito.Mockito.any; import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.eq; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.times; @@ -203,9 +203,9 @@ public void testDelayedReuseContainerBecomesAvailable() TestTaskSchedulerHelpers.waitForDelayedDrainNotify(drainNotifier); drainableAppCallback.drain(); verify(taskSchedulerManager).taskAllocated( - eq(0), eq(ta11), any(Object.class), eq(containerHost1)); + eq(0), eq(ta11), any(), eq(containerHost1)); verify(taskSchedulerManager).taskAllocated( - eq(0), eq(ta21), any(Object.class), eq(containerHost2)); + eq(0), eq(ta21), any(), eq(containerHost2)); // Adding the event later so that task1 assigned to containerHost1 // is deterministic. @@ -217,9 +217,9 @@ public void testDelayedReuseContainerBecomesAvailable() drainableAppCallback.drain(); verifyDeAllocateTask(taskScheduler, ta11, true, null, null); verify(taskSchedulerManager, times(1)).taskAllocated( - eq(0), eq(ta31), any(Object.class), eq(containerHost1)); + eq(0), eq(ta31), any(), eq(containerHost1)); verify(rmClient, times(0)).releaseAssignedContainer( - eq(containerHost1.getId())); + eq(containerHost1.getId())); eventHandler.verifyNoInvocations(AMContainerEventStopRequest.class); eventHandler.reset(); @@ -325,8 +325,8 @@ public void testDelayedReuseContainerNotAvailable() taskScheduler.onContainersAllocated(Lists.newArrayList(containerHost1, containerHost2)); TestTaskSchedulerHelpers.waitForDelayedDrainNotify(drainNotifier); drainableAppCallback.drain(); - verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta11), any(Object.class), eq(containerHost1)); - verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta21), any(Object.class), + verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta11), any(), eq(containerHost1)); + verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta21), any(), eq(containerHost2)); // Adding the event later so that task1 assigned to containerHost1 is deterministic. @@ -338,7 +338,7 @@ public void testDelayedReuseContainerNotAvailable() drainableAppCallback.drain(); verifyDeAllocateTask(taskScheduler, ta21, true, null, null); verify(taskSchedulerManager, times(0)).taskAllocated( - eq(0), eq(ta31), any(Object.class), eq(containerHost2)); + eq(0), eq(ta31), any(), eq(containerHost2)); verify(rmClient, times(1)).releaseAssignedContainer( eq(containerHost2.getId())); eventHandler.verifyInvocation(AMContainerEventStopRequest.class); @@ -426,7 +426,7 @@ public void testSimpleReuse() throws IOException, InterruptedException, Executio taskScheduler.onContainersAllocated(Collections.singletonList(container1)); TestTaskSchedulerHelpers.waitForDelayedDrainNotify(drainNotifier); drainableAppCallback.drain(); - verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta11), any(Object.class), + verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta11), any(), eq(container1)); // Task assigned to container completed successfully. Container should be re-used. @@ -435,7 +435,7 @@ public void testSimpleReuse() throws IOException, InterruptedException, Executio null, 0)); drainableAppCallback.drain(); verifyDeAllocateTask(taskScheduler, ta11, true, null, null); - verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta12), any(Object.class), eq(container1)); + verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta12), any(), eq(container1)); verify(rmClient, times(0)).releaseAssignedContainer(eq(container1.getId())); eventHandler.verifyNoInvocations(AMContainerEventStopRequest.class); eventHandler.reset(); @@ -447,7 +447,7 @@ public void testSimpleReuse() throws IOException, InterruptedException, Executio null, 0)); drainableAppCallback.drain(); verifyDeAllocateTask(taskScheduler, ta12, true, null, null); - verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta13), any(Object.class), + verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta13), any(), eq(container1)); verify(rmClient, times(0)).releaseAssignedContainer(eq(container1.getId())); eventHandler.verifyNoInvocations(AMContainerEventStopRequest.class); @@ -458,7 +458,7 @@ public void testSimpleReuse() throws IOException, InterruptedException, Executio new AMSchedulerEventTAEnded(ta13, container1.getId(), TaskAttemptState.FAILED, null, "TIMEOUT", 0)); drainableAppCallback.drain(); - verify(taskSchedulerManager, times(0)).taskAllocated(eq(0), eq(ta14), any(Object.class), + verify(taskSchedulerManager, times(0)).taskAllocated(eq(0), eq(ta14), any(), eq(container1)); verifyDeAllocateTask(taskScheduler, ta13, false, null, "TIMEOUT"); verify(rmClient).releaseAssignedContainer(eq(container1.getId())); @@ -471,7 +471,7 @@ public void testSimpleReuse() throws IOException, InterruptedException, Executio taskScheduler.onContainersAllocated(Collections.singletonList(container2)); TestTaskSchedulerHelpers.waitForDelayedDrainNotify(drainNotifier); drainableAppCallback.drain(); - verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta14), any(Object.class), + verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta14), any(), eq(container2)); // Task assigned to container completed successfully. No pending requests. Container should be released. @@ -571,7 +571,7 @@ public void testReuseWithTaskSpecificLaunchCmdOption() throws IOException, Inter taskScheduler.onContainersAllocated(Collections.singletonList(container1)); TestTaskSchedulerHelpers.waitForDelayedDrainNotify(drainNotifier); drainableAppCallback.drain(); - verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta11), any(Object.class), + verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta11), any(), eq(container1)); // First task had profiling on. This container can not be reused further. @@ -580,7 +580,7 @@ public void testReuseWithTaskSpecificLaunchCmdOption() throws IOException, Inter null, 0)); drainableAppCallback.drain(); verifyDeAllocateTask(taskScheduler, ta11, true, null, null); - verify(taskSchedulerManager, times(0)).taskAllocated(eq(0), eq(ta12), any(Object.class), + verify(taskSchedulerManager, times(0)).taskAllocated(eq(0), eq(ta12), any(), eq(container1)); verify(rmClient, times(1)).releaseAssignedContainer(eq(container1.getId())); eventHandler.verifyInvocation(AMContainerEventStopRequest.class); @@ -614,7 +614,7 @@ public void testReuseWithTaskSpecificLaunchCmdOption() throws IOException, Inter taskScheduler.onContainersAllocated(Collections.singletonList(container2)); TestTaskSchedulerHelpers.waitForDelayedDrainNotify(drainNotifier); drainableAppCallback.drain(); - verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta13), any(Object.class), eq(container2)); + verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta13), any(), eq(container2)); // Verify that the container can not be reused when profiling option is turned on // Even for 2 tasks having same profiling option can have container reusability. @@ -623,7 +623,7 @@ public void testReuseWithTaskSpecificLaunchCmdOption() throws IOException, Inter null, 0)); drainableAppCallback.drain(); verifyDeAllocateTask(taskScheduler, ta13, true, null, null); - verify(taskSchedulerManager, times(0)).taskAllocated(eq(0), eq(ta14), any(Object.class), + verify(taskSchedulerManager, times(0)).taskAllocated(eq(0), eq(ta14), any(), eq(container2)); verify(rmClient, times(1)).releaseAssignedContainer(eq(container2.getId())); eventHandler.verifyInvocation(AMContainerEventStopRequest.class); @@ -661,7 +661,7 @@ public void testReuseWithTaskSpecificLaunchCmdOption() throws IOException, Inter taskScheduler.onContainersAllocated(Collections.singletonList(container3)); TestTaskSchedulerHelpers.waitForDelayedDrainNotify(drainNotifier); drainableAppCallback.drain(); - verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta15), any(Object.class), + verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta15), any(), eq(container3)); //Ensure task 6 (of vertex 1) is allocated to same container @@ -670,7 +670,7 @@ public void testReuseWithTaskSpecificLaunchCmdOption() throws IOException, Inter null, 0)); drainableAppCallback.drain(); verifyDeAllocateTask(taskScheduler, ta15, true, null, null); - verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta16), any(Object.class), eq(container3)); + verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta16), any(), eq(container3)); eventHandler.reset(); taskScheduler.shutdown(); @@ -759,7 +759,7 @@ public void testReuseNonLocalRequest() TestTaskSchedulerHelpers.waitForDelayedDrainNotify(drainNotifier); drainableAppCallback.drain(); verify(taskSchedulerManager).taskAllocated( - eq(0), eq(ta11), any(Object.class), eq(container1)); + eq(0), eq(ta11), any(), eq(container1)); // Send launch request for task2 (vertex2) taskSchedulerManager.handleEvent(lrEvent12); @@ -773,7 +773,7 @@ public void testReuseNonLocalRequest() drainableAppCallback.drain(); verifyDeAllocateTask(taskScheduler, ta11, true, null, null); verify(taskSchedulerManager, times(0)).taskAllocated( - eq(0), eq(ta12), any(Object.class), eq(container1)); + eq(0), eq(ta12), any(), eq(container1)); verify(rmClient, times(0)).releaseAssignedContainer(eq(container1.getId())); eventHandler.verifyNoInvocations(AMContainerEventStopRequest.class); eventHandler.reset(); @@ -781,7 +781,7 @@ public void testReuseNonLocalRequest() TestTaskSchedulerHelpers.waitForDelayedDrainNotify(drainNotifier); drainableAppCallback.drain(); verify(taskSchedulerManager).taskAllocated( - eq(0), eq(ta12), any(Object.class), eq(container1)); + eq(0), eq(ta12), any(), eq(container1)); // TA12 completed. taskSchedulerManager.handleEvent( @@ -887,7 +887,7 @@ public void testReuseAcrossVertices() TestTaskSchedulerHelpers.waitForDelayedDrainNotify(drainNotifier); drainableAppCallback.drain(); verify(taskSchedulerManager).taskAllocated( - eq(0), eq(ta11), any(Object.class), eq(container1)); + eq(0), eq(ta11), any(), eq(container1)); // Send launch request for task2 (vertex2) taskSchedulerManager.handleEvent(lrEvent21); @@ -900,7 +900,7 @@ public void testReuseAcrossVertices() drainableAppCallback.drain(); verifyDeAllocateTask(taskScheduler, ta11, true, null, null); verify(taskSchedulerManager).taskAllocated( - eq(0), eq(ta21), any(Object.class), eq(container1)); + eq(0), eq(ta21), any(), eq(container1)); verify(rmClient, times(0)).releaseAssignedContainer(eq(container1.getId())); // Task 2 completes. @@ -999,7 +999,7 @@ public void testReuseLocalResourcesChanged() throws IOException, InterruptedExce taskScheduler.onContainersAllocated(Collections.singletonList(container1)); TestTaskSchedulerHelpers.waitForDelayedDrainNotify(drainNotifier); drainableAppCallback.drain(); - verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta111), any(Object.class), eq(container1)); + verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta111), any(), eq(container1)); assignEvent = (AMContainerEventAssignTA) eventHandler.verifyInvocation(AMContainerEventAssignTA.class); assertEquals(1, assignEvent.getRemoteTaskLocalResources().size()); @@ -1009,7 +1009,7 @@ public void testReuseLocalResourcesChanged() throws IOException, InterruptedExce null, 0)); drainableAppCallback.drain(); verifyDeAllocateTask(taskScheduler, ta111, true, null, null); - verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta112), any(Object.class), eq(container1)); + verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta112), any(), eq(container1)); verify(rmClient, times(0)).releaseAssignedContainer(eq(container1.getId())); eventHandler.verifyNoInvocations(AMContainerEventStopRequest.class); assignEvent = (AMContainerEventAssignTA) eventHandler.verifyInvocation(AMContainerEventAssignTA.class); @@ -1052,7 +1052,7 @@ public void testReuseLocalResourcesChanged() throws IOException, InterruptedExce TestTaskSchedulerHelpers.waitForDelayedDrainNotify(drainNotifier); drainableAppCallback.drain(); - verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta211), any(Object.class), + verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta211), any(), eq(container1)); verify(rmClient, times(0)).releaseAssignedContainer(eq(container1.getId())); eventHandler.verifyNoInvocations(AMContainerEventStopRequest.class); @@ -1065,7 +1065,7 @@ public void testReuseLocalResourcesChanged() throws IOException, InterruptedExce null, 0)); drainableAppCallback.drain(); verifyDeAllocateTask(taskScheduler, ta211, true, null, null); - verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta212), any(Object.class), eq(container1)); + verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta212), any(), eq(container1)); verify(rmClient, times(0)).releaseAssignedContainer(eq(container1.getId())); eventHandler.verifyNoInvocations(AMContainerEventStopRequest.class); assignEvent = (AMContainerEventAssignTA) eventHandler.verifyInvocation(AMContainerEventAssignTA.class); @@ -1186,7 +1186,7 @@ public void testReuseConflictLocalResources() throws IOException, InterruptedExc taskScheduler.onContainersAllocated(Collections.singletonList(container1)); TestTaskSchedulerHelpers.waitForDelayedDrainNotify(drainNotifier); drainableAppCallback.drain(); - verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta111), any(Object.class), eq(container1)); + verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta111), any(), eq(container1)); assignEvent = (AMContainerEventAssignTA) eventHandler.verifyInvocation(AMContainerEventAssignTA.class); assertEquals(1, assignEvent.getRemoteTaskLocalResources().size()); @@ -1196,7 +1196,7 @@ public void testReuseConflictLocalResources() throws IOException, InterruptedExc null, 0)); drainableAppCallback.drain(); verifyDeAllocateTask(taskScheduler, ta111, true, null, null); - verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta112), any(Object.class), eq(container1)); + verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta112), any(), eq(container1)); verify(rmClient, times(0)).releaseAssignedContainer(eq(container1.getId())); eventHandler.verifyNoInvocations(AMContainerEventStopRequest.class); assignEvent = (AMContainerEventAssignTA) eventHandler.verifyInvocation(AMContainerEventAssignTA.class); @@ -1217,7 +1217,7 @@ public void testReuseConflictLocalResources() throws IOException, InterruptedExc taskSchedulerManager.handleEvent(lrEvent13); TestTaskSchedulerHelpers.waitForDelayedDrainNotify(drainNotifier); drainableAppCallback.drain(); - verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta113), any(Object.class), eq(container1)); + verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta113), any(), eq(container1)); verify(rmClient, times(0)).releaseAssignedContainer(eq(container1.getId())); eventHandler.verifyNoInvocations(AMContainerEventStopRequest.class); eventHandler.reset(); @@ -1234,7 +1234,7 @@ public void testReuseConflictLocalResources() throws IOException, InterruptedExc taskSchedulerManager.handleEvent(lrEvent14); TestTaskSchedulerHelpers.waitForDelayedDrainNotify(drainNotifier); drainableAppCallback.drain(); - verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta114), any(Object.class), eq(container1)); + verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta114), any(), eq(container1)); verify(rmClient, times(0)).releaseAssignedContainer(eq(container1.getId())); eventHandler.verifyNoInvocations(AMContainerEventStopRequest.class); eventHandler.reset(); @@ -1276,7 +1276,7 @@ public void testReuseConflictLocalResources() throws IOException, InterruptedExc taskScheduler.onContainersAllocated(Collections.singletonList(container2)); TestTaskSchedulerHelpers.waitForDelayedDrainNotify(drainNotifier); drainableAppCallback.drain(); - verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta211), any(Object.class), eq(container2)); + verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta211), any(), eq(container2)); eventHandler.reset(); taskSchedulerManager.handleEvent(new AMSchedulerEventTAEnded(ta211, container2.getId(), @@ -1305,7 +1305,7 @@ public void testReuseConflictLocalResources() throws IOException, InterruptedExc taskSchedulerManager.handleEvent(lrEvent31); TestTaskSchedulerHelpers.waitForDelayedDrainNotify(drainNotifier); drainableAppCallback.drain(); - verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta311), any(Object.class), eq(container2)); + verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta311), any(), eq(container2)); eventHandler.reset(); taskScheduler.shutdown(); @@ -1376,7 +1376,7 @@ public void testAssignmentOnShutdown() taskScheduler.onContainersAllocated(Collections.singletonList(container1)); drainableAppCallback.drain(); verify(taskSchedulerManager, times(0)).taskAllocated(eq(0), eq(ta11), - any(Object.class), eq(container1)); + any(), eq(container1)); taskScheduler.shutdown(); taskSchedulerManager.close(); } @@ -1467,20 +1467,20 @@ eventHandler, rmClient, new AlwaysMatchesContainerMatcher(), taskScheduler.onContainersAllocated(Collections.singletonList(container1)); TestTaskSchedulerHelpers.waitForDelayedDrainNotify(drainNotifier); drainableAppCallback.drain(); - verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta11), any(Object.class), eq(container1)); + verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta11), any(), eq(container1)); // Second container allocated, should start ta13 taskScheduler.onContainersAllocated(Collections.singletonList(container2)); TestTaskSchedulerHelpers.waitForDelayedDrainNotify(drainNotifier); drainableAppCallback.drain(); - verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta13), any(Object.class), eq(container2)); + verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta13), any(), eq(container2)); // ta11 finished, should start ta12 taskSchedulerManager.handleEvent(new AMSchedulerEventTAEnded(ta11, container1.getId(), TaskAttemptState.SUCCEEDED, null, null, 0)); drainableAppCallback.drain(); verifyDeAllocateTask(taskScheduler, ta11, true, null, null); - verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta12), any(Object.class), eq(container1)); + verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta12), any(), eq(container1)); verify(rmClient, times(0)).releaseAssignedContainer(eq(container1.getId())); eventHandler.verifyNoInvocations(AMContainerEventStopRequest.class); eventHandler.reset(); @@ -1490,7 +1490,7 @@ eventHandler, rmClient, new AlwaysMatchesContainerMatcher(), TaskAttemptState.SUCCEEDED, null, null, 0)); drainableAppCallback.drain(); verifyDeAllocateTask(taskScheduler, ta13, true, null, null); - verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta14), any(Object.class), eq(container2)); + verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta14), any(), eq(container2)); verify(rmClient, times(0)).releaseAssignedContainer(eq(container2.getId())); eventHandler.verifyNoInvocations(AMContainerEventStopRequest.class); eventHandler.reset(); @@ -1609,14 +1609,14 @@ eventHandler, rmClient, new ContainerContextMatcher(), taskScheduler.onContainersAllocated(Collections.singletonList(container1)); TestTaskSchedulerHelpers.waitForDelayedDrainNotify(drainNotifier); drainableAppCallback.drain(); - verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta11), any(Object.class), eq(container1)); + verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta11), any(), eq(container1)); // finish ta11, should start ta13 taskSchedulerManager.handleEvent(new AMSchedulerEventTAEnded(ta11, container1.getId(), TaskAttemptState.SUCCEEDED, null, null, 0)); drainableAppCallback.drain(); verifyDeAllocateTask(taskScheduler, ta11, true, null, null); - verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta13), any(Object.class), eq(container1)); + verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta13), any(), eq(container1)); verify(rmClient, times(0)).releaseAssignedContainer(eq(container1.getId())); eventHandler.verifyNoInvocations(AMContainerEventStopRequest.class); eventHandler.reset(); @@ -1634,7 +1634,7 @@ eventHandler, rmClient, new ContainerContextMatcher(), taskScheduler.onContainersAllocated(Collections.singletonList(container2)); TestTaskSchedulerHelpers.waitForDelayedDrainNotify(drainNotifier); drainableAppCallback.drain(); - verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta12), any(Object.class), eq(container2)); + verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta12), any(), eq(container2)); // ta12 finished, cannot reuse container, should release container2 taskSchedulerManager.handleEvent(new AMSchedulerEventTAEnded(ta12, container2.getId(), @@ -1649,7 +1649,7 @@ eventHandler, rmClient, new ContainerContextMatcher(), taskScheduler.onContainersAllocated(Collections.singletonList(container3)); TestTaskSchedulerHelpers.waitForDelayedDrainNotify(drainNotifier); drainableAppCallback.drain(); - verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta14), any(Object.class), eq(container3)); + verify(taskSchedulerManager).taskAllocated(eq(0), eq(ta14), any(), eq(container3)); // ta14 finished, should release container3 taskSchedulerManager.handleEvent(new AMSchedulerEventTAEnded(ta14, container3.getId(), diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestDagAwareYarnTaskScheduler.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestDagAwareYarnTaskScheduler.java index 48dd93825b..1465bfaaf6 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestDagAwareYarnTaskScheduler.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestDagAwareYarnTaskScheduler.java @@ -77,11 +77,11 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.anyInt; -import static org.mockito.Matchers.anyListOf; -import static org.mockito.Matchers.eq; -import static org.mockito.Matchers.isNull; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.anyInt; +import static org.mockito.Mockito.anyList; +import static org.mockito.Mockito.eq; +import static org.mockito.Mockito.isNull; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.never; import static org.mockito.Mockito.spy; @@ -170,23 +170,23 @@ public void testNoReuse() throws Exception { racks, mockPriority, null, mockCookie1); drainableAppCallback.drain(); verify(mockRMClient, times(1)). - addContainerRequest(any(TaskRequest.class)); + addContainerRequest(any()); // returned from task requests before allocation happens assertFalse(scheduler.deallocateTask(mockTask1, true, null, null)); - verify(mockApp, times(0)).containerBeingReleased(any(ContainerId.class)); + verify(mockApp, times(0)).containerBeingReleased(any()); verify(mockRMClient, times(1)). - removeContainerRequest(any(TaskRequest.class)); + removeContainerRequest(any()); verify(mockRMClient, times(0)). - releaseAssignedContainer((ContainerId) any()); + releaseAssignedContainer(any()); // deallocating unknown task assertFalse(scheduler.deallocateTask(mockTask1, true, null, null)); - verify(mockApp, times(0)).containerBeingReleased(any(ContainerId.class)); + verify(mockApp, times(0)).containerBeingReleased(any()); verify(mockRMClient, times(1)). - removeContainerRequest(any(TaskRequest.class)); + removeContainerRequest(any()); verify(mockRMClient, times(0)). - releaseAssignedContainer((ContainerId) any()); + releaseAssignedContainer(any()); // allocate tasks Object mockTask2 = new MockTask("task2"); @@ -237,7 +237,7 @@ public void testNoReuse() throws Exception { verify(mockApp).taskAllocated(mockTask2, mockCookie2, mockContainer2); verify(mockApp).taskAllocated(mockTask3, mockCookie3, mockContainer3); // no other allocations returned - verify(mockApp, times(3)).taskAllocated(any(), any(), (Container) any()); + verify(mockApp, times(3)).taskAllocated(any(), any(), any()); verify(mockRMClient).removeContainerRequest(request1); verify(mockRMClient).removeContainerRequest(request2); verify(mockRMClient).removeContainerRequest(request3); @@ -253,7 +253,7 @@ public void testNoReuse() throws Exception { assertEquals(mockTask2, scheduler.deallocateContainer(mockCId2)); drainableAppCallback.drain(); verify(mockRMClient).releaseAssignedContainer(mockCId2); - verify(mockRMClient, times(3)).releaseAssignedContainer((ContainerId) any()); + verify(mockRMClient, times(3)).releaseAssignedContainer(any()); List statuses = new ArrayList<>(); ContainerStatus mockStatus1 = mock(ContainerStatus.class); @@ -277,16 +277,16 @@ public void testNoReuse() throws Exception { // currently allocated container status returned and not released verify(mockApp).containerCompleted(mockTask3, mockStatus3); // no other statuses returned - verify(mockApp, times(3)).containerCompleted(any(), (ContainerStatus) any()); - verify(mockRMClient, times(3)).releaseAssignedContainer((ContainerId) any()); + verify(mockApp, times(3)).containerCompleted(any(), any()); + verify(mockRMClient, times(3)).releaseAssignedContainer(any()); // verify blacklisting - verify(mockRMClient, times(0)).updateBlacklist(anyListOf(String.class), anyListOf(String.class)); + verify(mockRMClient, times(0)).updateBlacklist(anyList(), anyList()); String badHost = "host6"; NodeId badNodeId = NodeId.newInstance(badHost, 1); scheduler.blacklistNode(badNodeId); List badNodeList = Collections.singletonList(badHost); - verify(mockRMClient, times(1)).updateBlacklist(eq(badNodeList), isNull(List.class)); + verify(mockRMClient, times(1)).updateBlacklist(eq(badNodeList), isNull()); Object mockTask4 = new MockTask("task4"); Object mockCookie4 = new Object(); scheduler.allocateTask(mockTask4, mockCapability, null, @@ -300,10 +300,10 @@ public void testNoReuse() throws Exception { scheduler.onContainersAllocated(containers); drainableAppCallback.drain(); // no new allocation - verify(mockApp, times(3)).taskAllocated(any(), any(), (Container) any()); + verify(mockApp, times(3)).taskAllocated(any(), any(), any()); // verify blacklisted container released verify(mockRMClient).releaseAssignedContainer(mockCId5); - verify(mockRMClient, times(4)).releaseAssignedContainer((ContainerId) any()); + verify(mockRMClient, times(4)).releaseAssignedContainer(any()); // verify request added back verify(mockRMClient, times(6)).addContainerRequest(requestCaptor.capture()); NodeId host6 = NodeId.newInstance("host6", 6); @@ -314,17 +314,17 @@ public void testNoReuse() throws Exception { scheduler.onContainersAllocated(containers); drainableAppCallback.drain(); // new allocation - verify(mockApp, times(4)).taskAllocated(any(), any(), (Container) any()); + verify(mockApp, times(4)).taskAllocated(any(), any(), any()); verify(mockApp).taskAllocated(mockTask4, mockCookie4, mockContainer6); // deallocate allocated task assertTrue(scheduler.deallocateTask(mockTask4, true, null, null)); drainableAppCallback.drain(); verify(mockApp).containerBeingReleased(mockCId6); verify(mockRMClient).releaseAssignedContainer(mockCId6); - verify(mockRMClient, times(5)).releaseAssignedContainer((ContainerId) any()); + verify(mockRMClient, times(5)).releaseAssignedContainer(any()); // test unblacklist scheduler.unblacklistNode(badNodeId); - verify(mockRMClient, times(1)).updateBlacklist(isNull(List.class), eq(badNodeList)); + verify(mockRMClient, times(1)).updateBlacklist(isNull(), eq(badNodeList)); assertEquals(0, scheduler.getNumBlacklistedNodes()); float progress = 0.5f; @@ -335,16 +335,16 @@ public void testNoReuse() throws Exception { scheduler.allocateTask(mockTask1, mockCapability, hosts, racks, mockPriority, null, mockCookie1); drainableAppCallback.drain(); - verify(mockRMClient, times(7)).addContainerRequest(any(TaskRequest.class)); + verify(mockRMClient, times(7)).addContainerRequest(any()); verify(mockRMClient, times(6)). - removeContainerRequest(any(TaskRequest.class)); + removeContainerRequest(any()); scheduler.allocateTask(mockTask1, mockCapability, hosts, racks, mockPriority, null, mockCookie1); drainableAppCallback.drain(); // old request removed and new one added verify(mockRMClient, times(7)). - removeContainerRequest(any(TaskRequest.class)); - verify(mockRMClient, times(8)).addContainerRequest(any(TaskRequest.class)); + removeContainerRequest(any()); + verify(mockRMClient, times(8)).addContainerRequest(any()); assertFalse(scheduler.deallocateTask(mockTask1, true, null, null)); // test speculative node adjustment @@ -360,9 +360,9 @@ public void testNoReuse() throws Exception { mockPriority, null, mockCookie5); drainableAppCallback.drain(); // no new allocation - verify(mockApp, times(4)).taskAllocated(any(), any(), (Container) any()); + verify(mockApp, times(4)).taskAllocated(any(), any(), any()); // verify container released - verify(mockRMClient, times(5)).releaseAssignedContainer((ContainerId) any()); + verify(mockRMClient, times(5)).releaseAssignedContainer(any()); // verify request added back verify(mockRMClient, times(9)).addContainerRequest(requestCaptor.capture()); @@ -377,7 +377,7 @@ public void testNoReuse() throws Exception { drainableAppCallback.drain(); verify(mockApp) .reportError(eq(YarnTaskSchedulerServiceError.RESOURCEMANAGER_ERROR), argumentCaptor.capture(), - any(DagInfo.class)); + any()); assertTrue(argumentCaptor.getValue().contains("mockexception")); scheduler.onShutdownRequest(); @@ -468,8 +468,8 @@ public void testSimpleReuseLocalMatching() throws Exception { assertTrue(scheduler.deallocateTask(taskv0t0.task, true, null, null)); clock.incrementTime(10000); drainableAppCallback.drain(); - verify(mockApp, never()).containerBeingReleased(any(ContainerId.class)); - verify(mockRMClient, never()).releaseAssignedContainer(any(ContainerId.class)); + verify(mockApp, never()).containerBeingReleased(any()); + verify(mockRMClient, never()).releaseAssignedContainer(any()); verify(mockApp).taskAllocated(taskv0t2.task, taskv0t2.cookie, container1); verify(mockRMClient).removeContainerRequest(reqv0t2); @@ -477,8 +477,8 @@ public void testSimpleReuseLocalMatching() throws Exception { assertTrue(scheduler.deallocateTask(taskv0t2.task, true, null, null)); clock.incrementTime(10000); drainableAppCallback.drain(); - verify(mockApp, never()).containerBeingReleased(any(ContainerId.class)); - verify(mockRMClient, never()).releaseAssignedContainer(any(ContainerId.class)); + verify(mockApp, never()).containerBeingReleased(any()); + verify(mockRMClient, never()).releaseAssignedContainer(any()); verify(mockApp).taskAllocated(taskv1t0.task, taskv1t0.cookie, container1); verify(mockRMClient).removeContainerRequest(reqv1t0); @@ -575,8 +575,8 @@ public void testSimpleReuseRackMatching() throws Exception { assertTrue(scheduler.deallocateTask(taskv0t0.task, true, null, null)); clock.incrementTime(10000); drainableAppCallback.drain(); - verify(mockApp, never()).containerBeingReleased(any(ContainerId.class)); - verify(mockRMClient, never()).releaseAssignedContainer(any(ContainerId.class)); + verify(mockApp, never()).containerBeingReleased(any()); + verify(mockRMClient, never()).releaseAssignedContainer(any()); verify(mockApp).taskAllocated(taskv1t0.task, taskv1t0.cookie, container1); verify(mockRMClient).removeContainerRequest(reqv1t0); @@ -584,8 +584,8 @@ public void testSimpleReuseRackMatching() throws Exception { assertTrue(scheduler.deallocateTask(taskv1t0.task, true, null, null)); clock.incrementTime(10000); drainableAppCallback.drain(); - verify(mockApp, never()).containerBeingReleased(any(ContainerId.class)); - verify(mockRMClient, never()).releaseAssignedContainer(any(ContainerId.class)); + verify(mockApp, never()).containerBeingReleased(any()); + verify(mockRMClient, never()).releaseAssignedContainer(any()); verify(mockApp).taskAllocated(taskv0t1.task, taskv0t1.cookie, container1); verify(mockRMClient).removeContainerRequest(reqv0t1); @@ -682,8 +682,8 @@ public void testSimpleReuseAnyMatching() throws Exception { assertTrue(scheduler.deallocateTask(taskv0t0.task, true, null, null)); clock.incrementTime(10000); drainableAppCallback.drain(); - verify(mockApp, never()).containerBeingReleased(any(ContainerId.class)); - verify(mockRMClient, never()).releaseAssignedContainer(any(ContainerId.class)); + verify(mockApp, never()).containerBeingReleased(any()); + verify(mockRMClient, never()).releaseAssignedContainer(any()); verify(mockApp).taskAllocated(taskv1t0.task, taskv1t0.cookie, container1); verify(mockRMClient).removeContainerRequest(reqv1t0); @@ -691,8 +691,8 @@ public void testSimpleReuseAnyMatching() throws Exception { assertTrue(scheduler.deallocateTask(taskv1t0.task, true, null, null)); clock.incrementTime(10000); drainableAppCallback.drain(); - verify(mockApp, never()).containerBeingReleased(any(ContainerId.class)); - verify(mockRMClient, never()).releaseAssignedContainer(any(ContainerId.class)); + verify(mockApp, never()).containerBeingReleased(any()); + verify(mockRMClient, never()).releaseAssignedContainer(any()); verify(mockApp).taskAllocated(taskv0t1.task, taskv0t1.cookie, container1); verify(mockRMClient).removeContainerRequest(reqv0t1); @@ -700,8 +700,8 @@ public void testSimpleReuseAnyMatching() throws Exception { assertTrue(scheduler.deallocateTask(taskv0t1.task, true, null, null)); clock.incrementTime(10000); drainableAppCallback.drain(); - verify(mockApp, never()).containerBeingReleased(any(ContainerId.class)); - verify(mockRMClient, never()).releaseAssignedContainer(any(ContainerId.class)); + verify(mockApp, never()).containerBeingReleased(any()); + verify(mockRMClient, never()).releaseAssignedContainer(any()); verify(mockApp).taskAllocated(taskv0t2.task, taskv0t2.cookie, container1); verify(mockRMClient).removeContainerRequest(reqv0t2); @@ -795,8 +795,8 @@ public void testReuseWithAffinity() throws Exception { assertTrue(scheduler.deallocateTask(taskv0t0.task, true, null, null)); clock.incrementTime(10000); drainableAppCallback.drain(); - verify(mockApp, never()).containerBeingReleased(any(ContainerId.class)); - verify(mockRMClient, never()).releaseAssignedContainer(any(ContainerId.class)); + verify(mockApp, never()).containerBeingReleased(any()); + verify(mockRMClient, never()).releaseAssignedContainer(any()); verify(mockApp).taskAllocated(taskv1t0.task, taskv1t0.cookie, container1); verify(mockRMClient).removeContainerRequest(reqv1t0); @@ -895,8 +895,8 @@ public void testReuseVertexDescendants() throws Exception { assertTrue(scheduler.deallocateTask(taskv0t0.task, true, null, null)); clock.incrementTime(10000); drainableAppCallback.drain(); - verify(mockApp, never()).containerBeingReleased(any(ContainerId.class)); - verify(mockRMClient, never()).releaseAssignedContainer(any(ContainerId.class)); + verify(mockApp, never()).containerBeingReleased(any()); + verify(mockRMClient, never()).releaseAssignedContainer(any()); verify(mockApp).taskAllocated(taskv2t0.task, taskv2t0.cookie, container1); verify(mockRMClient).removeContainerRequest(reqv2t0); @@ -905,8 +905,8 @@ public void testReuseVertexDescendants() throws Exception { assertTrue(scheduler.deallocateTask(taskv2t0.task, true, null, null)); clock.incrementTime(10000); drainableAppCallback.drain(); - verify(mockApp, never()).containerBeingReleased(any(ContainerId.class)); - verify(mockRMClient, never()).releaseAssignedContainer(any(ContainerId.class)); + verify(mockApp, never()).containerBeingReleased(any()); + verify(mockRMClient, never()).releaseAssignedContainer(any()); verify(mockApp).taskAllocated(taskv0t1.task, taskv0t1.cookie, container1); verify(mockRMClient).removeContainerRequest(reqv0t1); @@ -914,8 +914,8 @@ public void testReuseVertexDescendants() throws Exception { assertTrue(scheduler.deallocateTask(taskv0t1.task, true, null, null)); clock.incrementTime(10000); drainableAppCallback.drain(); - verify(mockApp, never()).containerBeingReleased(any(ContainerId.class)); - verify(mockRMClient, never()).releaseAssignedContainer(any(ContainerId.class)); + verify(mockApp, never()).containerBeingReleased(any()); + verify(mockRMClient, never()).releaseAssignedContainer(any()); verify(mockApp).taskAllocated(taskv1t0.task, taskv1t0.cookie, container1); verify(mockRMClient).removeContainerRequest(reqv1t0); @@ -1070,15 +1070,15 @@ public void testSessionContainers() throws Exception { assertTrue(scheduler.deallocateTask(taskv0t4.task, true, null, null)); assertTrue(scheduler.deallocateTask(taskv0t5.task, true, null, null)); assertTrue(scheduler.deallocateTask(taskv0t6.task, true, null, null)); - verify(mockApp, never()).containerBeingReleased(any(ContainerId.class)); - verify(mockRMClient, never()).releaseAssignedContainer(any(ContainerId.class)); + verify(mockApp, never()).containerBeingReleased(any()); + verify(mockRMClient, never()).releaseAssignedContainer(any()); // verify only two of the containers were released after idle expiration // and the rest were spread across the nodes and racks clock.incrementTime(5000); drainableAppCallback.drain(); - verify(mockApp, times(2)).containerBeingReleased(any(ContainerId.class)); - verify(mockRMClient, times(2)).releaseAssignedContainer(any(ContainerId.class)); + verify(mockApp, times(2)).containerBeingReleased(any()); + verify(mockRMClient, times(2)).releaseAssignedContainer(any()); Set hosts = new HashSet<>(); Set racks = new HashSet<>(); for (HeldContainer hc : scheduler.getSessionContainers()) { @@ -1202,7 +1202,7 @@ public void testPreemptionNoHeadroom() throws Exception { scheduler.getProgress(); scheduler.getProgress(); drainableAppCallback.drain(); - verify(mockApp, times(1)).preemptContainer(any(ContainerId.class)); + verify(mockApp, times(1)).preemptContainer(any()); verify(mockApp).preemptContainer(cid2); assertEquals(taskv1t1.task, scheduler.deallocateContainer(cid2)); drainableAppCallback.drain(); @@ -1226,13 +1226,13 @@ public void testPreemptionNoHeadroom() throws Exception { scheduler.getProgress(); scheduler.getProgress(); drainableAppCallback.drain(); - verify(mockApp, times(1)).preemptContainer(any(ContainerId.class)); + verify(mockApp, times(1)).preemptContainer(any()); // adding request for v0 should trigger preemption on next heartbeat taskRequestCaptor.scheduleTask(taskv0t1); scheduler.getProgress(); drainableAppCallback.drain(); - verify(mockApp, times(2)).preemptContainer(any(ContainerId.class)); + verify(mockApp, times(2)).preemptContainer(any()); verify(mockApp).preemptContainer(cid1); assertEquals(taskv1t0.task, scheduler.deallocateContainer(cid1)); drainableAppCallback.drain(); @@ -1346,7 +1346,7 @@ public void testPreemptionWhenBlocked() throws Exception { scheduler.getProgress(); scheduler.getProgress(); drainableAppCallback.drain(); - verify(mockApp, times(1)).preemptContainer(any(ContainerId.class)); + verify(mockApp, times(1)).preemptContainer(any()); verify(mockApp).preemptContainer(cid1); String appMsg = "success"; AppFinalStatus finalStatus = 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 70e31f33e2..e193ee98f2 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 @@ -190,12 +190,12 @@ public void preemptDescendantsOnly() { Answer answer = new Answer() { @Override public Void answer(InvocationOnMock invocation) { - ContainerId containerId = invocation.getArgumentAt(0, ContainerId.class); + ContainerId containerId = invocation.getArgument(0, ContainerId.class); taskSchedulerService.deallocateContainer(containerId); return null; } }; - doAnswer(answer).when(mockContext).preemptContainer(any(ContainerId.class)); + doAnswer(answer).when(mockContext).preemptContainer(any()); taskSchedulerService.initialize(); taskSchedulerService.start(); 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 965b8d76a4..ff34084d37 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 @@ -23,11 +23,12 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.anyInt; -import static org.mockito.Matchers.anyString; -import static org.mockito.Matchers.eq; import static org.mockito.Mockito.RETURNS_DEEP_STUBS; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.anyInt; +import static org.mockito.Mockito.anyString; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.eq; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.times; @@ -70,7 +71,6 @@ import org.apache.tez.dag.app.rm.TestTaskSchedulerHelpers.TaskSchedulerWithDrainableContext; import org.apache.tez.dag.app.rm.YarnTaskSchedulerService.CookieContainerRequest; import org.apache.tez.dag.app.rm.YarnTaskSchedulerService.HeldContainer; -import org.apache.tez.serviceplugins.api.DagInfo; import org.apache.tez.serviceplugins.api.TaskSchedulerContext; import org.apache.tez.serviceplugins.api.TaskSchedulerContext.AppFinalStatus; import org.junit.After; @@ -79,7 +79,6 @@ import org.junit.BeforeClass; import org.junit.Test; import org.mockito.ArgumentCaptor; -import org.mockito.Mockito; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; @@ -174,7 +173,7 @@ public void testTaskSchedulerNoReuse() throws Exception { // returned from task requests before allocation happens assertFalse(scheduler.deallocateTask(mockTask1, true, null, null)); - verify(mockApp, times(0)).containerBeingReleased(any(ContainerId.class)); + verify(mockApp, times(0)).containerBeingReleased(any()); verify(mockRMClient, times(1)). removeContainerRequest((CookieContainerRequest) any()); verify(mockRMClient, times(0)). @@ -182,7 +181,7 @@ public void testTaskSchedulerNoReuse() throws Exception { // deallocating unknown task assertFalse(scheduler.deallocateTask(mockTask1, true, null, null)); - verify(mockApp, times(0)).containerBeingReleased(any(ContainerId.class)); + verify(mockApp, times(0)).containerBeingReleased(any()); verify(mockRMClient, times(1)). removeContainerRequest((CookieContainerRequest) any()); verify(mockRMClient, times(0)). @@ -378,7 +377,7 @@ public void testTaskSchedulerNoReuse() throws Exception { drainableAppCallback.drain(); verify(mockApp) .reportError(eq(YarnTaskSchedulerServiceError.RESOURCEMANAGER_ERROR), argumentCaptor.capture(), - any(DagInfo.class)); + any()); assertTrue(argumentCaptor.getValue().contains("mockexception")); scheduler.onShutdownRequest(); @@ -744,7 +743,7 @@ public void testTaskSchedulerWithReuse() throws Exception { scheduler.onError(mockException); drainableAppCallback.drain(); verify(mockApp).reportError(eq(YarnTaskSchedulerServiceError.RESOURCEMANAGER_ERROR), argumentCaptor.capture(), - any(DagInfo.class)); + any()); assertTrue(argumentCaptor.getValue().contains("mockexception")); scheduler.onShutdownRequest(); @@ -791,43 +790,43 @@ public void testTaskSchedulerDetermineMinHeldContainers() throws Exception { Resource r = Resource.newInstance(0, 0); ContainerId mockCId1 = ContainerId.newInstance(appId, 0); Container c1 = Container.newInstance(mockCId1, emptyHost, null, r, null, null); - HeldContainer hc1 = Mockito.spy(new HeldContainer(c1, 0, 0, null, containerSignatureMatcher)); + HeldContainer hc1 = spy(new HeldContainer(c1, 0, 0, null, containerSignatureMatcher)); when(hc1.getNode()).thenReturn(node1Rack1); when(hc1.getRack()).thenReturn(rack1); when(hc1.getContainer()).thenReturn(c1); ContainerId mockCId2 = ContainerId.newInstance(appId, 1); Container c2 = Container.newInstance(mockCId2, emptyHost, null, r, null, null); - HeldContainer hc2 = Mockito.spy(new HeldContainer(c2, 0, 0, null, containerSignatureMatcher)); + HeldContainer hc2 = spy(new HeldContainer(c2, 0, 0, null, containerSignatureMatcher)); when(hc2.getNode()).thenReturn(node2Rack1); when(hc2.getRack()).thenReturn(rack1); when(hc2.getContainer()).thenReturn(c2); ContainerId mockCId3 = ContainerId.newInstance(appId, 2); Container c3 = Container.newInstance(mockCId3, emptyHost, null, r, null, null); - HeldContainer hc3 = Mockito.spy(new HeldContainer(c3, 0, 0, null, containerSignatureMatcher)); + HeldContainer hc3 = spy(new HeldContainer(c3, 0, 0, null, containerSignatureMatcher)); when(hc3.getNode()).thenReturn(node1Rack1); when(hc3.getRack()).thenReturn(rack1); when(hc3.getContainer()).thenReturn(c3); ContainerId mockCId4 = ContainerId.newInstance(appId, 3); Container c4 = Container.newInstance(mockCId4, emptyHost, null, r, null, null); - HeldContainer hc4 = Mockito.spy(new HeldContainer(c4, 0, 0, null, containerSignatureMatcher)); + HeldContainer hc4 = spy(new HeldContainer(c4, 0, 0, null, containerSignatureMatcher)); when(hc4.getNode()).thenReturn(node2Rack1); when(hc4.getRack()).thenReturn(rack1); when(hc4.getContainer()).thenReturn(c4); ContainerId mockCId5 = ContainerId.newInstance(appId, 4); Container c5 = Container.newInstance(mockCId5, emptyHost, null, r, null, null); - HeldContainer hc5 = Mockito.spy(new HeldContainer(c5, 0, 0, null, containerSignatureMatcher)); + HeldContainer hc5 = spy(new HeldContainer(c5, 0, 0, null, containerSignatureMatcher)); when(hc5.getNode()).thenReturn(node1Rack2); when(hc5.getRack()).thenReturn(rack2); when(hc5.getContainer()).thenReturn(c5); ContainerId mockCId6 = ContainerId.newInstance(appId, 5); Container c6 = Container.newInstance(mockCId6, emptyHost, null, r, null, null); - HeldContainer hc6 = Mockito.spy(new HeldContainer(c6, 0, 0, null, containerSignatureMatcher)); + HeldContainer hc6 = spy(new HeldContainer(c6, 0, 0, null, containerSignatureMatcher)); when(hc6.getNode()).thenReturn(node2Rack2); when(hc6.getRack()).thenReturn(rack2); when(hc6.getContainer()).thenReturn(c6); ContainerId mockCId7 = ContainerId.newInstance(appId, 6); Container c7 = Container.newInstance(mockCId7, emptyHost, null, r, null, null); - HeldContainer hc7 = Mockito.spy(new HeldContainer(c7, 0, 0, null, containerSignatureMatcher)); + HeldContainer hc7 = spy(new HeldContainer(c7, 0, 0, null, containerSignatureMatcher)); when(hc7.getNode()).thenReturn(node1Rack3); when(hc7.getRack()).thenReturn(rack3); when(hc7.getContainer()).thenReturn(c7); @@ -1143,7 +1142,7 @@ public List> answer( }); - Mockito.doAnswer(new Answer() { + doAnswer(new Answer() { public Object answer(InvocationOnMock invocation) { Object[] args = invocation.getArguments(); ContainerId cId = (ContainerId) args[0]; @@ -1342,7 +1341,7 @@ public void testTaskSchedulerPreemption2() throws Exception { List containers = new ArrayList(); containers.add(mockContainer1); - Mockito.doAnswer(new Answer() { + doAnswer(new Answer() { public Object answer(InvocationOnMock invocation) { Object[] args = invocation.getArguments(); ContainerId cId = (ContainerId) args[0]; @@ -1603,7 +1602,7 @@ public void testContainerExpired() throws Exception { statuses.add(mockStatus2); scheduler.onContainersCompleted(statuses); - verify(mockApp, times(0)).taskAllocated(any(), any(), any(Container.class)); + verify(mockApp, times(0)).taskAllocated(any(), any(), any()); verify(mockRMClient, times(3)).addContainerRequest(requestCaptor.capture()); CookieContainerRequest resubmitRequest = requestCaptor.getValue(); assertEquals(request2.getCookie().getTask(), resubmitRequest.getCookie().getTask()); @@ -1616,7 +1615,7 @@ public void testContainerExpired() throws Exception { assertFalse(scheduler.deallocateTask(mockTask2, true, null, null)); scheduler.onContainersAllocated(containers); scheduler.onContainersCompleted(statuses); - verify(mockApp, times(0)).taskAllocated(any(), any(), any(Container.class)); + verify(mockApp, times(0)).taskAllocated(any(), any(), any()); verify(mockRMClient, times(3)).addContainerRequest(requestCaptor.capture()); } diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerManager.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerManager.java index 3ddd1861d4..095bd4e287 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerManager.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerManager.java @@ -23,10 +23,10 @@ import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.eq; import static org.mockito.Mockito.RETURNS_DEEP_STUBS; +import static org.mockito.Mockito.any; import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.eq; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.reset; import static org.mockito.Mockito.spy; @@ -661,7 +661,7 @@ mockTaskAttempt1, mock(TaskLocationHint.class), 1, mock(ContainerContext.class), tseh.handle(launchRequest1); verify(tseh.getTestTaskScheduler(0)).allocateTask(eq(mockTaskAttempt1), eq(resource), - any(String[].class), any(String[].class), any(Priority.class), any(Object.class), + any(String[].class), any(String[].class), any(), any(), eq(launchRequest1)); AMSchedulerEventTALaunchRequest launchRequest2 = @@ -670,7 +670,7 @@ mockTaskAttempt2, mock(TaskLocationHint.class), 1, mock(ContainerContext.class), 0); tseh.handle(launchRequest2); verify(tseh.getTestTaskScheduler(1)).allocateTask(eq(mockTaskAttempt2), eq(resource), - any(String[].class), any(String[].class), any(Priority.class), any(Object.class), + any(String[].class), any(String[].class), any(), any(), eq(launchRequest2)); } 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 d3614d9ff2..3061ceaa88 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 @@ -23,10 +23,10 @@ import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.eq; -import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.any; import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.eq; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.never; import static org.mockito.Mockito.reset; @@ -1311,7 +1311,7 @@ protected void mockDAGID() { */ @SuppressWarnings("unchecked") public void verifyNoOutgoingEvents() { - verify(eventHandler, never()).handle(any(Event.class)); + verify(eventHandler, never()).handle(any()); } /** diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/web/TestAMWebController.java b/tez-dag/src/test/java/org/apache/tez/dag/app/web/TestAMWebController.java index 94c8df6233..ced9ca0abe 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/web/TestAMWebController.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/web/TestAMWebController.java @@ -18,11 +18,11 @@ package org.apache.tez.dag.app.web; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.anyString; -import static org.mockito.Matchers.eq; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.anyString; import static org.mockito.Mockito.doNothing; import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.eq; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.reset; import static org.mockito.Mockito.spy; diff --git a/tez-dag/src/test/java/org/apache/tez/dag/history/recovery/TestRecoveryService.java b/tez-dag/src/test/java/org/apache/tez/dag/history/recovery/TestRecoveryService.java index 790e2d8059..5d1fff9121 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/history/recovery/TestRecoveryService.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/history/recovery/TestRecoveryService.java @@ -20,8 +20,8 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; -import static org.mockito.Matchers.anyInt; -import static org.mockito.Matchers.eq; +import static org.mockito.Mockito.anyInt; +import static org.mockito.Mockito.eq; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.times; diff --git a/tez-ext-service-tests/pom.xml b/tez-ext-service-tests/pom.xml index 8c2d84ee7c..609b896fd8 100644 --- a/tez-ext-service-tests/pom.xml +++ b/tez-ext-service-tests/pom.xml @@ -57,9 +57,13 @@ org.mockito - mockito-all + mockito-core test + + org.apache.tez + hadoop-shim + org.apache.tez tez-runtime-internals diff --git a/tez-mapreduce/pom.xml b/tez-mapreduce/pom.xml index ee7c318ff0..a22870fc38 100644 --- a/tez-mapreduce/pom.xml +++ b/tez-mapreduce/pom.xml @@ -39,6 +39,10 @@ test-jar test + + org.apache.tez + hadoop-shim + org.apache.tez tez-runtime-library @@ -91,7 +95,7 @@ org.mockito - mockito-all + mockito-core test diff --git a/tez-mapreduce/src/test/java/org/apache/hadoop/mapred/split/TestGroupedSplits.java b/tez-mapreduce/src/test/java/org/apache/hadoop/mapred/split/TestGroupedSplits.java index 3dce417459..9d700652a7 100644 --- a/tez-mapreduce/src/test/java/org/apache/hadoop/mapred/split/TestGroupedSplits.java +++ b/tez-mapreduce/src/test/java/org/apache/hadoop/mapred/split/TestGroupedSplits.java @@ -325,7 +325,7 @@ public void testGroupedSplitSize() throws IOException { for (int i=0; i org.mockito - mockito-all + mockito-core test diff --git a/tez-plugins/tez-aux-services/src/test/java/org/apache/tez/auxservices/TestShuffleHandler.java b/tez-plugins/tez-aux-services/src/test/java/org/apache/tez/auxservices/TestShuffleHandler.java index b91e0eb203..40b16857ad 100644 --- a/tez-plugins/tez-aux-services/src/test/java/org/apache/tez/auxservices/TestShuffleHandler.java +++ b/tez-plugins/tez-aux-services/src/test/java/org/apache/tez/auxservices/TestShuffleHandler.java @@ -26,6 +26,9 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.fail; import static org.junit.Assume.assumeTrue; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -97,7 +100,6 @@ import org.junit.Test; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; -import org.mockito.Mockito; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -1480,7 +1482,7 @@ public void testSendMapCount() throws Exception { final ChannelHandlerContext mockCtx = mock(ChannelHandlerContext.class); final Channel mockCh = mock(AbstractChannel.class); - final ChannelPipeline mockPipeline = Mockito.mock(ChannelPipeline.class); + final ChannelPipeline mockPipeline = mock(ChannelPipeline.class); // Mock HttpRequest and ChannelFuture final FullHttpRequest httpRequest = createHttpRequest(); @@ -1490,11 +1492,11 @@ public void testSendMapCount() throws Exception { new ShuffleHandler.TimeoutHandler(); // Mock Netty Channel Context and Channel behavior - Mockito.doReturn(mockCh).when(mockCtx).channel(); - Mockito.when(mockCh.pipeline()).thenReturn(mockPipeline); - Mockito.when(mockPipeline.get(Mockito.any(String.class))).thenReturn(timerHandler); + doReturn(mockCh).when(mockCtx).channel(); + when(mockCh.pipeline()).thenReturn(mockPipeline); + when(mockPipeline.get(any(String.class))).thenReturn(timerHandler); when(mockCtx.channel()).thenReturn(mockCh); - Mockito.doReturn(mockFuture).when(mockCh).writeAndFlush(Mockito.any(Object.class)); + doReturn(mockFuture).when(mockCh).writeAndFlush(any()); when(mockCh.writeAndFlush(Object.class)).thenReturn(mockFuture); final ShuffleHandler sh = new MockShuffleHandler(); @@ -1570,8 +1572,8 @@ public ChannelFuture createMockChannelFuture(Channel mockCh, final List listenerList) { final ChannelFuture mockFuture = mock(ChannelFuture.class); when(mockFuture.channel()).thenReturn(mockCh); - Mockito.doReturn(true).when(mockFuture).isSuccess(); - Mockito.doAnswer(new Answer() { + doReturn(true).when(mockFuture).isSuccess(); + doAnswer(new Answer() { @Override public Object answer(InvocationOnMock invocation) throws Throwable { //Add ReduceMapFileCount listener to a list @@ -1581,7 +1583,7 @@ public Object answer(InvocationOnMock invocation) throws Throwable { invocation.getArguments()[0]); return null; } - }).when(mockFuture).addListener(Mockito.any( + }).when(mockFuture).addListener(any( ShuffleHandler.ReduceMapFileCount.class)); return mockFuture; } diff --git a/tez-plugins/tez-history-parser/pom.xml b/tez-plugins/tez-history-parser/pom.xml index 5240bba1d8..f0758dcd15 100644 --- a/tez-plugins/tez-history-parser/pom.xml +++ b/tez-plugins/tez-history-parser/pom.xml @@ -132,7 +132,7 @@ org.mockito - mockito-all + mockito-core test diff --git a/tez-plugins/tez-protobuf-history-plugin/pom.xml b/tez-plugins/tez-protobuf-history-plugin/pom.xml index 9d82ed3965..607a31f95b 100644 --- a/tez-plugins/tez-protobuf-history-plugin/pom.xml +++ b/tez-plugins/tez-protobuf-history-plugin/pom.xml @@ -47,7 +47,7 @@ org.mockito - mockito-all + mockito-core test 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 50ed0bd77d..0b61b7d958 100644 --- a/tez-plugins/tez-yarn-timeline-history-with-acls/pom.xml +++ b/tez-plugins/tez-yarn-timeline-history-with-acls/pom.xml @@ -126,7 +126,7 @@ org.mockito - mockito-all + mockito-core test diff --git a/tez-plugins/tez-yarn-timeline-history-with-fs/pom.xml b/tez-plugins/tez-yarn-timeline-history-with-fs/pom.xml index 4ba760f1aa..75df73f3a6 100644 --- a/tez-plugins/tez-yarn-timeline-history-with-fs/pom.xml +++ b/tez-plugins/tez-yarn-timeline-history-with-fs/pom.xml @@ -36,6 +36,10 @@ org.apache.tez tez-dag + + org.apache.tez + hadoop-shim + org.apache.tez tez-yarn-timeline-history @@ -130,7 +134,7 @@ org.mockito - mockito-all + mockito-core test @@ -155,4 +159,3 @@ - diff --git a/tez-plugins/tez-yarn-timeline-history-with-fs/src/test/java/org/apache/tez/dag/history/logging/ats/TestATSV15HistoryLoggingService.java b/tez-plugins/tez-yarn-timeline-history-with-fs/src/test/java/org/apache/tez/dag/history/logging/ats/TestATSV15HistoryLoggingService.java index 65ce91aab2..7ba14fffef 100644 --- a/tez-plugins/tez-yarn-timeline-history-with-fs/src/test/java/org/apache/tez/dag/history/logging/ats/TestATSV15HistoryLoggingService.java +++ b/tez-plugins/tez-yarn-timeline-history-with-fs/src/test/java/org/apache/tez/dag/history/logging/ats/TestATSV15HistoryLoggingService.java @@ -24,10 +24,8 @@ import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; -import static org.mockito.Matchers.anyObject; -import static org.mockito.Matchers.anyString; -import static org.mockito.Matchers.anyVararg; import static org.mockito.Mockito.any; +import static org.mockito.Mockito.anyString; import static org.mockito.Mockito.eq; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.times; @@ -42,7 +40,6 @@ import java.util.Map; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.security.token.Token; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ContainerId; @@ -53,7 +50,6 @@ import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse; import org.apache.hadoop.yarn.client.api.TimelineClient; import org.apache.hadoop.yarn.exceptions.YarnException; -import org.apache.hadoop.yarn.security.client.TimelineDelegationTokenIdentifier; import org.apache.tez.common.security.DAGAccessControls; import org.apache.tez.common.security.HistoryACLPolicyManager; import org.apache.tez.dag.api.TezConfiguration; @@ -71,7 +67,6 @@ import org.apache.tez.dag.records.TezVertexID; import org.apache.tez.hadoop.shim.HadoopShim; import org.junit.Test; -import org.mockito.Matchers; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; @@ -448,17 +443,17 @@ private ATSV15HistoryLoggingService createService(int numDagsPerGroup) throws IO entityLog = new HashMap<>(); //timelineClient.init(conf); when(timelineClient.getDelegationToken(anyString())).thenReturn(null); - when(timelineClient.renewDelegationToken(Matchers.>any())).thenReturn(0L); - when(timelineClient.putEntities(Matchers.anyVararg())).thenAnswer(new Answer() { + when(timelineClient.renewDelegationToken(any())).thenReturn(0L); + when(timelineClient.putEntities(any())).thenAnswer(new Answer() { @Override public TimelinePutResponse answer(InvocationOnMock invocation) throws Throwable { return putEntityHelper(DEFAULT_GROUP_ID, invocation.getArguments(), 0); } }); - when(timelineClient.putEntities(any(ApplicationAttemptId.class), any(TimelineEntityGroupId.class), Matchers.anyVararg())).thenAnswer(new Answer() { + when(timelineClient.putEntities(any(), any(), any())).thenAnswer(new Answer() { @Override public TimelinePutResponse answer(InvocationOnMock invocation) throws Throwable { - return putEntityHelper(invocation.getArgumentAt(1, TimelineEntityGroupId.class), invocation.getArguments(), 2); + return putEntityHelper(invocation.getArgument(1, TimelineEntityGroupId.class), invocation.getArguments(), 2); } }); service.timelineClient = timelineClient; diff --git a/tez-plugins/tez-yarn-timeline-history/pom.xml b/tez-plugins/tez-yarn-timeline-history/pom.xml index 78b83dd877..3f17e97327 100644 --- a/tez-plugins/tez-yarn-timeline-history/pom.xml +++ b/tez-plugins/tez-yarn-timeline-history/pom.xml @@ -120,7 +120,7 @@ org.mockito - mockito-all + mockito-core test diff --git a/tez-plugins/tez-yarn-timeline-history/src/test/java/org/apache/tez/dag/history/logging/ats/TestATSHistoryLoggingService.java b/tez-plugins/tez-yarn-timeline-history/src/test/java/org/apache/tez/dag/history/logging/ats/TestATSHistoryLoggingService.java index 6603f4f668..0536568ab4 100644 --- a/tez-plugins/tez-yarn-timeline-history/src/test/java/org/apache/tez/dag/history/logging/ats/TestATSHistoryLoggingService.java +++ b/tez-plugins/tez-yarn-timeline-history/src/test/java/org/apache/tez/dag/history/logging/ats/TestATSHistoryLoggingService.java @@ -18,6 +18,8 @@ package org.apache.tez.dag.history.logging.ats; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.conf.Configuration; @@ -25,11 +27,9 @@ import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.NodeId; -import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity; import org.apache.hadoop.yarn.client.api.TimelineClient; import org.apache.hadoop.yarn.util.SystemClock; import org.apache.hadoop.yarn.conf.YarnConfiguration; -import org.apache.tez.common.security.DAGAccessControls; import org.apache.tez.common.security.HistoryACLPolicyManager; import org.apache.tez.dag.api.TezConfiguration; import org.apache.tez.dag.api.records.DAGProtos.DAGPlan; @@ -49,13 +49,9 @@ import org.junit.Assert; import org.junit.Before; import org.junit.Test; -import org.mockito.Matchers; -import org.mockito.invocation.InvocationOnMock; -import org.mockito.stubbing.Answer; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.eq; import static org.mockito.Mockito.any; +import static org.mockito.Mockito.eq; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; @@ -102,7 +98,7 @@ public void setup() throws Exception { when(appContext.getCurrentDAGID()).thenReturn(null); when(appContext.getApplicationID()).thenReturn(appId); when(atsHistoryLoggingService.timelineClient.putEntities( - Matchers.anyVararg())).thenAnswer( + any())).thenAnswer( new Answer() { @Override public Object answer(InvocationOnMock invocation) throws Throwable { @@ -184,8 +180,7 @@ public void testTimelineServiceDisable() throws Exception { atsHistoryLoggingService1.setAppContext(appContext); atsHistoryLoggingService1.timelineClient = mock(TimelineClient.class); - when(atsHistoryLoggingService1.timelineClient.putEntities( - Matchers.anyVararg())).thenAnswer( + when(atsHistoryLoggingService1.timelineClient.putEntities(any())).thenAnswer( new Answer() { @Override public Object answer(InvocationOnMock invocation) throws Throwable { @@ -227,12 +222,11 @@ public Object answer(InvocationOnMock invocation) throws Throwable { @Test(timeout=10000) public void testNonSessionDomains() throws Exception { - when(historyACLPolicyManager.setupSessionACLs((Configuration)any(), (ApplicationId)any())) - .thenReturn( - Collections.singletonMap(TezConfiguration.YARN_ATS_ACL_SESSION_DOMAIN_ID, "session-id")); + when(historyACLPolicyManager.setupSessionACLs(any(), any())) + .thenReturn( + Collections.singletonMap(TezConfiguration.YARN_ATS_ACL_SESSION_DOMAIN_ID, "session-id")); atsHistoryLoggingService.start(); - verify(historyACLPolicyManager, times(1)).setupSessionACLs( - (Configuration)any(), (ApplicationId)any()); + verify(historyACLPolicyManager, times(1)).setupSessionACLs(any(), any()); // Send the event and wait for completion. TezDAGID dagId1 = TezDAGID.getInstance(appId, 0); @@ -245,7 +239,7 @@ public void testNonSessionDomains() throws Exception { } // No dag domain were created. verify(historyACLPolicyManager, times(0)) - .setupSessionDAGACLs((Configuration)any(), eq(appId), eq("0"), (DAGAccessControls)any()); + .setupSessionDAGACLs(any(), eq(appId), eq("0"), any()); // All calls made with session domain id. verify(historyACLPolicyManager, times(6)).updateTimelineEntityDomain(any(), eq("session-id")); @@ -253,11 +247,10 @@ public void testNonSessionDomains() throws Exception { @Test(timeout=10000) public void testNonSessionDomainsFailed() throws Exception { - when(historyACLPolicyManager.setupSessionACLs((Configuration)any(), (ApplicationId)any())) - .thenThrow(new IOException()); + when(historyACLPolicyManager.setupSessionACLs(any(), any())) + .thenThrow(new IOException()); atsHistoryLoggingService.start(); - verify(historyACLPolicyManager, times(1)).setupSessionACLs( - (Configuration)any(), (ApplicationId)any()); + verify(historyACLPolicyManager, times(1)).setupSessionACLs(any(), any()); // Send the event and wait for completion. TezDAGID dagId1 = TezDAGID.getInstance(appId, 0); @@ -269,7 +262,7 @@ public void testNonSessionDomainsFailed() throws Exception { } // No dag domain were created. verify(historyACLPolicyManager, times(0)) - .setupSessionDAGACLs((Configuration)any(), eq(appId), eq("0"), (DAGAccessControls)any()); + .setupSessionDAGACLs(any(), eq(appId), eq("0"), any()); // All calls made with session domain id. verify(historyACLPolicyManager, times(0)).updateTimelineEntityDomain(any(), eq("session-id")); @@ -278,11 +271,10 @@ public void testNonSessionDomainsFailed() throws Exception { @Test(timeout=10000) public void testNonSessionDomainsAclNull() throws Exception { - when(historyACLPolicyManager.setupSessionACLs((Configuration)any(), (ApplicationId)any())) - .thenReturn(null); + when(historyACLPolicyManager.setupSessionACLs(any(), any())) + .thenReturn(null); atsHistoryLoggingService.start(); - verify(historyACLPolicyManager, times(1)).setupSessionACLs( - (Configuration)any(), (ApplicationId)any()); + verify(historyACLPolicyManager, times(1)).setupSessionACLs(any(), any()); // Send the event and wait for completion. TezDAGID dagId1 = TezDAGID.getInstance(appId, 0); @@ -295,7 +287,7 @@ public void testNonSessionDomainsAclNull() throws Exception { } // No dag domain were created. verify(historyACLPolicyManager, times(0)) - .setupSessionDAGACLs((Configuration)any(), eq(appId), eq("0"), (DAGAccessControls)any()); + .setupSessionDAGACLs(any(), eq(appId), eq("0"), any()); // All calls made with session domain id. verify(historyACLPolicyManager, times(0)).updateTimelineEntityDomain(any(), eq("session-id")); @@ -304,19 +296,15 @@ public void testNonSessionDomainsAclNull() throws Exception { @Test(timeout=10000) public void testSessionDomains() throws Exception { - when(historyACLPolicyManager.setupSessionACLs((Configuration)any(), (ApplicationId)any())) - .thenReturn( - Collections.singletonMap(TezConfiguration.YARN_ATS_ACL_SESSION_DOMAIN_ID, "test-domain")); + when(historyACLPolicyManager.setupSessionACLs(any(), any())) + .thenReturn(Collections.singletonMap(TezConfiguration.YARN_ATS_ACL_SESSION_DOMAIN_ID, "test-domain")); - when(historyACLPolicyManager.setupSessionDAGACLs( - (Configuration)any(), (ApplicationId)any(), eq("0"), (DAGAccessControls)any())) - .thenReturn( - Collections.singletonMap(TezConfiguration.YARN_ATS_ACL_DAG_DOMAIN_ID, "dag-domain")); + when(historyACLPolicyManager.setupSessionDAGACLs(any(), any(), eq("0"), any())) + .thenReturn(Collections.singletonMap(TezConfiguration.YARN_ATS_ACL_DAG_DOMAIN_ID, "dag-domain")); when(appContext.isSession()).thenReturn(true); atsHistoryLoggingService.start(); - verify(historyACLPolicyManager, times(1)).setupSessionACLs((Configuration)any(), - (ApplicationId)any()); + verify(historyACLPolicyManager, times(1)).setupSessionACLs(any(), any()); // Send the event and wait for completion. TezDAGID dagId1 = TezDAGID.getInstance(appId, 0); @@ -329,7 +317,7 @@ public void testSessionDomains() throws Exception { } // No dag domain were created. verify(historyACLPolicyManager, times(1)) - .setupSessionDAGACLs((Configuration)any(), eq(appId), eq("0"), (DAGAccessControls)any()); + .setupSessionDAGACLs(any(), eq(appId), eq("0"), any()); // All calls made with session domain id. verify(historyACLPolicyManager, times(1)).updateTimelineEntityDomain(any(), eq("test-domain")); @@ -338,18 +326,15 @@ public void testSessionDomains() throws Exception { @Test(timeout=10000) public void testSessionDomainsFailed() throws Exception { - when(historyACLPolicyManager.setupSessionACLs((Configuration)any(), (ApplicationId)any())) - .thenThrow(new IOException()); + when(historyACLPolicyManager.setupSessionACLs(any(), any())) + .thenThrow(new IOException()); - when(historyACLPolicyManager.setupSessionDAGACLs( - (Configuration)any(), (ApplicationId)any(), eq("0"), (DAGAccessControls)any())) - .thenReturn( - Collections.singletonMap(TezConfiguration.YARN_ATS_ACL_DAG_DOMAIN_ID, "dag-domain")); + when(historyACLPolicyManager.setupSessionDAGACLs(any(), any(), eq("0"), any())) + .thenReturn(Collections.singletonMap(TezConfiguration.YARN_ATS_ACL_DAG_DOMAIN_ID, "dag-domain")); when(appContext.isSession()).thenReturn(true); atsHistoryLoggingService.start(); - verify(historyACLPolicyManager, times(1)).setupSessionACLs((Configuration)any(), - (ApplicationId)any()); + verify(historyACLPolicyManager, times(1)).setupSessionACLs(any(), any()); // Send the event and wait for completion. TezDAGID dagId1 = TezDAGID.getInstance(appId, 0); @@ -361,27 +346,24 @@ public void testSessionDomainsFailed() throws Exception { } // No dag domain were created. verify(historyACLPolicyManager, times(0)) - .setupSessionDAGACLs((Configuration)any(), eq(appId), eq("0"), (DAGAccessControls)any()); + .setupSessionDAGACLs(any(), eq(appId), eq("0"), any()); // No calls were made for domains. - verify(historyACLPolicyManager, times(0)).updateTimelineEntityDomain(any(), (String)any()); + verify(historyACLPolicyManager, times(0)).updateTimelineEntityDomain(any(), any()); Assert.assertEquals(0, atsEntitiesCounter); } @Test(timeout=10000) public void testSessionDomainsDagFailed() throws Exception { - when(historyACLPolicyManager.setupSessionACLs((Configuration)any(), (ApplicationId)any())) - .thenReturn(Collections.singletonMap( - TezConfiguration.YARN_ATS_ACL_SESSION_DOMAIN_ID, "session-domain")); + when(historyACLPolicyManager.setupSessionACLs(any(), any())) + .thenReturn(Collections.singletonMap(TezConfiguration.YARN_ATS_ACL_SESSION_DOMAIN_ID, "session-domain")); - when(historyACLPolicyManager.setupSessionDAGACLs( - (Configuration)any(), (ApplicationId)any(), eq("0"), (DAGAccessControls)any())) - .thenThrow(new IOException()); + when(historyACLPolicyManager.setupSessionDAGACLs(any(), any(), eq("0"), any())) + .thenThrow(new IOException()); when(appContext.isSession()).thenReturn(true); atsHistoryLoggingService.start(); - verify(historyACLPolicyManager, times(1)).setupSessionACLs((Configuration)any(), - (ApplicationId)any()); + verify(historyACLPolicyManager, times(1)).setupSessionACLs(any(), any()); // Send the event and wait for completion. TezDAGID dagId1 = TezDAGID.getInstance(appId, 0); @@ -394,29 +376,27 @@ public void testSessionDomainsDagFailed() throws Exception { } // DAG domain was called once. verify(historyACLPolicyManager, times(1)) - .setupSessionDAGACLs((Configuration)any(), eq(appId), eq("0"), (DAGAccessControls)any()); + .setupSessionDAGACLs(any(), eq(appId), eq("0"), any()); // All calls made with session domain id. verify(historyACLPolicyManager, times(1)) .updateTimelineEntityDomain(any(), eq("session-domain")); verify(historyACLPolicyManager, times(1)) - .updateTimelineEntityDomain(any(), (String)any()); + .updateTimelineEntityDomain(any(), any()); Assert.assertEquals(1, atsEntitiesCounter); } @Test(timeout=10000) public void testSessionDomainsAclNull() throws Exception { - when(historyACLPolicyManager.setupSessionACLs((Configuration)any(), (ApplicationId)any())) - .thenReturn(null); + when(historyACLPolicyManager.setupSessionACLs(any(), any())) + .thenReturn(null); - when(historyACLPolicyManager.setupSessionDAGACLs( - (Configuration)any(), (ApplicationId)any(), eq("0"), (DAGAccessControls)any())) - .thenReturn(null); + when(historyACLPolicyManager.setupSessionDAGACLs(any(), any(), eq("0"), any())) + .thenReturn(null); when(appContext.isSession()).thenReturn(true); atsHistoryLoggingService.start(); - verify(historyACLPolicyManager, times(1)).setupSessionACLs((Configuration)any(), - (ApplicationId)any()); + verify(historyACLPolicyManager, times(1)).setupSessionACLs(any(), any()); // Send the event and wait for completion. TezDAGID dagId1 = TezDAGID.getInstance(appId, 0); @@ -429,10 +409,10 @@ public void testSessionDomainsAclNull() throws Exception { } // No dag domain were created. verify(historyACLPolicyManager, times(1)) - .setupSessionDAGACLs((Configuration)any(), eq(appId), eq("0"), (DAGAccessControls)any()); + .setupSessionDAGACLs(any(), eq(appId), eq("0"), any()); // All calls made with session domain id. - verify(historyACLPolicyManager, times(0)).updateTimelineEntityDomain(any(), (String)any()); + verify(historyACLPolicyManager, times(0)).updateTimelineEntityDomain(any(), any()); Assert.assertEquals(6, atsEntitiesCounter); } diff --git a/tez-runtime-internals/pom.xml b/tez-runtime-internals/pom.xml index 65841cf666..0619ca1e16 100644 --- a/tez-runtime-internals/pom.xml +++ b/tez-runtime-internals/pom.xml @@ -25,6 +25,10 @@ tez-runtime-internals + + commons-lang + commons-lang + org.apache.tez tez-api @@ -71,7 +75,7 @@ org.mockito - mockito-all + mockito-core test diff --git a/tez-runtime-internals/src/test/java/org/apache/tez/runtime/TestLogicalIOProcessorRuntimeTask.java b/tez-runtime-internals/src/test/java/org/apache/tez/runtime/TestLogicalIOProcessorRuntimeTask.java index ba9a66d0f1..1524cac357 100644 --- a/tez-runtime-internals/src/test/java/org/apache/tez/runtime/TestLogicalIOProcessorRuntimeTask.java +++ b/tez-runtime-internals/src/test/java/org/apache/tez/runtime/TestLogicalIOProcessorRuntimeTask.java @@ -22,7 +22,11 @@ import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +import static org.mockito.Mockito.anyList; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.only; +import static org.mockito.Mockito.verify; import java.io.IOException; import java.nio.ByteBuffer; @@ -75,7 +79,6 @@ import com.google.common.collect.Lists; import com.google.common.collect.Multimap; -import org.mockito.Mockito; public class TestLogicalIOProcessorRuntimeTask { @@ -190,7 +193,7 @@ public void testEventsCantBeSentInCleanup() throws Exception { runner.call(); // We verify that no events were sent - Mockito.verify(umbilical, Mockito.only()).addEvents(Collections. emptyList()); + verify(umbilical, only()).addEvents(Collections. emptyList()); } /** @@ -229,7 +232,7 @@ public void testExceptionHappensInClose() throws Exception { fail("RuntimeException should have been thrown"); } catch (RuntimeException e) { // No events should be sent thorught the umbilical protocol - Mockito.verify(umbilical, Mockito.never()).addEvents(Mockito.anyList()); + verify(umbilical, never()).addEvents(anyList()); } } finally { sharedExecutor.shutdownNow(); diff --git a/tez-runtime-internals/src/test/java/org/apache/tez/runtime/task/TestTaskReporter.java b/tez-runtime-internals/src/test/java/org/apache/tez/runtime/task/TestTaskReporter.java index 9f514f405b..147d17655b 100644 --- a/tez-runtime-internals/src/test/java/org/apache/tez/runtime/task/TestTaskReporter.java +++ b/tez-runtime-internals/src/test/java/org/apache/tez/runtime/task/TestTaskReporter.java @@ -18,7 +18,7 @@ package org.apache.tez.runtime.task; -import static org.mockito.Matchers.any; +import static org.mockito.Mockito.any; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.isA; @@ -84,7 +84,7 @@ public Object answer(InvocationOnMock invocation) throws Throwable { throw new TezUncheckedException("Invalid request id for test: " + request.getRequestId()); } } - }).when(mockUmbilical).heartbeat(any(TezHeartbeatRequest.class)); + }).when(mockUmbilical).heartbeat(any()); TezTaskAttemptID mockTaskAttemptId = mock(TezTaskAttemptID.class); LogicalIOProcessorRuntimeTask mockTask = mock(LogicalIOProcessorRuntimeTask.class); @@ -105,10 +105,10 @@ public Object answer(InvocationOnMock invocation) throws Throwable { lock.wait(); } } - verify(mockUmbilical, times(3)).heartbeat(any(TezHeartbeatRequest.class)); + verify(mockUmbilical, times(3)).heartbeat(any()); Thread.sleep(200l); // Sleep for less than the callable sleep time. No more invocations. - verify(mockUmbilical, times(3)).heartbeat(any(TezHeartbeatRequest.class)); + verify(mockUmbilical, times(3)).heartbeat(any()); } finally { executor.shutdownNow(); } diff --git a/tez-runtime-library/pom.xml b/tez-runtime-library/pom.xml index e43f90f415..c913e87d43 100644 --- a/tez-runtime-library/pom.xml +++ b/tez-runtime-library/pom.xml @@ -102,7 +102,7 @@ org.mockito - mockito-all + mockito-core test diff --git a/tez-runtime-library/src/test/java/org/apache/tez/dag/library/vertexmanager/TestFairShuffleVertexManager.java b/tez-runtime-library/src/test/java/org/apache/tez/dag/library/vertexmanager/TestFairShuffleVertexManager.java index de857bc06f..5108b8f9e6 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/dag/library/vertexmanager/TestFairShuffleVertexManager.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/dag/library/vertexmanager/TestFairShuffleVertexManager.java @@ -28,7 +28,6 @@ import org.apache.tez.dag.api.InputDescriptor; import org.apache.tez.dag.api.OutputDescriptor; import org.apache.tez.dag.api.TezUncheckedException; -import org.apache.tez.dag.api.VertexLocationHint; import org.apache.tez.dag.api.VertexManagerPluginContext; import org.apache.tez.dag.api.event.VertexState; import org.apache.tez.dag.api.event.VertexStateUpdate; @@ -311,7 +310,7 @@ private void testSchedulingWithPartitionStats( doAnswer(new reconfigVertexAnswer(mockContext, mockManagedVertexId, newEdgeManagers)).when(mockContext).reconfigureVertex( - anyInt(), any(VertexLocationHint.class), anyMap()); + anyInt(), any(), anyMap()); // check initialization manager = createFairShuffleVertexManager(conf, mockContext, diff --git a/tez-runtime-library/src/test/java/org/apache/tez/dag/library/vertexmanager/TestInputReadyVertexManager.java b/tez-runtime-library/src/test/java/org/apache/tez/dag/library/vertexmanager/TestInputReadyVertexManager.java index d59439e735..2eaaba4871 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/dag/library/vertexmanager/TestInputReadyVertexManager.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/dag/library/vertexmanager/TestInputReadyVertexManager.java @@ -30,7 +30,6 @@ import org.apache.tez.dag.api.InputDescriptor; import org.apache.tez.dag.api.OutputDescriptor; import org.apache.tez.dag.api.TezUncheckedException; -import org.apache.tez.dag.api.VertexLocationHint; import org.apache.tez.dag.api.VertexManagerPluginContext; import org.apache.tez.dag.api.EdgeProperty.SchedulingType; import org.apache.tez.dag.api.VertexManagerPluginContext.ScheduleTaskRequest; @@ -275,8 +274,7 @@ public void testComplex() throws Exception { } catch (TezUncheckedException e) { e.getMessage().contains("1-1 source vertices must have identical concurrency"); } - verify(mockContext, times(1)).reconfigureVertex(anyInt(), (VertexLocationHint) any(), - anyMap()); // not invoked + verify(mockContext, times(1)).reconfigureVertex(anyInt(), any(), any()); // not invoked when(mockContext.getVertexNumTasks(mockSrcVertexId3)).thenReturn(3); @@ -288,8 +286,7 @@ public void testComplex() throws Exception { manager.onVertexStateUpdated(new VertexStateUpdate(mockSrcVertexId1, VertexState.CONFIGURED)); manager.onVertexStateUpdated(new VertexStateUpdate(mockSrcVertexId2, VertexState.CONFIGURED)); manager.onVertexStateUpdated(new VertexStateUpdate(mockSrcVertexId3, VertexState.CONFIGURED)); - verify(mockContext, times(1)).reconfigureVertex(anyInt(), (VertexLocationHint) any(), - anyMap()); // not invoked + verify(mockContext, times(1)).reconfigureVertex(anyInt(), any(), any()); // not invoked verify(mockContext, times(2)).doneReconfiguringVertex(); manager.onVertexStarted(initialCompletions); // all 1-1 0's done but not scheduled because v1 is not done diff --git a/tez-runtime-library/src/test/java/org/apache/tez/dag/library/vertexmanager/TestShuffleVertexManager.java b/tez-runtime-library/src/test/java/org/apache/tez/dag/library/vertexmanager/TestShuffleVertexManager.java index b824d0b908..fcbcb42ac4 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/dag/library/vertexmanager/TestShuffleVertexManager.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/dag/library/vertexmanager/TestShuffleVertexManager.java @@ -36,12 +36,12 @@ import java.util.List; import java.util.Map; -import static org.mockito.Matchers.eq; import static org.mockito.Mockito.any; import static org.mockito.Mockito.anyInt; import static org.mockito.Mockito.anyList; import static org.mockito.Mockito.anyMap; import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.eq; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; @@ -142,8 +142,8 @@ public void testLargeDataSize() throws IOException { manager.onVertexManagerEventReceived(vmEvent); manager.onSourceTaskCompleted(createTaskAttemptIdentifier(mockSrcVertexId2, 1)); // Auto-reduce is triggered - verify(mockContext, times(1)).reconfigureVertex(anyInt(), any(VertexLocationHint.class), anyMap()); - verify(mockContext, times(1)).reconfigureVertex(eq(2), any(VertexLocationHint.class), anyMap()); + verify(mockContext, times(1)).reconfigureVertex(anyInt(), any(), anyMap()); + verify(mockContext, times(1)).reconfigureVertex(eq(2), any(), anyMap()); Assert.assertEquals(2, newEdgeManagers.size()); Assert.assertEquals(0, manager.pendingTasks.size()); // all tasks scheduled Assert.assertEquals(2, scheduledTasks.size()); diff --git a/tez-runtime-library/src/test/java/org/apache/tez/dag/library/vertexmanager/TestShuffleVertexManagerBase.java b/tez-runtime-library/src/test/java/org/apache/tez/dag/library/vertexmanager/TestShuffleVertexManagerBase.java index 9c3a5b34d8..75866248c3 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/dag/library/vertexmanager/TestShuffleVertexManagerBase.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/dag/library/vertexmanager/TestShuffleVertexManagerBase.java @@ -27,7 +27,6 @@ import org.apache.tez.dag.api.InputDescriptor; import org.apache.tez.dag.api.OutputDescriptor; import org.apache.tez.dag.api.TezUncheckedException; -import org.apache.tez.dag.api.VertexLocationHint; import org.apache.tez.dag.api.VertexManagerPluginContext; import org.apache.tez.dag.api.event.VertexState; import org.apache.tez.dag.api.event.VertexStateUpdate; @@ -114,8 +113,7 @@ public void testZeroSourceTasksWithVertexStartedFirst() { manager.onVertexStateUpdated(new VertexStateUpdate(mockSrcVertexId2, VertexState.CONFIGURED)); manager.onVertexStateUpdated(new VertexStateUpdate(mockSrcVertexId3, VertexState.CONFIGURED)); Assert.assertTrue(manager.pendingTasks.isEmpty()); - verify(mockContext, times(1)).reconfigureVertex(eq(1), any - (VertexLocationHint.class), anyMap()); + verify(mockContext, times(1)).reconfigureVertex(eq(1), any(), anyMap()); verify(mockContext, times(1)).doneReconfiguringVertex(); // reconfig done Assert.assertTrue(scheduledTasks.size() == 1); // all tasks scheduled and parallelism changed scheduledTasks.clear(); @@ -153,8 +151,7 @@ public void testZeroSourceTasksWithVertexStateUpdatedFirst() { // trigger start and processing of pending notification events manager.onVertexStarted(emptyCompletions); Assert.assertTrue(manager.bipartiteSources == 2); - verify(mockContext, times(1)).reconfigureVertex(eq(1), any - (VertexLocationHint.class), anyMap()); + verify(mockContext, times(1)).reconfigureVertex(eq(1), any(), anyMap()); verify(mockContext, times(1)).doneReconfiguringVertex(); // reconfig done Assert.assertTrue(manager.pendingTasks.isEmpty()); Assert.assertTrue(scheduledTasks.size() == 1); // all tasks scheduled and parallelism changed @@ -322,8 +319,8 @@ public void testTez978() throws IOException { vmEvent = getVertexManagerEvent(null, 160 * MB, mockSrcVertexId2); manager.onVertexManagerEventReceived(vmEvent); Assert.assertTrue(manager.determineParallelismAndApply(0.25f)); //ensure parallelism is determined - verify(mockContext, times(1)).reconfigureVertex(anyInt(), any(VertexLocationHint.class), anyMap()); - verify(mockContext, times(1)).reconfigureVertex(eq(2), any(VertexLocationHint.class), anyMap()); + verify(mockContext, times(1)).reconfigureVertex(anyInt(), any(), anyMap()); + verify(mockContext, times(1)).reconfigureVertex(eq(2), any(), anyMap()); manager.onSourceTaskCompleted(createTaskAttemptIdentifier(mockSrcVertexId2, 0)); Assert.assertEquals(0, manager.pendingTasks.size()); Assert.assertEquals(2, scheduledTasks.size()); @@ -340,7 +337,7 @@ public void testTez978() throws IOException { //min/max fraction of 0.0/0.2 manager = createManager(conf, mockContext, 0.0f, 0.2f); // initial invocation count == 3 - verify(mockContext, times(1)).reconfigureVertex(anyInt(), any(VertexLocationHint.class), anyMap()); + verify(mockContext, times(1)).reconfigureVertex(anyInt(), any(), anyMap()); manager.onVertexStarted(emptyCompletions); manager.onVertexStateUpdated(new VertexStateUpdate(mockSrcVertexId1, VertexState.CONFIGURED)); manager.onVertexStateUpdated(new VertexStateUpdate(mockSrcVertexId2, VertexState.CONFIGURED)); @@ -354,18 +351,18 @@ public void testTez978() throws IOException { manager.onVertexManagerEventReceived(getVertexManagerEvent(null, 10 * MB, mockSrcVertexId1)); manager.onSourceTaskCompleted(createTaskAttemptIdentifier(mockSrcVertexId1, i)); //should not change parallelism - verify(mockContext, times(1)).reconfigureVertex(anyInt(), any(VertexLocationHint.class), anyMap()); + verify(mockContext, times(1)).reconfigureVertex(anyInt(), any(), anyMap()); } for(int i=0;i<3;i++) { manager.onSourceTaskCompleted(createTaskAttemptIdentifier(mockSrcVertexId2, i)); - verify(mockContext, times(1)).reconfigureVertex(anyInt(), any(VertexLocationHint.class), anyMap()); + verify(mockContext, times(1)).reconfigureVertex(anyInt(), any(), anyMap()); } //Since max threshold (40 * 0.2 = 8) is met, vertex manager should determine parallelism manager.onSourceTaskCompleted(createTaskAttemptIdentifier(mockSrcVertexId2, 8)); // parallelism updated - verify(mockContext, times(2)).reconfigureVertex(anyInt(), any(VertexLocationHint.class), anyMap()); + verify(mockContext, times(2)).reconfigureVertex(anyInt(), any(), anyMap()); // check exact update value - 8 events with 100 each => 20 -> 2000 => 2 tasks (with 1000 per task) - verify(mockContext, times(2)).reconfigureVertex(eq(2), any(VertexLocationHint.class), anyMap()); + verify(mockContext, times(2)).reconfigureVertex(eq(2), any(), anyMap()); } @Test(timeout = 5000) @@ -418,8 +415,8 @@ public void testAutoParallelism() throws Exception { manager.onSourceTaskCompleted(createTaskAttemptIdentifier(mockSrcVertexId2, 1)); // managedVertex tasks reduced - verify(mockContext, times(1)).reconfigureVertex(anyInt(), any(VertexLocationHint.class), anyMap()); - verify(mockContext, times(1)).reconfigureVertex(eq(2), any(VertexLocationHint.class), anyMap()); + verify(mockContext, times(1)).reconfigureVertex(anyInt(), any(), anyMap()); + verify(mockContext, times(1)).reconfigureVertex(eq(2), any(), anyMap()); Assert.assertEquals(2, newEdgeManagers.size()); // TODO improve tests for parallelism Assert.assertEquals(0, manager.pendingTasks.size()); // all tasks scheduled @@ -432,7 +429,7 @@ public void testAutoParallelism() throws Exception { // more completions dont cause recalculation of parallelism manager.onSourceTaskCompleted(createTaskAttemptIdentifier(mockSrcVertexId2, 0)); - verify(mockContext, times(1)).reconfigureVertex(anyInt(), any(VertexLocationHint.class), anyMap()); + verify(mockContext, times(1)).reconfigureVertex(anyInt(), any(), anyMap()); Assert.assertEquals(2, newEdgeManagers.size()); EdgeManagerPluginOnDemand edgeManager = @@ -493,7 +490,7 @@ public void testShuffleVertexManagerSlowStart() { String mockManagedVertexId = "Vertex4"; VertexManagerPluginContext mockContext = mock(VertexManagerPluginContext.class); - when(mockContext.getVertexStatistics(any(String.class))).thenReturn(mock(VertexStatistics.class)); + when(mockContext.getVertexStatistics(any())).thenReturn(mock(VertexStatistics.class)); when(mockContext.getInputVertexEdgeProperties()).thenReturn(mockInputVertices); when(mockContext.getVertexName()).thenReturn(mockManagedVertexId); when(mockContext.getVertexNumTasks(mockManagedVertexId)).thenReturn(3); @@ -877,8 +874,7 @@ public void test_Tez1649_with_scatter_gather_edges() throws IOException { Assert.assertTrue(manager.totalNumBipartiteSourceTasks == 6); //Ensure that setVertexParallelism is not called for R2. - verify(mockContext_R2, times(0)).reconfigureVertex(anyInt(), any(VertexLocationHint.class), - anyMap()); + verify(mockContext_R2, times(0)).reconfigureVertex(anyInt(), any(), anyMap()); //ShuffleVertexManager's updatePendingTasks relies on getVertexNumTasks. Setting this for test when(mockContext_R2.getVertexNumTasks(mockManagedVertexId_R2)).thenReturn(1); @@ -886,9 +882,8 @@ public void test_Tez1649_with_scatter_gather_edges() throws IOException { // complete configuration of r1 triggers the scheduling manager.onVertexStateUpdated(new VertexStateUpdate(r1, VertexState.CONFIGURED)); Assert.assertTrue(manager.totalNumBipartiteSourceTasks == 9); - verify(mockContext_R2, times(1)).reconfigureVertex(eq(1), any(VertexLocationHint.class), - anyMap()); - + verify(mockContext_R2, times(1)).reconfigureVertex(eq(1), any(), anyMap()); + Assert.assertTrue(manager.pendingTasks.size() == 0); // all tasks scheduled Assert.assertTrue(scheduledTasks.size() == 1); diff --git a/tez-runtime-library/src/test/java/org/apache/tez/dag/library/vertexmanager/TestShuffleVertexManagerUtils.java b/tez-runtime-library/src/test/java/org/apache/tez/dag/library/vertexmanager/TestShuffleVertexManagerUtils.java index 92812224cd..44adc462bc 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/dag/library/vertexmanager/TestShuffleVertexManagerUtils.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/dag/library/vertexmanager/TestShuffleVertexManagerUtils.java @@ -32,7 +32,6 @@ import org.apache.tez.dag.api.InputDescriptor; import org.apache.tez.dag.api.OutputDescriptor; import org.apache.tez.dag.api.UserPayload; -import org.apache.tez.dag.api.VertexLocationHint; import org.apache.tez.dag.api.VertexManagerPluginContext; import org.apache.tez.dag.library.vertexmanager.FairShuffleVertexManager.FairRoutingType; import org.apache.tez.dag.library.vertexmanager.FairShuffleVertexManager.FairShuffleVertexManagerConfigBuilder; @@ -56,10 +55,10 @@ import java.util.List; import java.util.Map; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.anyInt; -import static org.mockito.Matchers.anyList; -import static org.mockito.Matchers.anyMap; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.anyInt; +import static org.mockito.Mockito.anyList; +import static org.mockito.Mockito.anyMap; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -113,7 +112,7 @@ VertexManagerPluginContext createVertexManagerContext( mockContext).scheduleTasks(anyList()); doAnswer(new reconfigVertexAnswer(mockContext, mockManagedVertexId, newEdgeManagers)).when(mockContext).reconfigureVertex( - anyInt(), any(VertexLocationHint.class), anyMap()); + anyInt(), any(), anyMap()); return mockContext; } diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/cartesianproduct/TestCartesianProductVertexManagerPartitioned.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/cartesianproduct/TestCartesianProductVertexManagerPartitioned.java index 1012a36fb4..a77e2c712e 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/cartesianproduct/TestCartesianProductVertexManagerPartitioned.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/cartesianproduct/TestCartesianProductVertexManagerPartitioned.java @@ -21,7 +21,6 @@ import org.apache.tez.dag.api.EdgeProperty; import org.apache.tez.dag.api.TezReflectionException; import org.apache.tez.dag.api.UserPayload; -import org.apache.tez.dag.api.VertexLocationHint; import org.apache.tez.dag.api.VertexManagerPluginContext; import org.apache.tez.dag.api.VertexManagerPluginContext.ScheduleTaskRequest; import org.apache.tez.dag.api.event.VertexState; @@ -37,7 +36,6 @@ import org.junit.Test; import org.mockito.ArgumentCaptor; import org.mockito.Captor; -import org.mockito.Matchers; import org.mockito.MockitoAnnotations; import java.util.ArrayList; @@ -48,8 +46,9 @@ import static org.apache.tez.dag.api.EdgeProperty.DataMovementType.BROADCAST; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; -import static org.mockito.Matchers.eq; -import static org.mockito.Matchers.isNull; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.eq; +import static org.mockito.Mockito.isNull; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.never; import static org.mockito.Mockito.times; @@ -75,7 +74,7 @@ public void setup() throws TezReflectionException { private void setupWithConfig(CartesianProductConfigProto config) throws TezReflectionException { - MockitoAnnotations.initMocks(this); + MockitoAnnotations.openMocks(this); context = mock(VertexManagerPluginContext.class); when(context.getVertexName()).thenReturn("cp"); when(context.getVertexNumTasks("cp")).thenReturn(-1); @@ -110,7 +109,7 @@ private void testReconfigureVertexHelper(CartesianProductConfigProto config, vertexManager.onVertexStateUpdated(new VertexStateUpdate("v0", VertexState.CONFIGURED)); verify(context, times(1)).reconfigureVertex(parallelismCaptor.capture(), - isNull(VertexLocationHint.class), edgePropertiesCaptor.capture()); + isNull(), edgePropertiesCaptor.capture()); assertEquals((int)parallelismCaptor.getValue(), parallelism); assertNull(edgePropertiesCaptor.getValue()); } @@ -134,12 +133,12 @@ public void testScheduling() throws Exception { vertexManager.onSourceTaskCompleted(allCompletions.get(0)); vertexManager.onSourceTaskCompleted(allCompletions.get(1)); - verify(context, never()).scheduleTasks(Matchers.>any()); + verify(context, never()).scheduleTasks(any()); List scheduleTaskRequests; vertexManager.onSourceTaskCompleted(allCompletions.get(2)); // shouldn't start schedule because broadcast src is not in RUNNING state - verify(context, never()).scheduleTasks(Matchers.>any()); + verify(context, never()).scheduleTasks(any()); vertexManager.onVertexStateUpdated(new VertexStateUpdate("v2", VertexState.RUNNING)); verify(context, times(1)).scheduleTasks(scheduleTaskRequestCaptor.capture()); @@ -161,7 +160,7 @@ public void testScheduling() throws Exception { for (int i = 6; i < 8; i++) { vertexManager.onSourceTaskCompleted(allCompletions.get(i)); - verify(context, times(4)).scheduleTasks(Matchers.>any()); + verify(context, times(4)).scheduleTasks(any()); } } @@ -191,7 +190,7 @@ private void testOnVertexStartHelper(boolean broadcastRunning) throws Exception vertexManager.onVertexStarted(completions); if (!broadcastRunning) { - verify(context, never()).scheduleTasks(Matchers.>any()); + verify(context, never()).scheduleTasks(any()); vertexManager.onVertexStateUpdated(new VertexStateUpdate("v2", VertexState.RUNNING)); } diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/cartesianproduct/TestFairCartesianProductVertexManager.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/cartesianproduct/TestFairCartesianProductVertexManager.java index 6219a158d0..54d771ad08 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/cartesianproduct/TestFairCartesianProductVertexManager.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/cartesianproduct/TestFairCartesianProductVertexManager.java @@ -22,7 +22,6 @@ import com.google.protobuf.InvalidProtocolBufferException; import org.apache.tez.dag.api.EdgeManagerPluginDescriptor; import org.apache.tez.dag.api.EdgeProperty; -import org.apache.tez.dag.api.VertexLocationHint; import org.apache.tez.dag.api.VertexManagerPluginContext; import org.apache.tez.dag.api.VertexManagerPluginContext.ScheduleTaskRequest; import org.apache.tez.dag.api.event.VertexState; @@ -52,10 +51,10 @@ import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.anyInt; -import static org.mockito.Matchers.anyMapOf; -import static org.mockito.Matchers.eq; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.anyInt; +import static org.mockito.Mockito.anyMap; +import static org.mockito.Mockito.eq; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.never; import static org.mockito.Mockito.times; @@ -72,7 +71,7 @@ public class TestFairCartesianProductVertexManager { @Before public void setup() { - MockitoAnnotations.initMocks(this); + MockitoAnnotations.openMocks(this); ctx = mock(VertexManagerPluginContext.class); vertexManager = new FairCartesianProductVertexManager(ctx); } @@ -224,11 +223,11 @@ public void testDAGVertexOnlyGroupByMaxParallelism() throws Exception { vertexManager.onVertexManagerEventReceived(getVMEvent(250, "v0", 0)); verify(ctx, never()).reconfigureVertex( - anyInt(), any(VertexLocationHint.class), anyMapOf(String.class, EdgeProperty.class)); + anyInt(), any(), anyMap()); vertexManager.onVertexManagerEventReceived(getVMEvent(200, "v1", 0)); verify(ctx, times(1)).reconfigureVertex( - eq(30), any(VertexLocationHint.class), edgePropertiesCaptor.capture()); + eq(30), any(), edgePropertiesCaptor.capture()); Map edgeProperties = edgePropertiesCaptor.getValue(); verifyEdgeProperties(edgeProperties.get("v0"), new String[]{"v0", "v1"}, new int[]{5, 6}, 30); verifyVertexGroupInfo(edgeProperties.get("v0"), 0); @@ -259,7 +258,7 @@ public void testDAGVertexOnlyGroupByMinOpsPerWorker() throws Exception { } verify(ctx, times(1)).reconfigureVertex( - eq(12), any(VertexLocationHint.class), edgePropertiesCaptor.capture()); + eq(12), any(), edgePropertiesCaptor.capture()); Map edgeProperties = edgePropertiesCaptor.getValue(); verifyEdgeProperties(edgeProperties.get("v0"), new String[]{"v0", "v1"}, new int[]{4, 3}, 100); verifyEdgeProperties(edgeProperties.get("v1"), new String[]{"v0", "v1"}, new int[]{4, 3}, 100); @@ -289,7 +288,7 @@ public void testDAGVertexGroup() throws Exception { vertexManager.onVertexManagerEventReceived(getVMEvent(5, "v2", 0)); vertexManager.onVertexManagerEventReceived(getVMEvent(5, "v2", 1)); verify(ctx, times(1)).reconfigureVertex( - eq(100), any(VertexLocationHint.class), edgePropertiesCaptor.capture()); + eq(100), any(), edgePropertiesCaptor.capture()); Map edgeProperties = edgePropertiesCaptor.getValue(); for (int i = 0; i < 3; i++) { verifyEdgeProperties(edgeProperties.get("v" + i), new String[]{"v0", "g0"}, @@ -323,7 +322,7 @@ public void testDAGVertexGroupOnly() throws Exception { vertexManager.onVertexManagerEventReceived(getVMEvent(16, "v3", 0)); verify(ctx, times(1)).reconfigureVertex( - eq(100), any(VertexLocationHint.class), edgePropertiesCaptor.capture()); + eq(100), any(), edgePropertiesCaptor.capture()); Map edgeProperties = edgePropertiesCaptor.getValue(); for (int i = 0; i < 4; i++) { verifyEdgeProperties(edgeProperties.get("v" + i), new String[]{"g0", "g1"}, @@ -352,7 +351,7 @@ public void testSchedulingVertexOnlyWithBroadcast() throws Exception { vertexManager.onVertexManagerEventReceived(getVMEvent(250, "v0", 0)); vertexManager.onVertexManagerEventReceived(getVMEvent(200, "v1", 0)); verify(ctx, times(1)).reconfigureVertex( - eq(30), any(VertexLocationHint.class), edgePropertiesCaptor.capture()); + eq(30), any(), edgePropertiesCaptor.capture()); assertFalse(edgePropertiesCaptor.getValue().containsKey("v2")); vertexManager.onVertexStarted(null); @@ -400,7 +399,7 @@ public void testZeroSrcTask() throws Exception { vertexManager.initialize(config); vertexManager.onVertexStateUpdated(new VertexStateUpdate("v0", VertexState.CONFIGURED)); vertexManager.onVertexStateUpdated(new VertexStateUpdate("v1", VertexState.CONFIGURED)); - vertexManager.onVertexStarted(new ArrayList()); + vertexManager.onVertexStarted(new ArrayList<>()); vertexManager.onSourceTaskCompleted(getTaId("v0", 0)); vertexManager.onSourceTaskCompleted(getTaId("v0", 1)); } @@ -430,11 +429,11 @@ public void testGroupingFraction() throws Exception { vertexManager.onSourceTaskCompleted(getTaId("v1", i)); } verify(ctx, never()).reconfigureVertex( - anyInt(), any(VertexLocationHint.class), anyMapOf(String.class, EdgeProperty.class)); + anyInt(), any(), anyMap()); vertexManager.onSourceTaskCompleted(getTaId("v1", 14)); verify(ctx, times(1)).reconfigureVertex( - eq(24), any(VertexLocationHint.class), edgePropertiesCaptor.capture()); + eq(24), any(), edgePropertiesCaptor.capture()); } @Test(timeout = 5000) @@ -448,7 +447,7 @@ public void testGroupFractionWithZeroStats() throws Exception { vertexManager.onSourceTaskCompleted(getTaId("v1", i)); } verify(ctx, never()).reconfigureVertex( - anyInt(), any(VertexLocationHint.class), anyMapOf(String.class, EdgeProperty.class)); + anyInt(), any(), anyMap()); } @Test(timeout = 5000) @@ -462,7 +461,7 @@ public void testGroupingFractionWithZeroOutput() throws Exception { vertexManager.onSourceTaskCompleted(getTaId("v1", i)); } verify(ctx, times(1)).reconfigureVertex( - eq(0), any(VertexLocationHint.class), edgePropertiesCaptor.capture()); + eq(0), any(), edgePropertiesCaptor.capture()); } @Test(timeout = 5000) @@ -476,7 +475,7 @@ public void testZeroSrcOutput() throws Exception { vertexManager.onVertexManagerEventReceived(getVMEvent(0, "v1", 1)); vertexManager.onVertexManagerEventReceived(getVMEvent(0, "v1", 2)); verify(ctx, times(1)).reconfigureVertex( - eq(0), any(VertexLocationHint.class), edgePropertiesCaptor.capture()); + eq(0), any(), edgePropertiesCaptor.capture()); } @Test(timeout = 5000) @@ -495,7 +494,7 @@ public void testDisableGrouping() throws Exception { vertexManager.onVertexManagerEventReceived(getVMEvent(250, "v0", 0)); vertexManager.onVertexManagerEventReceived(getVMEvent(200, "v1", 0)); verify(ctx, times(1)).reconfigureVertex( - eq(6), any(VertexLocationHint.class), edgePropertiesCaptor.capture()); + eq(6), any(), edgePropertiesCaptor.capture()); } @Test(timeout = 5000) @@ -511,7 +510,7 @@ public void testParallelismTwoSkewedSource() throws Exception { } verify(ctx, times(1)).reconfigureVertex( - eq(99), any(VertexLocationHint.class), edgePropertiesCaptor.capture()); + eq(99), any(), edgePropertiesCaptor.capture()); Map edgeProperties = edgePropertiesCaptor.getValue(); verifyEdgeProperties(edgeProperties.get("v0"), new String[]{"v0", "v1"}, new int[]{99, 1}, 100); @@ -539,9 +538,9 @@ public void testParallelismThreeSkewedSource() throws Exception { } verify(ctx, times(1)).reconfigureVertex( - eq(93), any(VertexLocationHint.class), edgePropertiesCaptor.capture()); + eq(93), any(), edgePropertiesCaptor.capture()); Map edgeProperties = edgePropertiesCaptor.getValue(); verifyEdgeProperties(edgeProperties.get("v0"), new String[]{"v0", "v1", "v2"}, new int[]{31, 3, 1}, 100); } -} \ No newline at end of file +} 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 338f640f91..efba8ea6a7 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 @@ -20,14 +20,15 @@ import org.apache.hadoop.fs.RawLocalFileSystem; import org.apache.tez.runtime.library.common.CompositeInputAttemptIdentifier; -import static org.mockito.Matchers.anyInt; -import static org.mockito.Matchers.anyLong; -import static org.mockito.Matchers.anyString; -import static org.mockito.Matchers.eq; + import static org.mockito.Mockito.any; +import static org.mockito.Mockito.anyInt; +import static org.mockito.Mockito.anyLong; +import static org.mockito.Mockito.anyString; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doNothing; import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.eq; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.never; import static org.mockito.Mockito.spy; @@ -193,7 +194,7 @@ public Path answer(InvocationOnMock invocation) throws Throwable { Object[] args = invocation.getArguments(); return new Path(SHUFFLE_INPUT_FILE_PREFIX + args[0]); } - }).when(fetcher).getShuffleInputFileName(anyString(), anyString()); + }).when(fetcher).getShuffleInputFileName(anyString(), any()); doAnswer(new Answer() { @Override diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/TestShuffleUtils.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/TestShuffleUtils.java index 446801ac2f..3363cb6627 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/TestShuffleUtils.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/TestShuffleUtils.java @@ -36,7 +36,7 @@ import org.junit.Assert; import org.junit.Before; import org.junit.Test; -import org.mockito.Matchers; +import org.mockito.ArgumentMatchers; import org.slf4j.Logger; import java.io.ByteArrayInputStream; @@ -257,28 +257,28 @@ public void testGenerateOnSpillEvent_With_All_EmptyPartitions() throws Exception outputContext, spillId, new TezSpillRecord(indexFile, conf), physicalOutputs, true, pathComponent, null, false, auxiliaryService, TezCommonUtils.newBestCompressionDeflater()); - Assert.assertTrue(events.size() == 2); //one for VM + Assert.assertEquals(2, events.size()); //one for VM Assert.assertTrue(events.get(0) instanceof VertexManagerEvent); Assert.assertTrue(events.get(1) instanceof CompositeDataMovementEvent); CompositeDataMovementEvent cdme = (CompositeDataMovementEvent) events.get(1); - Assert.assertTrue(cdme.getCount() == physicalOutputs); - Assert.assertTrue(cdme.getSourceIndexStart() == 0); + Assert.assertEquals(cdme.getCount(), physicalOutputs); + Assert.assertEquals(0, cdme.getSourceIndexStart()); ShuffleUserPayloads.DataMovementEventPayloadProto dmeProto = ShuffleUserPayloads.DataMovementEventPayloadProto.parseFrom(ByteString.copyFrom( cdme.getUserPayload())); //spill details should be present - Assert.assertTrue(dmeProto.getSpillId() == 0); + Assert.assertEquals(0, dmeProto.getSpillId()); Assert.assertTrue(dmeProto.hasLastEvent() && dmeProto.getLastEvent()); - Assert.assertTrue(dmeProto.getPathComponent().equals("")); + Assert.assertEquals("", dmeProto.getPathComponent()); byte[] emptyPartitions = TezCommonUtils.decompressByteStringToByteArray(dmeProto .getEmptyPartitions()); BitSet emptyPartitionsBitSet = TezUtilsInternal.fromByteArray(emptyPartitions); - Assert.assertTrue("emptyPartitionBitSet cardinality (expecting 10) = " + emptyPartitionsBitSet - .cardinality(), emptyPartitionsBitSet.cardinality() == 10); + Assert.assertEquals("emptyPartitionBitSet cardinality (expecting 10) = " + emptyPartitionsBitSet + .cardinality(), 10, emptyPartitionsBitSet.cardinality()); } @@ -289,10 +289,10 @@ public void testInternalErrorTranslation() throws Exception { when(mockCodecStream.read(any(byte[].class), anyInt(), anyInt())) .thenThrow(new InternalError(codecErrorMsg)); Decompressor mockDecoder = mock(Decompressor.class); - CompressionCodec mockCodec = mock(ConfigurableCodecForTest.class); - when(((ConfigurableCodecForTest) mockCodec).getConf()).thenReturn(mock(Configuration.class)); + ConfigurableCodecForTest mockCodec = mock(ConfigurableCodecForTest.class); + when(mockCodec.getConf()).thenReturn(mock(Configuration.class)); when(mockCodec.createDecompressor()).thenReturn(mockDecoder); - when(mockCodec.createInputStream(any(InputStream.class), any(Decompressor.class))) + when(mockCodec.createInputStream(any(), any())) .thenReturn(mockCodecStream); byte[] header = new byte[] { (byte) 'T', (byte) 'I', (byte) 'F', (byte) 1}; try { @@ -312,10 +312,10 @@ public void testExceptionTranslation() throws Exception { when(mockCodecStream.read(any(byte[].class), anyInt(), anyInt())) .thenThrow(new IllegalArgumentException(codecErrorMsg)); Decompressor mockDecoder = mock(Decompressor.class); - CompressionCodec mockCodec = mock(ConfigurableCodecForTest.class); - when(((ConfigurableCodecForTest) mockCodec).getConf()).thenReturn(mock(Configuration.class)); + ConfigurableCodecForTest mockCodec = mock(ConfigurableCodecForTest.class); + when(mockCodec.getConf()).thenReturn(mock(Configuration.class)); when(mockCodec.createDecompressor()).thenReturn(mockDecoder); - when(mockCodec.createInputStream(any(InputStream.class), any(Decompressor.class))) + when(mockCodec.createInputStream(any(), any())) .thenReturn(mockCodecStream); byte[] header = new byte[] { (byte) 'T', (byte) 'I', (byte) 'F', (byte) 1}; try { @@ -329,10 +329,10 @@ public void testExceptionTranslation() throws Exception { CompressionInputStream mockCodecStream1 = mock(CompressionInputStream.class); when(mockCodecStream1.read(any(byte[].class), anyInt(), anyInt())) .thenThrow(new SocketTimeoutException(codecErrorMsg)); - CompressionCodec mockCodec1 = mock(ConfigurableCodecForTest.class); - when(((ConfigurableCodecForTest) mockCodec1).getConf()).thenReturn(mock(Configuration.class)); + ConfigurableCodecForTest mockCodec1 = mock(ConfigurableCodecForTest.class); + when(mockCodec1.getConf()).thenReturn(mock(Configuration.class)); when(mockCodec1.createDecompressor()).thenReturn(mockDecoder); - when(mockCodec1.createInputStream(any(InputStream.class), any(Decompressor.class))) + when(mockCodec1.createInputStream(any(), any())) .thenReturn(mockCodecStream1); try { ShuffleUtils.shuffleToMemory(new byte[1024], new ByteArrayInputStream(header), @@ -345,10 +345,10 @@ public void testExceptionTranslation() throws Exception { CompressionInputStream mockCodecStream2 = mock(CompressionInputStream.class); when(mockCodecStream2.read(any(byte[].class), anyInt(), anyInt())) .thenThrow(new InternalError(codecErrorMsg)); - CompressionCodec mockCodec2 = mock(ConfigurableCodecForTest.class); - when(((ConfigurableCodecForTest) mockCodec2).getConf()).thenReturn(mock(Configuration.class)); + ConfigurableCodecForTest mockCodec2 = mock(ConfigurableCodecForTest.class); + when(mockCodec2.getConf()).thenReturn(mock(Configuration.class)); when(mockCodec2.createDecompressor()).thenReturn(mockDecoder); - when(mockCodec2.createInputStream(any(InputStream.class), any(Decompressor.class))) + when(mockCodec2.createInputStream(any(), any())) .thenReturn(mockCodecStream2); try { ShuffleUtils.shuffleToMemory(new byte[1024], new ByteArrayInputStream(header), @@ -394,14 +394,14 @@ public void testFetchStatsLogger() throws Exception { logger.logIndividualFetchComplete(10, 100, 1000, "testType", ident); } verify(activeLogger, times(0)).info(anyString()); - verify(aggregateLogger, times(1)).info(anyString(), Matchers.anyVararg()); + verify(aggregateLogger, times(1)).info(anyString(), ArgumentMatchers.any()); when(activeLogger.isInfoEnabled()).thenReturn(true); for (int i = 0; i < 1000; i++) { logger.logIndividualFetchComplete(10, 100, 1000, "testType", ident); } verify(activeLogger, times(1000)).info(anyString()); - verify(aggregateLogger, times(1)).info(anyString(), Matchers.anyVararg()); + verify(aggregateLogger, times(1)).info(anyString(), ArgumentMatchers.any()); } /** diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/impl/TestShuffleInputEventHandlerImpl.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/impl/TestShuffleInputEventHandlerImpl.java index 8a774dc1a5..9f6b4a3a43 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/impl/TestShuffleInputEventHandlerImpl.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/impl/TestShuffleInputEventHandlerImpl.java @@ -18,11 +18,11 @@ package org.apache.tez.runtime.library.common.shuffle.impl; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.anyInt; -import static org.mockito.Matchers.anyString; -import static org.mockito.Matchers.eq; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.anyInt; +import static org.mockito.Mockito.anyString; import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.eq; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.times; @@ -199,8 +199,8 @@ private InputContext createInputContext() throws IOException { @Override public ExecutorService answer(InvocationOnMock invocation) throws Throwable { return sharedExecutor.createExecutorService( - invocation.getArgumentAt(0, Integer.class), - invocation.getArgumentAt(1, String.class)); + invocation.getArgument(0, Integer.class), + invocation.getArgument(1, String.class)); } }); return inputContext; @@ -265,7 +265,7 @@ public void testPipelinedShuffleEvents() throws IOException { //0--> 1 with spill id 1 (attemptNum 1). This should report exception dme = createDataMovementEvent(true, 0, 1, 1, false, new BitSet(), 4, 1); handler.handleEvents(Collections.singletonList(dme)); - verify(inputContext).killSelf(any(Throwable.class), anyString()); + verify(inputContext).killSelf(any(), anyString()); } /** @@ -297,7 +297,7 @@ public void testPipelinedShuffleEvents_WithOutOfOrderAttempts() throws IOExcepti //Now send attemptNum 0. This should throw exception, because attempt #1 is already added dme = createDataMovementEvent(true, 0, 1, 0, false, new BitSet(), 4, 0); handler.handleEvents(Collections.singletonList(dme)); - verify(inputContext).killSelf(any(Throwable.class), anyString()); + verify(inputContext).killSelf(any(), anyString()); } /** @@ -338,7 +338,7 @@ public void testPipelinedShuffleEvents_WithEmptyPartitions() throws IOException //Now send attemptNum 1. This should throw exception, because attempt #1 is already added dme = createDataMovementEvent(true, 0, 1, 0, false, new BitSet(), 4, 1); handler.handleEvents(Collections.singletonList(dme)); - verify(inputContext).killSelf(any(Throwable.class), anyString()); + verify(inputContext).killSelf(any(), anyString()); } private Event createDataMovementEvent(boolean addSpillDetails, int srcIdx, int targetIdx, diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/impl/TestShuffleManager.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/impl/TestShuffleManager.java index fda2c896d1..ba854b9c14 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/impl/TestShuffleManager.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/impl/TestShuffleManager.java @@ -20,9 +20,9 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.anyInt; -import static org.mockito.Matchers.anyString; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.anyInt; +import static org.mockito.Mockito.anyString; import static org.mockito.Mockito.atLeast; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doReturn; @@ -179,8 +179,8 @@ private InputContext createInputContext() throws IOException { @Override public ExecutorService answer(InvocationOnMock invocation) throws Throwable { return sharedExecutor.createExecutorService( - invocation.getArgumentAt(0, Integer.class), - invocation.getArgumentAt(1, String.class)); + invocation.getArgument(0, Integer.class), + invocation.getArgument(1, String.class)); } }); return inputContext; 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 f283780c12..068c8f7b02 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 @@ -19,15 +19,15 @@ 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; -import static org.mockito.Matchers.anyLong; -import static org.mockito.Matchers.anyString; -import static org.mockito.Matchers.eq; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.anyBoolean; +import static org.mockito.Mockito.anyInt; +import static org.mockito.Mockito.anyLong; +import static org.mockito.Mockito.anyString; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doNothing; import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.eq; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.never; import static org.mockito.Mockito.spy; @@ -163,7 +163,7 @@ public void testLocalFetchModeSetting1() throws Exception { spyFetcher.fetchNext(); verify(spyFetcher, times(1)).setupLocalDiskFetch(mapHost); - verify(spyFetcher, never()).copyFromHost(any(MapHost.class)); + verify(spyFetcher, never()).copyFromHost(any()); // if hostname does not match use http mapHost = new MapHost(HOST + "_OTHER", PORT, 0, 1); @@ -177,7 +177,7 @@ public void testLocalFetchModeSetting1() throws Exception { spyFetcher.fetchNext(); - verify(spyFetcher, never()).setupLocalDiskFetch(any(MapHost.class)); + verify(spyFetcher, never()).setupLocalDiskFetch(any()); verify(spyFetcher, times(1)).copyFromHost(mapHost); // if port does not match use http @@ -192,7 +192,7 @@ public void testLocalFetchModeSetting1() throws Exception { spyFetcher.fetchNext(); - verify(spyFetcher, never()).setupLocalDiskFetch(any(MapHost.class)); + verify(spyFetcher, never()).setupLocalDiskFetch(any()); verify(spyFetcher, times(1)).copyFromHost(mapHost); //if local fetch is not enabled @@ -266,8 +266,8 @@ public MapOutput answer(InvocationOnMock invocation) throws Throwable { return mapOutput; } }).when(spyFetcher) - .getMapOutputForDirectDiskFetch(any(InputAttemptIdentifier.class), any(Path.class), - any(TezIndexRecord.class)); + .getMapOutputForDirectDiskFetch(any(), any(), + any()); doAnswer(new Answer() { @Override @@ -275,7 +275,7 @@ public Path answer(InvocationOnMock invocation) throws Throwable { Object[] args = invocation.getArguments(); return new Path(SHUFFLE_INPUT_FILE_PREFIX + args[0]); } - }).when(spyFetcher).getShuffleInputFileName(anyString(), anyString()); + }).when(spyFetcher).getShuffleInputFileName(any(), any()); for (int i = 0; i < host.getPartitionCount(); i++) { doAnswer(new Answer() { @@ -291,7 +291,7 @@ public TezIndexRecord answer(InvocationOnMock invocation) throws Throwable { // match with params for copySucceeded below. return new TezIndexRecord(p * 10, (p+1) * 1000, (p+2) * 100); } - }).when(spyFetcher).getIndexRecord(anyString(), eq(host.getPartitionId() + i)); + }).when(spyFetcher).getIndexRecord(any(), eq(host.getPartitionId() + i)); } doNothing().when(scheduler).copySucceeded(any(InputAttemptIdentifier.class), any(MapHost.class), @@ -369,7 +369,7 @@ public Path answer(InvocationOnMock invocation) throws Throwable { Object[] args = invocation.getArguments(); return new Path(SHUFFLE_INPUT_FILE_PREFIX + args[0]); } - }).when(spyFetcher).getShuffleInputFileName(anyString(), anyString()); + }).when(spyFetcher).getShuffleInputFileName(any(), anyString()); for (int i = 0; i < host.getPartitionCount(); i++) { doAnswer(new Answer() { @@ -459,7 +459,7 @@ public Path answer(InvocationOnMock invocation) throws Throwable { Object[] args = invocation.getArguments(); return new Path(SHUFFLE_INPUT_FILE_PREFIX + args[0]); } - }).when(spyFetcher).getShuffleInputFileName(anyString(), anyString()); + }).when(spyFetcher).getShuffleInputFileName(any(), any()); for (int i = 0; i < host.getPartitionCount(); i++) { doAnswer(new Answer() { @@ -477,7 +477,7 @@ public TezIndexRecord answer(InvocationOnMock invocation) throws Throwable { // match with params for copySucceeded below. return new TezIndexRecord(p * 10, (p + 1) * 1000, (p + 2) * 100); } - }).when(spyFetcher).getIndexRecord(anyString(), eq(host.getPartitionId() + i)); + }).when(spyFetcher).getIndexRecord(any(), eq(host.getPartitionId() + i)); } doNothing().when(scheduler).copySucceeded(any(InputAttemptIdentifier.class), any(MapHost.class), @@ -591,7 +591,7 @@ public void testWithRetry() throws Exception { new InputAttemptIdentifier(3, 4, InputAttemptIdentifier.PATH_PREFIX + "pathComponent_3") ); doReturn(srcAttempts).when(scheduler).getMapsForHost(host); - doReturn(true).when(fetcher).setupConnection(any(MapHost.class), any(Collection.class)); + doReturn(true).when(fetcher).setupConnection(any(), any()); URL url = ShuffleUtils.constructInputURL("http://" + HOST + ":" + PORT + "/mapOutput?job=job_123&&reduce=1&map=", srcAttempts, false); fetcher.httpConnection = new FakeHttpConnection(url, null, "", null); @@ -605,7 +605,7 @@ public MapOutput answer(InvocationOnMock invocation) throws Throwable { doReturn(args[0]).when(mapOutput).getAttemptIdentifier(); return mapOutput; } - }).when(merger).reserve(any(InputAttemptIdentifier.class), anyInt(), anyInt(), anyInt()); + }).when(merger).reserve(any(), anyInt(), anyInt(), anyInt()); //Create read timeout when reading data doAnswer(new Answer() { @@ -614,7 +614,7 @@ public MapOutput answer(InvocationOnMock invocation) throws Throwable { // Simulate read timeout by throwing proper exception throw new FetcherReadTimeoutException("creating fetcher socket read timeout exception"); } - }).when(fetcher).copyMapOutput(any(MapHost.class), any(DataInputStream.class), any(InputAttemptIdentifier.class)); + }).when(fetcher).copyMapOutput(any(), any(), any()); try { fetcher.copyFromHost(host); diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestShuffle.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestShuffle.java index 0a9c37e9ea..590affc078 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestShuffle.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestShuffle.java @@ -16,10 +16,10 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; -import static org.mockito.Matchers.anyInt; -import static org.mockito.Matchers.anyString; -import static org.mockito.Matchers.eq; +import static org.mockito.Mockito.anyInt; +import static org.mockito.Mockito.anyString; import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.eq; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; @@ -160,8 +160,8 @@ private InputContext createTezInputContext() throws IOException { @Override public ExecutorService answer(InvocationOnMock invocation) throws Throwable { return sharedExecutor.createExecutorService( - invocation.getArgumentAt(0, Integer.class), - invocation.getArgumentAt(1, String.class)); + invocation.getArgument(0, Integer.class), + invocation.getArgument(1, String.class)); } }); return inputContext; diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestShuffleInputEventHandlerOrderedGrouped.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestShuffleInputEventHandlerOrderedGrouped.java index cf4ff6a826..8da4adcf48 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestShuffleInputEventHandlerOrderedGrouped.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestShuffleInputEventHandlerOrderedGrouped.java @@ -37,11 +37,11 @@ import java.util.concurrent.ExecutorService; import static org.junit.Assert.assertTrue; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.anyInt; -import static org.mockito.Matchers.anyString; -import static org.mockito.Matchers.eq; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.anyInt; +import static org.mockito.Mockito.anyString; import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.eq; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.times; @@ -95,8 +95,8 @@ private InputContext createTezInputContext() throws IOException { @Override public ExecutorService answer(InvocationOnMock invocation) throws Throwable { return sharedExecutor.createExecutorService( - invocation.getArgumentAt(0, Integer.class), - invocation.getArgumentAt(1, String.class)); + invocation.getArgument(0, Integer.class), + invocation.getArgument(1, String.class)); } }); return inputContext; @@ -272,7 +272,7 @@ public void testPiplinedShuffleEvents_WithOutofOrderAttempts() throws IOExceptio handler.handleEvents(Collections.singletonList(dme2)); // task should issue kill request - verify(scheduler, times(1)).killSelf(any(IOException.class), any(String.class)); + verify(scheduler, times(1)).killSelf(any(), any()); } @Test (timeout = 5000) @@ -307,7 +307,7 @@ public void testPipelinedShuffle_WithObsoleteEvents() throws IOException, Interr handler.handleEvents(events); // task should issue kill request, as inputs are scheduled for download already. - verify(scheduler, times(1)).killSelf(any(IOException.class), any(String.class)); + verify(scheduler, times(1)).killSelf(any(), any()); } @Test(timeout = 5000) @@ -348,8 +348,8 @@ public void testAllPartitionsEmpty() throws IOException { events.add(dme); handler.handleEvents(events); InputAttemptIdentifier expectedIdentifier = new InputAttemptIdentifier(targetIdx, 0); - verify(scheduler).copySucceeded(eq(expectedIdentifier), any(MapHost.class), eq(0l), - eq(0l), eq(0l), any(MapOutput.class), eq(true)); + verify(scheduler).copySucceeded(eq(expectedIdentifier), any(), eq(0L), + eq(0L), eq(0L), any(), eq(true)); } @Test(timeout = 5000) @@ -362,8 +362,8 @@ public void testCurrentPartitionEmpty() throws IOException { events.add(dme); handler.handleEvents(events); InputAttemptIdentifier expectedIdentifier = new InputAttemptIdentifier(targetIdx, 0); - verify(scheduler).copySucceeded(eq(expectedIdentifier), any(MapHost.class), eq(0l), - eq(0l), eq(0l), any(MapOutput.class), eq(true)); + verify(scheduler).copySucceeded(eq(expectedIdentifier), any(), eq(0L), + eq(0L), eq(0L), any(), eq(true)); } @Test(timeout = 5000) diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestShuffleScheduler.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestShuffleScheduler.java index b89ffb0ce9..9df9aaf69b 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestShuffleScheduler.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestShuffleScheduler.java @@ -16,9 +16,9 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.anyInt; -import static org.mockito.Matchers.anyString; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.anyInt; +import static org.mockito.Mockito.anyString; import static org.mockito.Mockito.atLeast; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doReturn; @@ -267,10 +267,10 @@ public void _testReducerHealth_1(Configuration conf) throws IOException { if (minFailurePerHost <= 4) { //As per test threshold. Should fail & retrigger shuffle - verify(shuffle, atLeast(0)).reportException(any(Throwable.class)); + verify(shuffle, atLeast(0)).reportException(any()); } else if (minFailurePerHost > 100) { //host failure is so high that this would not retrigger shuffle re-execution - verify(shuffle, atLeast(1)).reportException(any(Throwable.class)); + verify(shuffle, atLeast(1)).reportException(any()); } } @@ -335,7 +335,7 @@ public void testReducerHealth_2() throws IOException, InterruptedException { } //Shuffle has not stalled. so no issues. - verify(scheduler.reporter, times(0)).reportException(any(Throwable.class)); + verify(scheduler.reporter, times(0)).reportException(any()); //stall shuffle scheduler.lastProgressTime = System.currentTimeMillis() - 250000; @@ -346,7 +346,7 @@ public void testReducerHealth_2() throws IOException, InterruptedException { new MapHost("host" + (190 % totalProducerNodes), 10000, 190, 1), false, true); //Even when it is stalled, need (320 - 300 = 20) * 3 = 60 failures - verify(scheduler.reporter, times(0)).reportException(any(Throwable.class)); + verify(scheduler.reporter, times(0)).reportException(any()); assertEquals(11, scheduler.failedShufflesSinceLastCompletion); @@ -370,7 +370,7 @@ public void testReducerHealth_2() throws IOException, InterruptedException { assertEquals(61, scheduler.failedShufflesSinceLastCompletion); assertEquals(10, scheduler.remainingMaps.get()); - verify(shuffle, atLeast(0)).reportException(any(Throwable.class)); + verify(shuffle, atLeast(0)).reportException(any()); //fail another 30 for (int i = 110; i < 120; i++) { @@ -388,7 +388,7 @@ public void testReducerHealth_2() throws IOException, InterruptedException { // Should fail now due to fetcherHealthy. (stall has already happened and // these are the only pending tasks) - verify(shuffle, atLeast(1)).reportException(any(Throwable.class)); + verify(shuffle, atLeast(1)).reportException(any()); } @@ -452,7 +452,7 @@ public void testReducerHealth_3() throws IOException { false, true); // failedShufflesSinceLastCompletion has crossed the limits. Throw error - verify(shuffle, times(0)).reportException(any(Throwable.class)); + verify(shuffle, times(0)).reportException(any()); } @Test(timeout = 60000) @@ -538,13 +538,13 @@ public void testReducerHealth_4() throws IOException { // failedShufflesSinceLastCompletion has crossed the limits. 20% of other nodes had failures as // well. However, it has failed only in one host. So this should proceed // until AM decides to restart the producer. - verify(shuffle, times(0)).reportException(any(Throwable.class)); + verify(shuffle, times(0)).reportException(any()); //stall the shuffle (but within limits) scheduler.lastProgressTime = System.currentTimeMillis() - 300000; scheduler.copyFailed(InputAttemptFetchFailure.fromAttempt(inputAttemptIdentifier), new MapHost("host" + (319 % totalProducerNodes), 10000, 319, 1), false, true); - verify(shuffle, times(1)).reportException(any(Throwable.class)); + verify(shuffle, times(1)).reportException(any()); } @@ -607,7 +607,7 @@ public void testReducerHealth_5() throws IOException { false, true); //Shuffle has not received the events completely. So do not bail out yet. - verify(shuffle, times(0)).reportException(any(Throwable.class)); + verify(shuffle, times(0)).reportException(any()); } @@ -675,7 +675,7 @@ public void _testReducerHealth_6(Configuration conf) throws IOException { assertEquals(scheduler.remainingMaps.get(), 310); //Do not bail out (number of failures is just 5) - verify(scheduler.reporter, times(0)).reportException(any(Throwable.class)); + verify(scheduler.reporter, times(0)).reportException(any()); //5 fetches fail repeatedly for (int i = 10; i < 15; i++) { @@ -696,10 +696,10 @@ public void _testReducerHealth_6(Configuration conf) throws IOException { // Now bail out, as Shuffle has crossed the // failedShufflesSinceLastCompletion limits. (even // though reducerHeathly is - verify(shuffle, atLeast(1)).reportException(any(Throwable.class)); + verify(shuffle, atLeast(1)).reportException(any()); } else { //Do not bail out yet. - verify(shuffle, atLeast(0)).reportException(any(Throwable.class)); + verify(shuffle, atLeast(0)).reportException(any()); } } @@ -755,7 +755,7 @@ public void testReducerHealth_7() throws IOException { false, true); } - verify(shuffle, atLeast(1)).reportException(any(Throwable.class)); + verify(shuffle, atLeast(1)).reportException(any()); } private ShuffleSchedulerForTest createScheduler(long startTime, int @@ -1016,8 +1016,8 @@ private InputContext createTezInputContext() throws IOException { @Override public ExecutorService answer(InvocationOnMock invocation) throws Throwable { return sharedExecutor.createExecutorService( - invocation.getArgumentAt(0, Integer.class), - invocation.getArgumentAt(1, String.class)); + invocation.getArgument(0, Integer.class), + invocation.getArgument(1, String.class)); } }); return inputContext; 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 bb0f22fabd..84ec143808 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 @@ -66,7 +66,7 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; -import static org.mockito.Matchers.any; +import static org.mockito.Mockito.any; import static org.mockito.Mockito.atLeastOnce; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; 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 754fbfa497..a56536dfe8 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 @@ -26,8 +26,8 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.anyLong; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.anyLong; import static org.mockito.Mockito.atLeastOnce; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doReturn; @@ -63,7 +63,6 @@ import org.apache.tez.dag.api.UserPayload; import org.apache.tez.runtime.api.Event; import org.apache.tez.runtime.api.ExecutionContext; -import org.apache.tez.runtime.api.MemoryUpdateCallback; import org.apache.tez.runtime.api.OutputContext; import org.apache.tez.runtime.api.OutputStatisticsReporter; import org.apache.tez.runtime.api.events.CompositeDataMovementEvent; @@ -703,7 +702,7 @@ private OutputContext createTezOutputContext() throws IOException { callback.memoryAssigned(requestedSize); return null; } - }).when(context).requestInitialMemory(anyLong(), any(MemoryUpdateCallback.class)); + }).when(context).requestInitialMemory(anyLong(), any()); return context; } -} \ No newline at end of file +} diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/writers/TestUnorderedPartitionedKVWriter.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/writers/TestUnorderedPartitionedKVWriter.java index c39bf3f707..dc7357c9cc 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/writers/TestUnorderedPartitionedKVWriter.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/writers/TestUnorderedPartitionedKVWriter.java @@ -21,13 +21,13 @@ 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.Matchers.anyListOf; -import static org.mockito.Matchers.eq; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.anyList; import static org.mockito.Mockito.atLeast; import static org.mockito.Mockito.atMost; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.eq; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.never; import static org.mockito.Mockito.times; @@ -55,13 +55,13 @@ import org.apache.hadoop.fs.CommonConfigurationKeys; import org.apache.hadoop.fs.permission.FsAction; import org.apache.tez.dag.api.TezConfiguration; -import org.apache.tez.runtime.api.TaskFailureType; import org.apache.tez.runtime.api.events.VertexManagerEvent; import org.apache.tez.runtime.library.common.Constants; import org.apache.tez.runtime.library.common.writers.UnorderedPartitionedKVWriter.SpillInfo; import org.apache.tez.runtime.library.shuffle.impl.ShuffleUserPayloads; import org.apache.tez.runtime.library.shuffle.impl.ShuffleUserPayloads.VertexManagerEventPayloadProto; import org.apache.tez.runtime.library.utils.DATA_RANGE_IN_MB; +import org.mockito.invocation.InvocationOnMock; import org.roaringbitmap.RoaringBitmap; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -107,7 +107,6 @@ import com.google.common.collect.Multimap; import com.google.common.collect.Sets; import org.mockito.ArgumentCaptor; -import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; @RunWith(value = Parameterized.class) @@ -418,7 +417,7 @@ public void textTest(int numRegularRecords, int numPartitions, long availableMem numRecordsWritten++; } if (pipeliningEnabled) { - verify(outputContext, times(numLargeKeys)).sendEvents(anyListOf(Event.class)); + verify(outputContext, times(numLargeKeys)).sendEvents(anyList()); } // Write Large val records @@ -434,7 +433,7 @@ public void textTest(int numRegularRecords, int numPartitions, long availableMem numRecordsWritten++; } if (pipeliningEnabled) { - verify(outputContext, times(numLargevalues + numLargeKeys)).sendEvents(anyListOf(Event.class)); + verify(outputContext, times(numLargevalues + numLargeKeys)).sendEvents(anyList()); } // Write records where key + val are large (but both can fit in the buffer individually) @@ -451,11 +450,11 @@ public void textTest(int numRegularRecords, int numPartitions, long availableMem } if (pipeliningEnabled) { verify(outputContext, times(numLargevalues + numLargeKeys + numLargeKvPairs)) - .sendEvents(anyListOf(Event.class)); + .sendEvents(anyList()); } List events = kvWriter.close(); - verify(outputContext, never()).reportFailure(any(TaskFailureType.class), any(Throwable.class), any(String.class)); + verify(outputContext, never()).reportFailure(any(), any(), any()); if (!pipeliningEnabled) { VertexManagerEvent vmEvent = null; @@ -732,8 +731,8 @@ private void baseTestWithPipelinedTransfer(int numRecords, int numPartitions, Se } verifyPartitionStats(VMEvent, partitionsWithData); - verify(outputContext, never()).reportFailure(any(TaskFailureType.class), - any(Throwable.class), any(String.class)); + verify(outputContext, never()).reportFailure(any(), + any(), any()); assertNull(kvWriter.currentBuffer); assertEquals(0, kvWriter.availableBuffers.size()); @@ -976,8 +975,8 @@ private void baseTestWithFinalMergeDisabled(int numRecords, int numPartitions, } } - verify(outputContext, never()).reportFailure(any(TaskFailureType.class), - any(Throwable.class), any(String.class)); + verify(outputContext, never()).reportFailure(any(), + any(), any()); assertNull(kvWriter.currentBuffer); assertEquals(0, kvWriter.availableBuffers.size()); @@ -1187,7 +1186,7 @@ private void baseTest(int numRecords, int numPartitions, Set skippedPar int recordsPerBuffer = sizePerBuffer / sizePerRecordWithOverhead; int numExpectedSpills = numRecordsWritten / recordsPerBuffer / kvWriter.spillLimit; - verify(outputContext, never()).reportFailure(any(TaskFailureType.class), any(Throwable.class), any(String.class)); + verify(outputContext, never()).reportFailure(any(), any(), any()); assertNull(kvWriter.currentBuffer); assertEquals(0, kvWriter.availableBuffers.size()); diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/input/TestOrderedGroupedKVInput.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/input/TestOrderedGroupedKVInput.java index 56b6805a63..c1cdf7cfa0 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/input/TestOrderedGroupedKVInput.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/input/TestOrderedGroupedKVInput.java @@ -15,7 +15,7 @@ package org.apache.tez.runtime.library.input; import static org.junit.Assert.assertEquals; -import static org.mockito.Matchers.any; +import static org.mockito.Mockito.any; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.doThrow; diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/OutputTestHelpers.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/OutputTestHelpers.java index a7c7ca28cd..573b3e9a18 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/OutputTestHelpers.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/OutputTestHelpers.java @@ -14,8 +14,8 @@ package org.apache.tez.runtime.library.output; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.anyLong; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.anyLong; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; @@ -28,7 +28,6 @@ import org.apache.tez.common.counters.TezCounters; import org.apache.tez.dag.api.TezConfiguration; import org.apache.tez.dag.api.UserPayload; -import org.apache.tez.runtime.api.MemoryUpdateCallback; import org.apache.tez.runtime.api.OutputContext; import org.apache.tez.runtime.api.OutputStatisticsReporter; import org.apache.tez.runtime.api.impl.ExecutionContextImpl; @@ -66,7 +65,7 @@ static OutputContext createOutputContext(Configuration conf, Configuration userP callback.memoryAssigned(requestedSize); return null; } - }).when(ctx).requestInitialMemory(anyLong(), any(MemoryUpdateCallback.class)); + }).when(ctx).requestInitialMemory(anyLong(), any()); doReturn(conf).when(ctx).getContainerConfiguration(); doReturn(TezUtils.createUserPayloadFromConf(userPayloadConf)).when(ctx).getUserPayload(); doReturn("taskVertex").when(ctx).getTaskVertexName(); diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/TestOnFileSortedOutput.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/TestOnFileSortedOutput.java index 7999d45fcd..47f841c3d6 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/TestOnFileSortedOutput.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/TestOnFileSortedOutput.java @@ -32,7 +32,6 @@ import org.apache.tez.dag.api.UserPayload; import org.apache.tez.runtime.api.ExecutionContext; import org.apache.tez.runtime.api.Event; -import org.apache.tez.runtime.api.MemoryUpdateCallback; import org.apache.tez.runtime.api.OutputContext; import org.apache.tez.runtime.api.OutputStatisticsReporter; import org.apache.tez.runtime.api.events.CompositeDataMovementEvent; @@ -68,8 +67,8 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.anyLong; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.anyLong; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; @@ -420,7 +419,7 @@ private OutputContext createTezOutputContext() throws IOException { callback.memoryAssigned(requestedSize); return null; } - }).when(context).requestInitialMemory(anyLong(), any(MemoryUpdateCallback.class)); + }).when(context).requestInitialMemory(anyLong(), any()); ExecutionContext ExecutionContext = mock(ExecutionContext.class); doReturn(HOST).when(ExecutionContext).getHostName(); doReturn(reporter).when(context).getStatisticsReporter(); diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/TestOnFileUnorderedKVOutput.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/TestOnFileUnorderedKVOutput.java index 963300cd40..cdb246d069 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/TestOnFileUnorderedKVOutput.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/TestOnFileUnorderedKVOutput.java @@ -18,8 +18,8 @@ package org.apache.tez.runtime.library.output; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.anyLong; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.anyLong; import static org.mockito.Mockito.atLeast; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.mock; @@ -61,7 +61,6 @@ import org.apache.tez.dag.records.TezVertexID; import org.apache.tez.runtime.LogicalIOProcessorRuntimeTask; import org.apache.tez.runtime.api.Event; -import org.apache.tez.runtime.api.MemoryUpdateCallback; import org.apache.tez.runtime.api.OutputContext; import org.apache.tez.runtime.api.events.CompositeDataMovementEvent; import org.apache.tez.runtime.api.impl.ExecutionContextImpl; @@ -282,7 +281,7 @@ null, auxEnv, new MemoryDistributor(1, 1, payloadConf), outputDescriptor, null, callback.memoryAssigned(requestedSize); return null; } - }).when(outputContext).requestInitialMemory(anyLong(), any(MemoryUpdateCallback.class)); + }).when(outputContext).requestInitialMemory(anyLong(), any()); return outputContext; } diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/utils/TestCodecUtils.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/utils/TestCodecUtils.java index afa645974f..5141598200 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/utils/TestCodecUtils.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/utils/TestCodecUtils.java @@ -49,7 +49,8 @@ import org.apache.tez.runtime.library.common.sort.impl.IFileInputStream; import org.junit.Assert; import org.junit.Test; -import org.mockito.Mockito; + +import static org.mockito.Mockito.mock; public class TestCodecUtils { @@ -80,7 +81,7 @@ private void testConcurrentDecompressorCreationWithModifiedBuffersizeOnCodec( Decompressor decompressor = CodecUtils.getDecompressor(codec); DecompressorStream stream = (DecompressorStream) CodecUtils.getDecompressedInputStreamWithBufferSize(codec, - Mockito.mock(IFileInputStream.class), decompressor, modifiedBufferSize); + mock(IFileInputStream.class), decompressor, modifiedBufferSize); Assert.assertEquals("stream buffer size is incorrect", modifiedBufferSize, getBufferSize(stream)); @@ -128,7 +129,7 @@ private void testConcurrentCompressorDecompressorCreationOnCodec(CompressionCode Decompressor decompressor = CodecUtils.getDecompressor(codec); CompressionInputStream stream = (CompressionInputStream) CodecUtils.getDecompressedInputStreamWithBufferSize(codec, - Mockito.mock(IFileInputStream.class), decompressor, modifiedBufferSize); + mock(IFileInputStream.class), decompressor, modifiedBufferSize); Assert.assertEquals("stream buffer size is incorrect", modifiedBufferSize, getBufferSize(stream)); @@ -146,7 +147,7 @@ private void testConcurrentCompressorDecompressorCreationOnCodec(CompressionCode Compressor compressor = CodecUtils.getCompressor(codec); CompressionOutputStream stream = - CodecUtils.createOutputStream(codec, Mockito.mock(OutputStream.class), compressor); + CodecUtils.createOutputStream(codec, mock(OutputStream.class), compressor); Assert.assertEquals("stream buffer size is incorrect", CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_DEFAULT, getBufferSize(stream)); @@ -164,7 +165,7 @@ private void testConcurrentCompressorDecompressorCreationOnCodec(CompressionCode Decompressor decompressor = CodecUtils.getDecompressor(codec); CompressionInputStream stream = - CodecUtils.createInputStream(codec, Mockito.mock(InputStream.class), decompressor); + CodecUtils.createInputStream(codec, mock(InputStream.class), decompressor); Assert.assertEquals("stream buffer size is incorrect", CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_DEFAULT, getBufferSize(stream)); diff --git a/tez-tools/analyzers/job-analyzer/pom.xml b/tez-tools/analyzers/job-analyzer/pom.xml index 89ae9088ff..0cc85834cb 100644 --- a/tez-tools/analyzers/job-analyzer/pom.xml +++ b/tez-tools/analyzers/job-analyzer/pom.xml @@ -46,6 +46,14 @@ org.apache.tez tez-dag + + org.apache.tez + hadoop-shim + + + org.apache.tez + tez-runtime-library + org.apache.tez tez-tests @@ -136,7 +144,7 @@ org.mockito - mockito-all + mockito-core test From 86dc4fd83f64494f5584c154e6812a9cee3cf4fb Mon Sep 17 00:00:00 2001 From: johnnyCake1 <92083002+johnnyCake1@users.noreply.github.com> Date: Fri, 10 Jun 2022 22:39:17 +0200 Subject: [PATCH 114/137] TEZ-4038: Add a /prof profiler endpoint like HiveServer2 has (#212) (Zheenbek Akimzhanov reviewed by Laszlo Bodor) --- .../apache/tez/common/TezUtilsInternal.java | 20 + .../tez/common/web/ProfileOutputServlet.java | 66 ++++ .../apache/tez/common/web/ProfileServlet.java | 362 ++++++++++++++++++ .../web/ServletToControllerAdapters.java | 13 + .../apache/tez/dag/app/web/WebUIService.java | 20 + .../test/java/org/apache/tez/test/TestAM.java | 26 +- 6 files changed, 504 insertions(+), 3 deletions(-) create mode 100644 tez-common/src/main/java/org/apache/tez/common/web/ProfileOutputServlet.java create mode 100644 tez-common/src/main/java/org/apache/tez/common/web/ProfileServlet.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 adcae8a964..c2efb29cb6 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 @@ -21,6 +21,7 @@ import java.io.FileInputStream; import java.io.FileNotFoundException; import java.io.IOException; +import java.lang.management.ManagementFactory; import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; import java.nio.charset.Charset; @@ -322,6 +323,25 @@ public static > Set getEnums(Configuration conf, String con return enums; } + public static Integer getPid() { + String pidStr = null; + String name = ManagementFactory.getRuntimeMXBean().getName(); + if (name != null) { + int idx = name.indexOf("@"); + if (idx != -1) { + pidStr = name.substring(0, name.indexOf("@")); + } + } + try { + if (pidStr != null) { + return Integer.valueOf(pidStr); + } + } catch (NumberFormatException nfe) { + LOG.info("Couldn't parse \"{}\" into integer pid", pidStr); + } + return null; + } + @Private public static void setHadoopCallerContext(HadoopShim hadoopShim, TezTaskAttemptID attemptID) { hadoopShim.setHadoopCallerContext("tez_ta:" + attemptID.toString()); diff --git a/tez-common/src/main/java/org/apache/tez/common/web/ProfileOutputServlet.java b/tez-common/src/main/java/org/apache/tez/common/web/ProfileOutputServlet.java new file mode 100644 index 0000000000..2fac77cdc8 --- /dev/null +++ b/tez-common/src/main/java/org/apache/tez/common/web/ProfileOutputServlet.java @@ -0,0 +1,66 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.tez.common.web; + +import org.apache.hadoop.yarn.webapp.MimeType; +import org.eclipse.jetty.servlet.DefaultServlet; + +import javax.servlet.ServletException; +import javax.servlet.http.HttpServletRequest; +import javax.servlet.http.HttpServletResponse; +import java.io.File; +import java.io.IOException; +import java.io.PrintWriter; +import java.nio.file.Files; +import java.nio.file.Paths; + +/** + * Servlet to serve files generated by {@link ProfileServlet}. + */ +public class ProfileOutputServlet extends DefaultServlet { + public static final String FILE_QUERY_PARAM = "file"; + + public void doGet(HttpServletRequest request, HttpServletResponse response) throws ServletException, IOException { + String queriedFile = request.getParameter(FILE_QUERY_PARAM); + if (queriedFile == null) { + writeMessage(response, "Run the profiler to be able to receive its output"); + return; + } + File outputFile = new File(ProfileServlet.OUTPUT_DIR, queriedFile); + if (!outputFile.exists()) { + writeMessage(response, "Requested file does not exist: " + queriedFile); + return; + } + if (outputFile.length() < 100) { + response.setIntHeader("Refresh", 2); + writeMessage(response, "This page auto-refreshes every 2 seconds until output file is ready..."); + return; + } + response.setContentType(MimeType.HTML); + response.getOutputStream().write(Files.readAllBytes(Paths.get(outputFile.getPath()))); + response.getOutputStream().flush(); + response.getOutputStream().close(); + } + + private void writeMessage(HttpServletResponse response, String message) throws IOException { + response.setContentType(MimeType.TEXT); + PrintWriter out = response.getWriter(); + out.println(message); + out.close(); + } +} diff --git a/tez-common/src/main/java/org/apache/tez/common/web/ProfileServlet.java b/tez-common/src/main/java/org/apache/tez/common/web/ProfileServlet.java new file mode 100644 index 0000000000..1cdddfbf9c --- /dev/null +++ b/tez-common/src/main/java/org/apache/tez/common/web/ProfileServlet.java @@ -0,0 +1,362 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.tez.common.web; + +import com.google.common.base.Joiner; + +import org.apache.hadoop.http.HttpServer2; +import org.apache.tez.common.TezUtilsInternal; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.servlet.ServletException; +import javax.servlet.http.HttpServlet; +import javax.servlet.http.HttpServletRequest; +import javax.servlet.http.HttpServletResponse; +import java.io.File; +import java.io.IOException; +import java.io.PrintStream; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; + +/** + *
+ * Servlet that runs async-profiler as web-endpoint.
+ * Following options from async-profiler can be specified as query paramater.
+ * //  -e event          profiling event: cpu|alloc|lock|cache-misses etc.
+ * //  -d duration       run profiling for{@literal } seconds (integer)
+ * //  -i interval       sampling interval in nanoseconds (long)
+ * //  -j jstackdepth    maximum Java stack depth (integer)
+ * //  -b bufsize        frame buffer size (long)
+ * //  -t                profile different threads separately
+ * //  -s                simple class names instead of FQN
+ * //  -o fmt[,fmt...]   output format: summary|traces|flat|collapsed|svg|tree|jfr
+ * //  --width px        SVG width pixels (integer)
+ * //  --height px       SVG frame height pixels (integer)
+ * //  --minwidth px     skip frames smaller than px (double)
+ * //  --reverse         generate stack-reversed FlameGraph / Call tree
+ * Example:
+ * - To collect 30 second CPU profile of current process (returns FlameGraph svg)
+ * {@literal curl "http://localhost:10002/prof"}
+ * - To collect 1 minute CPU profile of current process and output in tree format (html)
+ * {@literal curl  "http://localhost:10002/prof?output=tree&duration=60"}
+ * - To collect 30 second heap allocation profile of current process (returns FlameGraph svg)
+ * {@literal curl "http://localhost:10002/prof?event=alloc"}
+ * - To collect lock contention profile of current process (returns FlameGraph svg)
+ * {@literal curl "http://localhost:10002/prof?event=lock"}
+ * Following event types are supported (default is 'cpu') (NOTE: not all OS'es support all events)
+ * // Perf events:
+ * //    cpu
+ * //    page-faults
+ * //    context-switches
+ * //    cycles
+ * //    instructions
+ * //    cache-references
+ * //    cache-misses
+ * //    branches
+ * //    branch-misses
+ * //    bus-cycles
+ * //    L1-dcache-load-misses
+ * //    LLC-load-misses
+ * //    dTLB-load-misses
+ * //    mem:breakpoint
+ * //    trace:tracepoint
+ * // Java events:
+ * //    alloc
+ * //    lock
+ * 
+ */ +public class ProfileServlet extends HttpServlet { + private static final long serialVersionUID = 1L; + private static final Logger LOG = LoggerFactory.getLogger(ProfileServlet.class); + private static final String ACCESS_CONTROL_ALLOW_METHODS = "Access-Control-Allow-Methods"; + private static final String ALLOWED_METHODS = "GET"; + private static final String ACCESS_CONTROL_ALLOW_ORIGIN = "Access-Control-Allow-Origin"; + private static final String CONTENT_TYPE_TEXT = "text/plain; charset=utf-8"; + private static final String ASYNC_PROFILER_HOME_ENV = "ASYNC_PROFILER_HOME"; + private static final String ASYNC_PROFILER_HOME_SYSTEM_PROPERTY = "async.profiler.home"; + private static final String PROFILER_SCRIPT = "/profiler.sh"; + private static final int DEFAULT_DURATION_SECONDS = 10; + private static final AtomicInteger ID_GEN = new AtomicInteger(0); + public static final String OUTPUT_DIR = System.getProperty("java.io.tmpdir") + "/prof-output"; + + enum Event { + CPU("cpu"), + ALLOC("alloc"), + LOCK("lock"), + PAGE_FAULTS("page-faults"), + CONTEXT_SWITCHES("context-switches"), + CYCLES("cycles"), + INSTRUCTIONS("instructions"), + CACHE_REFERENCES("cache-references"), + CACHE_MISSES("cache-misses"), + BRANCHES("branches"), + BRANCH_MISSES("branch-misses"), + BUS_CYCLES("bus-cycles"), + L1_DCACHE_LOAD_MISSES("L1-dcache-load-misses"), + LLC_LOAD_MISSES("LLC-load-misses"), + DTLB_LOAD_MISSES("dTLB-load-misses"), + MEM_BREAKPOINT("mem:breakpoint"), + TRACE_TRACEPOINT("trace:tracepoint"); + + private final String internalName; + + Event(final String internalName) { + this.internalName = internalName; + } + + public String getInternalName() { + return internalName; + } + + public static Event fromInternalName(final String name) { + for (Event event : values()) { + if (event.getInternalName().equalsIgnoreCase(name)) { + return event; + } + } + return null; + } + } + + enum Output { + SUMMARY, TRACES, FLAT, COLLAPSED, SVG, TREE, JFR + } + + private final Lock profilerLock = new ReentrantLock(); + private Integer pid; + private String asyncProfilerHome; + private transient Process process; + + public ProfileServlet() { + this.asyncProfilerHome = getAsyncProfilerHome(); + this.pid = TezUtilsInternal.getPid(); + LOG.info("Servlet process PID: {} asyncProfilerHome: {}", pid, asyncProfilerHome); + } + + public void doGet(HttpServletRequest request, HttpServletResponse response) throws ServletException, IOException { + response.setContentType("text/plain; charset=UTF-8"); + PrintStream out = new PrintStream(response.getOutputStream(), false, "UTF-8"); + if (!HttpServer2.isInstrumentationAccessAllowed(this.getServletContext(), request, response)) { + response.setStatus(HttpServletResponse.SC_UNAUTHORIZED); + setResponseHeader(response); + out.println("Unauthorized: Instrumentation access is not allowed!"); + out.close(); + return; + } + + // make sure async profiler home is set + if (asyncProfilerHome == null || asyncProfilerHome.trim().isEmpty()) { + response.setStatus(HttpServletResponse.SC_INTERNAL_SERVER_ERROR); + setResponseHeader(response); + out.println("ASYNC_PROFILER_HOME env is not set"); + out.close(); + return; + } + + // if pid is explicitly specified, use it else default to current process + pid = getInteger(request, "pid", pid); + // if pid is not specified in query param and if current process pid cannot be determined + if (pid == null) { + response.setStatus(HttpServletResponse.SC_INTERNAL_SERVER_ERROR); + setResponseHeader(response); + out.println("'pid' query parameter unspecified or unable to determine PID of current process."); + out.close(); + return; + } + + final int duration = getInteger(request, "duration", DEFAULT_DURATION_SECONDS); + final Output output = getOutput(request); + final Event event = getEvent(request); + final Long interval = getLong(request, "interval"); + final Integer jstackDepth = getInteger(request, "jstackdepth", null); + final Long bufsize = getLong(request, "bufsize"); + final boolean thread = request.getParameterMap().containsKey("thread"); + final boolean simple = request.getParameterMap().containsKey("simple"); + final Integer width = getInteger(request, "width", null); + final Integer height = getInteger(request, "height", null); + final Double minwidth = getMinWidth(request); + final boolean reverse = request.getParameterMap().containsKey("reverse"); + if (process == null || !process.isAlive()) { + try { + int lockTimeoutSecs = 3; + if (profilerLock.tryLock(lockTimeoutSecs, TimeUnit.SECONDS)) { + try { + File outputFile = new File(OUTPUT_DIR, + "async-prof-pid-" + pid + "-" + event.name().toLowerCase() + "-" + ID_GEN.incrementAndGet() + "." + + output.name().toLowerCase()); + List cmd = new ArrayList<>(); + cmd.add(asyncProfilerHome + PROFILER_SCRIPT); + cmd.add("-e"); + cmd.add(event.getInternalName()); + cmd.add("-d"); + cmd.add("" + duration); + cmd.add("-o"); + cmd.add(output.name().toLowerCase()); + cmd.add("-f"); + cmd.add(outputFile.getAbsolutePath()); + if (interval != null) { + cmd.add("-i"); + cmd.add(interval.toString()); + } + if (jstackDepth != null) { + cmd.add("-j"); + cmd.add(jstackDepth.toString()); + } + if (bufsize != null) { + cmd.add("-b"); + cmd.add(bufsize.toString()); + } + if (thread) { + cmd.add("-t"); + } + if (simple) { + cmd.add("-s"); + } + if (width != null) { + cmd.add("--width"); + cmd.add(width.toString()); + } + if (height != null) { + cmd.add("--height"); + cmd.add(height.toString()); + } + if (minwidth != null) { + cmd.add("--minwidth"); + cmd.add(minwidth.toString()); + } + if (reverse) { + cmd.add("--reverse"); + } + cmd.add(pid.toString()); + process = new ProcessBuilder(cmd).start(); + + // set response and set refresh header to output location + setResponseHeader(response); + response.setStatus(HttpServletResponse.SC_ACCEPTED); + String relativeUrl = "/prof-output"; + // to avoid auto-refresh by ProfileOutputServlet, refreshDelay can be specified via url param + int refreshDelay = getInteger(request, "refreshDelay", 0); + // instead of sending redirect, set auto-refresh so that browsers will refresh with redirected url + response.setHeader("Refresh", (duration + refreshDelay) + "; URL=" + relativeUrl + '?' + + ProfileOutputServlet.FILE_QUERY_PARAM + '=' + outputFile.getName()); + + out.println("Profiled PID: " + pid); + out.println("Started [" + event.getInternalName() + + "] profiling. This page will automatically redirect to " + + relativeUrl + " after " + duration + " seconds.\n\ncommand:\n" + Joiner.on(" ").join(cmd)); + out.flush(); + } finally { + profilerLock.unlock(); + } + } else { + setResponseHeader(response); + response.setStatus(HttpServletResponse.SC_INTERNAL_SERVER_ERROR); + out.println("Unable to acquire lock. Another instance of profiler might be running."); + LOG.warn("Unable to acquire lock in {} seconds. Another instance of profiler might be running.", + lockTimeoutSecs); + } + } catch (InterruptedException e) { + LOG.warn("Interrupted while acquiring profile lock.", e); + response.setStatus(HttpServletResponse.SC_INTERNAL_SERVER_ERROR); + } + } else { + setResponseHeader(response); + response.setStatus(HttpServletResponse.SC_INTERNAL_SERVER_ERROR); + out.println("Another instance of profiler is already running."); + } + out.close(); + } + + private Integer getInteger(final HttpServletRequest req, final String param, final Integer defaultValue) { + final String value = req.getParameter(param); + if (value != null) { + try { + return Integer.valueOf(value); + } catch (NumberFormatException e) { + return defaultValue; + } + } + return defaultValue; + } + + private Long getLong(final HttpServletRequest req, final String param) { + final String value = req.getParameter(param); + if (value != null) { + try { + return Long.valueOf(value); + } catch (NumberFormatException e) { + return null; + } + } + return null; + } + + private Double getMinWidth(final HttpServletRequest req) { + final String value = req.getParameter("minwidth"); + if (value != null) { + try { + return Double.valueOf(value); + } catch (NumberFormatException e) { + return null; + } + } + return null; + } + + private Event getEvent(final HttpServletRequest req) { + final String eventArg = req.getParameter("event"); + if (eventArg != null) { + Event event = Event.fromInternalName(eventArg); + return event == null ? Event.CPU : event; + } + return Event.CPU; + } + + private Output getOutput(final HttpServletRequest req) { + final String outputArg = req.getParameter("output"); + if (outputArg != null) { + try { + return Output.valueOf(outputArg.trim().toUpperCase()); + } catch (IllegalArgumentException e) { + LOG.warn("Output format value is invalid, returning with default SVG"); + return Output.SVG; + } + } + return Output.SVG; + } + + private void setResponseHeader(final HttpServletResponse response) { + response.setHeader(ACCESS_CONTROL_ALLOW_METHODS, ALLOWED_METHODS); + response.setHeader(ACCESS_CONTROL_ALLOW_ORIGIN, "*"); + response.setContentType(CONTENT_TYPE_TEXT); + } + + public static String getAsyncProfilerHome() { + String asyncProfilerHome = System.getenv(ASYNC_PROFILER_HOME_ENV); + // if ENV is not set, see if -Dasync.profiler.home=/path/to/async/profiler/home is set + if (asyncProfilerHome == null || asyncProfilerHome.trim().isEmpty()) { + asyncProfilerHome = System.getProperty(ASYNC_PROFILER_HOME_SYSTEM_PROPERTY); + } + return asyncProfilerHome; + } +} diff --git a/tez-common/src/main/java/org/apache/tez/common/web/ServletToControllerAdapters.java b/tez-common/src/main/java/org/apache/tez/common/web/ServletToControllerAdapters.java index 35ca1b6408..304e9a9118 100644 --- a/tez-common/src/main/java/org/apache/tez/common/web/ServletToControllerAdapters.java +++ b/tez-common/src/main/java/org/apache/tez/common/web/ServletToControllerAdapters.java @@ -42,4 +42,17 @@ public StackServletController() throws ServletException { this.servlet = new StackServlet(); } } + + public static class ProfileServletController extends AbstractServletToControllerAdapter { + public ProfileServletController() throws ServletException { + this.servlet = new ProfileServlet(); + } + } + + public static class ProfileOutputServletController extends AbstractServletToControllerAdapter { + public ProfileOutputServletController() throws ServletException { + this.servlet = new ProfileOutputServlet(); + } + } + } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/web/WebUIService.java b/tez-dag/src/main/java/org/apache/tez/dag/app/web/WebUIService.java index bf94a73002..da7e66d037 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/web/WebUIService.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/web/WebUIService.java @@ -20,12 +20,19 @@ import static org.apache.hadoop.yarn.util.StringHelper.pajoin; +import java.io.IOException; import java.net.InetSocketAddress; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; import org.apache.tez.common.Preconditions; +import org.apache.tez.common.web.ProfileServlet; import org.apache.tez.common.web.ServletToControllerAdapters.ConfServletController; import org.apache.tez.common.web.ServletToControllerAdapters.JMXJsonServletController; import org.apache.tez.common.web.ServletToControllerAdapters.StackServletController; +import org.apache.tez.common.web.ServletToControllerAdapters.ProfileServletController; +import org.apache.tez.common.web.ServletToControllerAdapters.ProfileOutputServletController; import com.google.inject.name.Names; import org.slf4j.Logger; @@ -235,6 +242,19 @@ public void setup() { route("/jmx", JMXJsonServletController.class); route("/conf", ConfServletController.class); route("/stacks", StackServletController.class); + final String asyncProfilerHome = ProfileServlet.getAsyncProfilerHome(); + if (asyncProfilerHome != null && !asyncProfilerHome.trim().isEmpty()) { + Path tmpDir = Paths.get(ProfileServlet.OUTPUT_DIR); + try { + Files.createDirectories(tmpDir); + route("/prof", ProfileServletController.class); + route("/prof-output", ProfileOutputServletController.class); + } catch (IOException e) { + LOG.info("Could not create directory for profiler output: {} Disabling /prof endpoint... ", tmpDir); + } + } else { + LOG.info("ASYNC_PROFILER_HOME env or -Dasync.profiler.home not specified. Disabling /prof endpoint.."); + } } } } diff --git a/tez-tests/src/test/java/org/apache/tez/test/TestAM.java b/tez-tests/src/test/java/org/apache/tez/test/TestAM.java index 3e8a58cf7a..93fb0b4739 100644 --- a/tez-tests/src/test/java/org/apache/tez/test/TestAM.java +++ b/tez-tests/src/test/java/org/apache/tez/test/TestAM.java @@ -19,7 +19,7 @@ import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; - +import java.io.File; import java.io.IOException; import java.net.HttpURLConnection; import java.net.URL; @@ -74,8 +74,13 @@ public static void setup() throws IOException { Configuration tezClusterConf = new Configuration(); tezClusterConf.set("fs.defaultFS", remoteFs.getUri().toString()); // use HDFS tezClusterConf.setInt("yarn.nodemanager.delete.debug-delay-sec", 20000); - tezClusterConf.setLong(TezConfiguration.TEZ_AM_SLEEP_TIME_BEFORE_EXIT_MILLIS, 1000); + tezClusterConf.setLong(TezConfiguration.TEZ_AM_SLEEP_TIME_BEFORE_EXIT_MILLIS, 2000); tezClusterConf.set(YarnConfiguration.PROXY_ADDRESS, "localhost"); + //provide temporary profiler script to test /prof endpoint + File profiler = getProfiler(); + profiler.createNewFile(); + profiler.setExecutable(true, false); + tezClusterConf.set(TezConfiguration.TEZ_AM_LAUNCH_CMD_OPTS, ("-Dasync.profiler.home=" + getProfilerHomePath())); tezCluster.init(tezClusterConf); tezCluster.start(); } @@ -91,6 +96,7 @@ public static void tearDown() { dfsCluster.shutdown(); dfsCluster = null; } + getProfiler().delete(); } @Test(timeout = 60000) @@ -122,6 +128,8 @@ public void testAMWebUIService() throws TezException, IOException, InterruptedEx checkAddress(webUIAddress + "/jmx"); checkAddress(webUIAddress + "/conf"); checkAddress(webUIAddress + "/stacks"); + checkAddress(webUIAddress + "/prof", 202); + checkAddress(webUIAddress + "/prof-output"); URL url = new URL(webUIAddress); IntegerRanges portRange = conf.getRange(TezConfiguration.TEZ_AM_WEBSERVICE_PORT_RANGE, @@ -133,14 +141,26 @@ public void testAMWebUIService() throws TezException, IOException, InterruptedEx } private void checkAddress(String url) { + checkAddress(url, 200); + } + + private void checkAddress(String url, int expectedCode) { boolean success = false; try { HttpURLConnection connection = (HttpURLConnection) new URL(url).openConnection(); connection.connect(); - success = (connection.getResponseCode() == 200); + success = (connection.getResponseCode() == expectedCode); } catch (Exception e) { LOG.error("Error while checking url: " + url, e); } assertTrue(url + " should be available", success); } + + private static File getProfiler() { + return new File(getProfilerHomePath(), "profiler.sh"); + } + + private static String getProfilerHomePath() { + return System.getProperty("java.io.tmpdir"); + } } From 3da8438e68263f71462e99eb0b728e121702f5bf Mon Sep 17 00:00:00 2001 From: Aman Raj <104416558+amanraj2520@users.noreply.github.com> Date: Wed, 15 Jun 2022 14:55:38 +0530 Subject: [PATCH 115/137] [TEZ-4422] [CVE-2021-43138] Upgrade async from 2.3.0 to 2.6.4 to fix the vulnerability. Also an upgrade of yarn version to 1.6.0 and frontend maven plugin to 1.8.0 was done. (#217) (Aman Raj reviewed by Laszlo Bodor) --- pom.xml | 2 +- tez-ui/pom.xml | 2 +- tez-ui/src/main/webapp/package.json | 3 +++ tez-ui/src/main/webapp/yarn.lock | 18 +++++++++--------- 4 files changed, 14 insertions(+), 11 deletions(-) diff --git a/pom.xml b/pom.xml index c496e0ce87..9fd644bc9a 100644 --- a/pom.xml +++ b/pom.xml @@ -67,7 +67,7 @@ 0.7.45 ${env.PROTOC_PATH} scm:git:https://gitbox.apache.org/repos/asf/tez.git - 1.4 + 1.8.0 3.0.5 3.1.1 8.35 diff --git a/tez-ui/pom.xml b/tez-ui/pom.xml index 80cb05944b..ea4b520222 100644 --- a/tez-ui/pom.xml +++ b/tez-ui/pom.xml @@ -374,7 +374,7 @@ ${nodeVersion} - v0.21.3 + v1.6.0 diff --git a/tez-ui/src/main/webapp/package.json b/tez-ui/src/main/webapp/package.json index ad3aa74c5d..613660211b 100644 --- a/tez-ui/src/main/webapp/package.json +++ b/tez-ui/src/main/webapp/package.json @@ -62,5 +62,8 @@ }, "dependencies": { "em-tgraph": "0.0.14" + }, + "resolutions": { + "**/form-data/async": "2.6.4" } } diff --git a/tez-ui/src/main/webapp/yarn.lock b/tez-ui/src/main/webapp/yarn.lock index 660ac80d87..a8c96491d5 100644 --- a/tez-ui/src/main/webapp/yarn.lock +++ b/tez-ui/src/main/webapp/yarn.lock @@ -212,16 +212,16 @@ async@0.9.0: version "0.9.0" resolved "https://registry.yarnpkg.com/async/-/async-0.9.0.tgz#ac3613b1da9bed1b47510bb4651b8931e47146c7" +async@2.6.4, async@^2.0.1: + version "2.6.4" + resolved "https://registry.yarnpkg.com/async/-/async-2.6.4.tgz#706b7ff6084664cd7eae713f6f965433b5504221" + dependencies: + lodash "^4.17.14" + async@^1.0.0: version "1.5.2" resolved "https://registry.yarnpkg.com/async/-/async-1.5.2.tgz#ec6a61ae56480c0c3cb241c95618e20892f9672a" -async@^2.0.1: - version "2.3.0" - resolved "https://registry.yarnpkg.com/async/-/async-2.3.0.tgz#1013d1051047dd320fe24e494d5c66ecaf6147d9" - dependencies: - lodash "^4.14.0" - async@~0.2.6, async@~0.2.9: version "0.2.10" resolved "https://registry.yarnpkg.com/async/-/async-0.2.10.tgz#b6bbe0b0674b9d719708ca38de8c237cb526c3d1" @@ -3115,9 +3115,9 @@ lodash@^3.10.0, lodash@^3.6.0, lodash@^3.9.3: version "3.10.1" resolved "https://registry.yarnpkg.com/lodash/-/lodash-3.10.1.tgz#5bf45e8e49ba4189e17d482789dfd15bd140b7b6" -lodash@^4.14.0: - version "4.17.4" - resolved "https://registry.yarnpkg.com/lodash/-/lodash-4.17.4.tgz#78203a4d1c328ae1d86dca6460e369b57f4055ae" +lodash@^4.17.14: + version "4.17.21" + resolved "https://registry.yarnpkg.com/lodash/-/lodash-4.17.21.tgz#679591c564c3bffaae8454cf0b3df370c3d6911c" lodash@~2.3.0: version "2.3.0" From c6eb047aa2df2b60a673c2f852594295b5cb416a Mon Sep 17 00:00:00 2001 From: Aman Raj <104416558+amanraj2520@users.noreply.github.com> Date: Wed, 15 Jun 2022 18:52:43 +0530 Subject: [PATCH 116/137] [TEZ-4423][CVE-2021-44906] Upgrade minimist version from 0.0.8 to 1.2.6 to fix the vulnerability (#220) --- tez-ui/src/main/webapp/package.json | 4 +++- tez-ui/src/main/webapp/yarn.lock | 6 +++--- 2 files changed, 6 insertions(+), 4 deletions(-) diff --git a/tez-ui/src/main/webapp/package.json b/tez-ui/src/main/webapp/package.json index 613660211b..47ac36c110 100644 --- a/tez-ui/src/main/webapp/package.json +++ b/tez-ui/src/main/webapp/package.json @@ -64,6 +64,8 @@ "em-tgraph": "0.0.14" }, "resolutions": { - "**/form-data/async": "2.6.4" + "**/form-data/async": "2.6.4", + "**/mkdirp/minimist": "1.2.6", + "**/optimist/minimist": "1.2.6" } } diff --git a/tez-ui/src/main/webapp/yarn.lock b/tez-ui/src/main/webapp/yarn.lock index a8c96491d5..32030ce87a 100644 --- a/tez-ui/src/main/webapp/yarn.lock +++ b/tez-ui/src/main/webapp/yarn.lock @@ -3290,9 +3290,9 @@ minimatch@~0.2.9: lru-cache "2" sigmund "~1.0.0" -minimist@0.0.8, minimist@~0.0.1: - version "0.0.8" - resolved "https://registry.yarnpkg.com/minimist/-/minimist-0.0.8.tgz#857fcabfc3397d2625b8228262e86aa7a011b05d" +minimist@0.0.8, minimist@1.2.6, minimist@~0.0.1: + version "1.2.6" + resolved "https://registry.yarnpkg.com/minimist/-/minimist-1.2.6.tgz#8637a5b759ea0d6e98702cfb3a9283323c93af44" minimist@^1.1.0, minimist@^1.1.1: version "1.2.0" From 8ffa32df467c115ef6358ceef9c1703147d10a6a Mon Sep 17 00:00:00 2001 From: Aman Raj <104416558+amanraj2520@users.noreply.github.com> Date: Mon, 20 Jun 2022 12:15:25 +0530 Subject: [PATCH 117/137] [TEZ-4424][CVE-2021-3918] Upgrade json-schema from 0.2.3 to 0.4.0 to fix the vulnerability (#222) (Aman Raj reviewed by Laszlo Bodor) --- tez-ui/src/main/webapp/package.json | 3 ++- tez-ui/src/main/webapp/yarn.lock | 6 +++--- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/tez-ui/src/main/webapp/package.json b/tez-ui/src/main/webapp/package.json index 47ac36c110..c5e8e8d6f0 100644 --- a/tez-ui/src/main/webapp/package.json +++ b/tez-ui/src/main/webapp/package.json @@ -66,6 +66,7 @@ "resolutions": { "**/form-data/async": "2.6.4", "**/mkdirp/minimist": "1.2.6", - "**/optimist/minimist": "1.2.6" + "**/optimist/minimist": "1.2.6", + "**/jsprim/json-schema": "0.4.0" } } diff --git a/tez-ui/src/main/webapp/yarn.lock b/tez-ui/src/main/webapp/yarn.lock index 32030ce87a..b9f1220508 100644 --- a/tez-ui/src/main/webapp/yarn.lock +++ b/tez-ui/src/main/webapp/yarn.lock @@ -2789,9 +2789,9 @@ json-parse-helpfulerror@^1.0.2: dependencies: jju "^1.1.0" -json-schema@0.2.3: - version "0.2.3" - resolved "https://registry.yarnpkg.com/json-schema/-/json-schema-0.2.3.tgz#b480c892e59a2f05954ce727bd3f2a4e882f9e13" +json-schema@0.2.3, json-schema@0.4.0: + version "0.4.0" + resolved "https://registry.yarnpkg.com/json-schema/-/json-schema-0.4.0.tgz#f7de4cf6efab838ebaeb3236474cbba5a1930ab5" json-stable-stringify@^1.0.0, json-stable-stringify@^1.0.1: version "1.0.1" From 1ab7fc97c91cd9db5c8150e46ee7f98375bcced9 Mon Sep 17 00:00:00 2001 From: Aman Raj <104416558+amanraj2520@users.noreply.github.com> Date: Tue, 21 Jun 2022 00:41:30 +0530 Subject: [PATCH 118/137] [TEZ-4425][WS-2020-0345] Upgrade jsonpointer version from 4.0.1 to 4.1.0 (#226) (Aman Raj reviewed by Laszlo Bodor) --- tez-ui/src/main/webapp/package.json | 3 ++- tez-ui/src/main/webapp/yarn.lock | 6 +++--- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/tez-ui/src/main/webapp/package.json b/tez-ui/src/main/webapp/package.json index c5e8e8d6f0..4e4aff67eb 100644 --- a/tez-ui/src/main/webapp/package.json +++ b/tez-ui/src/main/webapp/package.json @@ -67,6 +67,7 @@ "**/form-data/async": "2.6.4", "**/mkdirp/minimist": "1.2.6", "**/optimist/minimist": "1.2.6", - "**/jsprim/json-schema": "0.4.0" + "**/jsprim/json-schema": "0.4.0", + "jsonpointer": "4.1.0" } } diff --git a/tez-ui/src/main/webapp/yarn.lock b/tez-ui/src/main/webapp/yarn.lock index b9f1220508..a3734301f2 100644 --- a/tez-ui/src/main/webapp/yarn.lock +++ b/tez-ui/src/main/webapp/yarn.lock @@ -2821,9 +2821,9 @@ jsonify@~0.0.0: version "0.0.0" resolved "https://registry.yarnpkg.com/jsonify/-/jsonify-0.0.0.tgz#2c74b6ee41d93ca51b7b5aaee8f503631d252a73" -jsonpointer@^4.0.0: - version "4.0.1" - resolved "https://registry.yarnpkg.com/jsonpointer/-/jsonpointer-4.0.1.tgz#4fd92cb34e0e9db3c89c8622ecf51f9b978c6cb9" +jsonpointer@4.1.0, jsonpointer@^4.0.0: + version "4.1.0" + resolved "https://registry.yarnpkg.com/jsonpointer/-/jsonpointer-4.1.0.tgz#501fb89986a2389765ba09e6053299ceb4f2c2cc" jsprim@^1.2.2: version "1.4.0" From 5f181ea32a630f583c2830101dd4969e6ab8d77d Mon Sep 17 00:00:00 2001 From: Ayush Saxena Date: Tue, 21 Jun 2022 00:43:55 +0530 Subject: [PATCH 119/137] TEZ-4429: Upgrade guava to 31.1 to fix CVE-2020-8908. (#223) (Ayush Saxena reviewed by Laszlo Bodor) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 9fd644bc9a..ebc1198d6b 100644 --- a/pom.xml +++ b/pom.xml @@ -57,7 +57,7 @@ https://repository.apache.org/service/local/staging/deploy/maven2 ${user.home}/clover.license - 27.0-jre + 31.1-jre 3.3.1 4.1.72.Final 0.13.0 From f25831ee40904813f31b65f794228b58a4800b8e Mon Sep 17 00:00:00 2001 From: Mark Bathori <57758037+mbathori-cloudera@users.noreply.github.com> Date: Tue, 21 Jun 2022 09:34:59 +0200 Subject: [PATCH 120/137] TEZ-4428: Use protoc-jar-maven-plugin to generate protobuf classes (#218) (Mark Bathori reviewed by Martin Tzvetanov Grigorov, Sylwester Lachiewicz, Laszlo Bodor) --- .travis.yml | 3 - BUILDING.txt | 9 +- build-tools/docker/Dockerfile | 2 - build-tools/install-protobuf.sh | 88 ------------------- build-tools/protocw | 32 ------- pom.xml | 22 ----- tez-api/pom.xml | 53 +++-------- tez-dag/pom.xml | 35 ++++---- tez-ext-service-tests/pom.xml | 31 +++---- tez-mapreduce/pom.xml | 29 +++--- .../tez-protobuf-history-plugin/pom.xml | 44 +++------- tez-runtime-internals/pom.xml | 29 +++--- tez-runtime-library/pom.xml | 31 +++---- 13 files changed, 109 insertions(+), 299 deletions(-) delete mode 100755 build-tools/install-protobuf.sh delete mode 100755 build-tools/protocw diff --git a/.travis.yml b/.travis.yml index 3637a0defe..65eaf77783 100644 --- a/.travis.yml +++ b/.travis.yml @@ -22,9 +22,6 @@ sudo: required env: MAVEN_OPTS="-Xmx2G -XX:MaxPermSize=512M" -before_install: - - ./build-tools/install-protobuf.sh - script: - jdk_switcher use oraclejdk8 - mvn -B clean install package -DskipTests=true -Dmaven.javadoc.skip=true diff --git a/BUILDING.txt b/BUILDING.txt index ae81d9c6c8..8b89a559de 100644 --- a/BUILDING.txt +++ b/BUILDING.txt @@ -122,10 +122,11 @@ The version of Protocol Buffer compiler, protoc, can be defined on-the-fly as: The default version is defined in the root pom.xml. -If you have multiple versions of protoc in your system, you can set in your -build shell the PROTOC_PATH environment variable to point to the one you -want to use for the Tez build. If you don't define this environment variable, -protoc is looked up in the PATH. +If you have multiple versions of protoc in your system, you can set in your +build shell the PROTOC_PATH environment variable to point to the one you +want to use for the Tez build. If you don't define this environment variable then the +embedded protoc compiler will be used with the version defined in ${protobuf.version}. +It detects the platform and executes the corresponding protoc binary at build time. You can also specify the path to protoc while building using -Dprotoc.path diff --git a/build-tools/docker/Dockerfile b/build-tools/docker/Dockerfile index 3e792604ec..f51cc3bd2b 100644 --- a/build-tools/docker/Dockerfile +++ b/build-tools/docker/Dockerfile @@ -55,8 +55,6 @@ RUN apt-get -q update && apt-get -q install --no-install-recommends -y \ rsync \ software-properties-common \ ssh-client \ - sudo \ - wget \ xz-utils \ && apt-get clean \ && rm -rf /var/lib/apt/lists/* diff --git a/build-tools/install-protobuf.sh b/build-tools/install-protobuf.sh deleted file mode 100755 index c28729a8a7..0000000000 --- a/build-tools/install-protobuf.sh +++ /dev/null @@ -1,88 +0,0 @@ -#!/bin/bash - -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -# This script attempts to install an arbitrary version of protobuf if needed. -# The desired version should be the first parameter: $1. -# Typical usage: the script is automatically called from tez-api (by maven) during the build process. - -# This script runs from build-tools folder. The user can remove -# the dynamically installed protobuf anytime like: -# rm -rf ./build-tools/protobuf/ #from root folder - -set -x -PROTOBUF_VERSION=${1:-2.5.0} -PROTOBUF_MAJOR_VERSION=$(echo "$PROTOBUF_VERSION" | cut -d. -f1) -if [ -n "$ZSH_VERSION" ]; then - SCRIPT_DIR="${0:a:h}" -else - SCRIPT_DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )" -fi - -function install_protobuf { - # before protobuf 3, there is no pre-compiled executables are host on github, let's try to build and make it - if (( PROTOBUF_MAJOR_VERSION < 3 )); then - wget "https://github.com/google/protobuf/releases/download/v$PROTOBUF_VERSION/protobuf-$PROTOBUF_VERSION.tar.gz" - tar -xzvf "protobuf-$PROTOBUF_VERSION.tar.gz" - rm "protobuf-$PROTOBUF_VERSION.tar.gz" - cd "protobuf-$PROTOBUF_VERSION" && ./configure --prefix=/usr && make && sudo make install - # since protobuf 3, there are precompiled protoc executables on github, let's quickly download and use it - else - ARCH=`uname -m` - case "$(uname -s)" in - Darwin) - FILE_NAME="protoc-$PROTOBUF_VERSION-osx-$ARCH" - ;; - Linux) - if test $ARCH = "aarch64"; then - ARCH="aarch_64" - fi - FILE_NAME="protoc-$PROTOBUF_VERSION-linux-$ARCH" - ;; - *) - echo "Unsupported OS returned by uname -s, you'll have to install protobuf 3.x manually" - exit 1 - ;; - esac - rm -f "$FILE_NAME.zip" #cleanup unfinished file if any - wget "https://github.com/google/protobuf/releases/download/v$PROTOBUF_VERSION/$FILE_NAME.zip" - mkdir "$SCRIPT_DIR/protobuf" - unzip -o "$FILE_NAME.zip" -d "$SCRIPT_DIR/protobuf" - rm "$FILE_NAME.zip" - fi -} - -if test -f "$SCRIPT_DIR/protobuf/bin/protoc"; then - PROTOBUF_INSTALLED_VERSION=$("$SCRIPT_DIR/protobuf/bin/protoc" --version) -else - PROTOBUF_INSTALLED_VERSION=$(protoc --version) -fi - -PROTOC_EXIT_CODE=$? - -if [ $PROTOC_EXIT_CODE -eq 0 ]; then - PROTOBUF_INSTALLED_VERSION=$(echo "$PROTOBUF_INSTALLED_VERSION" | tr -s ' ' | cut -d ' ' -f 2) - if [ "$PROTOBUF_INSTALLED_VERSION" == "$PROTOBUF_VERSION" ]; then - echo "Current protobuf version is equal to the requested ($PROTOBUF_INSTALLED_VERSION), exiting..." - else - echo "Current protobuf version ($PROTOBUF_INSTALLED_VERSION) is not equal to the requested ($PROTOBUF_VERSION), installing $PROTOBUF_VERSION" - install_protobuf - fi -else - echo "protoc --version command had non-zero return value, need to install probuf" - install_protobuf -fi diff --git a/build-tools/protocw b/build-tools/protocw deleted file mode 100755 index 6196071815..0000000000 --- a/build-tools/protocw +++ /dev/null @@ -1,32 +0,0 @@ -#!/bin/bash - -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -### This is a protoc wrapper for tez, which can dinamically call protoc from a downloaded protobuf. - -if [ -n "$ZSH_VERSION" ]; then - SCRIPT_DIR="${0:a:h}" -else - SCRIPT_DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )" -fi - -if test -f "$SCRIPT_DIR/protobuf/bin/protoc"; then - "$SCRIPT_DIR/protobuf/bin/protoc" "$@" -else - protoc "$@" -fi -exit $? diff --git a/pom.xml b/pom.xml index ebc1198d6b..5f381cdd27 100644 --- a/pom.xml +++ b/pom.xml @@ -1310,28 +1310,6 @@
- - - protoc-path-env-variable-not-defined - - - !env.PROTOC_PATH - - - - ${basedir}/../build-tools/protocw - - diff --git a/tez-api/pom.xml b/tez-api/pom.xml index cd5abdb03a..1d10bc1f85 100644 --- a/tez-api/pom.xml +++ b/tez-api/pom.xml @@ -145,53 +145,28 @@ org.apache.rat apache-rat-plugin - - exec-maven-plugin - org.codehaus.mojo - 1.6.0 + com.github.os72 + protoc-jar-maven-plugin + 3.11.4 - Install protobuf - initialize - - exec - - - ${basedir}/../build-tools/install-protobuf.sh - ${protobuf.version} - - - - - - org.apache.hadoop - hadoop-maven-plugins - - - compile-protoc generate-sources - protoc + run - ${protobuf.version} + com.google.protobuf:protoc:${protobuf.version} ${protoc.path} - - ${basedir}/src/main/proto - - - ${basedir}/src/main/proto - - DAGApiRecords.proto - DAGClientAMProtocol.proto - Events.proto - - - ${project.build.directory}/generated-sources/java + none + + ${basedir}/src/main/proto + + + + ${project.build.directory}/generated-sources/java + + diff --git a/tez-dag/pom.xml b/tez-dag/pom.xml index 356ed2c2e9..fda9fb2a54 100644 --- a/tez-dag/pom.xml +++ b/tez-dag/pom.xml @@ -211,30 +211,31 @@ - org.apache.hadoop - hadoop-maven-plugins + com.github.os72 + protoc-jar-maven-plugin + 3.11.4 - compile-protoc generate-sources - protoc + run - ${protobuf.version} + com.google.protobuf:protoc:${protobuf.version} ${protoc.path} - - ${basedir}/src/main/proto - ${basedir}/../tez-api/src/main/proto - ${basedir}/../tez-runtime-internals/src/main/proto - - - ${basedir}/src/main/proto - - HistoryEvents.proto - - - ${project.build.directory}/generated-sources/java + none + + ${basedir}/../tez-api/src/main/proto + ${basedir}/../tez-runtime-internals/src/main/proto + + + ${basedir}/src/main/proto + + + + ${project.build.directory}/generated-sources/java + + diff --git a/tez-ext-service-tests/pom.xml b/tez-ext-service-tests/pom.xml index 609b896fd8..a0bcbe32c0 100644 --- a/tez-ext-service-tests/pom.xml +++ b/tez-ext-service-tests/pom.xml @@ -165,29 +165,30 @@ apache-rat-plugin - org.apache.hadoop - hadoop-maven-plugins + com.github.os72 + protoc-jar-maven-plugin + 3.11.4 - compile-protoc generate-sources - protoc + run - ${protobuf.version} + com.google.protobuf:protoc:${protobuf.version} ${protoc.path} - - ${basedir}/src/test/proto + none + ${basedir}/../tez-api/src/main/proto - - - ${basedir}/src/test/proto - - TezDaemonProtocol.proto - - - ${project.build.directory}/generated-test-sources/java + + + ${basedir}/src/test/proto + + + + ${project.build.directory}/generated-test-sources/java + + diff --git a/tez-mapreduce/pom.xml b/tez-mapreduce/pom.xml index a22870fc38..1e35d2ea11 100644 --- a/tez-mapreduce/pom.xml +++ b/tez-mapreduce/pom.xml @@ -137,28 +137,27 @@ apache-rat-plugin - org.apache.hadoop - hadoop-maven-plugins + com.github.os72 + protoc-jar-maven-plugin + 3.11.4 - compile-protoc generate-sources - protoc + run - ${protobuf.version} + com.google.protobuf:protoc:${protobuf.version} ${protoc.path} - - ${basedir}/src/main/proto - - - ${basedir}/src/main/proto - - MRRuntimeProtos.proto - - - ${project.build.directory}/generated-sources/java + none + + ${basedir}/src/main/proto + + + + ${project.build.directory}/generated-sources/java + + diff --git a/tez-plugins/tez-protobuf-history-plugin/pom.xml b/tez-plugins/tez-protobuf-history-plugin/pom.xml index 607a31f95b..0fa526413e 100644 --- a/tez-plugins/tez-protobuf-history-plugin/pom.xml +++ b/tez-plugins/tez-protobuf-history-plugin/pom.xml @@ -59,47 +59,31 @@ apache-rat-plugin - org.apache.hadoop - hadoop-maven-plugins + com.github.os72 + protoc-jar-maven-plugin + 3.11.4 - compile-protoc generate-sources - protoc + run - ${protobuf.version} + com.google.protobuf:protoc:${protobuf.version} ${protoc.path} - - ${basedir}/src/main/proto - - - ${basedir}/src/main/proto - - HistoryLogger.proto - - - ${project.build.directory}/generated-sources/java + none + + ${basedir}/src/main/proto + + + + ${project.build.directory}/generated-sources/java + + - - - - - protoc-path-env-variable-not-defined - - - !env.PROTOC_PATH - - - - ${basedir}/../../build-tools/protocw - - - diff --git a/tez-runtime-internals/pom.xml b/tez-runtime-internals/pom.xml index 0619ca1e16..efe01e058e 100644 --- a/tez-runtime-internals/pom.xml +++ b/tez-runtime-internals/pom.xml @@ -91,28 +91,27 @@ apache-rat-plugin - org.apache.hadoop - hadoop-maven-plugins + com.github.os72 + protoc-jar-maven-plugin + 3.11.4 - compile-protoc generate-sources - protoc + run - ${protobuf.version} + com.google.protobuf:protoc:${protobuf.version} ${protoc.path} - - ${basedir}/src/main/proto - - - ${basedir}/src/main/proto - - RuntimeEvents.proto - - - ${project.build.directory}/generated-sources/java + none + + ${basedir}/src/main/proto + + + + ${project.build.directory}/generated-sources/java + + diff --git a/tez-runtime-library/pom.xml b/tez-runtime-library/pom.xml index c913e87d43..e6db47e535 100644 --- a/tez-runtime-library/pom.xml +++ b/tez-runtime-library/pom.xml @@ -114,30 +114,27 @@ apache-rat-plugin - org.apache.hadoop - hadoop-maven-plugins + com.github.os72 + protoc-jar-maven-plugin + 3.11.4 - compile-protoc generate-sources - protoc + run - ${protobuf.version} + com.google.protobuf:protoc:${protobuf.version} ${protoc.path} - - ${basedir}/src/main/proto - - - ${basedir}/src/main/proto - - ShufflePayloads.proto - CartesianProductPayload.proto - FairShufflePayloads.proto - - - ${project.build.directory}/generated-sources/java + none + + ${basedir}/src/main/proto + + + + ${project.build.directory}/generated-sources/java + + From adcc3138d3f79b056131bb2c7864a2b9a39848ea Mon Sep 17 00:00:00 2001 From: Aman Raj <104416558+amanraj2520@users.noreply.github.com> Date: Tue, 21 Jun 2022 13:11:57 +0530 Subject: [PATCH 121/137] TEZ-4426: [CVE-2018-1000620] Upgrade cryptiles from 2.0.5 to 4.1.2 (#227) (Aman Raj reviewed by Nikhil Gupta, Laszlo Bodor) --- tez-ui/pom.xml | 2 +- tez-ui/src/main/webapp/package.json | 3 ++- tez-ui/src/main/webapp/yarn.lock | 18 ++++++++++++++---- 3 files changed, 17 insertions(+), 6 deletions(-) diff --git a/tez-ui/pom.xml b/tez-ui/pom.xml index ea4b520222..59a02a2ab0 100644 --- a/tez-ui/pom.xml +++ b/tez-ui/pom.xml @@ -29,7 +29,7 @@ src/main/webapp - v5.12.0 + v8.9.0 ${basedir}/src/main/webapp/node/node node/yarn/dist/bin/yarn.js diff --git a/tez-ui/src/main/webapp/package.json b/tez-ui/src/main/webapp/package.json index 4e4aff67eb..f79e99e589 100644 --- a/tez-ui/src/main/webapp/package.json +++ b/tez-ui/src/main/webapp/package.json @@ -68,6 +68,7 @@ "**/mkdirp/minimist": "1.2.6", "**/optimist/minimist": "1.2.6", "**/jsprim/json-schema": "0.4.0", - "jsonpointer": "4.1.0" + "jsonpointer": "4.1.0", + "cryptiles": "4.1.2" } } diff --git a/tez-ui/src/main/webapp/yarn.lock b/tez-ui/src/main/webapp/yarn.lock index a3734301f2..3a1513e314 100644 --- a/tez-ui/src/main/webapp/yarn.lock +++ b/tez-ui/src/main/webapp/yarn.lock @@ -473,6 +473,12 @@ boom@2.x.x: dependencies: hoek "2.x.x" +boom@7.x.x: + version "7.3.0" + resolved "https://registry.yarnpkg.com/boom/-/boom-7.3.0.tgz#733a6d956d33b0b1999da3fe6c12996950d017b9" + dependencies: + hoek "6.x.x" + bower-config@0.6.1: version "0.6.1" resolved "https://registry.yarnpkg.com/bower-config/-/bower-config-0.6.1.tgz#7093155688bef44079bf4cb32d189312c87ded60" @@ -1180,11 +1186,11 @@ cross-spawn-async@^2.0.0: lru-cache "^4.0.0" which "^1.2.8" -cryptiles@2.x.x: - version "2.0.5" - resolved "https://registry.yarnpkg.com/cryptiles/-/cryptiles-2.0.5.tgz#3bdfecdc608147c1c67202fa291e7dca59eaa3b8" +cryptiles@2.x.x, cryptiles@4.1.2: + version "4.1.2" + resolved "https://registry.yarnpkg.com/cryptiles/-/cryptiles-4.1.2.tgz#363c9ab5c859da9d2d6fb901b64d980966181184" dependencies: - boom "2.x.x" + boom "7.x.x" ctype@0.5.3: version "0.5.3" @@ -2472,6 +2478,10 @@ hoek@2.x.x: version "2.16.3" resolved "https://registry.yarnpkg.com/hoek/-/hoek-2.16.3.tgz#20bb7403d3cea398e91dc4710a8ff1b8274a25ed" +hoek@6.x.x: + version "6.1.3" + resolved "https://registry.yarnpkg.com/hoek/-/hoek-6.1.3.tgz#73b7d33952e01fe27a38b0457294b79dd8da242c" + home-or-tmp@^1.0.0: version "1.0.0" resolved "https://registry.yarnpkg.com/home-or-tmp/-/home-or-tmp-1.0.0.tgz#4b9f1e40800c3e50c6c27f781676afcce71f3985" From be40104104796e265947b6c9666a5e1cd3ca848e Mon Sep 17 00:00:00 2001 From: Aman Raj <104416558+amanraj2520@users.noreply.github.com> Date: Thu, 30 Jun 2022 15:47:03 +0530 Subject: [PATCH 122/137] [TEZ-4427][CVE-2019-10744] Upgrade lodash.merge version to 4.6.2 to fix vulnerability (#229) (Aman Raj reviewed by Nikhil Gupta, Laszlo Bodor) --- tez-ui/src/main/webapp/package.json | 3 +- tez-ui/src/main/webapp/yarn.lock | 56 ++--------------------------- 2 files changed, 5 insertions(+), 54 deletions(-) diff --git a/tez-ui/src/main/webapp/package.json b/tez-ui/src/main/webapp/package.json index f79e99e589..3500b8aaff 100644 --- a/tez-ui/src/main/webapp/package.json +++ b/tez-ui/src/main/webapp/package.json @@ -69,6 +69,7 @@ "**/optimist/minimist": "1.2.6", "**/jsprim/json-schema": "0.4.0", "jsonpointer": "4.1.0", - "cryptiles": "4.1.2" + "cryptiles": "4.1.2", + "lodash.merge": "4.6.2" } } diff --git a/tez-ui/src/main/webapp/yarn.lock b/tez-ui/src/main/webapp/yarn.lock index 3a1513e314..2df3036ac9 100644 --- a/tez-ui/src/main/webapp/yarn.lock +++ b/tez-ui/src/main/webapp/yarn.lock @@ -2926,14 +2926,6 @@ lodash-node@^3.4.0: version "3.10.2" resolved "https://registry.yarnpkg.com/lodash-node/-/lodash-node-3.10.2.tgz#2598d5b1b54e6a68b4cb544e5c730953cbf632f7" -lodash._arraycopy@^3.0.0: - version "3.0.0" - resolved "https://registry.yarnpkg.com/lodash._arraycopy/-/lodash._arraycopy-3.0.0.tgz#76e7b7c1f1fb92547374878a562ed06a3e50f6e1" - -lodash._arrayeach@^3.0.0: - version "3.0.0" - resolved "https://registry.yarnpkg.com/lodash._arrayeach/-/lodash._arrayeach-3.0.0.tgz#bab156b2a90d3f1bbd5c653403349e5e5933ef9e" - lodash._baseassign@^3.0.0: version "3.2.0" resolved "https://registry.yarnpkg.com/lodash._baseassign/-/lodash._baseassign-3.2.0.tgz#8c38a099500f215ad09e59f1722fd0c52bfe0a4e" @@ -2954,10 +2946,6 @@ lodash._basecopy@^3.0.0: version "3.0.1" resolved "https://registry.yarnpkg.com/lodash._basecopy/-/lodash._basecopy-3.0.1.tgz#8da0e6a876cf344c0ad8a54882111dd3c5c7ca36" -lodash._basefor@^3.0.0: - version "3.0.3" - resolved "https://registry.yarnpkg.com/lodash._basefor/-/lodash._basefor-3.0.3.tgz#7550b4e9218ef09fad24343b612021c79b4c20c2" - lodash._baseindexof@^3.0.0: version "3.1.0" resolved "https://registry.yarnpkg.com/lodash._baseindexof/-/lodash._baseindexof-3.1.0.tgz#fe52b53a1c6761e42618d654e4a25789ed61822c" @@ -3031,14 +3019,6 @@ lodash.isarray@^3.0.0: version "3.0.4" resolved "https://registry.yarnpkg.com/lodash.isarray/-/lodash.isarray-3.0.4.tgz#79e4eb88c36a8122af86f844aa9bcd851b5fbb55" -lodash.isplainobject@^3.0.0: - version "3.2.0" - resolved "https://registry.yarnpkg.com/lodash.isplainobject/-/lodash.isplainobject-3.2.0.tgz#9a8238ae16b200432960cd7346512d0123fbf4c5" - dependencies: - lodash._basefor "^3.0.0" - lodash.isarguments "^3.0.0" - lodash.keysin "^3.0.0" - lodash.istypedarray@^3.0.0: version "3.0.6" resolved "https://registry.yarnpkg.com/lodash.istypedarray/-/lodash.istypedarray-3.0.6.tgz#c9a477498607501d8e8494d283b87c39281cef62" @@ -3051,32 +3031,9 @@ lodash.keys@^3.0.0: lodash.isarguments "^3.0.0" lodash.isarray "^3.0.0" -lodash.keysin@^3.0.0: - version "3.0.8" - resolved "https://registry.yarnpkg.com/lodash.keysin/-/lodash.keysin-3.0.8.tgz#22c4493ebbedb1427962a54b445b2c8a767fb47f" - dependencies: - lodash.isarguments "^3.0.0" - lodash.isarray "^3.0.0" - -lodash.merge@^3.0.2, lodash.merge@^3.3.2: - version "3.3.2" - resolved "https://registry.yarnpkg.com/lodash.merge/-/lodash.merge-3.3.2.tgz#0d90d93ed637b1878437bb3e21601260d7afe994" - dependencies: - lodash._arraycopy "^3.0.0" - lodash._arrayeach "^3.0.0" - lodash._createassigner "^3.0.0" - lodash._getnative "^3.0.0" - lodash.isarguments "^3.0.0" - lodash.isarray "^3.0.0" - lodash.isplainobject "^3.0.0" - lodash.istypedarray "^3.0.0" - lodash.keys "^3.0.0" - lodash.keysin "^3.0.0" - lodash.toplainobject "^3.0.0" - -lodash.merge@^4.5.1: - version "4.6.0" - resolved "https://registry.yarnpkg.com/lodash.merge/-/lodash.merge-4.6.0.tgz#69884ba144ac33fe699737a6086deffadd0f89c5" +lodash.merge@4.6.2, lodash.merge@^3.0.2, lodash.merge@^3.3.2, lodash.merge@^4.5.1: + version "4.6.2" + resolved "https://registry.yarnpkg.com/lodash.merge/-/lodash.merge-4.6.2.tgz#558aa53b43b661e1925a0afdfa36a9a1085fe57a" lodash.pad@^4.1.0: version "4.5.1" @@ -3100,13 +3057,6 @@ lodash.restparam@^3.0.0: version "3.6.1" resolved "https://registry.yarnpkg.com/lodash.restparam/-/lodash.restparam-3.6.1.tgz#936a4e309ef330a7645ed4145986c85ae5b20805" -lodash.toplainobject@^3.0.0: - version "3.0.0" - resolved "https://registry.yarnpkg.com/lodash.toplainobject/-/lodash.toplainobject-3.0.0.tgz#28790ad942d293d78aa663a07ecf7f52ca04198d" - dependencies: - lodash._basecopy "^3.0.0" - lodash.keysin "^3.0.0" - lodash.uniq@^3.2.2: version "3.2.2" resolved "https://registry.yarnpkg.com/lodash.uniq/-/lodash.uniq-3.2.2.tgz#146c36f25e75d19501ba402e88ba14937f63cd8b" From 06fff5c9cec13bc656c9c2e1678ae8a8e2271208 Mon Sep 17 00:00:00 2001 From: Ganesha Shreedhara Date: Mon, 4 Jul 2022 14:08:37 +0530 Subject: [PATCH 123/137] TEZ-4430: Fix tez.task.launch.cmd-opts property not working (#228) (Ganesha Shreedhara reviewed by Laszlo Bodor) --- .../src/main/java/org/apache/tez/client/TezClientUtils.java | 3 ++- .../test/java/org/apache/tez/client/TestTezClientUtils.java | 3 ++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java b/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java index d0fc374a81..d78ccf3e89 100644 --- a/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java +++ b/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java @@ -29,6 +29,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; +import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; @@ -767,7 +768,7 @@ static void maybeAddDefaultLoggingJavaOpts(String logLevel, List vargs) public static String maybeAddDefaultLoggingJavaOpts(String logLevel, String javaOpts) { List vargs = new ArrayList(5); if (javaOpts != null) { - vargs.add(javaOpts); + Collections.addAll(vargs, javaOpts.split(" ")); } else { vargs.add(""); } diff --git a/tez-api/src/test/java/org/apache/tez/client/TestTezClientUtils.java b/tez-api/src/test/java/org/apache/tez/client/TestTezClientUtils.java index 29a6769716..0341f27f66 100644 --- a/tez-api/src/test/java/org/apache/tez/client/TestTezClientUtils.java +++ b/tez-api/src/test/java/org/apache/tez/client/TestTezClientUtils.java @@ -684,12 +684,13 @@ public void testDefaultLoggingJavaOpts() { @Test public void testDefaultLoggingJavaOptsWithRootLogger() { - String origJavaOpts = "-D" + TezConstants.TEZ_ROOT_LOGGER_NAME + "=INFO"; + String origJavaOpts = "-D" + TezConstants.TEZ_ROOT_LOGGER_NAME + "=INFO -DtestProperty=value"; String javaOpts = TezClientUtils.maybeAddDefaultLoggingJavaOpts("FOOBAR", origJavaOpts); Assert.assertNotNull(javaOpts); Assert.assertTrue(javaOpts.contains("-D" + TezConstants.TEZ_ROOT_LOGGER_NAME + "=FOOBAR")); Assert.assertTrue(javaOpts.contains(TezConstants.TEZ_CONTAINER_LOG4J_PROPERTIES_FILE) && javaOpts.contains("-Dlog4j.configuratorClass=org.apache.tez.common.TezLog4jConfigurator")); + Assert.assertTrue(javaOpts.contains("-DtestProperty=value")); } @Test (timeout = 5000) From a192ec4443032c33269f5326755a292c87a98292 Mon Sep 17 00:00:00 2001 From: Laszlo Bodor Date: Mon, 4 Jul 2022 11:04:34 +0200 Subject: [PATCH 124/137] Revert "TEZ-4397 Open Tez Input splits asynchronously" This reverts commit f724c546069885e29e6446813805bb63bf0d5d9d. --- .../split/TezGroupedSplitsInputFormat.java | 72 ++----------------- .../mapreduce/grouper/TezSplitGrouper.java | 11 --- 2 files changed, 4 insertions(+), 79 deletions(-) diff --git a/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/split/TezGroupedSplitsInputFormat.java b/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/split/TezGroupedSplitsInputFormat.java index 6266ec1bcf..61ba560300 100644 --- a/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/split/TezGroupedSplitsInputFormat.java +++ b/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/split/TezGroupedSplitsInputFormat.java @@ -19,16 +19,8 @@ package org.apache.hadoop.mapred.split; import java.io.IOException; -import java.util.LinkedList; import java.util.Objects; -import java.util.Queue; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.Future; -import java.util.concurrent.atomic.AtomicInteger; -import com.google.common.util.concurrent.ThreadFactoryBuilder; -import org.apache.tez.mapreduce.grouper.TezSplitGrouper; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.classification.InterfaceAudience.Public; @@ -137,58 +129,14 @@ public class TezGroupedSplitsRecordReader implements RecordReader { int idx = 0; long progress; RecordReader curReader; - final AtomicInteger initIndex; - final int numReaders; - final ExecutorService initReaderExecService; - final Queue>> initedReaders; - + public TezGroupedSplitsRecordReader(TezGroupedSplit split, JobConf job, Reporter reporter) throws IOException { this.groupedSplit = split; this.job = job; this.reporter = reporter; - this.initIndex = new AtomicInteger(0); - int numThreads = conf.getInt(TezSplitGrouper.TEZ_GROUPING_SPLIT_INIT_THREADS, - TezSplitGrouper.TEZ_GROUPING_SPLIT_INIT_THREADS_DEFAULT); - this.numReaders = conf.getInt(TezSplitGrouper.TEZ_GROUPING_SPLIT_INIT_NUM_RECORDREADERS, - TezSplitGrouper.TEZ_GROUPING_SPLIT_INIT_NUM_RECORDREADERS_DEFAULT); - this.initReaderExecService = Executors.newFixedThreadPool(numThreads, - new ThreadFactoryBuilder() - .setDaemon(true) - .setPriority(Thread.MAX_PRIORITY) - .setNameFormat("TEZ-Split-Init-Thread-%d") - .build()); - this.initedReaders = new LinkedList<>(); - preInitReaders(); initNextRecordReader(); } - - private void preInitReaders() { - if (initReaderExecService == null) { - LOG.info("Init record reader threadpool is not initialized"); - return; - } - for (int i = 0; i < numReaders; i++) { - initedReaders.offer(this.initReaderExecService.submit(() -> { - try { - int index = initIndex.getAndIncrement(); - if (index >= groupedSplit.wrappedSplits.size()) { - return null; - } - InputSplit s = groupedSplit.wrappedSplits.get(index); - RecordReader reader = wrappedInputFormat.getRecordReader(s, job, reporter); - LOG.debug("Init Thread processed reader number {} initialization", index); - return reader; - } catch (Exception e) { - if (e instanceof InterruptedException) { - Thread.currentThread().interrupt(); - } - cancelsFutures(); - throw new RuntimeException(e); - } - })); - } - } @Override public boolean next(K key, V value) throws IOException { @@ -235,8 +183,6 @@ protected boolean initNextRecordReader() throws IOException { // if all chunks have been processed, nothing more to do. if (idx == groupedSplit.wrappedSplits.size()) { - LOG.info("Shutting down the init record reader threadpool"); - initReaderExecService.shutdownNow(); return false; } @@ -247,25 +193,15 @@ protected boolean initNextRecordReader() throws IOException { // get a record reader for the idx-th chunk try { - curReader = initedReaders.poll().get(); - preInitReaders(); + curReader = wrappedInputFormat.getRecordReader( + groupedSplit.wrappedSplits.get(idx), job, reporter); } catch (Exception e) { - if (e instanceof InterruptedException) { - Thread.currentThread().interrupt(); - } - cancelsFutures(); - throw new RuntimeException(e); + throw new RuntimeException (e); } idx++; return true; } - private void cancelsFutures() { - for (Future> f : initedReaders) { - f.cancel(true); - } - } - @Override public long getPos() throws IOException { long subprogress = 0; // bytes processed in current split diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/grouper/TezSplitGrouper.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/grouper/TezSplitGrouper.java index 3b2f17d1ff..a1d6b6c806 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/grouper/TezSplitGrouper.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/grouper/TezSplitGrouper.java @@ -102,17 +102,6 @@ public abstract class TezSplitGrouper { public static final String TEZ_GROUPING_NODE_LOCAL_ONLY = "tez.grouping.node.local.only"; public static final boolean TEZ_GROUPING_NODE_LOCAL_ONLY_DEFAULT = false; - /** - * Number of threads used to initialize the grouped splits, to asynchronously open the readers. - */ - public static final String TEZ_GROUPING_SPLIT_INIT_THREADS = "tez.grouping.split.init-threads"; - public static final int TEZ_GROUPING_SPLIT_INIT_THREADS_DEFAULT = 4; - - /** - * Number of record readers to asynchronously and proactively init. - */ - public static final String TEZ_GROUPING_SPLIT_INIT_NUM_RECORDREADERS = "tez.grouping.split.init.num-recordreaders"; - public static final int TEZ_GROUPING_SPLIT_INIT_NUM_RECORDREADERS_DEFAULT = 10; static class LocationHolder { List splits; From c386865e74d54a3a1b87a3e57b4cd14ccd332167 Mon Sep 17 00:00:00 2001 From: Mark Bathori <57758037+mark-bathori@users.noreply.github.com> Date: Fri, 22 Jul 2022 10:25:23 +0200 Subject: [PATCH 125/137] TEZ-4363: Bump protobuf dependency to 3.x (#192) (Mark Bathori reviewed by Laszlo Bodor, Aman Raj) --- BUILDING.txt | 2 +- build-tools/docker/Dockerfile | 8 ++++---- pom.xml | 2 +- .../apache/tez/dag/history/events/AMLaunchedEvent.java | 3 ++- .../org/apache/tez/dag/history/events/AMStartedEvent.java | 3 ++- .../tez/dag/history/events/ContainerLaunchedEvent.java | 4 +++- .../tez/dag/history/events/ContainerStoppedEvent.java | 4 +++- .../tez/dag/history/events/DAGCommitStartedEvent.java | 4 +++- .../apache/tez/dag/history/events/DAGFinishedEvent.java | 3 ++- .../tez/dag/history/events/DAGInitializedEvent.java | 4 +++- .../tez/dag/history/events/DAGKillRequestEvent.java | 4 +++- .../apache/tez/dag/history/events/DAGStartedEvent.java | 3 ++- .../apache/tez/dag/history/events/DAGSubmittedEvent.java | 3 ++- .../tez/dag/history/events/TaskAttemptFinishedEvent.java | 4 +++- .../tez/dag/history/events/TaskAttemptStartedEvent.java | 4 +++- .../apache/tez/dag/history/events/TaskFinishedEvent.java | 3 ++- .../apache/tez/dag/history/events/TaskStartedEvent.java | 3 ++- .../tez/dag/history/events/VertexCommitStartedEvent.java | 4 +++- .../dag/history/events/VertexConfigurationDoneEvent.java | 4 +++- .../tez/dag/history/events/VertexFinishedEvent.java | 4 +++- .../history/events/VertexGroupCommitFinishedEvent.java | 4 +++- .../dag/history/events/VertexGroupCommitStartedEvent.java | 4 +++- .../tez/dag/history/events/VertexInitializedEvent.java | 4 +++- .../apache/tez/dag/history/events/VertexStartedEvent.java | 3 ++- .../dag/history/logging/proto/ProtoMessageWritable.java | 3 ++- 25 files changed, 63 insertions(+), 28 deletions(-) diff --git a/BUILDING.txt b/BUILDING.txt index 8b89a559de..a917bfa1a2 100644 --- a/BUILDING.txt +++ b/BUILDING.txt @@ -9,7 +9,7 @@ Requirements: * JDK 1.8+ * Maven 3.1 or later * Findbugs 2.0.2 or later (if running findbugs) -* ProtocolBuffer 2.5.0 +* ProtocolBuffer 3.21.1 * Internet connection for first build (to fetch all dependencies) * Hadoop version should be 2.7.0 or higher. diff --git a/build-tools/docker/Dockerfile b/build-tools/docker/Dockerfile index f51cc3bd2b..a5f3e537aa 100644 --- a/build-tools/docker/Dockerfile +++ b/build-tools/docker/Dockerfile @@ -95,13 +95,13 @@ RUN add-apt-repository -y ppa:longsleep/golang-backports \ ############# ###### -# Install Google Protobuf 2.5.0 +# Install Google Protobuf 3.21.1 ###### -FROM tezbase AS protobuf250 +FROM tezbase AS protobuf SHELL ["/bin/bash", "-o", "pipefail", "-c"] RUN mkdir -p /opt/protobuf-src \ && curl -L -s -S \ - https://github.com/google/protobuf/releases/download/v2.5.0/protobuf-2.5.0.tar.gz \ + https://github.com/protocolbuffers/protobuf/releases/download/v21.1/protobuf-java-3.21.1.tar.gz \ -o /opt/protobuf.tar.gz \ && tar xzf /opt/protobuf.tar.gz --strip-components 1 -C /opt/protobuf-src WORKDIR /opt/protobuf-src @@ -261,7 +261,7 @@ RUN curl -sSL https://deb.nodesource.com/setup_14.x | bash - \ COPY --from=shellcheck /bin/shellcheck /bin/shellcheck COPY --from=hadolint /bin/hadolint /bin/hadolint COPY --from=buf /usr/local/bin/buf /usr/local/bin/buf -COPY --from=protobuf250 /opt/protobuf /opt/protobuf +COPY --from=protobuf /opt/protobuf /opt/protobuf ENV PROTOBUF_HOME /opt/protobuf ENV PROTOC_PATH /opt/protobuf/bin/protoc diff --git a/pom.xml b/pom.xml index 5f381cdd27..2e2cd8c128 100644 --- a/pom.xml +++ b/pom.xml @@ -63,7 +63,7 @@ 0.13.0 1.19 1.7.36 - 2.5.0 + 3.21.1 0.7.45 ${env.PROTOC_PATH} scm:git:https://gitbox.apache.org/repos/asf/tez.git diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/AMLaunchedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/AMLaunchedEvent.java index 001cbf0104..bb87fdb1fc 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/AMLaunchedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/AMLaunchedEvent.java @@ -22,6 +22,7 @@ import com.google.protobuf.CodedInputStream; import com.google.protobuf.CodedOutputStream; +import com.google.protobuf.ExtensionRegistry; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.util.ConverterUtils; import org.apache.tez.dag.history.HistoryEvent; @@ -90,7 +91,7 @@ public void toProtoStream(CodedOutputStream outputStream) throws IOException { @Override public void fromProtoStream(CodedInputStream inputStream) throws IOException { - AMLaunchedProto proto = inputStream.readMessage(AMLaunchedProto.PARSER, null); + AMLaunchedProto proto = inputStream.readMessage(AMLaunchedProto.PARSER, ExtensionRegistry.getEmptyRegistry()); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/AMStartedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/AMStartedEvent.java index 87daba68b8..6be528af96 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/AMStartedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/AMStartedEvent.java @@ -22,6 +22,7 @@ import com.google.protobuf.CodedInputStream; import com.google.protobuf.CodedOutputStream; +import com.google.protobuf.ExtensionRegistry; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.util.ConverterUtils; import org.apache.tez.dag.history.HistoryEvent; @@ -85,7 +86,7 @@ public void toProtoStream(CodedOutputStream outputStream) throws IOException { @Override public void fromProtoStream(CodedInputStream inputStream) throws IOException { - AMStartedProto proto = inputStream.readMessage(AMStartedProto.PARSER, null); + AMStartedProto proto = inputStream.readMessage(AMStartedProto.PARSER, ExtensionRegistry.getEmptyRegistry()); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/ContainerLaunchedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/ContainerLaunchedEvent.java index 11528e2f8c..9ec05926bb 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/ContainerLaunchedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/ContainerLaunchedEvent.java @@ -22,6 +22,7 @@ import com.google.protobuf.CodedInputStream; import com.google.protobuf.CodedOutputStream; +import com.google.protobuf.ExtensionRegistry; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.util.ConverterUtils; @@ -83,7 +84,8 @@ public void toProtoStream(CodedOutputStream outputStream) throws IOException { @Override public void fromProtoStream(CodedInputStream inputStream) throws IOException { - ContainerLaunchedProto proto = inputStream.readMessage(ContainerLaunchedProto.PARSER, null); + ContainerLaunchedProto proto = + inputStream.readMessage(ContainerLaunchedProto.PARSER, ExtensionRegistry.getEmptyRegistry()); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/ContainerStoppedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/ContainerStoppedEvent.java index 528f6293b6..9360e49643 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/ContainerStoppedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/ContainerStoppedEvent.java @@ -22,6 +22,7 @@ import com.google.protobuf.CodedInputStream; import com.google.protobuf.CodedOutputStream; +import com.google.protobuf.ExtensionRegistry; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.util.ConverterUtils; @@ -88,7 +89,8 @@ public void toProtoStream(CodedOutputStream outputStream) throws IOException { @Override public void fromProtoStream(CodedInputStream inputStream) throws IOException { - ContainerStoppedProto proto = inputStream.readMessage(ContainerStoppedProto.PARSER, null); + ContainerStoppedProto proto = + inputStream.readMessage(ContainerStoppedProto.PARSER, ExtensionRegistry.getEmptyRegistry()); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGCommitStartedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGCommitStartedEvent.java index 241dada2d6..694d9474ff 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGCommitStartedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGCommitStartedEvent.java @@ -23,6 +23,7 @@ import com.google.protobuf.CodedInputStream; import com.google.protobuf.CodedOutputStream; +import com.google.protobuf.ExtensionRegistry; import org.apache.tez.dag.history.HistoryEvent; import org.apache.tez.dag.history.HistoryEventType; import org.apache.tez.dag.history.SummaryEvent; @@ -76,7 +77,8 @@ public void toProtoStream(CodedOutputStream outputStream) throws IOException { @Override public void fromProtoStream(CodedInputStream inputStream) throws IOException { - DAGCommitStartedProto proto = inputStream.readMessage(DAGCommitStartedProto.PARSER, null); + DAGCommitStartedProto proto = + inputStream.readMessage(DAGCommitStartedProto.PARSER, ExtensionRegistry.getEmptyRegistry()); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGFinishedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGFinishedEvent.java index 6d1d6cd3ff..3364e1d2b2 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGFinishedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGFinishedEvent.java @@ -24,6 +24,7 @@ import com.google.protobuf.CodedInputStream; import com.google.protobuf.CodedOutputStream; +import com.google.protobuf.ExtensionRegistry; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.tez.common.counters.TezCounters; import org.apache.tez.dag.api.DagTypeConverters; @@ -129,7 +130,7 @@ public void toProtoStream(CodedOutputStream outputStream) throws IOException { @Override public void fromProtoStream(CodedInputStream inputStream) throws IOException { - DAGFinishedProto proto = inputStream.readMessage(DAGFinishedProto.PARSER, null); + DAGFinishedProto proto = inputStream.readMessage(DAGFinishedProto.PARSER, ExtensionRegistry.getEmptyRegistry()); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGInitializedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGInitializedEvent.java index 4df116ad37..31a53f714a 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGInitializedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGInitializedEvent.java @@ -23,6 +23,7 @@ import com.google.protobuf.CodedInputStream; import com.google.protobuf.CodedOutputStream; +import com.google.protobuf.ExtensionRegistry; import org.apache.tez.dag.history.HistoryEvent; import org.apache.tez.dag.history.HistoryEventType; import org.apache.tez.dag.records.DAGIDAware; @@ -91,7 +92,8 @@ public void toProtoStream(CodedOutputStream outputStream) throws IOException { @Override public void fromProtoStream(CodedInputStream inputStream) throws IOException { - DAGInitializedProto proto = inputStream.readMessage(DAGInitializedProto.PARSER, null); + DAGInitializedProto proto = + inputStream.readMessage(DAGInitializedProto.PARSER, ExtensionRegistry.getEmptyRegistry()); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGKillRequestEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGKillRequestEvent.java index c87f5ce7e7..b9e3da845c 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGKillRequestEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGKillRequestEvent.java @@ -22,6 +22,7 @@ import com.google.protobuf.CodedInputStream; import com.google.protobuf.CodedOutputStream; +import com.google.protobuf.ExtensionRegistry; import org.apache.tez.dag.history.HistoryEvent; import org.apache.tez.dag.history.HistoryEventType; import org.apache.tez.dag.history.SummaryEvent; @@ -76,7 +77,8 @@ public DAGKillRequestProto toProto() { @Override public void fromProtoStream(CodedInputStream inputStream) throws IOException { - DAGKillRequestProto proto = inputStream.readMessage(DAGKillRequestProto.PARSER, null); + DAGKillRequestProto proto = + inputStream.readMessage(DAGKillRequestProto.PARSER, ExtensionRegistry.getEmptyRegistry()); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGStartedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGStartedEvent.java index 9e1f6c4a50..d49ad3706f 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGStartedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGStartedEvent.java @@ -22,6 +22,7 @@ import com.google.protobuf.CodedInputStream; import com.google.protobuf.CodedOutputStream; +import com.google.protobuf.ExtensionRegistry; import org.apache.tez.dag.app.dag.DAGState; import org.apache.tez.dag.history.HistoryEvent; import org.apache.tez.dag.history.HistoryEventType; @@ -81,7 +82,7 @@ public void toProtoStream(CodedOutputStream outputStream) throws IOException { @Override public void fromProtoStream(CodedInputStream inputStream) throws IOException { - DAGStartedProto proto = inputStream.readMessage(DAGStartedProto.PARSER, null); + DAGStartedProto proto = inputStream.readMessage(DAGStartedProto.PARSER, ExtensionRegistry.getEmptyRegistry()); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGSubmittedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGSubmittedEvent.java index bed0f6f25a..ac9b923303 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGSubmittedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGSubmittedEvent.java @@ -24,6 +24,7 @@ import com.google.protobuf.CodedInputStream; import com.google.protobuf.CodedOutputStream; +import com.google.protobuf.ExtensionRegistry; import org.apache.tez.dag.records.DAGIDAware; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -134,7 +135,7 @@ public void toProtoStream(CodedOutputStream outputStream) throws IOException { @Override public void fromProtoStream(CodedInputStream inputStream) throws IOException { - DAGSubmittedProto proto = inputStream.readMessage(DAGSubmittedProto.PARSER, null); + DAGSubmittedProto proto = inputStream.readMessage(DAGSubmittedProto.PARSER, ExtensionRegistry.getEmptyRegistry()); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskAttemptFinishedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskAttemptFinishedEvent.java index ce2eb729f6..41b0ed6643 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskAttemptFinishedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskAttemptFinishedEvent.java @@ -24,6 +24,7 @@ import com.google.protobuf.CodedInputStream; import com.google.protobuf.CodedOutputStream; +import com.google.protobuf.ExtensionRegistry; import org.apache.tez.common.TezConverterUtils; import org.apache.tez.common.counters.CounterGroup; import org.apache.tez.common.counters.TezCounter; @@ -235,7 +236,8 @@ public void toProtoStream(CodedOutputStream outputStream) throws IOException { @Override public void fromProtoStream(CodedInputStream inputStream) throws IOException { - TaskAttemptFinishedProto proto = inputStream.readMessage(TaskAttemptFinishedProto.PARSER, null); + TaskAttemptFinishedProto proto = + inputStream.readMessage(TaskAttemptFinishedProto.PARSER, ExtensionRegistry.getEmptyRegistry()); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskAttemptStartedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskAttemptStartedEvent.java index 10ba439e1c..1442ff4b43 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskAttemptStartedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskAttemptStartedEvent.java @@ -22,6 +22,7 @@ import com.google.protobuf.CodedInputStream; import com.google.protobuf.CodedOutputStream; +import com.google.protobuf.ExtensionRegistry; import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.hadoop.yarn.util.ConverterUtils; @@ -98,7 +99,8 @@ public void toProtoStream(CodedOutputStream outputStream) throws IOException { @Override public void fromProtoStream(CodedInputStream inputStream) throws IOException { - TaskAttemptStartedProto proto = inputStream.readMessage(TaskAttemptStartedProto.PARSER, null); + TaskAttemptStartedProto proto = + inputStream.readMessage(TaskAttemptStartedProto.PARSER, ExtensionRegistry.getEmptyRegistry()); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskFinishedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskFinishedEvent.java index 215968274c..fd0a5fc6aa 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskFinishedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskFinishedEvent.java @@ -22,6 +22,7 @@ import com.google.protobuf.CodedInputStream; import com.google.protobuf.CodedOutputStream; +import com.google.protobuf.ExtensionRegistry; import org.apache.tez.common.counters.CounterGroup; import org.apache.tez.common.counters.TezCounter; import org.apache.tez.dag.records.TaskIDAware; @@ -116,7 +117,7 @@ public void toProtoStream(CodedOutputStream outputStream) throws IOException { @Override public void fromProtoStream(CodedInputStream inputStream) throws IOException { - TaskFinishedProto proto = inputStream.readMessage(TaskFinishedProto.PARSER, null); + TaskFinishedProto proto = inputStream.readMessage(TaskFinishedProto.PARSER, ExtensionRegistry.getEmptyRegistry()); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskStartedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskStartedEvent.java index 577aef963c..1379e04e19 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskStartedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskStartedEvent.java @@ -22,6 +22,7 @@ import com.google.protobuf.CodedInputStream; import com.google.protobuf.CodedOutputStream; +import com.google.protobuf.ExtensionRegistry; import org.apache.tez.dag.api.oldrecords.TaskState; import org.apache.tez.dag.history.HistoryEvent; import org.apache.tez.dag.history.HistoryEventType; @@ -90,7 +91,7 @@ public void toProtoStream(CodedOutputStream outputStream) throws IOException { @Override public void fromProtoStream(CodedInputStream inputStream) throws IOException { - TaskStartedProto proto = inputStream.readMessage(TaskStartedProto.PARSER, null); + TaskStartedProto proto = inputStream.readMessage(TaskStartedProto.PARSER, ExtensionRegistry.getEmptyRegistry()); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexCommitStartedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexCommitStartedEvent.java index ca2cafc834..cb05bdd60b 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexCommitStartedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexCommitStartedEvent.java @@ -24,6 +24,7 @@ import com.google.protobuf.CodedInputStream; import com.google.protobuf.CodedOutputStream; +import com.google.protobuf.ExtensionRegistry; import org.apache.tez.dag.history.HistoryEvent; import org.apache.tez.dag.history.HistoryEventType; import org.apache.tez.dag.history.SummaryEvent; @@ -80,7 +81,8 @@ public void toProtoStream(CodedOutputStream outputStream) throws IOException { @Override public void fromProtoStream(CodedInputStream inputStream) throws IOException { - VertexCommitStartedProto proto = inputStream.readMessage(VertexCommitStartedProto.PARSER, null); + VertexCommitStartedProto proto = + inputStream.readMessage(VertexCommitStartedProto.PARSER, ExtensionRegistry.getEmptyRegistry()); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexConfigurationDoneEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexConfigurationDoneEvent.java index 2f4fac079f..d5a1ce5b2d 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexConfigurationDoneEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexConfigurationDoneEvent.java @@ -24,6 +24,7 @@ import com.google.protobuf.CodedInputStream; import com.google.protobuf.CodedOutputStream; +import com.google.protobuf.ExtensionRegistry; import org.apache.tez.dag.api.DagTypeConverters; import org.apache.tez.dag.api.EdgeProperty; import org.apache.tez.dag.api.VertexLocationHint; @@ -162,7 +163,8 @@ public void toProtoStream(CodedOutputStream outputStream) throws IOException { @Override public void fromProtoStream(CodedInputStream inputStream) throws IOException { - VertexConfigurationDoneProto proto = inputStream.readMessage(VertexConfigurationDoneProto.PARSER, null); + VertexConfigurationDoneProto proto = + inputStream.readMessage(VertexConfigurationDoneProto.PARSER, ExtensionRegistry.getEmptyRegistry()); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexFinishedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexFinishedEvent.java index 24612a6a2e..4288a2f81a 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexFinishedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexFinishedEvent.java @@ -24,6 +24,7 @@ import com.google.protobuf.CodedInputStream; import com.google.protobuf.CodedOutputStream; +import com.google.protobuf.ExtensionRegistry; import org.apache.tez.dag.app.dag.impl.ServicePluginInfo; import org.apache.tez.dag.records.VertexIDAware; import org.slf4j.Logger; @@ -131,7 +132,8 @@ public void toProtoStream(CodedOutputStream outputStream) throws IOException { @Override public void fromProtoStream(CodedInputStream inputStream) throws IOException { - VertexFinishedProto proto = inputStream.readMessage(VertexFinishedProto.PARSER, null); + VertexFinishedProto proto = + inputStream.readMessage(VertexFinishedProto.PARSER, ExtensionRegistry.getEmptyRegistry()); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexGroupCommitFinishedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexGroupCommitFinishedEvent.java index c9d5aae770..7b3d0d4463 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexGroupCommitFinishedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexGroupCommitFinishedEvent.java @@ -24,6 +24,7 @@ import com.google.protobuf.CodedInputStream; import com.google.protobuf.CodedOutputStream; +import com.google.protobuf.ExtensionRegistry; import org.apache.tez.dag.history.HistoryEvent; import org.apache.tez.dag.history.HistoryEventType; import org.apache.tez.dag.history.SummaryEvent; @@ -101,7 +102,8 @@ public void toProtoStream(CodedOutputStream outputStream) throws IOException { @Override public void fromProtoStream(CodedInputStream inputStream) throws IOException { - VertexGroupCommitFinishedProto proto = inputStream.readMessage(VertexGroupCommitFinishedProto.PARSER, null); + VertexGroupCommitFinishedProto proto = + inputStream.readMessage(VertexGroupCommitFinishedProto.PARSER, ExtensionRegistry.getEmptyRegistry()); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexGroupCommitStartedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexGroupCommitStartedEvent.java index cdd11bcc4c..d615debeb3 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexGroupCommitStartedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexGroupCommitStartedEvent.java @@ -24,6 +24,7 @@ import com.google.protobuf.CodedInputStream; import com.google.protobuf.CodedOutputStream; +import com.google.protobuf.ExtensionRegistry; import org.apache.tez.dag.history.HistoryEvent; import org.apache.tez.dag.history.HistoryEventType; import org.apache.tez.dag.history.SummaryEvent; @@ -101,7 +102,8 @@ public void toProtoStream(CodedOutputStream outputStream) throws IOException { @Override public void fromProtoStream(CodedInputStream inputStream) throws IOException { - VertexGroupCommitStartedProto proto = inputStream.readMessage(VertexGroupCommitStartedProto.PARSER, null); + VertexGroupCommitStartedProto proto = + inputStream.readMessage(VertexGroupCommitStartedProto.PARSER, ExtensionRegistry.getEmptyRegistry()); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexInitializedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexInitializedEvent.java index a019fe3784..285c52001e 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexInitializedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexInitializedEvent.java @@ -25,6 +25,7 @@ import com.google.protobuf.CodedInputStream; import com.google.protobuf.CodedOutputStream; +import com.google.protobuf.ExtensionRegistry; import org.apache.tez.dag.api.DagTypeConverters; import org.apache.tez.dag.api.InputDescriptor; import org.apache.tez.dag.api.InputInitializerDescriptor; @@ -158,7 +159,8 @@ public void toProtoStream(CodedOutputStream outputStream) throws IOException { @Override public void fromProtoStream(CodedInputStream inputStream) throws IOException { - VertexInitializedProto proto = inputStream.readMessage(VertexInitializedProto.PARSER, null); + VertexInitializedProto proto = + inputStream.readMessage(VertexInitializedProto.PARSER, ExtensionRegistry.getEmptyRegistry()); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexStartedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexStartedEvent.java index cdd73b4483..53f00977a7 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexStartedEvent.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexStartedEvent.java @@ -22,6 +22,7 @@ import com.google.protobuf.CodedInputStream; import com.google.protobuf.CodedOutputStream; +import com.google.protobuf.ExtensionRegistry; import org.apache.tez.dag.app.dag.VertexState; import org.apache.tez.dag.history.HistoryEvent; import org.apache.tez.dag.history.HistoryEventType; @@ -81,7 +82,7 @@ public void toProtoStream(CodedOutputStream outputStream) throws IOException { @Override public void fromProtoStream(CodedInputStream inputStream) throws IOException { - VertexStartedProto proto = inputStream.readMessage(VertexStartedProto.PARSER, null); + VertexStartedProto proto = inputStream.readMessage(VertexStartedProto.PARSER, ExtensionRegistry.getEmptyRegistry()); if (proto == null) { throw new IOException("No data found in stream"); } diff --git a/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/ProtoMessageWritable.java b/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/ProtoMessageWritable.java index 34e47014e1..df5743660c 100644 --- a/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/ProtoMessageWritable.java +++ b/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/ProtoMessageWritable.java @@ -25,6 +25,7 @@ import java.io.InputStream; import java.io.OutputStream; +import com.google.protobuf.ExtensionRegistry; import org.apache.hadoop.io.Writable; import com.google.protobuf.CodedInputStream; @@ -96,6 +97,6 @@ public void readFields(DataInput in) throws IOException { cin.setSizeLimit(Integer.MAX_VALUE); } din.in = in; - message = cin.readMessage(parser, null); + message = cin.readMessage(parser, ExtensionRegistry.getEmptyRegistry()); } } From 5e31e4d970647aecba2b58e836ff65fedb144306 Mon Sep 17 00:00:00 2001 From: Aman Raj <104416558+amanraj2520@users.noreply.github.com> Date: Tue, 26 Jul 2022 12:04:04 +0530 Subject: [PATCH 126/137] [TEZ-4439] Update the protobuf documentation in TEZ codebase (#233) (Aman Raj reviewed by Laszlo Bodor) --- docs/src/site/markdown/install.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/src/site/markdown/install.md b/docs/src/site/markdown/install.md index b1cdcf9583..44ca862a82 100644 --- a/docs/src/site/markdown/install.md +++ b/docs/src/site/markdown/install.md @@ -34,11 +34,11 @@ or higher. 2. Build tez using `mvn clean package -DskipTests=true -Dmaven.javadoc.skip=true` - This assumes that you have already installed JDK8 or later and Maven 3 or later. - - Tez also requires Protocol Buffers 2.5.0, including the protoc-compiler. + - Tez also requires Protocol Buffers 3.19.4, including the protoc-compiler. * This can be downloaded from https://github.com/google/protobuf/tags/. * On Mac OS X with the homebrew package manager `brew install protobuf250` - * For rpm-based linux systems, the yum repos may not have the 2.5.0 version. - `rpm.pbone.net` has the protobuf-2.5.0 and protobuf-compiler-2.5.0 packages. + * For rpm-based linux systems, the yum repos may not have the 3.19.4 version. + `rpm.pbone.net` has the protobuf-3.19.4 and protobuf-compiler-3.19.4 packages. - If you prefer to run the unit tests, remove skipTests from the command above. - If you use Eclipse IDE, you can import the projects using From 621a83152c305ccb654f69f57fe711bc52d271be Mon Sep 17 00:00:00 2001 From: PJ Fanning Date: Tue, 2 Aug 2022 09:20:14 +0100 Subject: [PATCH 127/137] TEZ-4435: use jackson v2 - jackson v1 is EOL and full of security issues (#231) (PJ Fanning reviewed by Laszlo Bodor, Ayush Saxena) --- pom.xml | 29 ++++++++++++++++++- .../logging/proto/DagManifesFileScanner.java | 2 +- .../proto/HistoryEventProtoConverter.java | 2 +- 3 files changed, 30 insertions(+), 3 deletions(-) diff --git a/pom.xml b/pom.xml index 2e2cd8c128..3ec471b9c0 100644 --- a/pom.xml +++ b/pom.xml @@ -70,8 +70,10 @@ 1.8.0 3.0.5 3.1.1 + 3.0.0 8.35 1.3.6 + 2.0.0 ${project.build.directory}/tmp @@ -1001,7 +1003,14 @@ org.apache.maven.plugins maven-enforcer-plugin - false + ${maven-enforcer-plugin.version} + + + de.skuzzle.enforcer + restrict-imports-enforcer-rule + ${restrict-imports.enforcer.version} + + clean @@ -1019,6 +1028,24 @@ + + banned-illegal-imports + process-sources + + enforce + + + + + true + Use Fasterxml Jackson 2 dependency in place of org.codehaus Jackson 1 + + org.codehaus.jackson.** + + + + + diff --git a/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/DagManifesFileScanner.java b/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/DagManifesFileScanner.java index 697083c30c..addb148530 100644 --- a/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/DagManifesFileScanner.java +++ b/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/DagManifesFileScanner.java @@ -28,12 +28,12 @@ import java.util.List; import java.util.Map; +import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.commons.io.IOUtils; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.security.UserGroupInformation; import org.apache.tez.dag.api.TezConfiguration; import org.apache.tez.dag.history.logging.proto.HistoryLoggerProtos.ManifestEntryProto; -import org.codehaus.jackson.map.ObjectMapper; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/HistoryEventProtoConverter.java b/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/HistoryEventProtoConverter.java index f5f436eb9d..904c165684 100644 --- a/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/HistoryEventProtoConverter.java +++ b/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/HistoryEventProtoConverter.java @@ -24,6 +24,7 @@ import java.util.Map.Entry; import java.util.TreeMap; +import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.tez.common.ATSConstants; @@ -58,7 +59,6 @@ import org.apache.tez.dag.records.TezTaskAttemptID; import org.apache.tez.dag.records.TezTaskID; import org.apache.tez.dag.records.TezVertexID; -import org.codehaus.jackson.map.ObjectMapper; import org.slf4j.Logger; import org.slf4j.LoggerFactory; From db915ebafb6ad9bb9de062ee9b17c67eab67c7ae Mon Sep 17 00:00:00 2001 From: zhengchenyu Date: Mon, 29 Aug 2022 19:50:20 +0800 Subject: [PATCH 128/137] TEZ-4440. When tez app run in yarn fed cluster, may throw NPE (#235) (zhengchenyu reviewed by Laszlo Bodor) --- .../org/apache/tez/dag/app/rm/DagAwareYarnTaskScheduler.java | 3 ++- .../org/apache/tez/dag/app/rm/YarnTaskSchedulerService.java | 5 +++-- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/DagAwareYarnTaskScheduler.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/DagAwareYarnTaskScheduler.java index 29f350f9d0..1b8e19176d 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/DagAwareYarnTaskScheduler.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/DagAwareYarnTaskScheduler.java @@ -780,7 +780,8 @@ public void onError(Throwable e) { @Override public Resource getAvailableResources() { - return client.getAvailableResources(); + Resource resource = client.getAvailableResources(); + return resource == null ? Resource.newInstance(0, 0) : resource; } @Override 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 6f51566f75..ea0bdb92e8 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 @@ -254,7 +254,8 @@ public YarnTaskSchedulerService(TaskSchedulerContext taskSchedulerContext) { @Override public Resource getAvailableResources() { - return amRmClient.getAvailableResources(); + Resource resource = amRmClient.getAvailableResources(); + return resource == null ? Resource.newInstance(0, 0) : resource; } @Override @@ -1166,7 +1167,7 @@ boolean preemptIfNeeded() { ContainerId[] preemptedContainers = null; int numPendingRequestsToService = 0; synchronized (this) { - Resource freeResources = amRmClient.getAvailableResources(); + Resource freeResources = this.getAvailableResources(); if (LOG.isDebugEnabled()) { LOG.debug(constructPreemptionPeriodicLog(freeResources)); } else { From 55b6031fa2c38cd8af1d384414d29c4e19a875d7 Mon Sep 17 00:00:00 2001 From: zhengchenyu Date: Mon, 29 Aug 2022 19:52:29 +0800 Subject: [PATCH 129/137] TEZ-4441: TezAppMaster may stuck because of reportError skip send error event (#236) (zhengchenyu reviewed by Laszlo Bodor) --- .../org/apache/tez/dag/app/DAGAppMaster.java | 2 +- .../tez/dag/app/rm/TaskSchedulerManager.java | 8 +- .../dag/app/rm/TestTaskSchedulerHelpers.java | 17 ++++ .../dag/app/rm/TestTaskSchedulerManager.java | 89 +++++++++++++++++++ 4 files changed, 114 insertions(+), 2 deletions(-) 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 ca50b2ac6b..c8519201fd 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 @@ -2623,7 +2623,7 @@ public String getWebUIAddress() { } @VisibleForTesting - static void parseAllPlugins( + public static void parseAllPlugins( List taskSchedulerDescriptors, BiMap taskSchedulerPluginMap, List containerLauncherDescriptors, BiMap containerLauncherPluginMap, List taskCommDescriptors, BiMap taskCommPluginMap, diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerManager.java index 8a24ad3426..dfd48e6751 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerManager.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerManager.java @@ -910,7 +910,8 @@ public void reportError(int taskSchedulerIndex, ServicePluginError servicePlugin LOG.info("Error reported by scheduler {} - {}", Utils.getTaskSchedulerIdentifierString(taskSchedulerIndex, appContext) + ": " + diagnostics); - if (taskSchedulerDescriptors[taskSchedulerIndex].getClassName().equals(yarnSchedulerClassName)) { + if (taskSchedulerDescriptors[taskSchedulerIndex].getEntityName() + .equals(TezConstants.getTezYarnServicePluginName())) { LOG.warn( "Reporting a SchedulerServiceError to the DAGAppMaster since the error" + " was reported by the YARN task scheduler"); @@ -1078,4 +1079,9 @@ public String getTaskSchedulerClassName(int taskSchedulerIndex) { return taskSchedulers[taskSchedulerIndex].getTaskScheduler().getClass().getName(); } + @VisibleForTesting + public TaskScheduler getTaskScheduler(int taskSchedulerIndex) { + return taskSchedulers[taskSchedulerIndex].getTaskScheduler(); + } + } 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 b7acc6876c..490067a54c 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 @@ -161,6 +161,19 @@ public TaskSchedulerManagerForTest(AppContext appContext, this.defaultPayload = defaultPayload; } + TaskSchedulerManagerForTest(AppContext appContext, + EventHandler eventHandler, + TezAMRMClientAsync amrmClientAsync, + ContainerSignatureMatcher containerSignatureMatcher, + UserPayload defaultPayload, + List descriptors) { + super(appContext, null, eventHandler, containerSignatureMatcher, null, descriptors, + false, new HadoopShimsLoader(appContext.getAMConf()).getHadoopShim()); + this.amrmClientAsync = amrmClientAsync; + this.containerSignatureMatcher = containerSignatureMatcher; + this.defaultPayload = defaultPayload; + } + @SuppressWarnings("unchecked") @Override public void instantiateSchedulers(String host, int port, String trackingUrl, @@ -224,6 +237,10 @@ public Event verifyInvocation(Class eventClass) { fail("Expected Event: " + eventClass.getName() + " not sent"); return null; } + + public int getEventSize() { + return this.events.size(); + } } static class TaskSchedulerWithDrainableContext extends YarnTaskSchedulerService { diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerManager.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerManager.java index 095bd4e287..9d423c047c 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerManager.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestTaskSchedulerManager.java @@ -45,11 +45,17 @@ import java.util.List; import java.util.Set; import java.util.concurrent.Executors; +import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; +import com.google.common.base.Preconditions; +import com.google.common.base.Supplier; +import com.google.common.collect.BiMap; +import com.google.common.collect.HashBiMap; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.security.Credentials; +import org.apache.hadoop.util.Time; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.Container; import org.apache.hadoop.yarn.api.records.ContainerExitStatus; @@ -59,6 +65,8 @@ 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.client.api.AMRMClient; +import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.event.Event; import org.apache.hadoop.yarn.event.EventHandler; import org.apache.tez.common.ContainerSignatureMatcher; @@ -72,10 +80,16 @@ import org.apache.tez.dag.api.client.DAGClientServer; import org.apache.tez.dag.api.oldrecords.TaskAttemptState; import org.apache.tez.dag.app.AppContext; +import org.apache.tez.dag.app.ClusterInfo; import org.apache.tez.dag.app.ContainerContext; +import org.apache.tez.dag.app.ContainerHeartbeatHandler; +import org.apache.tez.dag.app.DAGAppMaster; +import org.apache.tez.dag.app.DAGAppMasterState; import org.apache.tez.dag.app.ServicePluginLifecycleAbstractService; +import org.apache.tez.dag.app.TaskCommunicatorManagerInterface; import org.apache.tez.dag.app.dag.DAG; import org.apache.tez.dag.app.dag.TaskAttempt; +import org.apache.tez.dag.app.dag.event.DAGAppMasterEventSchedulingServiceError; import org.apache.tez.dag.app.dag.event.DAGAppMasterEventType; import org.apache.tez.dag.app.dag.event.DAGAppMasterEventUserServiceFatalError; import org.apache.tez.dag.app.dag.event.DAGEventTerminateDag; @@ -89,6 +103,8 @@ import org.apache.tez.dag.app.rm.container.AMContainerEventType; import org.apache.tez.dag.app.rm.container.AMContainerMap; import org.apache.tez.dag.app.rm.container.AMContainerState; +import org.apache.tez.dag.app.rm.container.ContainerContextMatcher; +import org.apache.tez.dag.app.rm.node.AMNodeTracker; import org.apache.tez.dag.app.web.WebUIService; import org.apache.tez.dag.helpers.DagInfoImplForTest; import org.apache.tez.dag.records.TaskAttemptTerminationCause; @@ -839,6 +855,60 @@ protected void instantiateSchedulers(String host, int port, String trackingUrl, } } + @Test(timeout = 10000) + public void testHandleException() throws Exception { + Configuration tezConf = new Configuration(new YarnConfiguration()); + UserPayload defaultPayload = TezUtils.createUserPayloadFromConf(tezConf); + + // Parse plugins + List tsDescriptors = Lists.newLinkedList(); + BiMap tsMap = HashBiMap.create(); + DAGAppMaster.parseAllPlugins(tsDescriptors, tsMap, Lists.newLinkedList(), HashBiMap.create(), Lists.newLinkedList(), + HashBiMap.create(), null, false, defaultPayload); + + // Only TezYarn found. + Assert.assertEquals(1, tsDescriptors.size()); + Assert.assertEquals(TezConstants.getTezYarnServicePluginName(), tsDescriptors.get(0).getEntityName()); + + // Construct eventHandler + TestTaskSchedulerHelpers.CapturingEventHandler eventHandler = new TestTaskSchedulerHelpers.CapturingEventHandler(); + TezDAGID dagID = TezDAGID.getInstance("0", 0, 0); + + // Construct AMRMClient + AMRMClient rmClientCore = + new TestTaskSchedulerHelpers.AMRMClientForTest(); + TezAMRMClientAsync rmClient = + spy(new TestTaskSchedulerHelpers.AMRMClientAsyncForTest(rmClientCore, 100)); + + // Construct appContext + AppContext appContext = mock(AppContext.class); + doReturn(new Configuration(false)).when(appContext).getAMConf(); + AMContainerMap amContainerMap = new AMContainerMap(mock(ContainerHeartbeatHandler.class), + mock(TaskCommunicatorManagerInterface.class), new ContainerContextMatcher(), appContext); + AMNodeTracker amNodeTracker = new AMNodeTracker(eventHandler, appContext); + doReturn(amContainerMap).when(appContext).getAllContainers(); + doReturn(amNodeTracker).when(appContext).getNodeTracker(); + doReturn(DAGAppMasterState.RUNNING).when(appContext).getAMState(); + doReturn(dagID).when(appContext).getCurrentDAGID(); + doReturn(mock(ClusterInfo.class)).when(appContext).getClusterInfo(); + + // Construct TaskSchedulerManager + TaskSchedulerManager taskSchedulerManagerReal = + new TestTaskSchedulerHelpers.TaskSchedulerManagerForTest(appContext, eventHandler, rmClient, + new TestTaskSchedulerHelpers.AlwaysMatchesContainerMatcher(), defaultPayload, tsDescriptors); + TaskSchedulerManager taskSchedulerManager = spy(taskSchedulerManagerReal); + taskSchedulerManager.init(tezConf); + taskSchedulerManager.start(); + + // Send error to schedule, then expect DAGAppMasterEventSchedulingServiceError event. + YarnTaskSchedulerService scheduler = ((YarnTaskSchedulerService) taskSchedulerManager.getTaskScheduler(0)); + scheduler.onError(new Exception("Trigger by unit test")); + waitFor(() -> { + return eventHandler.getEventSize() > 0; + }, 1000, 5000); + eventHandler.verifyInvocation(DAGAppMasterEventSchedulingServiceError.class); + } + private static class ExceptionAnswer implements Answer { @Override public Object answer(InvocationOnMock invocation) throws Throwable { @@ -1107,4 +1177,23 @@ public boolean hasUnregistered() throws ServicePluginException { public void dagComplete() throws ServicePluginException { } } + + public static void waitFor(Supplier check, int checkEveryMillis, + int waitForMillis) throws TimeoutException, InterruptedException { + Preconditions.checkNotNull(check, "Input supplier interface should be initailized"); + Preconditions.checkArgument(waitForMillis >= checkEveryMillis, + "Total wait time should be greater than check interval time"); + + long st = Time.monotonicNow(); + boolean result = check.get(); + + while (!result && (Time.monotonicNow() - st < waitForMillis)) { + Thread.sleep(checkEveryMillis); + result = check.get(); + } + + if (!result) { + throw new TimeoutException("Timed out waiting for condition."); + } + } } From de88f2a2c5e8857360996c502f8827202d3e78ee Mon Sep 17 00:00:00 2001 From: Bodor Laszlo Date: Wed, 14 Sep 2022 12:44:25 +0200 Subject: [PATCH 130/137] TEZ-4447: Collect VertexStatus properly when DAGClientServer is not used (local mode without network) (#239) (Laszlo Bodor reviewed by Rajesh Balamohan) --- .../apache/tez/client/FrameworkClient.java | 7 +++- .../java/org/apache/tez/client/TezClient.java | 11 +----- .../tez/dag/api/client/DAGClientImpl.java | 8 +++- .../dag/api/client/DAGClientImplLocal.java | 11 +++++- .../org/apache/tez/client/LocalClient.java | 37 +++++++++++++------ .../org/apache/tez/client/MRTezClient.java | 6 --- .../tez/mapreduce/client/YARNRunner.java | 5 +-- .../library/processor/SleepProcessor.java | 1 + .../org/apache/tez/test/TestLocalMode.java | 23 ++++++++---- 9 files changed, 66 insertions(+), 43 deletions(-) diff --git a/tez-api/src/main/java/org/apache/tez/client/FrameworkClient.java b/tez-api/src/main/java/org/apache/tez/client/FrameworkClient.java index 2ec6d2864d..26c11fd084 100644 --- a/tez-api/src/main/java/org/apache/tez/client/FrameworkClient.java +++ b/tez-api/src/main/java/org/apache/tez/client/FrameworkClient.java @@ -145,7 +145,7 @@ public DAGClient submitDag(DAG dag, SubmitDAGRequestProto request, String client + ", applicationId=" + sessionAppId + ", dagId=" + dagId + ", dagName=" + dag.getName()); - return new DAGClientImpl(sessionAppId, dagId, tezConf, this, ugi); + return getDAGClient(sessionAppId, dagId, tezConf, ugi); } protected DAGClientAMProtocolBlockingPB waitForProxy(long clientTimeout, Configuration conf, @@ -186,4 +186,9 @@ protected DAGClientAMProtocolBlockingPB getProxy(Configuration conf, Application UserGroupInformation ugi) throws TezException, IOException { return TezClientUtils.getAMProxy(this, conf, sessionAppId, ugi); } + + public DAGClient getDAGClient(ApplicationId appId, String dagId, TezConfiguration tezConf, + UserGroupInformation ugi) { + return new DAGClientImpl(appId, dagId, tezConf, this, ugi); + } } diff --git a/tez-api/src/main/java/org/apache/tez/client/TezClient.java b/tez-api/src/main/java/org/apache/tez/client/TezClient.java index c37f0c181d..93807fdf46 100644 --- a/tez-api/src/main/java/org/apache/tez/client/TezClient.java +++ b/tez-api/src/main/java/org/apache/tez/client/TezClient.java @@ -76,7 +76,6 @@ import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolBlockingPB; import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.GetAMStatusRequestProto; import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.SubmitDAGRequestProto; -import org.apache.tez.dag.api.client.DAGClientImpl; import org.apache.tez.dag.api.records.DAGProtos.DAGPlan; import com.google.common.annotations.VisibleForTesting; @@ -1117,15 +1116,7 @@ private synchronized Map getTezJarResources(Credentials c @Private static DAGClient getDAGClient(ApplicationId appId, TezConfiguration tezConf, FrameworkClient frameworkClient, UserGroupInformation ugi) throws IOException, TezException { - return new DAGClientImpl(appId, getDefaultTezDAGID(appId), tezConf, frameworkClient, ugi); - } - - @Private // Used only for MapReduce compatibility code - static DAGClient getDAGClient(ApplicationId appId, TezConfiguration tezConf, - FrameworkClient frameworkClient) throws IOException, TezException { - UserGroupInformation ugi = - UserGroupInformation.createRemoteUser(UserGroupInformation.getCurrentUser().getUserName()); - return getDAGClient(appId, tezConf, frameworkClient, ugi); + return frameworkClient.getDAGClient(appId, getDefaultTezDAGID(appId), tezConf, ugi); } // DO NOT CHANGE THIS. This code is replicated from TezDAGID.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 bfea96b998..95dd85f388 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 @@ -296,9 +296,13 @@ public DAGStatus getDAGStatus(@Nullable Set statusOptions) throws } @Override - public VertexStatus getVertexStatus(String vertexName, Set statusOptions) throws - IOException, TezException { + public VertexStatus getVertexStatus(String vertexName, Set statusOptions) + throws IOException, TezException { + return getVertexStatusInternal(statusOptions, vertexName); + } + protected VertexStatus getVertexStatusInternal(Set statusOptions, String vertexName) + throws IOException, TezException { if (!dagCompleted) { VertexStatus vertexStatus = getVertexStatusViaAM(vertexName, statusOptions); diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClientImplLocal.java b/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClientImplLocal.java index a0509cdc5f..851bb687a1 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClientImplLocal.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClientImplLocal.java @@ -36,12 +36,15 @@ public class DAGClientImplLocal extends DAGClientImpl { private BiFunction, Long, DAGStatus> dagStatusFunction; + private BiFunction, String, VertexStatus> vertexStatusFunction; public DAGClientImplLocal(ApplicationId appId, String dagId, TezConfiguration conf, FrameworkClient frameworkClient, UserGroupInformation ugi, - BiFunction, Long, DAGStatus> dagStatusFunction) { + BiFunction, Long, DAGStatus> dagStatusFunction, + BiFunction, String, VertexStatus> vertexStatusFunction) { super(appId, dagId, conf, frameworkClient, ugi); this.dagStatusFunction = dagStatusFunction; + this.vertexStatusFunction = vertexStatusFunction; } @Override @@ -50,4 +53,10 @@ protected DAGStatus getDAGStatusInternal(@Nullable Set statusOpti return dagStatusFunction.apply(statusOptions == null ? new HashSet<>() : statusOptions, timeout); } + + @Override + protected VertexStatus getVertexStatusInternal(@Nullable Set statusOptions, String vertexName) + throws TezException, IOException { + return vertexStatusFunction.apply(statusOptions == null ? new HashSet<>() : statusOptions, vertexName); + } } diff --git a/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java b/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java index d0580bbae9..c9b3d7314b 100644 --- a/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java +++ b/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java @@ -60,6 +60,7 @@ import org.apache.tez.dag.api.client.DAGClientImplLocal; import org.apache.tez.dag.api.client.DAGStatus; import org.apache.tez.dag.api.client.StatusGetOpts; +import org.apache.tez.dag.api.client.VertexStatus; import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.SubmitDAGRequestProto; import org.apache.tez.dag.api.records.DAGProtos.AMPluginDescriptorProto; import org.apache.tez.dag.app.AppContext; @@ -426,20 +427,32 @@ public DAGClient submitDag(org.apache.tez.dag.api.DAG dag, SubmitDAGRequestProto } String dagId = dagAppMaster.submitDAGToAppMaster(request.getDAGPlan(), additionalResources); + return getDAGClient(sessionAppId, dagId, tezConf, ugi); + } + @Override + public DAGClient getDAGClient(ApplicationId appId, String dagId, TezConfiguration tezConf, + UserGroupInformation ugi) { return isLocalWithoutNetwork - ? new DAGClientImplLocal(sessionAppId, dagId, tezConf, this, - ugi, new BiFunction, Long, DAGStatus>() { - @Override - public DAGStatus apply(Set statusOpts, Long timeout) { - try { - return clientHandler.getDAGStatus(dagId, statusOpts, timeout); - } catch (TezException e) { - throw new RuntimeException(e); - } - } - }) - : new DAGClientImpl(sessionAppId, dagId, tezConf, this, ugi); + ? new DAGClientImplLocal(appId, dagId, tezConf, this, ugi, new BiFunction, Long, DAGStatus>() { + @Override + public DAGStatus apply(Set statusOpts, Long timeout) { + try { + return clientHandler.getDAGStatus(dagId, statusOpts, timeout); + } catch (TezException e) { + throw new RuntimeException(e); + } + } + }, new BiFunction, String, VertexStatus>() { + @Override + public VertexStatus apply(Set statusOpts, String vertexName) { + try { + return clientHandler.getVertexStatus(dagId, vertexName, statusOpts); + } catch (TezException e) { + throw new RuntimeException(e); + } + } + }) : new DAGClientImpl(appId, dagId, tezConf, this, ugi); } @Override diff --git a/tez-mapreduce/src/main/java/org/apache/tez/client/MRTezClient.java b/tez-mapreduce/src/main/java/org/apache/tez/client/MRTezClient.java index 86089e9a67..1057932e1d 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/client/MRTezClient.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/client/MRTezClient.java @@ -31,7 +31,6 @@ import org.apache.tez.dag.api.TezConfiguration; import org.apache.tez.dag.api.TezException; import org.apache.tez.dag.api.client.DAGClient; -import org.apache.tez.dag.api.client.MRDAGClient; @InterfaceAudience.Private public class MRTezClient extends TezClient { @@ -46,9 +45,4 @@ public DAGClient submitDAGApplication(ApplicationId appId, org.apache.tez.dag.ap throws TezException, IOException { return super.submitDAGApplication(appId, dag); } - - public static MRDAGClient getDAGClient(ApplicationId appId, TezConfiguration tezConf, FrameworkClient frameworkClient) - throws IOException, TezException { - return new MRDAGClient(TezClient.getDAGClient(appId, tezConf, frameworkClient)); - } } diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/client/YARNRunner.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/client/YARNRunner.java index 9dba357951..7aed4a04a8 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/client/YARNRunner.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/client/YARNRunner.java @@ -639,7 +639,7 @@ public JobStatus submitJob(JobID jobId, String jobSubmitDir, Credentials ts) tezClient = new MRTezClient("MapReduce", dagAMConf, false, jobLocalResources, ts); tezClient.start(); - tezClient.submitDAGApplication(appId, dag); + dagClient = new MRDAGClient(tezClient.submitDAGApplication(appId, dag)); tezClient.stop(); } catch (TezException e) { throw new IOException(e); @@ -702,9 +702,6 @@ public JobStatus getJobStatus(JobID jobID) throws IOException, String jobFile = MRApps.getJobFile(conf, user, jobID); DAGStatus dagStatus; try { - if(dagClient == null) { - dagClient = MRTezClient.getDAGClient(TypeConverter.toYarn(jobID).getAppId(), tezConf, null); - } dagStatus = dagClient.getDAGStatus(null); return new DAGJobStatus(dagClient.getApplicationReport(), dagStatus, jobFile); } catch (TezException e) { 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 3efcd21c92..7750a13a63 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 @@ -46,6 +46,7 @@ public class SleepProcessor extends AbstractLogicalIOProcessor { private static final Logger LOG = LoggerFactory.getLogger(SleepProcessor.class); + public static final String SLEEP_VERTEX_NAME = "Sleep"; private int timeToSleepMS; protected Map inputs; diff --git a/tez-tests/src/test/java/org/apache/tez/test/TestLocalMode.java b/tez-tests/src/test/java/org/apache/tez/test/TestLocalMode.java index bdb71ad177..00125fd547 100644 --- a/tez-tests/src/test/java/org/apache/tez/test/TestLocalMode.java +++ b/tez-tests/src/test/java/org/apache/tez/test/TestLocalMode.java @@ -39,6 +39,7 @@ 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.api.client.VertexStatus; import org.apache.tez.examples.OrderedWordCount; import org.apache.tez.runtime.api.AbstractLogicalIOProcessor; import org.apache.tez.runtime.api.Event; @@ -131,6 +132,8 @@ public void testMultipleClientsWithSession() throws TezException, InterruptedExc DAGClient dagClient1 = tezClient1.submitDAG(dag1); dagClient1.waitForCompletion(); assertEquals(DAGStatus.State.SUCCEEDED, dagClient1.getDAGStatus(null).getState()); + assertEquals(VertexStatus.State.SUCCEEDED, + dagClient1.getVertexStatus(SleepProcessor.SLEEP_VERTEX_NAME, null).getState()); dagClient1.close(); tezClient1.stop(); @@ -142,6 +145,8 @@ public void testMultipleClientsWithSession() throws TezException, InterruptedExc DAGClient dagClient2 = tezClient2.submitDAG(dag2); dagClient2.waitForCompletion(); assertEquals(DAGStatus.State.SUCCEEDED, dagClient2.getDAGStatus(null).getState()); + assertEquals(VertexStatus.State.SUCCEEDED, + dagClient2.getVertexStatus(SleepProcessor.SLEEP_VERTEX_NAME, null).getState()); assertFalse(dagClient1.getExecutionContext().equals(dagClient2.getExecutionContext())); dagClient2.close(); tezClient2.stop(); @@ -159,7 +164,8 @@ public void testMultipleClientsWithoutSession() throws TezException, Interrupted DAGClient dagClient1 = tezClient1.submitDAG(dag1); dagClient1.waitForCompletion(); assertEquals(DAGStatus.State.SUCCEEDED, dagClient1.getDAGStatus(null).getState()); - + assertEquals(VertexStatus.State.SUCCEEDED, + dagClient1.getVertexStatus(SleepProcessor.SLEEP_VERTEX_NAME, null).getState()); dagClient1.close(); tezClient1.stop(); @@ -171,6 +177,8 @@ public void testMultipleClientsWithoutSession() throws TezException, Interrupted DAGClient dagClient2 = tezClient2.submitDAG(dag2); dagClient2.waitForCompletion(); assertEquals(DAGStatus.State.SUCCEEDED, dagClient2.getDAGStatus(null).getState()); + assertEquals(VertexStatus.State.SUCCEEDED, + dagClient2.getVertexStatus(SleepProcessor.SLEEP_VERTEX_NAME, null).getState()); assertFalse(dagClient1.getExecutionContext().equals(dagClient2.getExecutionContext())); dagClient2.close(); tezClient2.stop(); @@ -189,7 +197,8 @@ public void testNoSysExitOnSuccessfulDAG() throws TezException, InterruptedExcep DAGClient dagClient1 = tezClient1.submitDAG(dag1); dagClient1.waitForCompletion(); assertEquals(DAGStatus.State.SUCCEEDED, dagClient1.getDAGStatus(null).getState()); - + assertEquals(VertexStatus.State.SUCCEEDED, + dagClient1.getVertexStatus(SleepProcessor.SLEEP_VERTEX_NAME, null).getState()); // Sleep for more time than is required for the DAG to complete. Thread.sleep((long) (TezConstants.TEZ_DAG_SLEEP_TIME_BEFORE_EXIT * 1.5)); @@ -210,7 +219,8 @@ public void testNoSysExitOnFailinglDAG() throws TezException, InterruptedExcepti DAGClient dagClient1 = tezClient1.submitDAG(dag1); dagClient1.waitForCompletion(); assertEquals(DAGStatus.State.FAILED, dagClient1.getDAGStatus(null).getState()); - + assertEquals(VertexStatus.State.FAILED, + dagClient1.getVertexStatus(SleepProcessor.SLEEP_VERTEX_NAME, null).getState()); // Sleep for more time than is required for the DAG to complete. Thread.sleep((long) (TezConstants.TEZ_DAG_SLEEP_TIME_BEFORE_EXIT * 1.5)); @@ -245,12 +255,11 @@ public void run(Map inputs, Map out } private DAG createSimpleDAG(String dagName, String processorName) { - DAG dag = DAG.create(dagName).addVertex(Vertex.create("Sleep", ProcessorDescriptor.create( - processorName).setUserPayload( - new SleepProcessor.SleepProcessorConfig(1).toUserPayload()), 1)); + DAG dag = DAG.create(dagName).addVertex(Vertex.create(SleepProcessor.SLEEP_VERTEX_NAME, ProcessorDescriptor + .create(processorName).setUserPayload(new SleepProcessor.SleepProcessorConfig(1).toUserPayload()), 1)); return dag; - } + @Test(timeout=30000) public void testMultiDAGsOnSession() throws IOException, TezException, InterruptedException { int dags = 2;//two dags will be submitted to session From 921de531f1f0a5b275844d693cb1b528595020ab Mon Sep 17 00:00:00 2001 From: Bodor Laszlo Date: Mon, 10 Oct 2022 12:20:03 +0200 Subject: [PATCH 131/137] TEZ-4431: Apache Tez Release 0.10.2: update SNAPSHOT versions, DOAP file (#234) (Laszlo Bodor reviewed by Rajesh Balamohan, Ayush Saxena) --- Tez_DOAP.rdf | 7 +++++++ docs/pom.xml | 2 +- hadoop-shim-impls/hadoop-shim-2.7/pom.xml | 2 +- hadoop-shim-impls/hadoop-shim-2.8/pom.xml | 2 +- hadoop-shim-impls/pom.xml | 2 +- hadoop-shim/pom.xml | 2 +- pom.xml | 2 +- tez-api/pom.xml | 2 +- tez-build-tools/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-aux-services/pom.xml | 2 +- tez-plugins/tez-history-parser/pom.xml | 2 +- tez-plugins/tez-protobuf-history-plugin/pom.xml | 2 +- tez-plugins/tez-yarn-timeline-cache-plugin/pom.xml | 2 +- tez-plugins/tez-yarn-timeline-history-with-acls/pom.xml | 2 +- tez-plugins/tez-yarn-timeline-history-with-fs/pom.xml | 2 +- tez-plugins/tez-yarn-timeline-history/pom.xml | 2 +- tez-runtime-internals/pom.xml | 2 +- tez-runtime-library/pom.xml | 2 +- tez-tests/pom.xml | 2 +- tez-tools/analyzers/job-analyzer/pom.xml | 2 +- tez-tools/analyzers/pom.xml | 2 +- tez-tools/pom.xml | 2 +- tez-tools/tez-javadoc-tools/pom.xml | 2 +- tez-tools/tez-tfile-parser/pom.xml | 2 +- tez-ui/pom.xml | 2 +- 32 files changed, 38 insertions(+), 31 deletions(-) diff --git a/Tez_DOAP.rdf b/Tez_DOAP.rdf index d4c473d493..94cae144d8 100644 --- a/Tez_DOAP.rdf +++ b/Tez_DOAP.rdf @@ -34,6 +34,13 @@ Java + + + Version 0.10.2 + 2022-07-30 + 0.10.2 + + Version 0.10.1 diff --git a/docs/pom.xml b/docs/pom.xml index 0d55177641..d82f003c41 100644 --- a/docs/pom.xml +++ b/docs/pom.xml @@ -27,7 +27,7 @@ org.apache.tez tez - 0.10.2-SNAPSHOT + 0.10.3-SNAPSHOT tez-docs pom diff --git a/hadoop-shim-impls/hadoop-shim-2.7/pom.xml b/hadoop-shim-impls/hadoop-shim-2.7/pom.xml index 2557e036c3..caccbdca5e 100644 --- a/hadoop-shim-impls/hadoop-shim-2.7/pom.xml +++ b/hadoop-shim-impls/hadoop-shim-2.7/pom.xml @@ -19,7 +19,7 @@ hadoop-shim-impls org.apache.tez - 0.10.2-SNAPSHOT + 0.10.3-SNAPSHOT 4.0.0 hadoop-shim-2.7 diff --git a/hadoop-shim-impls/hadoop-shim-2.8/pom.xml b/hadoop-shim-impls/hadoop-shim-2.8/pom.xml index 0a149cca04..4fdf57ff85 100644 --- a/hadoop-shim-impls/hadoop-shim-2.8/pom.xml +++ b/hadoop-shim-impls/hadoop-shim-2.8/pom.xml @@ -19,7 +19,7 @@ hadoop-shim-impls org.apache.tez - 0.10.2-SNAPSHOT + 0.10.3-SNAPSHOT 4.0.0 hadoop-shim-2.8 diff --git a/hadoop-shim-impls/pom.xml b/hadoop-shim-impls/pom.xml index 25cf3fd25a..83becc56f7 100644 --- a/hadoop-shim-impls/pom.xml +++ b/hadoop-shim-impls/pom.xml @@ -20,7 +20,7 @@ tez org.apache.tez - 0.10.2-SNAPSHOT + 0.10.3-SNAPSHOT hadoop-shim-impls pom diff --git a/hadoop-shim/pom.xml b/hadoop-shim/pom.xml index 18ddaca3df..15c86e52ed 100644 --- a/hadoop-shim/pom.xml +++ b/hadoop-shim/pom.xml @@ -20,7 +20,7 @@ tez org.apache.tez - 0.10.2-SNAPSHOT + 0.10.3-SNAPSHOT hadoop-shim diff --git a/pom.xml b/pom.xml index 3ec471b9c0..899dd0c103 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ org.apache.tez tez pom - 0.10.2-SNAPSHOT + 0.10.3-SNAPSHOT tez diff --git a/tez-api/pom.xml b/tez-api/pom.xml index 1d10bc1f85..7cabdf1160 100644 --- a/tez-api/pom.xml +++ b/tez-api/pom.xml @@ -20,7 +20,7 @@ org.apache.tez tez - 0.10.2-SNAPSHOT + 0.10.3-SNAPSHOT tez-api diff --git a/tez-build-tools/pom.xml b/tez-build-tools/pom.xml index 232a20be89..ae05507e63 100644 --- a/tez-build-tools/pom.xml +++ b/tez-build-tools/pom.xml @@ -20,7 +20,7 @@ org.apache.tez tez - 0.10.2-SNAPSHOT + 0.10.3-SNAPSHOT tez-build-tools diff --git a/tez-common/pom.xml b/tez-common/pom.xml index 9ccf02a38c..ecd4f7b4b7 100644 --- a/tez-common/pom.xml +++ b/tez-common/pom.xml @@ -20,7 +20,7 @@ org.apache.tez tez - 0.10.2-SNAPSHOT + 0.10.3-SNAPSHOT tez-common diff --git a/tez-dag/pom.xml b/tez-dag/pom.xml index fda9fb2a54..c2baf0c0d2 100644 --- a/tez-dag/pom.xml +++ b/tez-dag/pom.xml @@ -19,7 +19,7 @@ org.apache.tez tez - 0.10.2-SNAPSHOT + 0.10.3-SNAPSHOT tez-dag diff --git a/tez-dist/pom.xml b/tez-dist/pom.xml index 741ed9d1e9..ccc6c7589c 100644 --- a/tez-dist/pom.xml +++ b/tez-dist/pom.xml @@ -21,7 +21,7 @@ org.apache.tez tez - 0.10.2-SNAPSHOT + 0.10.3-SNAPSHOT tez-dist diff --git a/tez-examples/pom.xml b/tez-examples/pom.xml index f6ab04c020..e9040bec5a 100644 --- a/tez-examples/pom.xml +++ b/tez-examples/pom.xml @@ -20,7 +20,7 @@ org.apache.tez tez - 0.10.2-SNAPSHOT + 0.10.3-SNAPSHOT tez-examples diff --git a/tez-ext-service-tests/pom.xml b/tez-ext-service-tests/pom.xml index a0bcbe32c0..ddf5392fa4 100644 --- a/tez-ext-service-tests/pom.xml +++ b/tez-ext-service-tests/pom.xml @@ -20,7 +20,7 @@ tez org.apache.tez - 0.10.2-SNAPSHOT + 0.10.3-SNAPSHOT tez-ext-service-tests diff --git a/tez-mapreduce/pom.xml b/tez-mapreduce/pom.xml index 1e35d2ea11..bcebf81d03 100644 --- a/tez-mapreduce/pom.xml +++ b/tez-mapreduce/pom.xml @@ -20,7 +20,7 @@ org.apache.tez tez - 0.10.2-SNAPSHOT + 0.10.3-SNAPSHOT tez-mapreduce diff --git a/tez-plugins/pom.xml b/tez-plugins/pom.xml index 27f19289ce..8bec3bbdb6 100644 --- a/tez-plugins/pom.xml +++ b/tez-plugins/pom.xml @@ -21,7 +21,7 @@ org.apache.tez tez - 0.10.2-SNAPSHOT + 0.10.3-SNAPSHOT tez-plugins pom diff --git a/tez-plugins/tez-aux-services/pom.xml b/tez-plugins/tez-aux-services/pom.xml index 8b4ab74102..32a0f2de9c 100644 --- a/tez-plugins/tez-aux-services/pom.xml +++ b/tez-plugins/tez-aux-services/pom.xml @@ -20,7 +20,7 @@ tez-plugins org.apache.tez - 0.10.2-SNAPSHOT + 0.10.3-SNAPSHOT tez-aux-services diff --git a/tez-plugins/tez-history-parser/pom.xml b/tez-plugins/tez-history-parser/pom.xml index f0758dcd15..605daca345 100644 --- a/tez-plugins/tez-history-parser/pom.xml +++ b/tez-plugins/tez-history-parser/pom.xml @@ -21,7 +21,7 @@ org.apache.tez tez-plugins - 0.10.2-SNAPSHOT + 0.10.3-SNAPSHOT tez-history-parser diff --git a/tez-plugins/tez-protobuf-history-plugin/pom.xml b/tez-plugins/tez-protobuf-history-plugin/pom.xml index 0fa526413e..98678fd53e 100644 --- a/tez-plugins/tez-protobuf-history-plugin/pom.xml +++ b/tez-plugins/tez-protobuf-history-plugin/pom.xml @@ -19,7 +19,7 @@ org.apache.tez tez-plugins - 0.10.2-SNAPSHOT + 0.10.3-SNAPSHOT tez-protobuf-history-plugin diff --git a/tez-plugins/tez-yarn-timeline-cache-plugin/pom.xml b/tez-plugins/tez-yarn-timeline-cache-plugin/pom.xml index 942d43658a..a362e6bd4a 100644 --- a/tez-plugins/tez-yarn-timeline-cache-plugin/pom.xml +++ b/tez-plugins/tez-yarn-timeline-cache-plugin/pom.xml @@ -19,7 +19,7 @@ org.apache.tez tez-plugins - 0.10.2-SNAPSHOT + 0.10.3-SNAPSHOT tez-yarn-timeline-cache-plugin 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 0b61b7d958..5aedd28373 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 @@ org.apache.tez tez-plugins - 0.10.2-SNAPSHOT + 0.10.3-SNAPSHOT tez-yarn-timeline-history-with-acls diff --git a/tez-plugins/tez-yarn-timeline-history-with-fs/pom.xml b/tez-plugins/tez-yarn-timeline-history-with-fs/pom.xml index 75df73f3a6..8811e06011 100644 --- a/tez-plugins/tez-yarn-timeline-history-with-fs/pom.xml +++ b/tez-plugins/tez-yarn-timeline-history-with-fs/pom.xml @@ -19,7 +19,7 @@ org.apache.tez tez-plugins - 0.10.2-SNAPSHOT + 0.10.3-SNAPSHOT tez-yarn-timeline-history-with-fs diff --git a/tez-plugins/tez-yarn-timeline-history/pom.xml b/tez-plugins/tez-yarn-timeline-history/pom.xml index 3f17e97327..059b52b494 100644 --- a/tez-plugins/tez-yarn-timeline-history/pom.xml +++ b/tez-plugins/tez-yarn-timeline-history/pom.xml @@ -19,7 +19,7 @@ org.apache.tez tez-plugins - 0.10.2-SNAPSHOT + 0.10.3-SNAPSHOT tez-yarn-timeline-history diff --git a/tez-runtime-internals/pom.xml b/tez-runtime-internals/pom.xml index efe01e058e..66090c5764 100644 --- a/tez-runtime-internals/pom.xml +++ b/tez-runtime-internals/pom.xml @@ -20,7 +20,7 @@ org.apache.tez tez - 0.10.2-SNAPSHOT + 0.10.3-SNAPSHOT tez-runtime-internals diff --git a/tez-runtime-library/pom.xml b/tez-runtime-library/pom.xml index e6db47e535..f54b26b76a 100644 --- a/tez-runtime-library/pom.xml +++ b/tez-runtime-library/pom.xml @@ -20,7 +20,7 @@ org.apache.tez tez - 0.10.2-SNAPSHOT + 0.10.3-SNAPSHOT tez-runtime-library diff --git a/tez-tests/pom.xml b/tez-tests/pom.xml index 72f95d56ce..abf8674286 100644 --- a/tez-tests/pom.xml +++ b/tez-tests/pom.xml @@ -20,7 +20,7 @@ org.apache.tez tez - 0.10.2-SNAPSHOT + 0.10.3-SNAPSHOT tez-tests diff --git a/tez-tools/analyzers/job-analyzer/pom.xml b/tez-tools/analyzers/job-analyzer/pom.xml index 0cc85834cb..077de5cfb1 100644 --- a/tez-tools/analyzers/job-analyzer/pom.xml +++ b/tez-tools/analyzers/job-analyzer/pom.xml @@ -20,7 +20,7 @@ org.apache.tez tez-perf-analyzer - 0.10.2-SNAPSHOT + 0.10.3-SNAPSHOT tez-job-analyzer diff --git a/tez-tools/analyzers/pom.xml b/tez-tools/analyzers/pom.xml index 3c989e68ec..72b5aba3ae 100644 --- a/tez-tools/analyzers/pom.xml +++ b/tez-tools/analyzers/pom.xml @@ -20,7 +20,7 @@ org.apache.tez tez-tools - 0.10.2-SNAPSHOT + 0.10.3-SNAPSHOT tez-perf-analyzer pom diff --git a/tez-tools/pom.xml b/tez-tools/pom.xml index bffe9abc5d..f9987b3f97 100644 --- a/tez-tools/pom.xml +++ b/tez-tools/pom.xml @@ -21,7 +21,7 @@ org.apache.tez tez - 0.10.2-SNAPSHOT + 0.10.3-SNAPSHOT tez-tools pom diff --git a/tez-tools/tez-javadoc-tools/pom.xml b/tez-tools/tez-javadoc-tools/pom.xml index 6b0546674b..8106872c34 100644 --- a/tez-tools/tez-javadoc-tools/pom.xml +++ b/tez-tools/tez-javadoc-tools/pom.xml @@ -20,7 +20,7 @@ org.apache.tez tez-tools - 0.10.2-SNAPSHOT + 0.10.3-SNAPSHOT tez-javadoc-tools diff --git a/tez-tools/tez-tfile-parser/pom.xml b/tez-tools/tez-tfile-parser/pom.xml index a258a5e92b..281cbe9990 100644 --- a/tez-tools/tez-tfile-parser/pom.xml +++ b/tez-tools/tez-tfile-parser/pom.xml @@ -20,7 +20,7 @@ org.apache.tez tez-tools - 0.10.2-SNAPSHOT + 0.10.3-SNAPSHOT tez-tfile-parser diff --git a/tez-ui/pom.xml b/tez-ui/pom.xml index 59a02a2ab0..5b90d6d0fb 100644 --- a/tez-ui/pom.xml +++ b/tez-ui/pom.xml @@ -21,7 +21,7 @@ org.apache.tez tez - 0.10.2-SNAPSHOT + 0.10.3-SNAPSHOT tez-ui war From 40e9e0f6ab4201011ce8beb27eb80c8634c28f0b Mon Sep 17 00:00:00 2001 From: slfan1989 <55643692+slfan1989@users.noreply.github.com> Date: Sun, 23 Oct 2022 15:48:34 +0800 Subject: [PATCH 132/137] TEZ-4449: Upgrade jettison to 1.5.1 to fix CVE-2022-40149. (#242) (fanshilun reviewed by Laszlo Bodor) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 899dd0c103..5e03d2af85 100644 --- a/pom.xml +++ b/pom.xml @@ -738,7 +738,7 @@ org.codehaus.jettison jettison - 1.3.4 + 1.5.1 com.google.code.findbugs From ddb53cb014b68cb5935daedba06938a676063b7c Mon Sep 17 00:00:00 2001 From: Karel Kolman Date: Sun, 23 Oct 2022 10:41:27 +0200 Subject: [PATCH 133/137] TEZ-4448: Cannot submit Tez job when dag size is exceeds `ipc.maximum.data.length` and S3A is the filesystem (#241) (Karel Kolman reviewed by Laszlo Bodor) --- tez-api/src/main/java/org/apache/tez/client/TezClient.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tez-api/src/main/java/org/apache/tez/client/TezClient.java b/tez-api/src/main/java/org/apache/tez/client/TezClient.java index 93807fdf46..d54484927e 100644 --- a/tez-api/src/main/java/org/apache/tez/client/TezClient.java +++ b/tez-api/src/main/java/org/apache/tez/client/TezClient.java @@ -695,8 +695,8 @@ private DAGClient submitDAGSession(DAG dag) throws TezException, IOException { + ", max IPC message size= " + amConfig.getTezConfiguration().getInt( CommonConfigurationKeys.IPC_MAXIMUM_DATA_LENGTH, CommonConfigurationKeys.IPC_MAXIMUM_DATA_LENGTH_DEFAULT)); request.writeTo(fsDataOutputStream); - request = requestBuilder.clear().setSerializedRequestPath(fs.resolvePath(dagPlanPath).toString()).build(); } + request = requestBuilder.clear().setSerializedRequestPath(fs.resolvePath(dagPlanPath).toString()).build(); } return frameworkClient.submitDag(dag, request, clientName, sessionAppId, clientTimeout, From fc5b6e9df08cc860037eeacba8c57c909b570e64 Mon Sep 17 00:00:00 2001 From: slfan1989 <55643692+slfan1989@users.noreply.github.com> Date: Mon, 24 Oct 2022 17:12:00 +0800 Subject: [PATCH 134/137] TEZ-4453: Fix Typo of LocalClient. (#245) (slfan1989 reviewed by Laszlo Bodor) --- tez-dag/src/main/java/org/apache/tez/client/LocalClient.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java b/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java index c9b3d7314b..f6a626e64c 100644 --- a/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java +++ b/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java @@ -266,7 +266,7 @@ protected void startDAGAppMaster(final ApplicationSubmissionContext appContext) if (dagAMState.equals(DAGAppMasterState.NEW)) { LOG.info("DAGAppMaster is not started wait for 100ms..."); } else if (dagAMState.equals(DAGAppMasterState.INITED)) { - LOG.info("DAGAppMaster is not startetd wait for 100ms..."); + LOG.info("DAGAppMaster is not started wait for 100ms..."); } else if (dagAMState.equals(DAGAppMasterState.ERROR)) { throw new TezException("DAGAppMaster got an error during initialization"); } else if (dagAMState.equals(DAGAppMasterState.KILLED)) { From dbbd406aed3434e8866d1e1a64fce4ad9b3bf5a9 Mon Sep 17 00:00:00 2001 From: Bodor Laszlo Date: Wed, 26 Oct 2022 06:27:53 +0200 Subject: [PATCH 135/137] TEZ-4039: Tez should inject dag id, query id into MDC (#98) (Laszlo Bodor reviewed by Rajesh Balamohan, Prasanth Jayachandran) --- .../apache/tez/dag/api/TezConfiguration.java | 47 ++++++ .../apache/tez/common/TezUtilsInternal.java | 19 ++- .../org/apache/tez/util/LoggingUtils.java | 151 ++++++++++++++++++ .../org/apache/tez/dag/app/DAGAppMaster.java | 23 ++- .../org/apache/tez/runtime/task/TezChild.java | 25 ++- .../tez/runtime/task/TezTaskRunner2.java | 17 +- .../tez/runtime/task/TestTezTaskRunner2.java | 14 +- .../java/org/apache/tez/test/TestTezJobs.java | 57 ++++++- 8 files changed, 326 insertions(+), 27 deletions(-) create mode 100644 tez-common/src/main/java/org/apache/tez/util/LoggingUtils.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 1ffd70a3ff..cd6d02249e 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 @@ -260,6 +260,53 @@ public TezConfiguration(boolean loadDefaults) { public static final String TEZ_TASK_LOG_LEVEL = TEZ_TASK_PREFIX + "log.level"; public static final String TEZ_TASK_LOG_LEVEL_DEFAULT = "INFO"; + /** + * By this option, user can easily override the logging pattern which is applied in + * TezContainerLogAppender in AM, regardless of the environmental settings. + */ + @ConfigurationScope(Scope.AM) + @ConfigurationProperty + public static final String TEZ_LOG_PATTERN_LAYOUT_AM = TEZ_AM_PREFIX + "log.pattern.layout"; + + /** + * By this option, user can easily override the logging pattern which is applied in + * TezContainerLogAppender in tasks, regardless of the environmental settings. + */ + @ConfigurationScope(Scope.VERTEX) + @ConfigurationProperty + public static final String TEZ_LOG_PATTERN_LAYOUT_TASK = TEZ_TASK_PREFIX + "log.pattern.layout"; + + /** + * Set pattern to empty string to turn the custom log pattern feature off. + */ + public static final String TEZ_LOG_PATTERN_LAYOUT_DEFAULT = ""; + + /** + * Comma separated list of keys, which can used for defining keys in MDC. The corresponding values + * will be read from Configuration, see tez.mdc.custom.keys.conf.props for further details. + */ + @ConfigurationScope(Scope.AM) + @ConfigurationProperty + public static final String TEZ_MDC_CUSTOM_KEYS = TEZ_PREFIX + "mdc.custom.keys"; + public static final String TEZ_MDC_CUSTOM_KEYS_DEFAULT = ""; + + /** + * Comma separated list of Configuration keys. Tez will try to fill MDC with key value pairs in a + * way that a key will be the nth item in tez.mdc.custom.keys and the value will be the value from + * a Configuration object pointed by the nth key of tez.mdc.custom.keys.conf.props like below: + * + * tez.mdc.custom.keys=queryId,otherKey + * tez.mdc.custom.keys.conf.props=awesome.sql.app.query.id,awesome.sql.app.other.key + * + * So MDC will contain key -{@literal >} value pairs as: + * queryId -{@literal >} conf.get("awesome.sql.app.query.id") + * otherKey -{@literal >} conf.get("awesome.sql.app.other.key") + */ + @ConfigurationScope(Scope.AM) + @ConfigurationProperty + public static final String TEZ_MDC_CUSTOM_KEYS_CONF_PROPS = TEZ_MDC_CUSTOM_KEYS + ".conf.props"; + public static final String TEZ_MDC_CUSTOM_KEYS_CONF_PROPS_DEFAULT = ""; + /** * double value. Represents ratio of unique failed outputs / number of consumer * tasks. When this condition or value mentioned in {@link 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 c2efb29cb6..7ee5bb457e 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 @@ -47,6 +47,7 @@ import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.log4j.Appender; +import org.apache.log4j.PatternLayout; import org.apache.tez.common.io.NonSyncByteArrayOutputStream; import org.apache.tez.dag.api.DagTypeConverters; import org.apache.tez.dag.records.TezDAGID; @@ -157,17 +158,25 @@ private static String sanitizeString(String srcString) { return res; // Number starts allowed rightnow } - public static void updateLoggers(String addend) throws FileNotFoundException { + public static void updateLoggers(Configuration configuration, String addend, String patternString) + throws FileNotFoundException { LOG.info("Redirecting log file based on addend: " + addend); - Appender appender = org.apache.log4j.Logger.getRootLogger().getAppender( - TezConstants.TEZ_CONTAINER_LOGGER_NAME); + Appender appender = + org.apache.log4j.Logger.getRootLogger().getAppender(TezConstants.TEZ_CONTAINER_LOGGER_NAME); if (appender != null) { if (appender instanceof TezContainerLogAppender) { TezContainerLogAppender claAppender = (TezContainerLogAppender) appender; - claAppender.setLogFileName(constructLogFileName( - TezConstants.TEZ_CONTAINER_LOG_FILE_NAME, addend)); + claAppender + .setLogFileName(constructLogFileName(TezConstants.TEZ_CONTAINER_LOG_FILE_NAME, addend)); + + // there was a configured pattern + if (patternString != null) { + PatternLayout layout = (PatternLayout) claAppender.getLayout(); + layout.setConversionPattern(patternString); + } + claAppender.activateOptions(); } else { LOG.warn("Appender is a " + appender.getClass() + "; require an instance of " diff --git a/tez-common/src/main/java/org/apache/tez/util/LoggingUtils.java b/tez-common/src/main/java/org/apache/tez/util/LoggingUtils.java new file mode 100644 index 0000000000..e09b6b0964 --- /dev/null +++ b/tez-common/src/main/java/org/apache/tez/util/LoggingUtils.java @@ -0,0 +1,151 @@ +/** +* Licensed to the Apache Software Foundation (ASF) under one +* or more contributor license agreements. See the NOTICE file +* distributed with this work for additional information +* regarding copyright ownership. The ASF licenses this file +* to you under the Apache License, Version 2.0 (the +* "License"); you may not use this file except in compliance +* with the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, software +* distributed under the License is distributed on an "AS IS" BASIS, +* WITHOUT WARRANTIES OR CONDITIONS OF 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.lang.reflect.Constructor; +import java.lang.reflect.Field; +import java.lang.reflect.Modifier; +import java.util.Arrays; +import java.util.Hashtable; + +import org.apache.hadoop.conf.Configuration; +import org.apache.log4j.helpers.ThreadLocalMap; +import org.apache.tez.dag.api.TezConfiguration; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public final class LoggingUtils { + private static final Logger LOG = LoggerFactory.getLogger(LoggingUtils.class); + + private LoggingUtils() {} + + @SuppressWarnings("unchecked") + public static void initLoggingContext(ThreadLocalMap threadLocalMap, Configuration conf, + String dagId, String taskAttemptId) { + Hashtable data = (Hashtable) threadLocalMap.get(); + if (data == null) { + data = new NonClonableHashtable(); + threadLocalMap.set(data); + } + data.put("dagId", dagId == null ? "" : dagId); + data.put("taskAttemptId", taskAttemptId == null ? "" : taskAttemptId); + + String[] mdcKeys = conf.getStrings(TezConfiguration.TEZ_MDC_CUSTOM_KEYS, + TezConfiguration.TEZ_MDC_CUSTOM_KEYS_DEFAULT); + + if (mdcKeys == null || mdcKeys.length == 0) { + return; + } + + String[] mdcKeysValuesFrom = conf.getStrings(TezConfiguration.TEZ_MDC_CUSTOM_KEYS_CONF_PROPS, + TezConfiguration.TEZ_MDC_CUSTOM_KEYS_CONF_PROPS_DEFAULT); + LOG.info("MDC_LOGGING: setting up MDC keys: keys: {} / conf: {}", Arrays.asList(mdcKeys), + Arrays.asList(mdcKeysValuesFrom)); + + int i = 0; + for (String mdcKey : mdcKeys) { + // don't want to fail on incorrect mdc key settings, but warn in app logs + if (mdcKey.isEmpty() || mdcKeysValuesFrom.length < i + 1) { + LOG.warn("cannot set mdc key: {}", mdcKey); + break; + } + + String mdcValue = mdcKeysValuesFrom[i] == null ? "" : conf.get(mdcKeysValuesFrom[i]); + // MDC is backed by a Hashtable, let's prevent NPE because of null values + if (mdcValue != null) { + data.put(mdcKey, mdcValue); + } else { + LOG.warn("MDC_LOGGING: mdc value is null for key: {}, config key: {}", mdcKey, + mdcKeysValuesFrom[i]); + } + + i++; + } + } + + public static String getPatternForAM(Configuration conf) { + String pattern = + conf.get(TezConfiguration.TEZ_LOG_PATTERN_LAYOUT_AM, TezConfiguration.TEZ_LOG_PATTERN_LAYOUT_DEFAULT); + return pattern.isEmpty() ? null : pattern; + } + + public static String getPatternForTask(Configuration conf) { + String pattern = + conf.get(TezConfiguration.TEZ_LOG_PATTERN_LAYOUT_TASK, TezConfiguration.TEZ_LOG_PATTERN_LAYOUT_DEFAULT); + return pattern.isEmpty() ? null : pattern; + } + + /** + * This method is for setting a NonClonableHashtable into log4j's mdc. Reflection hacks are + * needed, because MDC.mdc is well protected (final static MDC mdc = new MDC();). The logic below + * is supposed to be called once per JVM, so it's not a subject to performance bottlenecks. For + * further details of this solution, please check NonClonableHashtable class, which is set into + * the ThreadLocalMap. A wrong outcome of this method (any kind of runtime/reflection problems) + * should not affect the DAGAppMaster/TezChild. In case of an exception a ThreadLocalMap is + * returned, but it won't affect the content of the MDC. + */ + @SuppressWarnings("unchecked") + public static ThreadLocalMap setupLog4j() { + ThreadLocalMap mdcContext = new ThreadLocalMap(); + mdcContext.set(new NonClonableHashtable()); + + try { + final Constructor[] constructors = org.apache.log4j.MDC.class.getDeclaredConstructors(); + for (Constructor c : constructors) { + c.setAccessible(true); + } + + org.apache.log4j.MDC mdc = (org.apache.log4j.MDC) constructors[0].newInstance(); + Field tlmField = org.apache.log4j.MDC.class.getDeclaredField("tlm"); + tlmField.setAccessible(true); + tlmField.set(mdc, mdcContext); + + Field mdcField = org.apache.log4j.MDC.class.getDeclaredField("mdc"); + mdcField.setAccessible(true); + + Field modifiers = Field.class.getDeclaredField("modifiers"); + modifiers.setAccessible(true); + modifiers.setInt(mdcField, mdcField.getModifiers() & ~Modifier.FINAL); + + mdcField.set(null, mdc); + + } catch (Exception e) { + LOG.warn("Cannot set log4j global MDC, mdcContext won't be applied to log4j's MDC class", e); + } + + return mdcContext; + } + + /** + * NonClonableHashtable is a special class for hacking the log4j MDC context. By design, log4j's + * MDC uses a ThreadLocalMap, which clones parent thread's context before propagating it to child + * thread (see: @see {@link org.apache.log4j.helpers.ThreadLocalMap#childValue()}). In our + * usecase, this is not suitable, as we want to maintain only one context globally (and set e.g. + * dagId, taskAttemptId), then update it as easy as possible when dag/taskattempt changes, without + * having to propagate the update parameters to all the threads in the JVM. + */ + private static class NonClonableHashtable extends Hashtable { + private static final long serialVersionUID = 1L; + + @Override + public synchronized Object clone() { + return this; + } + } +} 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 c8519201fd..a5d7b7db9f 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 @@ -112,6 +112,7 @@ import org.apache.hadoop.yarn.util.ConverterUtils; import org.apache.hadoop.yarn.util.ResourceCalculatorProcessTree; import org.apache.hadoop.yarn.util.SystemClock; +import org.apache.log4j.helpers.ThreadLocalMap; import org.apache.tez.common.AsyncDispatcher; import org.apache.tez.common.AsyncDispatcherConcurrent; import org.apache.tez.common.GcTimeUpdater; @@ -184,6 +185,7 @@ import org.apache.tez.dag.utils.Simple2LevelVersionComparator; import org.apache.tez.hadoop.shim.HadoopShim; import org.apache.tez.hadoop.shim.HadoopShimsLoader; +import org.apache.tez.util.LoggingUtils; import org.apache.tez.util.TezMxBeanResourceCalculator; import org.codehaus.jettison.json.JSONException; import org.slf4j.Logger; @@ -336,6 +338,7 @@ public class DAGAppMaster extends AbstractService { // must be LinkedHashMap to preserve order of service addition Map services = new LinkedHashMap(); + private ThreadLocalMap mdcContext; public DAGAppMaster(ApplicationAttemptId applicationAttemptId, ContainerId containerId, String nmHost, int nmPort, int nmHttpPort, @@ -343,6 +346,7 @@ public DAGAppMaster(ApplicationAttemptId applicationAttemptId, String [] localDirs, String[] logDirs, String clientVersion, Credentials credentials, String jobUserName, AMPluginDescriptorProto pluginDescriptorProto) { super(DAGAppMaster.class.getName()); + this.mdcContext = LoggingUtils.setupLog4j(); this.clock = clock; this.startTime = clock.getTime(); this.appSubmitTime = appSubmitTime; @@ -690,7 +694,7 @@ protected TaskSchedulerManager getTaskSchedulerManager() { private void handleInternalError(String errDiagnosticsPrefix, String errDiagDagEvent) { state = DAGAppMasterState.ERROR; if (currentDAG != null) { - _updateLoggers(currentDAG, "_post"); + updateLoggers(currentDAG, "_post"); LOG.info(errDiagnosticsPrefix + ". Aborting dag: " + currentDAG.getID()); // Inform the current DAG about the error sendEvent(new DAGEventInternalError(currentDAG.getID(), errDiagDagEvent)); @@ -760,7 +764,7 @@ protected synchronized void handle(DAGAppMasterEvent event) { if (!isSession) { LOG.info("Not a session, AM will unregister as DAG has completed"); this.taskSchedulerManager.setShouldUnregisterFlag(); - _updateLoggers(currentDAG, "_post"); + updateLoggers(currentDAG, "_post"); setStateOnDAGCompletion(); LOG.info("Shutting down on completion of dag:" + finishEvt.getDAGId()); shutdownHandler.shutdown(); @@ -768,7 +772,7 @@ protected synchronized void handle(DAGAppMasterEvent event) { LOG.info("DAG completed, dagId=" + finishEvt.getDAGId() + ", dagState=" + finishEvt.getDAGState()); lastDAGCompletionTime = clock.getTime(); - _updateLoggers(currentDAG, "_post"); + updateLoggers(currentDAG, "_post"); if (this.historyEventHandler.hasRecoveryFailed()) { String recoveryErrorMsg = "Recovery had a fatal error, shutting down session after" + " DAG completion"; @@ -879,9 +883,10 @@ protected synchronized void handle(DAGAppMasterEvent event) { } } - private void _updateLoggers(DAG dag, String appender) { + private void updateLoggers(DAG dag, String appender) { try { - TezUtilsInternal.updateLoggers(dag.getID().toString() + appender); + TezUtilsInternal.updateLoggers(dag.getConf(), dag.getID().toString() + appender, + LoggingUtils.getPatternForAM(dag.getConf())); } catch (FileNotFoundException e) { LOG.warn("Unable to update the logger. Continue with the old logger", e ); } @@ -2007,7 +2012,7 @@ public void serviceStart() throws Exception { + ", state=" + (recoveredDAGData.dagState == null ? "null" : recoveredDAGData.dagState) + ", failureReason=" + recoveredDAGData.reason); - _updateLoggers(recoveredDAGData.recoveredDAG, ""); + updateLoggers(recoveredDAGData.recoveredDAG, ""); if (recoveredDAGData.nonRecoverable) { addDiagnostic("DAG " + recoveredDAGData.recoveredDagID + " can not be recovered due to " + recoveredDAGData.reason); @@ -2042,7 +2047,7 @@ public void serviceStart() throws Exception { } } else { LOG.info("Found DAG to recover, dagId=" + recoveredDAGData.recoveredDAG.getID()); - _updateLoggers(recoveredDAGData.recoveredDAG, ""); + updateLoggers(recoveredDAGData.recoveredDAG, ""); DAGRecoveredEvent dagRecoveredEvent = new DAGRecoveredEvent(this.appAttemptID, recoveredDAGData.recoveredDAG.getID(), recoveredDAGData.recoveredDAG.getName(), recoveredDAGData.recoveredDAG.getUserName(), this.clock.getTime(), this.containerLogs); @@ -2467,7 +2472,9 @@ private void startDAG(DAGPlan dagPlan, Map additionalAMRe // /////////////////// Create the job itself. final DAG newDAG = createDAG(dagPlan); - _updateLoggers(newDAG, ""); + LoggingUtils.initLoggingContext(mdcContext, newDAG.getConf(), newDAG.getID().toString(), null); + + updateLoggers(newDAG, ""); if (LOG.isDebugEnabled()) { LOG.debug("Running a DAG with " + dagPlan.getVertexCount() + " vertices "); 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 b89b12db2b..7ab532ad33 100644 --- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java +++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java @@ -49,6 +49,7 @@ import org.apache.hadoop.yarn.YarnUncaughtExceptionHandler; import org.apache.hadoop.yarn.api.ApplicationConstants; import org.apache.hadoop.yarn.api.ApplicationConstants.Environment; +import org.apache.log4j.helpers.ThreadLocalMap; import org.apache.tez.common.ContainerContext; import org.apache.tez.common.ContainerTask; import org.apache.tez.common.TezCommonUtils; @@ -63,6 +64,7 @@ import org.apache.tez.dag.api.TezConfiguration; import org.apache.tez.dag.api.TezException; import org.apache.tez.dag.api.records.DAGProtos; +import org.apache.tez.dag.records.TezTaskAttemptID; import org.apache.tez.dag.records.TezVertexID; import org.apache.tez.dag.utils.RelocalizationUtils; import org.apache.tez.hadoop.shim.HadoopShim; @@ -71,6 +73,8 @@ import org.apache.tez.runtime.api.impl.ExecutionContextImpl; import org.apache.tez.runtime.common.objectregistry.ObjectRegistryImpl; import org.apache.tez.runtime.internals.api.TaskReporterInterface; +import org.apache.tez.util.LoggingUtils; + import org.apache.tez.util.TezRuntimeShutdownHandler; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -125,6 +129,7 @@ public class TezChild { private TezVertexID lastVertexID; private final HadoopShim hadoopShim; private final TezExecutors sharedExecutor; + private ThreadLocalMap mdcContext; public TezChild(Configuration conf, String host, int port, String containerIdentifier, String tokenIdentifier, int appAttemptNumber, String workingDir, String[] localDirs, @@ -133,6 +138,7 @@ public TezChild(Configuration conf, String host, int port, String containerIdent ExecutionContext executionContext, Credentials credentials, long memAvailable, String user, TezTaskUmbilicalProtocol umbilical, boolean updateSysCounters, HadoopShim hadoopShim) throws IOException, InterruptedException { + this.mdcContext = LoggingUtils.setupLog4j(); this.defaultConf = conf; this.containerIdString = containerIdentifier; this.appAttemptNumber = appAttemptNumber; @@ -216,7 +222,7 @@ public ContainerExecutionResult run() throws IOException, InterruptedException, while (!executor.isTerminated() && !isShutdown.get()) { if (taskCount > 0) { - TezUtilsInternal.updateLoggers(""); + TezUtilsInternal.updateLoggers(defaultConf, "", LoggingUtils.getPatternForTask(defaultConf)); } ListenableFuture getTaskFuture = executor.submit(containerReporter); boolean error = false; @@ -240,6 +246,19 @@ public ContainerExecutionResult run() throws IOException, InterruptedException, shutdown(); } } + + TezTaskAttemptID attemptId = containerTask.getTaskSpec().getTaskAttemptID(); + if (containerTask.getTaskSpec().getTaskConf() != null) { + Configuration copy = new Configuration(defaultConf); + TezTaskRunner2.mergeTaskSpecConfToConf(containerTask.getTaskSpec(), copy); + + LoggingUtils.initLoggingContext(mdcContext, copy, + attemptId.getTaskID().getVertexID().getDAGID().toString(), attemptId.toString()); + } else { + LoggingUtils.initLoggingContext(mdcContext, defaultConf, + attemptId.getTaskID().getVertexID().getDAGID().toString(), attemptId.toString()); + } + TezCommonUtils.logCredentials(LOG, containerTask.getCredentials(), "containerTask"); if (containerTask.shouldDie()) { LOG.info("ContainerTask returned shouldDie=true for container {}, Exiting", containerIdString); @@ -256,7 +275,8 @@ public ContainerExecutionResult run() throws IOException, InterruptedException, containerTask.getTaskSpec().getTaskAttemptID().toString()); TezUtilsInternal.setHadoopCallerContext(hadoopShim, containerTask.getTaskSpec().getTaskAttemptID()); - TezUtilsInternal.updateLoggers(loggerAddend); + TezUtilsInternal.updateLoggers(defaultConf, loggerAddend, LoggingUtils.getPatternForTask(defaultConf)); + FileSystem.clearStatistics(); childUGI = handleNewTaskCredentials(containerTask, childUGI); @@ -270,6 +290,7 @@ public ContainerExecutionResult run() throws IOException, InterruptedException, serviceConsumerMetadata, serviceProviderEnvMap, startedInputsMap, taskReporter, executor, objectRegistry, pid, executionContext, memAvailable, updateSysCounters, hadoopShim, sharedExecutor); + boolean shouldDie; try { TaskRunner2Result result = taskRunner.run(); 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 bbf037b8bf..ce379b5d0a 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 @@ -29,6 +29,7 @@ import java.util.concurrent.locks.ReentrantLock; import com.google.common.annotations.VisibleForTesting; + import org.apache.tez.common.Preconditions; import com.google.common.collect.Multimap; import org.apache.commons.lang.exception.ExceptionUtils; @@ -140,18 +141,22 @@ public TezTaskRunner2(Configuration tezConf, UserGroupInformation ugi, String[] this.umbilicalAndErrorHandler = new UmbilicalAndErrorHandler(); this.hadoopShim = hadoopShim; this.taskConf = new Configuration(tezConf); + mergeTaskSpecConfToConf(taskSpec, taskConf); + localExecutor = sharedExecutor == null ? new TezSharedExecutor(tezConf) : null; + this.task = new LogicalIOProcessorRuntimeTask(taskSpec, appAttemptNumber, taskConf, localDirs, + umbilicalAndErrorHandler, serviceConsumerMetadata, serviceProviderEnvMap, startedInputsMap, + objectRegistry, pid, executionContext, memAvailable, updateSysCounters, hadoopShim, + sharedExecutor == null ? localExecutor : sharedExecutor); + } + + static void mergeTaskSpecConfToConf(TaskSpec taskSpec, Configuration conf) { if (taskSpec.getTaskConf() != null) { Iterator> iter = taskSpec.getTaskConf().iterator(); while (iter.hasNext()) { Entry entry = iter.next(); - taskConf.set(entry.getKey(), entry.getValue()); + conf.set(entry.getKey(), entry.getValue()); } } - localExecutor = sharedExecutor == null ? new TezSharedExecutor(tezConf) : null; - this.task = new LogicalIOProcessorRuntimeTask(taskSpec, appAttemptNumber, taskConf, localDirs, - umbilicalAndErrorHandler, serviceConsumerMetadata, serviceProviderEnvMap, startedInputsMap, - objectRegistry, pid, executionContext, memAvailable, updateSysCounters, hadoopShim, - sharedExecutor == null ? localExecutor : sharedExecutor); } /** diff --git a/tez-runtime-internals/src/test/java/org/apache/tez/runtime/task/TestTezTaskRunner2.java b/tez-runtime-internals/src/test/java/org/apache/tez/runtime/task/TestTezTaskRunner2.java index 6876df93ec..a6d05beb5f 100644 --- a/tez-runtime-internals/src/test/java/org/apache/tez/runtime/task/TestTezTaskRunner2.java +++ b/tez-runtime-internals/src/test/java/org/apache/tez/runtime/task/TestTezTaskRunner2.java @@ -25,9 +25,14 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.tez.common.TezExecutors; import org.apache.tez.common.TezSharedExecutor; import org.apache.tez.dag.api.ProcessorDescriptor; +import org.apache.tez.dag.records.TezDAGID; +import org.apache.tez.dag.records.TezTaskAttemptID; +import org.apache.tez.dag.records.TezTaskID; +import org.apache.tez.dag.records.TezVertexID; import org.apache.tez.hadoop.shim.DefaultHadoopShim; import org.apache.tez.runtime.api.impl.InputSpec; import org.apache.tez.runtime.api.impl.OutputSpec; @@ -50,8 +55,13 @@ public void testTaskConfUsage() throws Exception { List inputSpecList = new ArrayList<>(); List outputSpecList = new ArrayList<>(); - TaskSpec taskSpec = new TaskSpec("dagName", "vertexName", 1, mock(ProcessorDescriptor.class), - inputSpecList, outputSpecList, null, taskConf); + TaskSpec taskSpec = + new TaskSpec( + TezTaskAttemptID.getInstance( + TezTaskID.getInstance(TezVertexID + .getInstance(TezDAGID.getInstance(ApplicationId.fromString("application_1_1"), 0), 0), 0), 0), + "dagName", "vertexName", 1, mock(ProcessorDescriptor.class), inputSpecList, + outputSpecList, null, taskConf); TezExecutors sharedExecutor = new TezSharedExecutor(conf); TezTaskRunner2 taskRunner2 = new TezTaskRunner2(conf, mock(UserGroupInformation.class), localDirs, taskSpec, 1, null, null, null, mock(TaskReporter.class), null, null, "pid", diff --git a/tez-tests/src/test/java/org/apache/tez/test/TestTezJobs.java b/tez-tests/src/test/java/org/apache/tez/test/TestTezJobs.java index 6de272358b..fd8f08b42c 100644 --- a/tez-tests/src/test/java/org/apache/tez/test/TestTezJobs.java +++ b/tez-tests/src/test/java/org/apache/tez/test/TestTezJobs.java @@ -164,10 +164,59 @@ public static void tearDown() { public void testHashJoinExample() throws Exception { HashJoinExample hashJoinExample = new HashJoinExample(); hashJoinExample.setConf(new Configuration(mrrTezCluster.getConfig())); - Path stagingDirPath = new Path("/tmp/tez-staging-dir"); - Path inPath1 = new Path("/tmp/hashJoin/inPath1"); - Path inPath2 = new Path("/tmp/hashJoin/inPath2"); - Path outPath = new Path("/tmp/hashJoin/outPath"); + runHashJoinExample(hashJoinExample); + } + + @Test(timeout = 60000) + public void testHashJoinExampleWithLogPattern() throws Exception { + HashJoinExample hashJoinExample = new HashJoinExample(); + + Configuration patternConfig = new Configuration(mrrTezCluster.getConfig()); + + patternConfig.set(TezConfiguration.TEZ_AM_LOG_LEVEL, "debug"); + patternConfig.set(TezConfiguration.TEZ_TASK_LOG_LEVEL, "debug"); + patternConfig.set(TezConfiguration.TEZ_LOG_PATTERN_LAYOUT_AM, + "%d{ISO8601} [%p] [%t (queryId=%X{queryId} dag=%X{dagId})] |%c{2}|: %m%n"); + patternConfig.set(TezConfiguration.TEZ_LOG_PATTERN_LAYOUT_TASK, + "%d{ISO8601} [%p] [%t (queryId=%X{queryId} dag=%X{dagId} task=%X{taskAttemptId})] |%c{2}|: %m%n"); + patternConfig.set(TezConfiguration.TEZ_MDC_CUSTOM_KEYS, "queryId"); + patternConfig.set(TezConfiguration.TEZ_MDC_CUSTOM_KEYS_CONF_PROPS, "hive.query.id"); + patternConfig.set("hive.query.id", "hello-upstream-application-12345"); + + //1. feature is on + //[main (queryId=hello-upstream-application-12345 dag=dag_1666683231618_0001_1)] |app.DAGAppMaster| + hashJoinExample.setConf(patternConfig); + runHashJoinExample(hashJoinExample); + + //2. feature is on, but custom keys are empty: expecting empty queryId with the same format + //[main (queryId= dag=dag_1666683231618_0002_1)] |app.DAGAppMaster| + patternConfig.set(TezConfiguration.TEZ_MDC_CUSTOM_KEYS, ""); + hashJoinExample.setConf(patternConfig); + runHashJoinExample(hashJoinExample); + + //3. feature is on, custom keys are defined but corresponding value is null in config: + //expecting empty queryId with the same format + //[main (queryId= dag=dag_1666683231618_0003_1)] |app.DAGAppMaster| + patternConfig.set(TezConfiguration.TEZ_MDC_CUSTOM_KEYS, "queryId"); + patternConfig.set(TezConfiguration.TEZ_MDC_CUSTOM_KEYS_CONF_PROPS, "hive.query.id.null"); + hashJoinExample.setConf(patternConfig); + runHashJoinExample(hashJoinExample); + + //4. feature is off: expecting to have properly formatted log lines with original log4j config (not empty string) + //[main] |app.DAGAppMaster| + patternConfig.set(TezConfiguration.TEZ_LOG_PATTERN_LAYOUT_AM, TezConfiguration.TEZ_LOG_PATTERN_LAYOUT_DEFAULT); + patternConfig.set(TezConfiguration.TEZ_LOG_PATTERN_LAYOUT_TASK, TezConfiguration.TEZ_LOG_PATTERN_LAYOUT_DEFAULT); + + hashJoinExample.setConf(patternConfig); + runHashJoinExample(hashJoinExample); + } + + private void runHashJoinExample(HashJoinExample hashJoinExample) throws Exception { + int random = new Random(System.currentTimeMillis()).nextInt(10000); + Path stagingDirPath = new Path(String.format("/tmp/tez-staging-dir%d", random)); + Path inPath1 = new Path(String.format("/tmp/hashJoin%d/inPath1", random)); + Path inPath2 = new Path(String.format("/tmp/hashJoin%d/inPath2", random)); + Path outPath = new Path(String.format("/tmp/hashJoin%d/outPath", random)); remoteFs.mkdirs(inPath1); remoteFs.mkdirs(inPath2); remoteFs.mkdirs(stagingDirPath); From 8ebc4b00f1d66ee88475b1e96691f658dab967be Mon Sep 17 00:00:00 2001 From: Ganesha Shreedhara Date: Wed, 26 Oct 2022 13:40:12 +0530 Subject: [PATCH 136/137] TEZ-4450: Shuffle data fetch fails when shuffle data is transferred via CompositeRoutedDataMovementEvent (#243) (Ganesha Shreedhara reviewed by Laszlo Bodor) --- .../impl/ShuffleInputEventHandlerImpl.java | 37 +++++----- .../TestShuffleInputEventHandlerImpl.java | 72 +++++++++++++++++++ .../org.mockito.plugins.MockMaker | 13 ++++ 3 files changed, 105 insertions(+), 17 deletions(-) create mode 100644 tez-runtime-library/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleInputEventHandlerImpl.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleInputEventHandlerImpl.java index ca1259f30a..4f42f57a1e 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleInputEventHandlerImpl.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleInputEventHandlerImpl.java @@ -171,8 +171,6 @@ public void logProgress(boolean updateOnClose) { private void processDataMovementEvent(DataMovementEvent dme, DataMovementEventPayloadProto shufflePayload, BitSet emptyPartitionsBitSet) throws IOException { int srcIndex = dme.getSourceIndex(); - String hostIdentifier = shufflePayload.getHost() + ":" + shufflePayload.getPort(); - if (LOG.isDebugEnabled()) { LOG.debug("DME srcIdx: " + srcIndex + ", targetIndex: " + dme.getTargetIndex() + ", attemptNum: " + dme.getVersion() + ", payload: " + ShuffleUtils @@ -198,20 +196,7 @@ private void processDataMovementEvent(DataMovementEvent dme, DataMovementEventPa CompositeInputAttemptIdentifier srcAttemptIdentifier = constructInputAttemptIdentifier(dme.getTargetIndex(), 1, dme.getVersion(), shufflePayload, (useSharedInputs && srcIndex == 0)); - if (shufflePayload.hasData()) { - DataProto dataProto = shufflePayload.getData(); - - FetchedInput fetchedInput = - inputAllocator.allocate(dataProto.getRawLength(), - dataProto.getCompressedLength(), srcAttemptIdentifier); - moveDataToFetchedInput(dataProto, fetchedInput, hostIdentifier); - shuffleManager.addCompletedInputWithData(srcAttemptIdentifier, fetchedInput); - - LOG.debug("Payload via DME : " + srcAttemptIdentifier); - } else { - shuffleManager.addKnownInput(shufflePayload.getHost(), shufflePayload.getPort(), - srcAttemptIdentifier, srcIndex); - } + processShufflePayload(shufflePayload, srcAttemptIdentifier, srcIndex); } private void moveDataToFetchedInput(DataProto dataProto, @@ -274,7 +259,25 @@ private void processCompositeRoutedDataMovementEvent(CompositeRoutedDataMovement CompositeInputAttemptIdentifier srcAttemptIdentifier = constructInputAttemptIdentifier(crdme.getTargetIndex(), crdme.getCount(), crdme.getVersion(), shufflePayload, (useSharedInputs && partitionId == 0)); - shuffleManager.addKnownInput(shufflePayload.getHost(), shufflePayload.getPort(), srcAttemptIdentifier, partitionId); + processShufflePayload(shufflePayload, srcAttemptIdentifier, partitionId); + } + + private void processShufflePayload(DataMovementEventPayloadProto shufflePayload, + CompositeInputAttemptIdentifier srcAttemptIdentifier, int srcIndex) throws IOException { + if (shufflePayload.hasData()) { + DataProto dataProto = shufflePayload.getData(); + String hostIdentifier = shufflePayload.getHost() + ":" + shufflePayload.getPort(); + FetchedInput fetchedInput = + inputAllocator.allocate(dataProto.getRawLength(), + dataProto.getCompressedLength(), srcAttemptIdentifier); + moveDataToFetchedInput(dataProto, fetchedInput, hostIdentifier); + shuffleManager.addCompletedInputWithData(srcAttemptIdentifier, fetchedInput); + + LOG.debug("Payload via DME : " + srcAttemptIdentifier); + } else { + shuffleManager.addKnownInput(shufflePayload.getHost(), shufflePayload.getPort(), + srcAttemptIdentifier, srcIndex); + } } private void processInputFailedEvent(InputFailedEvent ife) { diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/impl/TestShuffleInputEventHandlerImpl.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/impl/TestShuffleInputEventHandlerImpl.java index 9f6b4a3a43..f4ddf590ef 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/impl/TestShuffleInputEventHandlerImpl.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/impl/TestShuffleInputEventHandlerImpl.java @@ -19,17 +19,21 @@ package org.apache.tez.runtime.library.common.shuffle.impl; import static org.mockito.Mockito.any; +import static org.mockito.Mockito.anyBoolean; import static org.mockito.Mockito.anyInt; +import static org.mockito.Mockito.anyLong; import static org.mockito.Mockito.anyString; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.eq; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.mockStatic; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; import java.io.IOException; +import java.io.InputStream; import java.nio.ByteBuffer; import java.util.BitSet; import java.util.Collections; @@ -40,6 +44,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.DataOutputBuffer; +import org.apache.hadoop.io.compress.CompressionCodec; import org.apache.hadoop.security.token.Token; import org.apache.tez.common.TezCommonUtils; import org.apache.tez.common.TezExecutors; @@ -54,15 +59,20 @@ import org.apache.tez.runtime.api.Event; import org.apache.tez.runtime.api.ExecutionContext; import org.apache.tez.runtime.api.InputContext; +import org.apache.tez.runtime.api.events.CompositeRoutedDataMovementEvent; import org.apache.tez.runtime.api.events.DataMovementEvent; import org.apache.tez.runtime.library.common.CompositeInputAttemptIdentifier; import org.apache.tez.runtime.library.common.InputAttemptIdentifier; +import org.apache.tez.runtime.library.common.shuffle.FetchedInput; import org.apache.tez.runtime.library.common.shuffle.FetchedInputAllocator; +import org.apache.tez.runtime.library.common.shuffle.MemoryFetchedInput; import org.apache.tez.runtime.library.common.shuffle.ShuffleUtils; +import org.apache.tez.runtime.library.shuffle.impl.ShuffleUserPayloads; import org.apache.tez.runtime.library.shuffle.impl.ShuffleUserPayloads.DataMovementEventPayloadProto; import org.junit.After; import org.junit.Before; import org.junit.Test; +import org.mockito.MockedStatic; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; @@ -341,6 +351,53 @@ public void testPipelinedShuffleEvents_WithEmptyPartitions() throws IOException verify(inputContext).killSelf(any(), anyString()); } + /** + * Verify that data movement events with shuffle data are processed properly. + * + * @throws IOException + */ + @Test(timeout = 5000) + public void testDataMovementEventsWithShuffleData() throws IOException { + InputContext inputContext = mock(InputContext.class); + ShuffleManager shuffleManager = mock(ShuffleManager.class); + ShuffleManager compositeFetchShuffleManager = mock(ShuffleManager.class); + FetchedInputAllocator inputAllocator = mock(FetchedInputAllocator.class); + MemoryFetchedInput memoryFetchedInput = mock(MemoryFetchedInput.class); + + when(memoryFetchedInput.getType()).thenReturn(FetchedInput.Type.MEMORY); + when(memoryFetchedInput.getBytes()).thenReturn("data".getBytes()); + when(inputAllocator.allocate(anyLong(), anyLong(), any(InputAttemptIdentifier.class))) + .thenReturn(memoryFetchedInput); + + ShuffleInputEventHandlerImpl eventHandler = new ShuffleInputEventHandlerImpl(inputContext, + shuffleManager, inputAllocator, null, true, 4, false); + + ShuffleInputEventHandlerImpl compositeFetchEventHandler = new ShuffleInputEventHandlerImpl(inputContext, + compositeFetchShuffleManager, inputAllocator, null, true, 4, true); + + DataMovementEvent dataMovementEvent = (DataMovementEvent) createDataMovementEventWithShuffleData(false); + CompositeRoutedDataMovementEvent compositeRoutedDataMovementEvent = + (CompositeRoutedDataMovementEvent) createDataMovementEventWithShuffleData(true); + + List eventListWithDme = new LinkedList<>(); + eventListWithDme.add(dataMovementEvent); + eventListWithDme.add(compositeRoutedDataMovementEvent); + + try (MockedStatic shuffleUtils = mockStatic(ShuffleUtils.class)) { + shuffleUtils.when(() -> ShuffleUtils + .shuffleToMemory(any(byte[].class), any(InputStream.class), anyInt(), anyInt(), any(CompressionCodec.class), + anyBoolean(), anyInt(), any(), any(InputAttemptIdentifier.class))) + .thenAnswer((Answer) invocation -> null); + eventHandler.handleEvents(eventListWithDme); + compositeFetchEventHandler.handleEvents(eventListWithDme); + + verify(shuffleManager, times(2)) + .addCompletedInputWithData(any(InputAttemptIdentifier.class), any(FetchedInput.class)); + verify(compositeFetchShuffleManager, times(2)) + .addCompletedInputWithData(any(InputAttemptIdentifier.class), any(FetchedInput.class)); + } + } + private Event createDataMovementEvent(boolean addSpillDetails, int srcIdx, int targetIdx, int spillId, boolean isLastSpill, BitSet emptyPartitions, int numPartitions, int attemptNum) throws IOException { @@ -396,4 +453,19 @@ private ByteString createEmptyPartitionByteString(int... emptyPartitions) throws return emptyPartitionsBytesString; } + private Event createDataMovementEventWithShuffleData(boolean isComposite) { + DataMovementEventPayloadProto.Builder builder = DataMovementEventPayloadProto.newBuilder(); + builder.setHost(HOST); + builder.setPort(PORT); + builder.setPathComponent(PATH_COMPONENT); + ShuffleUserPayloads.DataProto.Builder dataProtoBuilder = ShuffleUserPayloads.DataProto.newBuilder() + .setData(ByteString.copyFromUtf8("data")); + builder.setData(dataProtoBuilder); + + Event dme = isComposite? + CompositeRoutedDataMovementEvent.create(0, 1, 1, 0, builder.build().toByteString().asReadOnlyByteBuffer()): + DataMovementEvent.create(0, 1, 0, builder.build().toByteString().asReadOnlyByteBuffer()); + return dme; + } + } diff --git a/tez-runtime-library/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker b/tez-runtime-library/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker new file mode 100644 index 0000000000..a258d79ad3 --- /dev/null +++ b/tez-runtime-library/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker @@ -0,0 +1,13 @@ +# 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. + +mock-maker-inline From 8bb27b845ce103b8d02a9a9d7fe2a1af10097e9c Mon Sep 17 00:00:00 2001 From: slfan1989 <55643692+slfan1989@users.noreply.github.com> Date: Wed, 9 Nov 2022 22:50:46 +0800 Subject: [PATCH 137/137] TEZ-4454: remove extra commas. (#246) (slfan1989 reviewed by Laszlo Bodor) --- .../test/java/org/apache/tez/common/TestTezCommonUtils.java | 2 +- .../java/org/apache/tez/dag/app/dag/impl/VertexImpl.java | 6 +++--- .../library/common/shuffle/orderedgrouped/MergeManager.java | 2 +- .../tez/runtime/library/common/sort/impl/TezMerger.java | 2 +- tez-tests/src/test/java/org/apache/tez/test/TestInput.java | 2 +- tez-tests/src/test/java/org/apache/tez/test/TestOutput.java | 2 +- 6 files changed, 8 insertions(+), 8 deletions(-) diff --git a/tez-api/src/test/java/org/apache/tez/common/TestTezCommonUtils.java b/tez-api/src/test/java/org/apache/tez/common/TestTezCommonUtils.java index d5dc6fd6b5..e1ae6cd2cb 100644 --- a/tez-api/src/test/java/org/apache/tez/common/TestTezCommonUtils.java +++ b/tez-api/src/test/java/org/apache/tez/common/TestTezCommonUtils.java @@ -49,7 +49,7 @@ public class TestTezCommonUtils { private static final File LOCAL_STAGING_DIR = new File(System.getProperty("test.build.data"), TestTezCommonUtils.class.getSimpleName()).getAbsoluteFile(); private static String RESOLVED_STAGE_DIR; - private static Configuration conf = new Configuration();; + private static Configuration conf = new Configuration(); private static String TEST_ROOT_DIR = "target" + Path.SEPARATOR + TestTezCommonUtils.class.getName() + "-tmpDir"; private static MiniDFSCluster dfsCluster = null; 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 e55b10a659..f8f2750267 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 @@ -681,7 +681,7 @@ private void augmentStateMachine() { .registerStateEnteredCallback(VertexState.RUNNING, STATE_CHANGED_CALLBACK) .registerStateEnteredCallback(VertexState.INITIALIZING, - STATE_CHANGED_CALLBACK);; + STATE_CHANGED_CALLBACK); } private final StateMachineTez stateMachine; @@ -3886,7 +3886,7 @@ private void commitCompleted(VertexEventCommitCompleted commitCompletedEvent) { } else { String diag = "Commit failed for output:" + commitCompletedEvent.getOutputName() + ", vertexId=" + logIdentifier + ", " - + ExceptionUtils.getStackTrace(commitCompletedEvent.getException());; + + ExceptionUtils.getStackTrace(commitCompletedEvent.getException()); LOG.info(diag); addDiagnostic(diag); trySetTerminationCause(VertexTerminationCause.COMMIT_FAILURE); @@ -4398,7 +4398,7 @@ public void setOutputVertices(Map outVertices) { addIO(vertex.getName()); } } finally { - writeLock.unlock();; + writeLock.unlock(); } } diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/MergeManager.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/MergeManager.java index 8d3f37dbcd..9da1276b8a 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/MergeManager.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/MergeManager.java @@ -322,7 +322,7 @@ void setupParentThread(Thread shuffleSchedulerThread) { if (this.memToMemMerger != null) { memToMemMerger.setParentThread(shuffleSchedulerThread); } - this.inMemoryMerger.setParentThread(shuffleSchedulerThread);; + this.inMemoryMerger.setParentThread(shuffleSchedulerThread); this.onDiskMerger.setParentThread(shuffleSchedulerThread); } diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezMerger.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezMerger.java index e83b34e114..12e5735199 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezMerger.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezMerger.java @@ -699,7 +699,7 @@ protected boolean lessThan(Object a, Object b) { int s1 = key1.getPosition(); int l1 = key1.getLength(); int s2 = key2.getPosition(); - int l2 = key2.getLength();; + int l2 = key2.getLength(); return comparator.compare(key1.getData(), s1, l1, key2.getData(), s2, l2) < 0; } diff --git a/tez-tests/src/test/java/org/apache/tez/test/TestInput.java b/tez-tests/src/test/java/org/apache/tez/test/TestInput.java index 36bb983d1d..7ad50b7e11 100644 --- a/tez-tests/src/test/java/org/apache/tez/test/TestInput.java +++ b/tez-tests/src/test/java/org/apache/tez/test/TestInput.java @@ -393,7 +393,7 @@ public void handleEvents(List inputEvents) throws Exception { @Override public List close() throws Exception { - getContext().getCounters().findCounter(COUNTER_NAME, COUNTER_NAME).increment(1);; + getContext().getCounters().findCounter(COUNTER_NAME, COUNTER_NAME).increment(1); return null; } diff --git a/tez-tests/src/test/java/org/apache/tez/test/TestOutput.java b/tez-tests/src/test/java/org/apache/tez/test/TestOutput.java index 8b292ab196..b595743f02 100644 --- a/tez-tests/src/test/java/org/apache/tez/test/TestOutput.java +++ b/tez-tests/src/test/java/org/apache/tez/test/TestOutput.java @@ -80,7 +80,7 @@ public void handleEvents(List outputEvents) { @Override public List close() throws Exception { LOG.info("Sending data movement event with value: " + output); - getContext().getCounters().findCounter(COUNTER_NAME, COUNTER_NAME).increment(1);; + getContext().getCounters().findCounter(COUNTER_NAME, COUNTER_NAME).increment(1); ByteBuffer result = ByteBuffer.allocate(4).putInt(output); result.flip(); List events = Lists.newArrayListWithCapacity(getNumPhysicalOutputs());