Skip to content

Commit

Permalink
refactor the shuffle status
Browse files Browse the repository at this point in the history
  • Loading branch information
zuston committed Jun 19, 2024
1 parent 4eee9a8 commit dea04ac
Show file tree
Hide file tree
Showing 10 changed files with 306 additions and 104 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -19,16 +19,68 @@

import java.util.Map;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;

import com.google.common.collect.Sets;
import org.apache.spark.SparkConf;
import org.apache.spark.shuffle.stage.RssShuffleStatus;
import org.apache.spark.shuffle.stage.RssShuffleStatusForReader;
import org.apache.spark.shuffle.stage.RssShuffleStatusForWriter;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import org.apache.uniffle.common.util.JavaUtils;

public class RssStageResubmitManager {

private static final Logger LOG = LoggerFactory.getLogger(RssStageResubmitManager.class);

private final SparkConf sparkConf = new SparkConf();
private final Map<Integer, RssShuffleStatusForReader> shuffleStatusForReader =
new ConcurrentHashMap<>();
private final Map<Integer, RssShuffleStatusForWriter> shuffleStatusForWriter =
new ConcurrentHashMap<>();

public void clear(int shuffleId) {
shuffleStatusForReader.remove(shuffleId);
shuffleStatusForWriter.remove(shuffleId);
}

public RssShuffleStatus getShuffleStatusForReader(int shuffleId, int stageId, int stageAttempt) {
RssShuffleStatus shuffleStatus =
shuffleStatusForReader.computeIfAbsent(
shuffleId, x -> new RssShuffleStatusForReader(stageId, shuffleId));
if (shuffleStatus.updateStageAttemptIfNecessary(stageAttempt)) {
return shuffleStatus;
}
return null;
}

public RssShuffleStatus getShuffleStatusForWriter(int shuffleId, int stageId, int stageAttempt) {
RssShuffleStatus shuffleStatus =
shuffleStatusForWriter.computeIfAbsent(
shuffleId, x -> new RssShuffleStatusForWriter(stageId, shuffleId));
if (shuffleStatus.updateStageAttemptIfNecessary(stageAttempt)) {
return shuffleStatus;
}
return null;
}

public boolean triggerStageRetry(RssShuffleStatus shuffleStatus) {
final String TASK_MAX_FAILURE = "spark.task.maxFailures";
int sparkTaskMaxFailures = sparkConf.getInt(TASK_MAX_FAILURE, 4);
if (shuffleStatus instanceof RssShuffleStatusForReader) {
if (shuffleStatus.getStageRetriedNumber() > 1) {
LOG.warn("The shuffleId:{}, stageId:{} has been retried. Ignore it.");
return false;
}
if (shuffleStatus.getTaskFailureAttemptCount() >= sparkTaskMaxFailures) {
shuffleStatus.markStageAttemptRetried();
return true;
}
}
return false;
}

/** Blacklist of the Shuffle Server when the write fails. */
private Set<String> serverIdBlackList;
/**
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,109 @@
/*
* 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.stage;

import java.util.HashSet;
import java.util.Set;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import java.util.function.Supplier;

/**
* This class is to track the stage attempt status to check whether to trigger the stage retry of
* Spark.
*/
public class RssShuffleStatus {
private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
private final ReentrantReadWriteLock.ReadLock readLock = lock.readLock();
private final ReentrantReadWriteLock.WriteLock writeLock = lock.writeLock();
private final int stageId;
private final int shuffleId;
// the retried stage attempt records
private final Set<Integer> stageAttemptRetriedRecords;

private int stageAttemptNumber;
// the failed task attempt numbers. Attention: these are not task attempt ids!
private Set<Integer> taskAttemptFailureRecords;

public RssShuffleStatus(int stageId, int shuffleId) {
this.shuffleId = shuffleId;
this.stageId = stageId;
this.stageAttemptRetriedRecords = new HashSet<>();
this.taskAttemptFailureRecords = new HashSet<>();
}

private <T> T withReadLock(Supplier<T> fn) {
readLock.lock();
try {
return fn.get();
} finally {
readLock.unlock();
}
}

private <T> T withWriteLock(Supplier<T> fn) {
writeLock.lock();
try {
return fn.get();
} finally {
writeLock.unlock();
}
}

public int getStageRetriedNumber() {
return withReadLock(() -> this.stageAttemptRetriedRecords.size());
}

public void markStageAttemptRetried() {
withWriteLock(
() -> {
this.stageAttemptRetriedRecords.add(stageAttemptNumber);
return null;
});
}

public int getStageAttempt() {
return withReadLock(() -> this.stageAttemptNumber);
}

public boolean updateStageAttemptIfNecessary(int stageAttempt) {
return withWriteLock(
() -> {
if (this.stageAttemptNumber < stageAttempt) {
// a new stage attempt is issued.
this.stageAttemptNumber = stageAttempt;
this.taskAttemptFailureRecords = new HashSet<>();
return true;
} else if (this.stageAttemptNumber > stageAttempt) {
return false;
}
return true;
});
}

public void incTaskFailure(int taskAttemptNumber) {
withWriteLock(
() -> {
taskAttemptFailureRecords.add(taskAttemptNumber);
return null;
});
}

public int getTaskFailureAttemptCount() {
return withReadLock(() -> taskAttemptFailureRecords.size());
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,24 @@
/*
* 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.stage;

public class RssShuffleStatusForReader extends RssShuffleStatus {
public RssShuffleStatusForReader(int stageId, int shuffleId) {
super(stageId, shuffleId);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,24 @@
/*
* 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.stage;

public class RssShuffleStatusForWriter extends RssShuffleStatus {
public RssShuffleStatusForWriter(int stageId, int shuffleId) {
super(stageId, shuffleId);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -987,4 +987,8 @@ protected RemoteStorageInfo getRemoteStorageInfo() {
public boolean isRssResubmitStage() {
return rssResubmitStage;
}

public RssStageResubmitManager getStageResubmitManager() {
return rssStageResubmitManager;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@
import java.util.Map;

import org.apache.spark.SparkException;
import org.apache.spark.shuffle.RssStageResubmitManager;
import org.apache.spark.shuffle.handle.MutableShuffleHandleInfo;
import org.apache.spark.shuffle.handle.ShuffleHandleInfo;

Expand Down Expand Up @@ -84,4 +85,6 @@ public interface RssShuffleManagerInterface {

MutableShuffleHandleInfo reassignOnBlockSendFailure(
int shuffleId, Map<Integer, List<ReceivingFailureServer>> partitionToFailureServers);

RssStageResubmitManager getStageResubmitManager();
}
Loading

0 comments on commit dea04ac

Please sign in to comment.