-
Notifications
You must be signed in to change notification settings - Fork 154
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
[#1796] fix(spark): Implicitly unregister map output on fetch failure #1797
Draft
zuston
wants to merge
23
commits into
apache:master
Choose a base branch
from
zuston:1796
base: master
Could not load branches
Branch not found: {{ refName }}
Loading
Could not load tags
Nothing to show
Loading
Are you sure you want to change the base?
Some commits from the old base branch may be removed from the timeline,
and old review comments may become outdated.
Draft
Changes from all commits
Commits
Show all changes
23 commits
Select commit
Hold shift + click to select a range
432304f
[#1796] fix(spark): Implicitly unregister map output on fetch failure
zuston 78448d7
refactor
zuston f1e69fe
refactor the shuffle status
zuston a67aa8f
log enhance
zuston 6bf6c96
fix
zuston 0664f5f
fix
zuston 46bc3ae
fix incorrect logic
zuston 9e0c053
refactor write side reassign
zuston 9996dbc
remove dead code
zuston 3d0450f
add unit tests
zuston c8384cd
make the reader fetch failure serverIds into blacklist
zuston 8523d6d
group the same stageId reader to calculate the task failure count
zuston 2d4e9a0
add support of catch the failure result fetch servers
zuston bce9598
remove unnecessary lock
zuston ab2ab2b
register on reassign to use the next attempt number
zuston 07b8790
send blocks with stage attempt number
zuston cbd71ff
fix some bugs
zuston 4bb973b
remove outdate handlers
zuston 55aebb7
get shuffle result with stage attempt number
zuston fee41dd
record the cache clear time
zuston 5c4c9e9
avoid adding the stage retry exception server into blacklist
zuston 5bf6010
avoid removing cost too much times
zuston 73e5020
draft all
zuston File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
130 changes: 130 additions & 0 deletions
130
client-spark/common/src/main/java/org/apache/spark/shuffle/stage/RssShuffleStatus.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,130 @@ | ||
/* | ||
* 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.Comparator; | ||
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 boolean isStageAttemptRetried(int stageAttempt) { | ||
return withReadLock(() -> stageAttemptRetriedRecords.contains(stageAttempt)); | ||
} | ||
|
||
public int getStageRetriedCount() { | ||
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()); | ||
} | ||
|
||
public int getMaxFailureAttemptNumber() { | ||
return withReadLock(() -> taskAttemptFailureRecords.stream().max(Comparator.comparing(Integer::intValue)).orElse(0)); | ||
} | ||
|
||
public Set<Integer> getTaskAttemptFailureRecords() { | ||
return withReadLock(() -> new HashSet<>(taskAttemptFailureRecords)); | ||
} | ||
|
||
public int getStageId() { | ||
return stageId; | ||
} | ||
|
||
public int getShuffleId() { | ||
return shuffleId; | ||
} | ||
} |
24 changes: 24 additions & 0 deletions
24
...-spark/common/src/main/java/org/apache/spark/shuffle/stage/RssShuffleStatusForReader.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
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); | ||
} | ||
} |
Oops, something went wrong.
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
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.
In one case, the Reader triggers retry, and the retry is recorded. After the Writer fails to write data for several times, the retry is triggered. However, this method returns that the retry has been performed.
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.
Yes. The same stageIdAttemptNumber retry will ocurr one time, is this incorrect? @yl09099