-
Notifications
You must be signed in to change notification settings - Fork 28.8k
[SPARK-53636][CORE] Fix thread-safety issue in SortShuffleManager.unregisterShuffle #52386
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
Open
Ngone51
wants to merge
1
commit into
apache:master
Choose a base branch
from
Ngone51:fix
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.
Open
Changes from all commits
Commits
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 hidden or 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 hidden or 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 |
---|---|---|
|
@@ -21,7 +21,7 @@ import java.io._ | |
import java.nio.ByteBuffer | ||
import java.nio.channels.Channels | ||
import java.nio.file.Files | ||
import java.util.{Collections, Map => JMap} | ||
import java.util.concurrent.ConcurrentHashMap | ||
|
||
import scala.collection.mutable.ArrayBuffer | ||
|
||
|
@@ -58,19 +58,19 @@ private[spark] class IndexShuffleBlockResolver( | |
conf: SparkConf, | ||
// var for testing | ||
var _blockManager: BlockManager, | ||
val taskIdMapsForShuffle: JMap[Int, OpenHashSet[Long]]) | ||
val taskIdMapsForShuffle: ConcurrentHashMap[Int, OpenHashSet[Long]]) | ||
extends ShuffleBlockResolver | ||
with Logging with MigratableResolver { | ||
|
||
def this(conf: SparkConf) = { | ||
this(conf, null, Collections.emptyMap()) | ||
this(conf, null, new ConcurrentHashMap[Int, OpenHashSet[Long]]()) | ||
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. class EmptyConcurrentMap<K, V> implements ConcurrentMap<K, V> {
private EmptyConcurrentMap() {
}
public V putIfAbsent(K key, V value) {
return null;
}
public boolean remove(Object key, Object value) {
return false;
}
public boolean replace(K key, V oldValue, V newValue) {
return false;
}
public V replace(K key, V value) {
return null;
}
public int size() {
return 0;
}
public boolean isEmpty() {
return true;
}
public boolean containsKey(Object key) {
return false;
}
public boolean containsValue(Object value) {
return false;
}
public V get(Object key) {
return null;
}
public V put(K key, V value) {
return null;
}
public V remove(Object key) {
return null;
}
public void putAll(Map<? extends K, ? extends V> m) {
}
public void clear() {
}
public Set<K> keySet() {
return Collections.emptySet();
}
public Collection<V> values() {
return Collections.emptySet();
}
public Set<Map.Entry<K, V>> entrySet() {
return Collections.emptySet();
}
} Is it possible to define a similar 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. Hmm, it seems to be overkill to me. |
||
} | ||
|
||
def this(conf: SparkConf, _blockManager: BlockManager) = { | ||
this(conf, _blockManager, Collections.emptyMap()) | ||
this(conf, _blockManager, new ConcurrentHashMap[Int, OpenHashSet[Long]]()) | ||
} | ||
|
||
def this(conf: SparkConf, taskIdMapsForShuffle: JMap[Int, OpenHashSet[Long]]) = { | ||
def this(conf: SparkConf, taskIdMapsForShuffle: ConcurrentHashMap[Int, OpenHashSet[Long]]) = { | ||
this(conf, null, taskIdMapsForShuffle) | ||
} | ||
|
||
|
This file contains hidden or 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 hidden or 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
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.
Can the type be the
ConcurrentMap
interface?