Skip to content
Closed
Show file tree
Hide file tree
Changes from 1 commit
Commits
Show all changes
27 commits
Select commit Hold shift + click to select a range
779c0f9
initial commit of sort-merge shuffle reader
jerryshao Sep 5, 2014
4f46dc0
Readability improvements to SortShuffleReader
sryza Oct 22, 2014
0861cf9
Clarify mergeWidth logic
sryza Oct 23, 2014
8f49b78
Add blocks remaining at level counter back in
sryza Oct 23, 2014
fcafa16
Small fix
sryza Oct 24, 2014
21dae69
Move merge to a separate class and use a priority queue instead of le…
sryza Oct 25, 2014
8e3766a
Rebase to the latest code and fix some conflicts
jerryshao Oct 30, 2014
98c039b
SortShuffleReader code improvement
jerryshao Nov 4, 2014
7d999ef
Changes to rebase to the latest master branch
jerryshao Nov 5, 2014
319e6d1
Don't spill more blocks than we need to
sryza Nov 5, 2014
96ef5c1
Fix bug: add to inMemoryBlocks
sryza Nov 5, 2014
d481c98
Fix another bug
sryza Nov 5, 2014
bf6a49d
Bug fix and revert ShuffleMemoryManager
jerryshao Nov 5, 2014
79dc823
Fix some bugs in spilling to disk
jerryshao Nov 7, 2014
2e04b85
Modify to use BlockObjectWriter to write data
jerryshao Nov 10, 2014
c1f97b6
Fix incorrect block size introduced bugs
jerryshao Nov 11, 2014
b5e472d
Address the comments
jerryshao Nov 12, 2014
40c59df
Fix some bugs
jerryshao Nov 12, 2014
42bf77d
Improve the failure process and expand ManagedBuffer
jerryshao Nov 14, 2014
a9eaef8
Copy the memory from off-heap to on-heap and some code style modifica…
jerryshao Nov 17, 2014
6f48c5c
Fix rebase introduced issue
jerryshao Nov 18, 2014
c2ddcce
Revert some unwanted changes
jerryshao Nov 18, 2014
f170db3
Clean up comments, break up large methods, spill based on actual bloc…
sryza Nov 24, 2014
123aea1
Log improve
jerryshao Nov 25, 2014
e035105
Fix scala style issue
jerryshao Nov 25, 2014
8b73701
Fix rebase issues
jerryshao Feb 22, 2015
d6c94da
Fix dead lock
jerryshao Apr 13, 2015
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Prev Previous commit
Next Next commit
Copy the memory from off-heap to on-heap and some code style modifica…
…tion
  • Loading branch information
jerryshao committed Apr 13, 2015
commit a9eaef8a695d85b4a80f2c4b3835d60352fab271
Original file line number Diff line number Diff line change
Expand Up @@ -18,15 +18,15 @@
package org.apache.spark.shuffle.sort

import java.io.FileOutputStream
import java.nio.ByteBuffer
import java.util.Comparator

import org.apache.spark.executor.ShuffleWriteMetrics

import scala.collection.mutable.{ArrayBuffer, HashMap, Queue}
import scala.util.{Failure, Success, Try}

