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

Spark 2 #30

Merged
merged 2 commits into from
Nov 17, 2024
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 @@ -101,6 +101,10 @@ private void initializeLifecycleManager(String appId) {
if (celebornConf.clientFetchThrowsFetchFailure()) {
MapOutputTrackerMaster mapOutputTracker =
(MapOutputTrackerMaster) SparkEnv.get().mapOutputTracker();

lifecycleManager.registerReportTaskShuffleFetchFailurePreCheck(
taskId -> !SparkUtils.taskAnotherAttemptRunning(taskId));

lifecycleManager.registerShuffleTrackerCallback(
shuffleId -> mapOutputTracker.unregisterAllMapOutput(shuffleId));
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@
import java.io.IOException;
import java.lang.reflect.Field;
import java.lang.reflect.Method;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.atomic.LongAdder;

import scala.Option;
Expand All @@ -35,6 +36,9 @@
import org.apache.spark.scheduler.MapStatus;
import org.apache.spark.scheduler.MapStatus$;
import org.apache.spark.scheduler.ShuffleMapStage;
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 @@ -47,7 +51,7 @@
import org.apache.celeborn.reflect.DynFields;

public class SparkUtils {
private static final Logger logger = LoggerFactory.getLogger(SparkUtils.class);
private static final Logger LOG = LoggerFactory.getLogger(SparkUtils.class);

public static final String FETCH_FAILURE_ERROR_MSG = "Celeborn FetchFailure with shuffle id ";

Expand Down Expand Up @@ -93,7 +97,7 @@ public static SQLMetric getUnsafeRowSerializerDataSizeMetric(UnsafeRowSerializer
field.setAccessible(true);
return (SQLMetric) field.get(serializer);
} catch (NoSuchFieldException | IllegalAccessException e) {
logger.warn("Failed to get dataSize metric, aqe won`t work properly.");
LOG.warn("Failed to get dataSize metric, aqe won`t work properly.");
}
return null;
}
Expand Down Expand Up @@ -200,7 +204,58 @@ public static void cancelShuffle(int shuffleId, String reason) {
scheduler.cancelStage(shuffleMapStage.get().id(), new Some<>(reason));
}
} else {
logger.error("Can not get active SparkContext, skip cancelShuffle.");
LOG.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();

public static boolean taskAnotherAttemptRunning(long taskId) {
if (SparkContext$.MODULE$.getActive().nonEmpty()) {
TaskSchedulerImpl taskScheduler =
(TaskSchedulerImpl) SparkContext$.MODULE$.getActive().get().taskScheduler();
ConcurrentHashMap<Long, TaskSetManager> taskIdToTaskSetManager =
TASK_ID_TO_TASK_SET_MANAGER_FIELD.bind(taskScheduler).get();
TaskSetManager taskSetManager = taskIdToTaskSetManager.get(taskId);
if (taskSetManager != null) {
scala.Option<TaskInfo> taskInfoOption =
TASK_INFOS_FIELD.bind(taskSetManager).get().get(taskId);
if (taskInfoOption.isDefined()) {
TaskInfo taskInfo = taskInfoOption.get();
return scala.collection.JavaConverters.asJavaCollectionConverter(
taskSetManager.taskAttempts()[taskInfo.index()])
.asJavaCollection().stream()
.anyMatch(
ti -> {
if (ti.running() && ti.attemptNumber() != taskInfo.attemptNumber()) {
LOG.info("Another attempt of task {} is running: {}.", taskInfo, ti);
return true;
} else {
return false;
}
});
} else {
LOG.error("Can not get TaskInfo for taskId: {}", taskId);
return false;
}
} else {
LOG.error("Can not get TaskSetManager for taskId: {}", taskId);
return false;
}
} else {
LOG.error("Can not get active SparkContext, skip checking.");
return false;
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -349,15 +349,18 @@ public static boolean taskAnotherAttemptRunning(long taskId) {
TASK_INFOS_FIELD.bind(taskSetManager).get().get(taskId);
if (taskInfoOption.isDefined()) {
TaskInfo taskInfo = taskInfoOption.get();
return taskSetManager.taskAttempts()[taskInfo.index()].exists(
ti -> {
if (ti.running() && ti.attemptNumber() != taskInfo.attemptNumber()) {
LOG.info("Another attempt of task {} is running: {}.", taskInfo, ti);
return true;
} else {
return false;
}
});
return scala.collection.JavaConverters.asJavaCollectionConverter(
taskSetManager.taskAttempts()[taskInfo.index()])
.asJavaCollection().stream()
.anyMatch(
ti -> {
if (ti.running() && ti.attemptNumber() != taskInfo.attemptNumber()) {
LOG.info("Another attempt of task {} is running: {}.", taskInfo, ti);
return true;
} else {
return false;
}
});
} else {
LOG.error("Can not get TaskInfo for taskId: {}", taskId);
return false;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1016,9 +1016,9 @@ class LifecycleManager(val appUniqueId: String, val conf: CelebornConf) extends

private def invokeReportTaskShuffleFetchFailurePreCheck(taskId: Long): Boolean = {
reportTaskShuffleFetchFailurePreCheck match {
case Some(precheck) =>
case Some(preCheck) =>
try {
precheck.apply(taskId)
preCheck.apply(taskId)
} catch {
case t: Throwable =>
logError(t.toString)
Expand Down Expand Up @@ -1789,9 +1789,10 @@ class LifecycleManager(val appUniqueId: String, val conf: CelebornConf) extends
}

@volatile private var reportTaskShuffleFetchFailurePreCheck
: Option[Function[java.lang.Long, Boolean]] = None
def registerReportTaskShuffleFetchFailurePreCheck(preCheck: Function[java.lang.Long, Boolean])
: Unit = {
: Option[java.util.function.Function[java.lang.Long, Boolean]] = None
def registerReportTaskShuffleFetchFailurePreCheck(preCheck: java.util.function.Function[
java.lang.Long,
Boolean]): Unit = {
reportTaskShuffleFetchFailurePreCheck = Some(preCheck)
}

Expand Down