Skip to content

Commit

Permalink
[apache#1579][part-1] fix(spark): Adjust reassigned time to ensure th…
Browse files Browse the repository at this point in the history
…at all previous data is cleared for stage retry
  • Loading branch information
yl09099 committed May 18, 2024
1 parent c3cbdec commit e7000e5
Show file tree
Hide file tree
Showing 26 changed files with 933 additions and 753 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -517,7 +517,8 @@ public void registerShuffle(
List<PartitionRange> partitionRanges,
RemoteStorageInfo remoteStorage,
ShuffleDataDistributionType distributionType,
int maxConcurrencyPerPartitionToWrite) {}
int maxConcurrencyPerPartitionToWrite,
int stageAttemptNumber) {}

@Override
public boolean sendCommit(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -504,7 +504,8 @@ public void registerShuffle(
List<PartitionRange> partitionRanges,
RemoteStorageInfo storageType,
ShuffleDataDistributionType distributionType,
int maxConcurrencyPerPartitionToWrite) {}
int maxConcurrencyPerPartitionToWrite,
int stageAttemptNumber) {}

@Override
public boolean sendCommit(
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,60 @@
/*
* 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;

import java.util.Map;
import java.util.Set;

import com.google.common.collect.Sets;

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

public class RssStageResubmitManager {
/** A list of shuffleServer for Write failures */
private Set<String> failuresShuffleServerIds;
/**
* Prevent multiple tasks from reporting FetchFailed, resulting in multiple ShuffleServer
* assignments, stageID, Attemptnumber Whether to reassign the combination flag;
*/
private Map<String, Boolean> serverAssignedInfos;

public RssStageResubmitManager() {
this.failuresShuffleServerIds = Sets.newConcurrentHashSet();
this.serverAssignedInfos = JavaUtils.newConcurrentMap();
}

public Set<String> getFailuresShuffleServerIds() {
return failuresShuffleServerIds;
}

public void setFailuresShuffleServerIds(Set<String> failuresShuffleServerIds) {
this.failuresShuffleServerIds = failuresShuffleServerIds;
}

public void recordFailuresShuffleServer(String shuffleServerId) {
failuresShuffleServerIds.add(shuffleServerId);
}

public boolean recordAndGetServerAssignedInfo(String stageIdAndAttempt) {
return serverAssignedInfos.computeIfAbsent(stageIdAndAttempt, id -> false);
}

public void recordAndGetServerAssignedInfo(String stageIdAndAttempt, boolean isRetried) {
serverAssignedInfos.put(stageIdAndAttempt, isRetried);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,127 @@
/*
* 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.handle;

import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.Set;

import com.google.common.collect.Lists;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import org.apache.uniffle.client.PartitionDataReplicaRequirementTracking;
import org.apache.uniffle.common.ShuffleServerInfo;
import org.apache.uniffle.proto.RssProtos;

public class ChainShuffleHandleInfo extends ShuffleHandleInfoBase {
private static final Logger LOGGER = LoggerFactory.getLogger(MutableShuffleHandleInfo.class);

private ShuffleHandleInfo current;
private LinkedList<ShuffleHandleInfo> historyHandles;

public ChainShuffleHandleInfo(ShuffleHandleInfo shuffleServerInfo) {
super(0, null);
this.current = shuffleServerInfo;
this.historyHandles = Lists.newLinkedList();
}

public ChainShuffleHandleInfo(
ShuffleHandleInfo currentShuffleServerInfo, LinkedList<ShuffleHandleInfo> historyHandles) {
super(0, null);
this.current = currentShuffleServerInfo;
this.historyHandles = historyHandles;
}

@Override
public Set<ShuffleServerInfo> getServers() {
return current.getServers();
}

@Override
public Map<Integer, List<ShuffleServerInfo>> getAvailablePartitionServersForWriter() {
return current.getAvailablePartitionServersForWriter();
}

@Override
public Map<Integer, List<ShuffleServerInfo>> getAllPartitionServersForReader() {
return current.getAllPartitionServersForReader();
}

@Override
public PartitionDataReplicaRequirementTracking createPartitionReplicaTracking() {
return current.createPartitionReplicaTracking();
}

/**
* When a Stage retry occurs, replace the current shuffleHandleInfo and record the historical
* shuffleHandleInfo.
*/
public void replaceCurrentShuffleHandleInfo(ShuffleHandleInfo shuffleHandleInfo) {
this.historyHandles.add(current);
this.current = shuffleHandleInfo;
}

public ShuffleHandleInfo getCurrent() {
return current;
}

public LinkedList<ShuffleHandleInfo> getHistoryHandles() {
return historyHandles;
}

public static RssProtos.ChainShuffleHandleInfo toProto(ChainShuffleHandleInfo handleInfo) {
synchronized (handleInfo) {
LinkedList<RssProtos.MutableShuffleHandleInfo> mutableShuffleHandleInfoLinkedList =
Lists.newLinkedList();
RssProtos.MutableShuffleHandleInfo currentMutableShuffleHandleInfo =
MutableShuffleHandleInfo.toProto((MutableShuffleHandleInfo) handleInfo.getCurrent());
for (ShuffleHandleInfo historyHandle : handleInfo.getHistoryHandles()) {
mutableShuffleHandleInfoLinkedList.add(
MutableShuffleHandleInfo.toProto((MutableShuffleHandleInfo) historyHandle));
}
RssProtos.ChainShuffleHandleInfo handleProto =
RssProtos.ChainShuffleHandleInfo.newBuilder()
.setCurrentMutableShuffleHandleInfo(currentMutableShuffleHandleInfo)
.addAllHistoryMutableShuffleHandleInfo(mutableShuffleHandleInfoLinkedList)
.build();
return handleProto;
}
}

public static ChainShuffleHandleInfo fromProto(RssProtos.ChainShuffleHandleInfo handleProto) {
if (handleProto == null) {
return null;
}

MutableShuffleHandleInfo mutableShuffleHandleInfo =
MutableShuffleHandleInfo.fromProto(handleProto.getCurrentMutableShuffleHandleInfo());
List<RssProtos.MutableShuffleHandleInfo> historyMutableShuffleHandleInfoList =
handleProto.getHistoryMutableShuffleHandleInfoList();
LinkedList<ShuffleHandleInfo> historyHandles = Lists.newLinkedList();
for (RssProtos.MutableShuffleHandleInfo shuffleHandleInfo :
historyMutableShuffleHandleInfoList) {
historyHandles.add(MutableShuffleHandleInfo.fromProto(shuffleHandleInfo));
}

ChainShuffleHandleInfo chainShuffleHandleInfo =
new ChainShuffleHandleInfo(mutableShuffleHandleInfo, historyHandles);
return chainShuffleHandleInfo;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -201,42 +201,40 @@ public void checkPartitionReassignServerNum(
}

public static RssProtos.MutableShuffleHandleInfo toProto(MutableShuffleHandleInfo handleInfo) {
synchronized (handleInfo) {
Map<Integer, RssProtos.PartitionReplicaServers> partitionToServers = new HashMap<>();
for (Map.Entry<Integer, Map<Integer, List<ShuffleServerInfo>>> entry :
handleInfo.partitionReplicaAssignedServers.entrySet()) {
int partitionId = entry.getKey();

Map<Integer, RssProtos.ReplicaServersItem> replicaServersProto = new HashMap<>();
Map<Integer, List<ShuffleServerInfo>> replicaServers = entry.getValue();
for (Map.Entry<Integer, List<ShuffleServerInfo>> replicaServerEntry :
replicaServers.entrySet()) {
RssProtos.ReplicaServersItem item =
RssProtos.ReplicaServersItem.newBuilder()
.addAllServerId(ShuffleServerInfo.toProto(replicaServerEntry.getValue()))
.build();
replicaServersProto.put(replicaServerEntry.getKey(), item);
}
Map<Integer, RssProtos.PartitionReplicaServers> partitionToServers = new HashMap<>();
for (Map.Entry<Integer, Map<Integer, List<ShuffleServerInfo>>> entry :
handleInfo.partitionReplicaAssignedServers.entrySet()) {
int partitionId = entry.getKey();

RssProtos.PartitionReplicaServers partitionReplicaServerProto =
RssProtos.PartitionReplicaServers.newBuilder()
.putAllReplicaServers(replicaServersProto)
Map<Integer, RssProtos.ReplicaServersItem> replicaServersProto = new HashMap<>();
Map<Integer, List<ShuffleServerInfo>> replicaServers = entry.getValue();
for (Map.Entry<Integer, List<ShuffleServerInfo>> replicaServerEntry :
replicaServers.entrySet()) {
RssProtos.ReplicaServersItem item =
RssProtos.ReplicaServersItem.newBuilder()
.addAllServerId(ShuffleServerInfo.toProto(replicaServerEntry.getValue()))
.build();
partitionToServers.put(partitionId, partitionReplicaServerProto);
replicaServersProto.put(replicaServerEntry.getKey(), item);
}

RssProtos.MutableShuffleHandleInfo handleProto =
RssProtos.MutableShuffleHandleInfo.newBuilder()
.setShuffleId(handleInfo.shuffleId)
.setRemoteStorageInfo(
RssProtos.RemoteStorageInfo.newBuilder()
.setPath(handleInfo.remoteStorage.getPath())
.putAllConfItems(handleInfo.remoteStorage.getConfItems())
.build())
.putAllPartitionToServers(partitionToServers)
RssProtos.PartitionReplicaServers partitionReplicaServerProto =
RssProtos.PartitionReplicaServers.newBuilder()
.putAllReplicaServers(replicaServersProto)
.build();
return handleProto;
partitionToServers.put(partitionId, partitionReplicaServerProto);
}

RssProtos.MutableShuffleHandleInfo handleProto =
RssProtos.MutableShuffleHandleInfo.newBuilder()
.setShuffleId(handleInfo.shuffleId)
.setRemoteStorageInfo(
RssProtos.RemoteStorageInfo.newBuilder()
.setPath(handleInfo.remoteStorage.getPath())
.putAllConfItems(handleInfo.remoteStorage.getConfItems())
.build())
.putAllPartitionToServers(partitionToServers)
.build();
return handleProto;
}

public static MutableShuffleHandleInfo fromProto(RssProtos.MutableShuffleHandleInfo handleProto) {
Expand Down

0 comments on commit e7000e5

Please sign in to comment.