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

[CELEBORN-1720] Prevent stage re-run if another task attempt is running or successful #2921

Open
wants to merge 1 commit into
base: main
Choose a base branch
from
Open
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 @@ -155,6 +155,7 @@ public void incReadTime(long time) {}
reduceId.getTaskID().getId(),
reduceId.getId(),
0,
0,
Integer.MAX_VALUE,
metricsCallback);
CelebornShuffleFetcher<K, V> shuffleReader =
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -101,6 +101,11 @@ private void initializeLifecycleManager(String appId) {
if (celebornConf.clientStageRerunEnabled()) {
MapOutputTrackerMaster mapOutputTracker =
(MapOutputTrackerMaster) SparkEnv.get().mapOutputTracker();

lifecycleManager.registerReportTaskShuffleFetchFailurePreCheck(
taskId -> !SparkUtils.taskAnotherAttemptRunningOrSuccessful(taskId));
SparkUtils.addSparkListener(new ShuffleFetchFailureReportTaskCleanListener());

lifecycleManager.registerShuffleTrackerCallback(
shuffleId -> mapOutputTracker.unregisterAllMapOutput(shuffleId));
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,12 +20,19 @@
import java.io.IOException;
import java.lang.reflect.Field;
import java.lang.reflect.Method;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.atomic.LongAdder;
import java.util.stream.Collectors;

import scala.Option;
import scala.Some;
import scala.Tuple2;

import com.google.common.annotations.VisibleForTesting;
import org.apache.spark.BarrierTaskContext;
import org.apache.spark.SparkConf;
import org.apache.spark.SparkContext;
Expand All @@ -35,6 +42,10 @@
import org.apache.spark.scheduler.MapStatus;
import org.apache.spark.scheduler.MapStatus$;
import org.apache.spark.scheduler.ShuffleMapStage;
import org.apache.spark.scheduler.SparkListener;
import org.apache.spark.scheduler.TaskInfo;
import org.apache.spark.scheduler.TaskSchedulerImpl;
import org.apache.spark.scheduler.TaskSetManager;
import org.apache.spark.sql.execution.UnsafeRowSerializer;
import org.apache.spark.sql.execution.metric.SQLMetric;
import org.apache.spark.storage.BlockManagerId;
Expand All @@ -43,6 +54,7 @@

import org.apache.celeborn.client.ShuffleClient;
import org.apache.celeborn.common.CelebornConf;
import org.apache.celeborn.common.util.JavaUtils;
import org.apache.celeborn.common.util.Utils;
import org.apache.celeborn.reflect.DynFields;

Expand Down Expand Up @@ -203,4 +215,135 @@ public static void cancelShuffle(int shuffleId, String reason) {
logger.error("Can not get active SparkContext, skip cancelShuffle.");
}
}

private static final DynFields.UnboundField<ConcurrentHashMap<Long, TaskSetManager>>
TASK_ID_TO_TASK_SET_MANAGER_FIELD =
DynFields.builder()
.hiddenImpl(TaskSchedulerImpl.class, "taskIdToTaskSetManager")
.defaultAlwaysNull()
.build();
private static final DynFields.UnboundField<scala.collection.mutable.HashMap<Long, TaskInfo>>
TASK_INFOS_FIELD =
DynFields.builder()
.hiddenImpl(TaskSetManager.class, "taskInfos")
.defaultAlwaysNull()
.build();

/**
* TaskSetManager - it is not designed to be used outside the spark scheduler. Please be careful.
*/
@VisibleForTesting
protected static TaskSetManager getTaskSetManager(TaskSchedulerImpl taskScheduler, long taskId) {
synchronized (taskScheduler) {
ConcurrentHashMap<Long, TaskSetManager> taskIdToTaskSetManager =
TASK_ID_TO_TASK_SET_MANAGER_FIELD.bind(taskScheduler).get();
return taskIdToTaskSetManager.get(taskId);
}
}

@VisibleForTesting
protected static Tuple2<TaskInfo, List<TaskInfo>> getTaskAttempts(
TaskSetManager taskSetManager, long taskId) {
if (taskSetManager != null) {
scala.Option<TaskInfo> taskInfoOption =
TASK_INFOS_FIELD.bind(taskSetManager).get().get(taskId);
turboFei marked this conversation as resolved.
Show resolved Hide resolved
if (taskInfoOption.isDefined()) {
TaskInfo taskInfo = taskInfoOption.get();
List<TaskInfo> taskAttempts =
scala.collection.JavaConverters.asJavaCollectionConverter(
taskSetManager.taskAttempts()[taskInfo.index()])
.asJavaCollection().stream()
.collect(Collectors.toList());
return Tuple2.apply(taskInfo, taskAttempts);
} else {
logger.error("Can not get TaskInfo for taskId: {}", taskId);
return null;
}
} else {
logger.error("Can not get TaskSetManager for taskId: {}", taskId);
return null;
}
}

protected static Map<String, Set<Long>> reportedStageShuffleFetchFailureTaskIds =
JavaUtils.newConcurrentHashMap();

protected static void removeStageReportedShuffleFetchFailureTaskIds(
int stageId, int stageAttemptId) {
reportedStageShuffleFetchFailureTaskIds.remove(stageId + "-" + stageAttemptId);
}

/**
* Only used to check for the shuffle fetch failure task whether another attempt is running or
* successful. If another attempt(excluding the reported shuffle fetch failure tasks in current
* stage) is running or successful, return true. Otherwise, return false.
*/
public static boolean taskAnotherAttemptRunningOrSuccessful(long taskId) {
SparkContext sparkContext = SparkContext$.MODULE$.getActive().getOrElse(null);
if (sparkContext == null) {
logger.error("Can not get active SparkContext.");
return false;
}
TaskSchedulerImpl taskScheduler = (TaskSchedulerImpl) sparkContext.taskScheduler();
synchronized (taskScheduler) {
TaskSetManager taskSetManager = getTaskSetManager(taskScheduler, taskId);
if (taskSetManager != null) {
int stageId = taskSetManager.stageId();
int stageAttemptId = taskSetManager.taskSet().stageAttemptId();
String stageUniqId = stageId + "-" + stageAttemptId;
Set<Long> reportedStageTaskIds =
reportedStageShuffleFetchFailureTaskIds.computeIfAbsent(
stageUniqId, k -> new HashSet<>());
reportedStageTaskIds.add(taskId);

Tuple2<TaskInfo, List<TaskInfo>> taskAttempts = getTaskAttempts(taskSetManager, taskId);

if (taskAttempts == null) return false;

TaskInfo taskInfo = taskAttempts._1();
for (TaskInfo ti : taskAttempts._2()) {
if (ti.taskId() != taskId) {
if (reportedStageTaskIds.contains(ti.taskId())) {
logger.info(
"StageId={} index={} taskId={} attempt={} another attempt {} has reported shuffle fetch failure, ignore it.",
stageId,
taskInfo.index(),
taskId,
taskInfo.attemptNumber(),
ti.attemptNumber());
} else if (ti.successful()) {
logger.info(
"StageId={} index={} taskId={} attempt={} another attempt {} is successful.",
stageId,
taskInfo.index(),
taskId,
taskInfo.attemptNumber(),
ti.attemptNumber());
return true;
} else if (ti.running()) {
logger.info(
"StageId={} index={} taskId={} attempt={} another attempt {} is running.",
stageId,
taskInfo.index(),
taskId,
taskInfo.attemptNumber(),
ti.attemptNumber());
return true;
}
}
}
return false;
} else {
logger.error("Can not get TaskSetManager for taskId: {}", taskId);
return false;
}
}
}

public static void addSparkListener(SparkListener listener) {
SparkContext sparkContext = SparkContext$.MODULE$.getActive().getOrElse(null);
if (sparkContext != null) {
sparkContext.addSparkListener(listener);
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -98,6 +98,7 @@ class CelebornShuffleReader[K, C](
shuffleId,
partitionId,
encodedAttemptId,
context.taskAttemptId(),
startMapIndex,
endMapIndex,
metricsCallback)
Expand All @@ -124,7 +125,10 @@ class CelebornShuffleReader[K, C](
exceptionRef.get() match {
case ce @ (_: CelebornIOException | _: PartitionUnRetryAbleException) =>
if (handle.throwsFetchFailure &&
shuffleClient.reportShuffleFetchFailure(handle.shuffleId, shuffleId)) {
shuffleClient.reportShuffleFetchFailure(
handle.shuffleId,
shuffleId,
context.taskAttemptId())) {
throw new FetchFailedException(
null,
handle.shuffleId,
Expand Down Expand Up @@ -158,7 +162,10 @@ class CelebornShuffleReader[K, C](
} catch {
case e @ (_: CelebornIOException | _: PartitionUnRetryAbleException) =>
if (handle.throwsFetchFailure &&
shuffleClient.reportShuffleFetchFailure(handle.shuffleId, shuffleId)) {
shuffleClient.reportShuffleFetchFailure(
handle.shuffleId,
shuffleId,
context.taskAttemptId())) {
throw new FetchFailedException(
null,
handle.shuffleId,
Expand Down
Original file line number Diff line number Diff line change
@@ -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.spark.shuffle.celeborn

import org.apache.spark.scheduler.{SparkListener, SparkListenerStageCompleted}

class ShuffleFetchFailureReportTaskCleanListener extends SparkListener {
override def onStageCompleted(stageCompleted: SparkListenerStageCompleted): Unit = {
SparkUtils.removeStageReportedShuffleFetchFailureTaskIds(
stageCompleted.stageInfo.stageId,
stageCompleted.stageInfo.attemptNumber())
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -144,6 +144,10 @@ private void initializeLifecycleManager(String appId) {
MapOutputTrackerMaster mapOutputTracker =
(MapOutputTrackerMaster) SparkEnv.get().mapOutputTracker();

lifecycleManager.registerReportTaskShuffleFetchFailurePreCheck(
taskId -> !SparkUtils.taskAnotherAttemptRunningOrSuccessful(taskId));
SparkUtils.addSparkListener(new ShuffleFetchFailureReportTaskCleanListener());

lifecycleManager.registerShuffleTrackerCallback(
shuffleId -> SparkUtils.unregisterAllMapOutput(mapOutputTracker, shuffleId));
}
Expand Down
Loading
Loading