Skip to content
Closed
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
@@ -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.spark

import scala.collection.JavaConverters._

import org.apache.spark.scheduler.{EventLoggingListener, SparkListenerInterface}

object CelebornSparkContextHelper {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Cleanup methods from this object which are not required ?
It looks like the only thing needed is eventLogger right now.


def eventLogger: Option[EventLoggingListener] = SparkContext.getActive.get.eventLogger

def env: SparkEnv = {
assert(SparkContext.getActive.isDefined)
SparkContext.getActive.get.env
}

def activeSparkContext(): Option[SparkContext] = {
SparkContext.getActive
}

def getListener(listenerClass: String): SparkListenerInterface = {
activeSparkContext().get.listenerBus.listeners.asScala.find(l =>
l.getClass.getCanonicalName.contains(listenerClass)).getOrElse(
throw new RuntimeException(
s"cannot find any listener containing $listenerClass in class name"))
}
}

Original file line number Diff line number Diff line change
@@ -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.
*/

package org.apache.spark.scheduler

import org.apache.spark.SparkContext

trait RunningStageManager {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Where is this being used ?

def isRunningStage(stageId: Int): Boolean
}
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

maybe you can go through this PR first.

#3109

I saw something duplicated.

Image

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

this is a version based on the original implementation of that PR, I will clean it up to merge to master branch


class RunningStageManagerImpl extends RunningStageManager {

private def dagScheduler = SparkContext.getActive.get.dagScheduler

override def isRunningStage(stageId: Int): Boolean = {
dagScheduler.runningStages.map(_.id).contains(stageId)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is unsafe - runningStages is expected to be used only from within DAGScheduler

}
}
Original file line number Diff line number Diff line change
Expand Up @@ -18,11 +18,15 @@
package org.apache.spark.shuffle.celeborn;

import java.io.IOException;
import java.lang.reflect.InvocationTargetException;
import java.util.concurrent.ConcurrentHashMap;

import org.apache.celeborn.spark.StageDependencyManager;
import org.apache.spark.*;
import org.apache.spark.launcher.SparkLauncher;
import org.apache.spark.rdd.DeterministicLevel;
import org.apache.spark.scheduler.RunningStageManager;
import org.apache.spark.scheduler.RunningStageManagerImpl;
import org.apache.spark.shuffle.*;
import org.apache.spark.shuffle.sort.SortShuffleManager;
import org.apache.spark.sql.internal.SQLConf;
Expand All @@ -31,6 +35,7 @@
import org.slf4j.LoggerFactory;

import org.apache.celeborn.client.LifecycleManager;
import org.apache.spark.listener.ListenerHelper;
import org.apache.celeborn.client.ShuffleClient;
import org.apache.celeborn.common.CelebornConf;
import org.apache.celeborn.common.protocol.ShuffleMode;
Expand Down Expand Up @@ -86,6 +91,30 @@ public class SparkShuffleManager implements ShuffleManager {
private long sendBufferPoolExpireTimeout;

private ExecutorShuffleIdTracker shuffleIdTracker = new ExecutorShuffleIdTracker();
private StageDependencyManager stageDepManager = null;

public static final String RUNNING_STAGE_CHECKER_CLASS =
"CELEBORN_TEST_RUNNING_STAGE_CHECKER_IMPL";

private RunningStageManager runningStageManager = null;

// for testing
public void buildRunningStageChecker()
throws ClassNotFoundException, NoSuchMethodException, InvocationTargetException,
InstantiationException, IllegalAccessException {
if (System.getProperty(RUNNING_STAGE_CHECKER_CLASS) == null) {
runningStageManager = new RunningStageManagerImpl();
} else {
String className = System.getProperty(RUNNING_STAGE_CHECKER_CLASS);
Class claz = Class.forName(className);
runningStageManager = (RunningStageManager) claz.getDeclaredConstructor().newInstance();
}
}

// for testing
public void initStageDepManager() {
this.stageDepManager = new StageDependencyManager(this);
}

public SparkShuffleManager(SparkConf conf, boolean isDriver) {
if (conf.getBoolean(SQLConf.LOCAL_SHUFFLE_READER_ENABLED().key(), true)) {
Expand Down Expand Up @@ -131,6 +160,47 @@ private void initializeLifecycleManager() {

lifecycleManager.registerShuffleTrackerCallback(
shuffleId -> SparkUtils.unregisterAllMapOutput(mapOutputTracker, shuffleId));

if (stageDepManager == null) {
stageDepManager = new StageDependencyManager(this);
}
stageDepManager.start();
try {
buildRunningStageChecker();
} catch (Exception re) {
throw new RuntimeException("cannot create running stage manager");
}
}
if (lifecycleManager.conf().clientShuffleEarlyDeletion()) {
logger.info("register early deletion callbacks");
ListenerHelper.addShuffleStatsTrackingListener();
lifecycleManager.registerStageToWriteCelebornShuffleCallback(
(celebornShuffleId, appShuffleIdentifier) ->
SparkUtils.addStageToWriteCelebornShuffleIdDep(
this, celebornShuffleId, appShuffleIdentifier));
lifecycleManager.registerCelebornToAppShuffleIdMappingCallback(
(celebornShuffleId, appShuffleIdentifier) ->
SparkUtils.addCelebornToSparkShuffleIdRef(
this, celebornShuffleId, appShuffleIdentifier));
lifecycleManager.registerGetCelebornShuffleIdForReaderCallback(
(celebornShuffleId, appShuffleIdentifier) ->
SparkUtils.addCelebornShuffleReadingStageDep(
this, celebornShuffleId, appShuffleIdentifier));
lifecycleManager.registerUpstreamAppShuffleIdsCallback(
(stageId) -> SparkUtils.getAllUpstreamAppShuffleIds(this, stageId));
lifecycleManager.registerGetAppShuffleIdByStageIdCallback(
(stageId) -> SparkUtils.getAppShuffleIdByStageId(this, stageId));
lifecycleManager.registerReaderStageToAppShuffleIdsCallback(
(appShuffleId, appShuffleIdentifier) ->
SparkUtils.addAppShuffleReadingStageDep(
this, appShuffleId, appShuffleIdentifier));
lifecycleManager.registerInvalidateAllUpstreamCheckCallback(
(appShuffleIdentifier) ->
SparkUtils.canInvalidateAllUpstream(this, appShuffleIdentifier));
if (stageDepManager == null) {
stageDepManager = new StageDependencyManager(this);
}
stageDepManager.start();
}
}
}
Expand Down Expand Up @@ -214,6 +284,7 @@ public void stop() {
_sortShuffleManager.stop();
_sortShuffleManager = null;
}
ListenerHelper.reset();
}

