-
Notifications
You must be signed in to change notification settings - Fork 135
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
[#1579][part-1] fix(spark): Adjust reassigned time to ensure that all previous data is cleared for stage retry #1584
base: master
Are you sure you want to change the base?
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
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.shuffle; | ||
|
||
public class RssStageResubmit { | ||
private String stageIdAndNumber; | ||
private boolean isReassigned; | ||
|
||
public RssStageResubmit(String stageIdAndNumber, boolean isReassigned) { | ||
this.stageIdAndNumber = stageIdAndNumber; | ||
this.isReassigned = isReassigned; | ||
} | ||
|
||
public String getStageIdAndNumber() { | ||
return stageIdAndNumber; | ||
} | ||
|
||
public void setStageIdAndNumber(String stageIdAndNumber) { | ||
this.stageIdAndNumber = stageIdAndNumber; | ||
} | ||
|
||
public boolean isReassigned() { | ||
return isReassigned; | ||
} | ||
|
||
public void setReassigned(boolean reassigned) { | ||
isReassigned = reassigned; | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,69 @@ | ||
/* | ||
* 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.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); | ||
/** 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<Integer, RssStageResubmit> 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 RssStageResubmit recordAndGetServerAssignedInfo(int shuffleId, String stageIdAndAttempt) { | ||
|
||
return serverAssignedInfos.computeIfAbsent( | ||
shuffleId, id -> new RssStageResubmit(stageIdAndAttempt, false)); | ||
} | ||
|
||
public void recordAndGetServerAssignedInfo( | ||
int shuffleId, String stageIdAndAttempt, boolean isRetried) { | ||
serverAssignedInfos | ||
.computeIfAbsent(shuffleId, id -> new RssStageResubmit(stageIdAndAttempt, false)) | ||
.setReassigned(isRetried); | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,129 @@ | ||
/* | ||
* 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 StageAttemptShuffleHandleInfo extends ShuffleHandleInfoBase { | ||
private static final Logger LOGGER = LoggerFactory.getLogger(StageAttemptShuffleHandleInfo.class); | ||
|
||
private ShuffleHandleInfo current; | ||
private LinkedList<ShuffleHandleInfo> historyHandles; | ||
|
||
public StageAttemptShuffleHandleInfo(ShuffleHandleInfo shuffleServerInfo) { | ||
super(0, null); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Why does this have to be a ShuffleHandleInfo when this is There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
This has been modified, the whole PR migrated to #1762. |
||
this.current = shuffleServerInfo; | ||
this.historyHandles = Lists.newLinkedList(); | ||
} | ||
|
||
public StageAttemptShuffleHandleInfo( | ||
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.StageAttemptShuffleHandleInfo toProto( | ||
StageAttemptShuffleHandleInfo 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.StageAttemptShuffleHandleInfo handleProto = | ||
RssProtos.StageAttemptShuffleHandleInfo.newBuilder() | ||
.setCurrentMutableShuffleHandleInfo(currentMutableShuffleHandleInfo) | ||
.addAllHistoryMutableShuffleHandleInfo(mutableShuffleHandleInfoLinkedList) | ||
.build(); | ||
return handleProto; | ||
} | ||
} | ||
|
||
public static StageAttemptShuffleHandleInfo fromProto( | ||
RssProtos.StageAttemptShuffleHandleInfo 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)); | ||
} | ||
|
||
StageAttemptShuffleHandleInfo stageAttemptShuffleHandleInfo = | ||
new StageAttemptShuffleHandleInfo(mutableShuffleHandleInfo, historyHandles); | ||
return stageAttemptShuffleHandleInfo; | ||
} | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Lack the default constructor to initialize these vars.