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

import scala.collection.JavaConverters._

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

object CelebornSparkContextHelper {

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,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.
*/

package org.apache.spark.celeborn

import org.apache.spark.shuffle.celeborn.SparkShuffleManager

import org.apache.celeborn.common.internal.Logging

class StageDependencyManager(shuffleManager: SparkShuffleManager) extends Logging {
def removeCelebornShuffleInternal(
celebornShuffleId: Int,
stageId: Option[Int]): Unit = {
throw new NotImplementedError("the method is not implemented")
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@

import org.apache.spark.SparkConf;
import org.apache.spark.TaskContext;
import org.apache.spark.celeborn.StageDependencyManager;
import org.apache.spark.shuffle.ShuffleHandle;
import org.apache.spark.shuffle.ShuffleReader;

Expand All @@ -42,4 +43,8 @@ public <K, C> ShuffleReader<K, C> getReader(
}
return super.getReader(handle, startPartition, endPartition, context);
}

public StageDependencyManager getStageDepManager() {
return new StageDependencyManager(this);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -22,8 +22,10 @@
import java.util.concurrent.ConcurrentHashMap;

import org.apache.spark.*;
import org.apache.spark.celeborn.StageDependencyManager;
import org.apache.spark.internal.config.package$;
import org.apache.spark.launcher.SparkLauncher;
import org.apache.spark.listener.ShuffleStatsTrackingListener;
import org.apache.spark.rdd.DeterministicLevel;
import org.apache.spark.shuffle.*;
import org.apache.spark.shuffle.sort.SortShuffleManager;
Expand Down Expand Up @@ -91,6 +93,13 @@ public class SparkShuffleManager implements ShuffleManager {

private ExecutorShuffleIdTracker shuffleIdTracker = new ExecutorShuffleIdTracker();

private StageDependencyManager stageDepManager = null;

// 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)) {
logger.warn(
Expand Down Expand Up @@ -177,6 +186,45 @@ private void initializeLifecycleManager(String appId) {
(celebornShuffleId) -> SparkUtils.removeCleanedShuffleId(this, celebornShuffleId));
}

if (lifecycleManager.conf().clientShuffleEarlyDeletion()) {
if (!lifecycleManager.conf().clientStageRerunEnabled()) {
throw new IllegalArgumentException(
CelebornConf.CLIENT_STAGE_RERUN_ENABLED().key()
+ " has to be "
+ "enabled, when "
+ CelebornConf.CLIENT_SHUFFLE_EARLY_DELETION().key()
+ " is set to true");
}
SparkUtils.addSparkListener(new ShuffleStatsTrackingListener());
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();
}

if (celebornConf.getReducerFileGroupBroadcastEnabled()) {
lifecycleManager.registerBroadcastGetReducerFileGroupResponseCallback(
(shuffleId, getReducerFileGroupResponse) ->
Expand Down Expand Up @@ -497,4 +545,8 @@ public LifecycleManager getLifecycleManager() {
public FailedShuffleCleaner getFailedShuffleCleaner() {
return this.failedShuffleCleaner;
}

public StageDependencyManager getStageDepManager() {
return this.stageDepManager;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -20,10 +20,7 @@
import java.io.ByteArrayInputStream;
import java.io.ObjectInputStream;
import java.io.ObjectOutputStream;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.*;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.LongAdder;
Expand Down Expand Up @@ -680,4 +677,56 @@ public static boolean isLocalMaster(SparkConf conf) {
String master = conf.get("spark.master", "");
return master.equals("local") || master.startsWith("local[");
}

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