Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

TEZ-4129: Delete intermediate attempt data for failed attempts for Shuffle Handler #72

Merged
merged 1 commit into from
Jan 27, 2022
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -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")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;

Expand All @@ -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);
}
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -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;
Expand Down Expand Up @@ -2738,4 +2740,7 @@ String buildPluginComponentLog(List<NamedEntityDescriptor> namedEntityDescriptor
return sb.toString();
}

public void taskAttemptFailed(TezTaskAttemptID attemptID, NodeId nodeId) {
getContainerLauncherManager().taskAttemptFailed(attemptID, jobTokenSecretManager, nodeId);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -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) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand All @@ -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;
Expand Down Expand Up @@ -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.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -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);
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -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 {

Expand All @@ -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
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -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) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -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<ContainerId, ListenableFuture<?>>
runningContainers =
Expand Down Expand Up @@ -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(
Expand Down Expand Up @@ -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);
}
}
}
Original file line number Diff line number Diff line change
@@ -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();
}
}
Loading