Skip to content
Snippets Groups Projects
Commit 8703898d authored by Aaron Davidson's avatar Aaron Davidson
Browse files

Address Reynold's comments

parent 3ca52309
No related branches found
No related tags found
No related merge requests found
......@@ -45,7 +45,7 @@ trait ShuffleBlocks {
}
/**
* Manages assigning disk-based block writers to shuffle tasks. Each shuffle task gets one file and
* Manages assigning disk-based block writers to shuffle tasks. Each shuffle task gets one file
* per reducer (this set of files is called a ShuffleFileGroup).
*
* As an optimization to reduce the number of physical shuffle files produced, multiple shuffle
......@@ -57,11 +57,13 @@ trait ShuffleBlocks {
* - bucketId: The id of the output partition (i.e., reducer id)
* - fileId: The unique id identifying a group of "combined shuffle files." Only one task at a
* time owns a particular fileId, and this id is returned to a pool when the task finishes.
* Each shuffle file is then mapped to a FileSegment, which is a 3-tuple (file, offset, length)
* that specifies where in a given file the actual block data is located.
*
* Shuffle file metadata is stored in a space-efficient manner. Rather than simply mapping
* ShuffleBlockIds to FileSegments, each ShuffleFile maintains a list of offsets for each block
* stored in that file. In order to find the location of a shuffle block, we search all ShuffleFiles
* destined for the block's reducer.
* ShuffleBlockIds directly to FileSegments, each ShuffleFile maintains a list of offsets for each
* block stored in that file. In order to find the location of a shuffle block, we search all
* ShuffleFiles destined for the block's reducer.
*
*/
private[spark]
......@@ -98,18 +100,22 @@ class ShuffleBlockManager(blockManager: BlockManager) extends Logging {
private
val metadataCleaner = new MetadataCleaner(MetadataCleanerType.SHUFFLE_BLOCK_MANAGER, this.cleanup)
def forShuffle(shuffleId: Int, numBuckets: Int, serializer: Serializer) = {
def forShuffle(shuffleId: Int, numBuckets: Int, serializer: Serializer): ShuffleBlocks = {
initializeShuffleMetadata(shuffleId, numBuckets)
new ShuffleBlocks {
override def acquireWriters(mapId: Int): ShuffleWriterGroup = {
val bufferSize = System.getProperty("spark.shuffle.file.buffer.kb", "100").toInt * 1024
val fileGroup = getUnusedFileGroup(shuffleId, mapId, numBuckets)
val writers = Array.tabulate[BlockObjectWriter](numBuckets) { bucketId =>
val blockId = ShuffleBlockId(shuffleId, mapId, bucketId)
if (consolidateShuffleFiles) {
var fileGroup: ShuffleFileGroup = null
val writers = if (consolidateShuffleFiles) {
fileGroup = getUnusedFileGroup(shuffleId, mapId, numBuckets)
Array.tabulate[BlockObjectWriter](numBuckets) { bucketId =>
val blockId = ShuffleBlockId(shuffleId, mapId, bucketId)
blockManager.getDiskWriter(blockId, fileGroup(bucketId).file, serializer, bufferSize)
} else {
}
} else {
Array.tabulate[BlockObjectWriter](numBuckets) { bucketId =>
val blockId = ShuffleBlockId(shuffleId, mapId, bucketId)
val blockFile = blockManager.diskBlockManager.getFile(blockId)
blockManager.getDiskWriter(blockId, blockFile, serializer, bufferSize)
}
......@@ -142,8 +148,6 @@ class ShuffleBlockManager(blockManager: BlockManager) extends Logging {
}
private def getUnusedFileGroup(shuffleId: Int, mapId: Int, numBuckets: Int): ShuffleFileGroup = {
if (!consolidateShuffleFiles) { return null }
val pool = shuffleToFileGroupPoolMap(shuffleId)
var fileGroup = pool.getUnusedFileGroup()
......
0% Loading or .
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment