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

[#1373][FOLLOWUP] fix(spark): shuffle manager rpc service invalid when partition data reassign is enabled #1583

Merged
merged 6 commits into from
Apr 1, 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 @@ -128,6 +128,10 @@ public class RssShuffleManager extends RssShuffleManagerBase {
JavaUtils.newConcurrentMap();
/** Whether to enable the dynamic shuffleServer function rewrite and reread functions */
private boolean rssResubmitStage;

private boolean taskBlockSendFailureRetry;

private boolean shuffleManagerRpcServiceEnabled;
/** A list of shuffleServer for Write failures */
private Set<String> failuresShuffleServerIds = Sets.newHashSet();
/**
Expand Down Expand Up @@ -222,11 +226,14 @@ public RssShuffleManager(SparkConf sparkConf, boolean isDriver) {
this.rssResubmitStage =
rssConf.getBoolean(RssClientConfig.RSS_RESUBMIT_STAGE, false)
&& RssSparkShuffleUtils.isStageResubmitSupported();
this.taskBlockSendFailureRetry =
rssConf.getBoolean(RssClientConf.RSS_CLIENT_BLOCK_SEND_FAILURE_RETRY_ENABLED);
this.shuffleManagerRpcServiceEnabled = taskBlockSendFailureRetry || rssResubmitStage;
if (!sparkConf.getBoolean(RssSparkConfig.RSS_TEST_FLAG.key(), false)) {
if (isDriver) {
heartBeatScheduledExecutorService =
ThreadUtils.getDaemonSingleThreadScheduledExecutor("rss-heartbeat");
if (rssResubmitStage) {
if (shuffleManagerRpcServiceEnabled) {
LOG.info("stage resubmit is supported and enabled");
// start shuffle manager server
rssConf.set(RPC_SERVER_PORT, 0);
Expand Down Expand Up @@ -376,7 +383,7 @@ public <K, V, C> ShuffleHandle registerShuffle(

shuffleIdToPartitionNum.putIfAbsent(shuffleId, dependency.partitioner().numPartitions());
shuffleIdToNumMapTasks.putIfAbsent(shuffleId, dependency.rdd().partitions().length);
if (rssResubmitStage) {
if (shuffleManagerRpcServiceEnabled) {
ShuffleHandleInfo handleInfo =
new ShuffleHandleInfo(shuffleId, partitionToServers, remoteStorage);
shuffleIdToShuffleHandleInfo.put(shuffleId, handleInfo);
Expand Down Expand Up @@ -473,7 +480,7 @@ public <K, V> ShuffleWriter<K, V> getWriter(
int shuffleId = rssHandle.getShuffleId();
String taskId = "" + context.taskAttemptId() + "_" + context.attemptNumber();
ShuffleHandleInfo shuffleHandleInfo;
if (rssResubmitStage) {
if (shuffleManagerRpcServiceEnabled) {
// Get the ShuffleServer list from the Driver based on the shuffleId
shuffleHandleInfo = getRemoteShuffleHandleInfo(shuffleId);
} else {
Expand Down Expand Up @@ -543,7 +550,7 @@ public <K, C> ShuffleReader<K, C> getReader(
+ "]");
start = System.currentTimeMillis();
ShuffleHandleInfo shuffleHandleInfo;
if (rssResubmitStage) {
if (shuffleManagerRpcServiceEnabled) {
// Get the ShuffleServer list from the Driver based on the shuffleId
shuffleHandleInfo = getRemoteShuffleHandleInfo(shuffleId);
} else {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -137,6 +137,10 @@ public class RssShuffleManager extends RssShuffleManagerBase {
private Map<Integer, ShuffleHandleInfo> shuffleIdToShuffleHandleInfo;
/** Whether to enable the dynamic shuffleServer function rewrite and reread functions */
private boolean rssResubmitStage;

private boolean taskBlockSendFailureRetryEnabled;

private boolean shuffleManagerRpcServiceEnabled;
/** A list of shuffleServer for Write failures */
private Set<String> failuresShuffleServerIds;
/**
Expand Down Expand Up @@ -239,10 +243,13 @@ public RssShuffleManager(SparkConf conf, boolean isDriver) {
this.rssResubmitStage =
rssConf.getBoolean(RssClientConfig.RSS_RESUBMIT_STAGE, false)
&& RssSparkShuffleUtils.isStageResubmitSupported();
this.taskBlockSendFailureRetryEnabled =
rssConf.getBoolean(RssClientConf.RSS_CLIENT_BLOCK_SEND_FAILURE_RETRY_ENABLED);
this.shuffleManagerRpcServiceEnabled = taskBlockSendFailureRetryEnabled || rssResubmitStage;
if (isDriver) {
heartBeatScheduledExecutorService =
ThreadUtils.getDaemonSingleThreadScheduledExecutor("rss-heartbeat");
if (rssResubmitStage) {
if (shuffleManagerRpcServiceEnabled) {
LOG.info("stage resubmit is supported and enabled");
// start shuffle manager server
rssConf.set(RPC_SERVER_PORT, 0);
Expand Down Expand Up @@ -471,7 +478,7 @@ public <K, V, C> ShuffleHandle registerShuffle(

shuffleIdToPartitionNum.putIfAbsent(shuffleId, dependency.partitioner().numPartitions());
shuffleIdToNumMapTasks.putIfAbsent(shuffleId, dependency.rdd().partitions().length);
if (rssResubmitStage) {
if (shuffleManagerRpcServiceEnabled) {
ShuffleHandleInfo handleInfo =
new ShuffleHandleInfo(shuffleId, partitionToServers, remoteStorage);
shuffleIdToShuffleHandleInfo.put(shuffleId, handleInfo);
Expand Down Expand Up @@ -509,7 +516,7 @@ public <K, V> ShuffleWriter<K, V> getWriter(
writeMetrics = context.taskMetrics().shuffleWriteMetrics();
}
ShuffleHandleInfo shuffleHandleInfo;
if (rssResubmitStage) {
if (shuffleManagerRpcServiceEnabled) {
// Get the ShuffleServer list from the Driver based on the shuffleId
shuffleHandleInfo = getRemoteShuffleHandleInfo(shuffleId);
} else {
Expand Down Expand Up @@ -651,7 +658,7 @@ public <K, C> ShuffleReader<K, C> getReaderImpl(
final int partitionNum = rssShuffleHandle.getDependency().partitioner().numPartitions();
int shuffleId = rssShuffleHandle.getShuffleId();
ShuffleHandleInfo shuffleHandleInfo;
if (rssResubmitStage) {
if (shuffleManagerRpcServiceEnabled) {
// Get the ShuffleServer list from the Driver based on the shuffleId
shuffleHandleInfo = getRemoteShuffleHandleInfo(shuffleId);
} else {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -112,7 +112,7 @@ public class RssShuffleWriter<K, V, C> extends ShuffleWriter<K, V> {
private final Set<Long> blockIds = Sets.newConcurrentHashSet();
private TaskContext taskContext;
private SparkConf sparkConf;
private boolean taskFailRetry;
private boolean blockSendFailureRetryEnabled;

/** used by columnar rss shuffle writer implementation */
protected final long taskAttemptId;
Expand Down Expand Up @@ -189,10 +189,11 @@ private RssShuffleWriter(
this.taskFailureCallback = taskFailureCallback;
this.taskContext = context;
this.sparkConf = sparkConf;
this.taskFailRetry =
this.blockSendFailureRetryEnabled =
sparkConf.getBoolean(
RssClientConf.RSS_TASK_FAILED_RETRY_ENABLED.key(),
RssClientConf.RSS_TASK_FAILED_RETRY_ENABLED.defaultValue());
RssSparkConfig.SPARK_RSS_CONFIG_PREFIX
+ RssClientConf.RSS_CLIENT_BLOCK_SEND_FAILURE_RETRY_ENABLED.key(),
RssClientConf.RSS_CLIENT_BLOCK_SEND_FAILURE_RETRY_ENABLED.defaultValue());
}

public RssShuffleWriter(
Expand Down Expand Up @@ -420,7 +421,7 @@ protected void checkBlockSendResult(Set<Long> blockIds) {

private void checkIfBlocksFailed() {
Set<Long> failedBlockIds = shuffleManager.getFailedBlockIds(taskId);
if (taskFailRetry && !failedBlockIds.isEmpty()) {
if (blockSendFailureRetryEnabled && !failedBlockIds.isEmpty()) {
Set<TrackingBlockStatus> shouldResendBlockSet = shouldResendBlockStatusSet(failedBlockIds);
try {
reSendFailedBlockIds(shouldResendBlockSet);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -203,10 +203,10 @@ public class RssClientConf {
"This option is only valid when the remote storage path is specified. If ture, "
+ "the remote storage conf will use the client side hadoop configuration loaded from the classpath.");

public static final ConfigOption<Boolean> RSS_TASK_FAILED_RETRY_ENABLED =
ConfigOptions.key("rss.task.failed.retry.enabled")
public static final ConfigOption<Boolean> RSS_CLIENT_BLOCK_SEND_FAILURE_RETRY_ENABLED =
Copy link
Member

Choose a reason for hiding this comment

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

rss.client.blockSendFailureRetry.enabled

Copy link
Contributor

Choose a reason for hiding this comment

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

rss.client.blockSendFailureRetry.enabled -> rss.client.blockSendFailureRetryEnabled.

ConfigOptions.key("rss.client.blockSendFailureRetry.enabled")
.booleanType()
.defaultValue(false)
.withDescription(
"Whether to support task write failed retry internal, default value is false.");
"Whether to support rss client block send failure retry, default value is false.");
}
Loading