Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
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
Original file line number Diff line number Diff line change
@@ -0,0 +1,42 @@
/*
* 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.sort

import org.apache.spark.{TaskContext, Logging}
import org.apache.spark.shuffle.{BaseShuffleHandle, ShuffleReader}
import org.apache.spark.shuffle.hash.HashShuffleReader

/**
* ShuffleReader that chooses SortShuffleReader or HashShuffleReader depending on whether there is
* a key ordering.
*/
private[spark] class MixedShuffleReader[K, C](
handle: BaseShuffleHandle[K, _, C],
startPartition: Int,
endPartition: Int,
context: TaskContext)
extends ShuffleReader[K, C] with Logging {

private val shuffleReader = if (handle.dependency.keyOrdering.isDefined) {
new SortShuffleReader[K, C](handle, startPartition, endPartition, context)
} else {
new HashShuffleReader[K, C](handle, startPartition, endPartition, context)
}

override def read(): Iterator[Product2[K, C]] = shuffleReader.read()
}
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,6 @@ import java.util.concurrent.ConcurrentHashMap

import org.apache.spark.{SparkConf, TaskContext, ShuffleDependency}
import org.apache.spark.shuffle._
import org.apache.spark.shuffle.hash.HashShuffleReader

private[spark] class SortShuffleManager(conf: SparkConf) extends ShuffleManager {

Expand All @@ -48,7 +47,7 @@ private[spark] class SortShuffleManager(conf: SparkConf) extends ShuffleManager
endPartition: Int,
context: TaskContext): ShuffleReader[K, C] = {
// We currently use the same block store shuffle fetcher as the hash-based shuffle.
new HashShuffleReader(
new MixedShuffleReader(
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can we make it configurable (as the experimental feature for now) for turn on/off the MixedShuffleReader? Let's say we enable the HashShuffleReader by default, not the MixedShuffleReader

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yeah, I will figure out a way which is more meaningful.

handle.asInstanceOf[BaseShuffleHandle[K, _, C]], startPartition, endPartition, context)
}

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,337 @@
/*
* 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.sort

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

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

import org.apache.spark._
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._
import org.apache.spark.util.{CompletionIterator, Utils}
import org.apache.spark.util.collection.{MergeUtil, TieredDiskMerger}

/**
* SortShuffleReader merges and aggregates shuffle data that has already been sorted within each
* map output block.
*
* As blocks are fetched, we store them in memory until we fail to acquire space from the
* ShuffleMemoryManager. When this occurs, we merge some in-memory blocks to disk and go back to
* fetching.
*
* TieredDiskMerger is responsible for managing the merged on-disk blocks and for supplying an
* iterator with their merged contents. The final iterator that is passed to user code merges this
* on-disk iterator with the in-memory blocks that have not yet been spilled.
*/
private[spark] class SortShuffleReader[K, C](
handle: BaseShuffleHandle[K, _, C],
startPartition: Int,
endPartition: Int,
context: TaskContext)
extends ShuffleReader[K, C] with Logging {

/** Manage the fetched in-memory shuffle block and related buffer */
case class MemoryShuffleBlock(blockId: BlockId, blockData: ManagedBuffer)

require(endPartition == startPartition + 1,
"Sort shuffle currently only supports fetching one partition")

private val dep = handle.dependency
private val conf = SparkEnv.get.conf
private val blockManager = SparkEnv.get.blockManager
private val ser = Serializer.getSerializer(dep.serializer)
private val shuffleMemoryManager = SparkEnv.get.shuffleMemoryManager

private val fileBufferSize = conf.getInt("spark.shuffle.file.buffer.kb", 32) * 1024

/** Queue to store in-memory shuffle blocks */
private val inMemoryBlocks = new Queue[MemoryShuffleBlock]()

/**
* Maintain block manager and reported size of each shuffle block. The block manager is used for
* error reporting. The reported size, which, because of size compression, may be slightly
* different than the size of the actual fetched block, is used for calculating how many blocks
* to spill.
*/
private val shuffleBlockMap = new HashMap[ShuffleBlockId, (BlockManagerId, Long)]()

/** keyComparator for mergeSort, id keyOrdering is not available,
* using hashcode of key to compare */
private val keyComparator: Comparator[K] = dep.keyOrdering.getOrElse(new Comparator[K] {
override def compare(a: K, b: K) = {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We never run into here right? As dep.keyOrdering should be always defined for the SortShuffleReader, simply throws an exception?

val h1 = if (a == null) 0 else a.hashCode()
val h2 = if (b == null) 0 else b.hashCode()
if (h1 < h2) -1 else if (h1 == h2) 0 else 1
}
})

/** A merge thread to merge on-disk blocks */
private val tieredMerger = new TieredDiskMerger(conf, dep, keyComparator, context)

/** Shuffle block fetcher iterator */
private var shuffleRawBlockFetcherItr: ShuffleRawBlockFetcherIterator = _

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

/** Number of bytes left to fetch */
private var unfetchedBytes: Long = 0L

def memoryBytesSpilled: Long = _memoryBytesSpilled

def diskBytesSpilled: Long = _diskBytesSpilled + tieredMerger.diskBytesSpilled

override def read(): Iterator[Product2[K, C]] = {
tieredMerger.start()

computeShuffleBlocks()

for ((blockId, blockOption) <- fetchShuffleBlocks()) {
val blockData = blockOption match {
case Success(b) => b
case Failure(e) =>
blockId match {
case b @ ShuffleBlockId(shuffleId, mapId, _) =>
val address = shuffleBlockMap(b)._1
throw new FetchFailedException (address, shuffleId.toInt, mapId.toInt, startPartition,
Utils.exceptionString (e))
case _ =>
throw new SparkException (
s"Failed to get block $blockId, which is not a shuffle block", e)
}
}

shuffleRawBlockFetcherItr.currentResult = null

// Try to fit block in memory. If this fails, merge in-memory blocks to disk.
val blockSize = blockData.size
val granted = shuffleMemoryManager.tryToAcquire(blockSize)
if (granted >= blockSize) {
if (blockData.isDirect) {
// If the shuffle block is allocated on a direct buffer, copy it to an on-heap buffer,
// otherwise off heap memory will be increased to the shuffle memory size.
val onHeapBuffer = ByteBuffer.allocate(blockSize.toInt)
onHeapBuffer.put(blockData.nioByteBuffer)

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

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

unfetchedBytes -= shuffleBlockMap(blockId.asInstanceOf[ShuffleBlockId])._2
}

// Make sure all the blocks have been fetched.
assert(unfetchedBytes == 0L)

tieredMerger.doneRegisteringOnDiskBlocks()

// Merge on-disk blocks with in-memory blocks to directly feed to the reducer.
val finalItrGroup = inMemoryBlocksToIterators(inMemoryBlocks) ++ Seq(tieredMerger.readMerged())
val mergedItr =
MergeUtil.mergeSort(finalItrGroup, keyComparator, dep.keyOrdering, dep.aggregator)

// Update the spill metrics and do cleanup work when task is finished.
context.taskMetrics().incMemoryBytesSpilled(memoryBytesSpilled)
context.taskMetrics().incDiskBytesSpilled(diskBytesSpilled)

def releaseFinalShuffleMemory(): Unit = {
inMemoryBlocks.foreach { block =>
block.blockData.release()
shuffleMemoryManager.release(block.blockData.size)
}
inMemoryBlocks.clear()
}
context.addTaskCompletionListener(_ => releaseFinalShuffleMemory())

// Release the in-memory block when iteration is completed.
val completionItr = CompletionIterator[Product2[K, C], Iterator[Product2[K, C]]](
mergedItr, releaseFinalShuffleMemory())

new InterruptibleIterator(context, completionItr.map(p => (p._1, p._2)))
}

/**
* Called when we've failed to acquire memory for a block we've just fetched. Figure out how many
* blocks to spill and then spill them.
*/
private def spillInMemoryBlocks(tippingBlock: MemoryShuffleBlock): Unit = {
val (tmpBlockId, file) = blockManager.diskBlockManager.createTempShuffleBlock()

// If the remaining unfetched data would fit inside our current allocation, we don't want to
// waste time spilling blocks beyond the space needed for it.
// Note that the number of unfetchedBytes is not exact, because of the compression used on the
// sizes of map output blocks.
var bytesToSpill = unfetchedBytes
val blocksToSpill = new ArrayBuffer[MemoryShuffleBlock]()
blocksToSpill += tippingBlock
bytesToSpill -= tippingBlock.blockData.size
while (bytesToSpill > 0 && !inMemoryBlocks.isEmpty) {
val block = inMemoryBlocks.dequeue()
blocksToSpill += block
bytesToSpill -= block.blockData.size
}

_memoryBytesSpilled += blocksToSpill.map(_.blockData.size()).sum

if (blocksToSpill.size > 1) {
spillMultipleBlocks(file, tmpBlockId, blocksToSpill, tippingBlock)
} else {
spillSingleBlock(file, blocksToSpill.head)
}

tieredMerger.registerOnDiskBlock(tmpBlockId, file)

logInfo(s"Merged ${blocksToSpill.size} in-memory blocks into file ${file.getName}")
}

private def spillSingleBlock(file: File, block: MemoryShuffleBlock): Unit = {
val fos = new FileOutputStream(file)
val buffer = block.blockData.nioByteBuffer()
var channel = fos.getChannel
var success = false

try {
while (buffer.hasRemaining) {
channel.write(buffer)
}
success = true
} finally {
if (channel != null) {
channel.close()
channel = null
}
if (!success) {
if (file.exists()) {
file.delete()
}
} else {
_diskBytesSpilled += file.length()
}
// When we spill a single block, it's the single tipping block that we never acquired memory
// from the shuffle memory manager for, so we don't need to release any memory from there.
block.blockData.release()
}
}

/**
* Merge multiple in-memory blocks to a single on-disk file.
*/
private def spillMultipleBlocks(file: File, tmpBlockId: BlockId,
blocksToSpill: Seq[MemoryShuffleBlock], tippingBlock: MemoryShuffleBlock): Unit = {
val itrGroup = inMemoryBlocksToIterators(blocksToSpill)
val partialMergedItr =
MergeUtil.mergeSort(itrGroup, keyComparator, dep.keyOrdering, dep.aggregator)
val curWriteMetrics = new ShuffleWriteMetrics()
var writer = blockManager.getDiskWriter(tmpBlockId, file, ser, fileBufferSize, curWriteMetrics)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

getDiskWriter had changed on #5606

var success = false

try {
partialMergedItr.foreach(writer.write)
success = true
} finally {
if (!success) {
if (writer != null) {
writer.revertPartialWritesAndClose()
writer = null
}
if (file.exists()) {
file.delete()
}
} else {
writer.commitAndClose()
writer = null
}
for (block <- blocksToSpill) {
block.blockData.release()
if (block != tippingBlock) {
shuffleMemoryManager.release(block.blockData.size)
}
}
}
_diskBytesSpilled += curWriteMetrics.shuffleBytesWritten
}

private def inMemoryBlocksToIterators(blocks: Seq[MemoryShuffleBlock])
: Seq[Iterator[Product2[K, C]]] = {
blocks.map{ case MemoryShuffleBlock(id, buf) =>
blockManager.dataDeserialize(id, buf.nioByteBuffer(), ser)
.asInstanceOf[Iterator[Product2[K, C]]]
}
}

/**
* Utility function to compute the shuffle blocks and related BlockManagerID, block size,
* also the total request shuffle size before starting to fetch the shuffle blocks.
*/
private def computeShuffleBlocks(): Unit = {
val statuses = SparkEnv.get.mapOutputTracker.getServerStatuses(handle.shuffleId, startPartition)

val splitsByAddress = new HashMap[BlockManagerId, ArrayBuffer[(Int, Long)]]()
for (((address, size), index) <- statuses.zipWithIndex) {
splitsByAddress.getOrElseUpdate(address, ArrayBuffer()) += ((index, size))
}

splitsByAddress.foreach { case (id, blocks) =>
blocks.foreach { case (idx, len) =>
shuffleBlockMap.put(ShuffleBlockId(handle.shuffleId, idx, startPartition), (id, len))
unfetchedBytes += len
}
}
}

private def fetchShuffleBlocks(): Iterator[(BlockId, Try[ManagedBuffer])] = {
val blocksByAddress = new HashMap[BlockManagerId, ArrayBuffer[(ShuffleBlockId, Long)]]()

shuffleBlockMap.foreach { case (block, (id, len)) =>
blocksByAddress.getOrElseUpdate(id,
ArrayBuffer[(ShuffleBlockId, Long)]()) += ((block, len))
}

shuffleRawBlockFetcherItr = new ShuffleRawBlockFetcherIterator(
context,
SparkEnv.get.blockManager.shuffleClient,
blockManager,
blocksByAddress.toSeq,
conf.getLong("spark.reducer.maxMbInFlight", 48) * 1024 * 1024)

val completionItr = CompletionIterator[
(BlockId, Try[ManagedBuffer]),
Iterator[(BlockId, Try[ManagedBuffer])]](shuffleRawBlockFetcherItr,
context.taskMetrics.updateShuffleReadMetrics())

new InterruptibleIterator[(BlockId, Try[ManagedBuffer])](context, completionItr)
}
}
Loading