import org.apache.spark._
import org.apache.spark.network.buffer.ManagedBuffer
import org.apache.spark.executor.ShuffleWriteMetrics
import org.apache.spark.network.buffer.{ManagedBuffer, NioManagedBuffer}
import org.apache.spark.serializer.Serializer
import org.apache.spark.shuffle.{BaseShuffleHandle, FetchFailedException, ShuffleReader}
import org.apache.spark.storage._
Expand Down Expand Up @@ -129,15 +129,24 @@ private[spark] class SortShuffleReader[K, C](
// Try to fit block in memory. If this fails, merge in-memory blocks to disk.
val blockSize = blockData.size
val granted = shuffleMemoryManager.tryToAcquire(blockSize)
val block = MemoryShuffleBlock(blockId, blockData)
if (granted >= blockSize) {
inMemoryBlocks += MemoryShuffleBlock(blockId, blockData)
if (blockData.isDirect) {
// If the memory shuffle block is allocated on direct buffer, copy it on heap,
// otherwise off heap memory will be increased out of control.
val onHeapBuffer = ByteBuffer.allocate(blockSize.toInt)
onHeapBuffer.put(blockData.nioByteBuffer)

inMemoryBlocks += MemoryShuffleBlock(blockId, new NioManagedBuffer(onHeapBuffer))
blockData.release()
} else {
inMemoryBlocks += MemoryShuffleBlock(blockId, blockData)
}
} else {
logInfo(s"Granted $granted memory is not enough to store shuffle block id $blockId, " +
logDebug(s"Granted $granted memory is not enough to store shuffle block id $blockId, " +
s"block size $blockSize, spilling in-memory blocks to release the memory")

shuffleMemoryManager.release(granted)
spillInMemoryBlocks(block)
spillInMemoryBlocks(MemoryShuffleBlock(blockId, blockData))
}

unfetchedBytes -= shuffleBlockMap(blockId.asInstanceOf[ShuffleBlockId])._2
Expand All @@ -164,7 +173,6 @@ private[spark] class SortShuffleReader[K, C](
}
inMemoryBlocks.clear()
}

context.addTaskCompletionListener(_ => releaseFinalShuffleMemory())

// Release the in-memory block when iteration is completed.
Expand Down Expand Up @@ -214,7 +222,7 @@ private[spark] class SortShuffleReader[K, C](
var success = false

try {
partialMergedItr.foreach(p => writer.write(p))
partialMergedItr.foreach(writer.write)
success = true
} finally {
if (!success) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -61,9 +61,6 @@ private[spark] class TieredDiskMerger[K, C](
private val blockManager = SparkEnv.get.blockManager
private val ser = Serializer.getSerializer(dep.serializer)

/** Number of bytes spilled on disk */
private var _diskBytesSpilled: Long = 0L

/** PriorityQueue to store the on-disk merging blocks, blocks are merged by size ordering */
private val onDiskBlocks = new PriorityBlockingQueue[DiskShuffleBlock]()

Expand All @@ -77,6 +74,11 @@ private[spark] class TieredDiskMerger[K, C](

@volatile private var doneRegistering = false

/** Number of bytes spilled on disk */
private var _diskBytesSpilled: Long = 0L

def diskBytesSpilled: Long = _diskBytesSpilled

def registerOnDiskBlock(blockId: BlockId, file: File): Unit = {
assert(!doneRegistering)
onDiskBlocks.put(new DiskShuffleBlock(blockId, file, file.length()))
Expand All @@ -88,8 +90,6 @@ private[spark] class TieredDiskMerger[K, C](
}
}

def diskBytesSpilled: Long = _diskBytesSpilled

/**
* Notify the merger that no more on disk blocks will be registered.
*/
Expand Down Expand Up @@ -184,9 +184,10 @@ private[spark] class TieredDiskMerger[K, C](
val blocksToMerge = new ArrayBuffer[DiskShuffleBlock]()
// Try to pick the smallest merge width that will result in the next merge being the final
// merge.
val mergeFactor = math.min(onDiskBlocks.size - maxMergeFactor + 1, maxMergeFactor)
(0 until mergeFactor).foreach {
var mergeFactor = math.min(onDiskBlocks.size - maxMergeFactor + 1, maxMergeFactor)
while (mergeFactor > 0) {
blocksToMerge += onDiskBlocks.take()
mergeFactor -= 1
}

// Merge the blocks
Expand All @@ -201,7 +202,7 @@ private[spark] class TieredDiskMerger[K, C](
var success = false

try {
partialMergedItr.foreach(p => writer.write(p))
partialMergedItr.foreach(writer.write)
success = true
} finally {
if (!success) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -69,9 +69,9 @@ public abstract class ManagedBuffer {
*/
public abstract Object convertToNetty() throws IOException;

/**
* Tell whether to not this byte buffer is direct
* @return
*/
/**
* Tell whether to not this byte buffer is direct
* @return
*/
public abstract boolean isDirect();
}
Original file line number Diff line number Diff line change
Expand Up @@ -101,4 +101,9 @@ public boolean equals(Object other) {
}
return false;
}

@Override
public boolean isDirect() {
return underlying.isDirect();
}
}