@Override
Expand Down Expand Up @@ -390,4 +461,12 @@ private int executorCores(SparkConf conf) {
public LifecycleManager getLifecycleManager() {
return this.lifecycleManager;
}

public RunningStageManager getRunningStageManager() {
return this.runningStageManager;
}

public StageDependencyManager getStageDepManager() {
return this.stageDepManager;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@

package org.apache.spark.shuffle.celeborn;

import java.util.Arrays;
import java.util.concurrent.atomic.LongAdder;

import scala.Tuple2;
Expand Down Expand Up @@ -274,4 +275,56 @@ public static void unregisterAllMapOutput(
throw new UnsupportedOperationException(
"unexpected! neither methods unregisterAllMapAndMergeOutput/unregisterAllMapOutput are found in MapOutputTrackerMaster");
}

public static Integer[] getAllUpstreamAppShuffleIds(
SparkShuffleManager sparkShuffleManager, int readerStageId) {
int[] upstreamShuffleIds =
sparkShuffleManager
.getStageDepManager()
.getAllUpstreamAppShuffleIdsByStageId(readerStageId);
return Arrays.stream(upstreamShuffleIds).boxed().toArray(Integer[]::new);
}

public static Integer getAppShuffleIdByStageId(
SparkShuffleManager sparkShuffleManager, int readerStageId) {
int writtenAppShuffleId =
sparkShuffleManager.getStageDepManager().getAppShuffleIdByStageId(readerStageId);
return writtenAppShuffleId;
}

public static void addCelebornShuffleReadingStageDep(
SparkShuffleManager sparkShuffleManager, int celebornShuffeId, String appShuffleIdentifier) {
sparkShuffleManager
.getStageDepManager()
.addCelebornShuffleIdReadingStageDep(celebornShuffeId, appShuffleIdentifier);
}

public static void addAppShuffleReadingStageDep(
SparkShuffleManager sparkShuffleManager, int appShuffleId, String appShuffleIdentifier) {
sparkShuffleManager
.getStageDepManager()
.addAppShuffleIdReadingStageDep(appShuffleId, appShuffleIdentifier);
}

public static boolean canInvalidateAllUpstream(
SparkShuffleManager sparkShuffleManager, String appShuffleIdentifier) {
String[] decodedAppShuffleIdentifier = appShuffleIdentifier.split("-");
return sparkShuffleManager
.getStageDepManager()
.hasAllUpstreamShuffleIdsInfo(Integer.valueOf(decodedAppShuffleIdentifier[1]));
}

public static void addStageToWriteCelebornShuffleIdDep(
SparkShuffleManager sparkShuffleManager, int celebornShuffeId, String appShuffleIdentifier) {
sparkShuffleManager
.getStageDepManager()
.addStageToCelebornShuffleIdRef(celebornShuffeId, appShuffleIdentifier);
}

public static void addCelebornToSparkShuffleIdRef(
SparkShuffleManager sparkShuffleManager, int celebornShuffeId, String appShuffleIdentifier) {
sparkShuffleManager
.getStageDepManager()
.addCelebornToAppShuffleIdMapping(celebornShuffeId, appShuffleIdentifier);
}
}
Loading
Loading