-
Notifications
You must be signed in to change notification settings - Fork 28.5k
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
[SPARK-4452][Core]Shuffle data structures can starve others on the same thread for memory #10024
Closed
Closed
Changes from 9 commits
Commits
Show all changes
21 commits
Select commit
Hold shift + click to select a range
8391185
init commit
lianhuiwang 34f2441
fix minor bug
lianhuiwang bedac89
fix some bugs
lianhuiwang b561641
fix minor style
lianhuiwang 16ca87b
fix minor style
lianhuiwang b9e7071
update code
lianhuiwang ae482a2
Merge branch 'apache-master' into SPARK-4452-2
lianhuiwang 49acacc
Merge branch 'apache-master' into SPARK-4452-2
lianhuiwang 7c36ef0
Merge branch 'apache-master' into SPARK-4452-2
lianhuiwang 70bcffa
fix thread safety & add ut
lianhuiwang b84ad96
fix Mima & minor bug
lianhuiwang dc632f5
fix Mima
lianhuiwang d1ed4e4
fix ut
lianhuiwang 743ef16
update comments
lianhuiwang 97fd174
fix thread safe
lianhuiwang e009d95
fix thread safe
lianhuiwang 7ea7274
fix SpillableIterator
lianhuiwang d16b5f3
merge with master
lianhuiwang e7a98d5
fix Mima
lianhuiwang ff3c2b8
update ut
lianhuiwang 792ff5a
Merge branch 'apache-master' into SPARK-4452-2
lianhuiwang 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
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 |
---|---|---|
|
@@ -93,10 +93,8 @@ private[spark] class ExternalSorter[K, V, C]( | |
partitioner: Option[Partitioner] = None, | ||
ordering: Option[Ordering[K]] = None, | ||
serializer: Serializer = SparkEnv.get.serializer) | ||
extends Logging | ||
with Spillable[WritablePartitionedPairCollection[K, C]] { | ||
|
||
override protected[this] def taskMemoryManager: TaskMemoryManager = context.taskMemoryManager() | ||
extends Spillable[WritablePartitionedPairCollection[K, C]](context.taskMemoryManager()) | ||
with Logging { | ||
|
||
private val conf = SparkEnv.get.conf | ||
|
||
|
@@ -137,6 +135,10 @@ private[spark] class ExternalSorter[K, V, C]( | |
private var _peakMemoryUsedBytes: Long = 0L | ||
def peakMemoryUsedBytes: Long = _peakMemoryUsedBytes | ||
|
||
private var isShuffleSort: Boolean = true | ||
var forceSpillFile: Option[SpilledFile] = None | ||
private var inMemoryOrDiskIterator: Iterator[((Int, K), C)] = null | ||
|
||
// A comparator for keys K that orders them within a partition to allow aggregation or sorting. | ||
// Can be a partial ordering by hash code if a total ordering is not provided through by the | ||
// user. (A partial ordering means that equal keys have comparator.compare(k, k) = 0, but some | ||
|
@@ -161,7 +163,7 @@ private[spark] class ExternalSorter[K, V, C]( | |
// Information about a spilled file. Includes sizes in bytes of "batches" written by the | ||
// serializer as we periodically reset its stream, as well as number of elements in each | ||
// partition, used to efficiently keep track of partitions when merging. | ||
private[this] case class SpilledFile( | ||
private[collection] case class SpilledFile( | ||
file: File, | ||
blockId: BlockId, | ||
serializerBatchSizes: Array[Long], | ||
|
@@ -235,6 +237,52 @@ private[spark] class ExternalSorter[K, V, C]( | |
* @param collection whichever collection we're using (map or buffer) | ||
*/ | ||
override protected[this] def spill(collection: WritablePartitionedPairCollection[K, C]): Unit = { | ||
val inMemoryIterator = collection.destructiveSortedWritablePartitionedIterator(comparator) | ||
val spillFile = spillMemoryIteratorToDisk(inMemoryIterator) | ||
spills.append(spillFile) | ||
} | ||
|
||
/** | ||
* Force to spilling the current in-memory collection to disk to release memory, | ||
* It will be called by TaskMemoryManager when there is not enough memory for the task. | ||
*/ | ||
override protected[this] def forceSpill(): Boolean = { | ||
if (isShuffleSort) { | ||
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 could be triggered by a different thread, so it should be thread safe. |
||
false | ||
} else { | ||
assert(inMemoryOrDiskIterator != null) | ||
val it = inMemoryOrDiskIterator | ||
val inMemoryIterator = new WritablePartitionedIterator { | ||
private[this] var cur = if (it.hasNext) it.next() else null | ||
|
||
def writeNext(writer: DiskBlockObjectWriter): Unit = { | ||
writer.write(cur._1._2, cur._2) | ||
cur = if (it.hasNext) it.next() else null | ||
} | ||
|
||
def hasNext(): Boolean = cur != null | ||
|
||
def nextPartition(): Int = cur._1._1 | ||
} | ||
logInfo(s"Task ${context.taskAttemptId} force spilling in-memory map to disk and " + | ||
s" it will release ${org.apache.spark.util.Utils.bytesToString(getUsed())} memory") | ||
forceSpillFile = Some(spillMemoryIteratorToDisk(inMemoryIterator)) | ||
val spillReader = new SpillReader(forceSpillFile.get) | ||
inMemoryOrDiskIterator = (0 until numPartitions).iterator.flatMap { p => | ||
val iterator = spillReader.readNextPartition() | ||
iterator.map(cur => ((p, cur._1), cur._2)) | ||
} | ||
map = null | ||
buffer = null | ||
true | ||
} | ||
} | ||
|
||
/** | ||
* Spill contents of in-memory iterator to a temporary file on disk. | ||
*/ | ||
private[this] def spillMemoryIteratorToDisk(inMemoryIterator: WritablePartitionedIterator) | ||
: SpilledFile = { | ||
// Because these files may be read during shuffle, their compression must be controlled by | ||
// spark.shuffle.compress instead of spark.shuffle.spill.compress, so we need to use | ||
// createTempShuffleBlock here; see SPARK-3426 for more context. | ||
|
@@ -271,12 +319,11 @@ private[spark] class ExternalSorter[K, V, C]( | |
|
||
var success = false | ||
try { | ||
val it = collection.destructiveSortedWritablePartitionedIterator(comparator) | ||
while (it.hasNext) { | ||
val partitionId = it.nextPartition() | ||
while (inMemoryIterator.hasNext) { | ||
val partitionId = inMemoryIterator.nextPartition() | ||
require(partitionId >= 0 && partitionId < numPartitions, | ||
s"partition Id: ${partitionId} should be in the range [0, ${numPartitions})") | ||
it.writeNext(writer) | ||
inMemoryIterator.writeNext(writer) | ||
elementsPerPartition(partitionId) += 1 | ||
objectsWritten += 1 | ||
|
||
|
@@ -308,7 +355,7 @@ private[spark] class ExternalSorter[K, V, C]( | |
} | ||
} | ||
|
||
spills.append(SpilledFile(file, blockId, batchSizes.toArray, elementsPerPartition)) | ||
SpilledFile(file, blockId, batchSizes.toArray, elementsPerPartition) | ||
} | ||
|
||
/** | ||
|
@@ -599,6 +646,25 @@ private[spark] class ExternalSorter[K, V, C]( | |
} | ||
} | ||
|
||
/** | ||
* Returns a destructive iterator for iterating over the entries of this map. | ||
* If this iterator is forced spill to disk to release memory when there is not enough memory, | ||
* it returns pairs from an on-disk map. | ||
*/ | ||
def destructiveIterator(memoryIterator: Iterator[((Int, K), C)]): Iterator[((Int, K), C)] = { | ||
if (isShuffleSort) { | ||
memoryIterator | ||
} else { | ||
inMemoryOrDiskIterator = memoryIterator | ||
new Iterator[((Int, K), C)] { | ||
|
||
override def hasNext = inMemoryOrDiskIterator.hasNext | ||
|
||
override def next() = inMemoryOrDiskIterator.next() | ||
} | ||
} | ||
} | ||
|
||
/** | ||
* Return an iterator over all the data written to this object, grouped by partition and | ||
* aggregated by the requested aggregator. For each partition we then have an iterator over its | ||
|
@@ -618,21 +684,26 @@ private[spark] class ExternalSorter[K, V, C]( | |
// we don't even need to sort by anything other than partition ID | ||
if (!ordering.isDefined) { | ||
// The user hasn't requested sorted keys, so only sort by partition ID, not key | ||
groupByPartition(collection.partitionedDestructiveSortedIterator(None)) | ||
groupByPartition(destructiveIterator(collection.partitionedDestructiveSortedIterator(None))) | ||
} else { | ||
// We do need to sort by both partition ID and key | ||
groupByPartition(collection.partitionedDestructiveSortedIterator(Some(keyComparator))) | ||
groupByPartition(destructiveIterator( | ||
collection.partitionedDestructiveSortedIterator(Some(keyComparator)))) | ||
} | ||
} else { | ||
// Merge spilled and in-memory data | ||
merge(spills, collection.partitionedDestructiveSortedIterator(comparator)) | ||
merge(spills, destructiveIterator( | ||
collection.partitionedDestructiveSortedIterator(comparator))) | ||
} | ||
} | ||
|
||
/** | ||
* Return an iterator over all the data written to this object, aggregated by our aggregator. | ||
*/ | ||
def iterator: Iterator[Product2[K, C]] = partitionedIterator.flatMap(pair => pair._2) | ||
def iterator: Iterator[Product2[K, C]] = { | ||
isShuffleSort = false | ||
partitionedIterator.flatMap(pair => pair._2) | ||
} | ||
|
||
/** | ||
* Write all the data added into this ExternalSorter into a file in the disk store. This is | ||
|
@@ -689,11 +760,18 @@ private[spark] class ExternalSorter[K, V, C]( | |
} | ||
|
||
def stop(): Unit = { | ||
map = null // So that the memory can be garbage-collected | ||
buffer = null // So that the memory can be garbage-collected | ||
spills.foreach(s => s.file.delete()) | ||
spills.clear() | ||
releaseMemory() | ||
forceSpillFile.foreach(_.file.delete()) | ||
if (map != null || buffer != null) { | ||
map = null // So that the memory can be garbage-collected | ||
buffer = null // So that the memory can be garbage-collected | ||
releaseMemory() | ||
} | ||
} | ||
|
||
override def toString(): String = { | ||
this.getClass.getName + "@" + java.lang.Integer.toHexString(this.hashCode()) | ||
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 this? |
||
} | ||
|
||
/** | ||
|
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.
This function does not actually create any object, I'd like to call it
acquireOnHeapMemory