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
SortShuffleReader code improvement
  • Loading branch information
jerryshao committed Apr 13, 2015
commit 98c039b21a03fddc3076d35cddd18d9439e99587
Original file line number Diff line number Diff line change
@@ -0,0 +1,40 @@
/*
* 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

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()

override def stop(): Unit = shuffleReader.stop()
}
Original file line number Diff line number Diff line change
Expand Up @@ -47,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 SortShuffleReader(
new MixedShuffleReader(
handle.asInstanceOf[BaseShuffleHandle[K, _, C]], startPartition, endPartition, context)
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,10 +26,9 @@ import org.apache.spark.{Logging, InterruptibleIterator, SparkEnv, TaskContext}
import org.apache.spark.network.buffer.ManagedBuffer
import org.apache.spark.serializer.Serializer
import org.apache.spark.shuffle.{ShuffleReader, BaseShuffleHandle}
import org.apache.spark.shuffle.hash.BlockStoreShuffleFetcher
import org.apache.spark.storage._
import org.apache.spark.util.CompletionIterator
import org.apache.spark.util.collection.{MergeUtil, TieredDiskMerger}
import org.apache.spark.util.collection.{TieredDiskMerger, MergeUtil}

/**
* SortShuffleReader merges and aggregates shuffle data that has already been sorted within each
Expand All @@ -50,13 +49,13 @@ private[spark] class SortShuffleReader[K, C](
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 fileBufferSize = conf.getInt("spark.shuffle.file.buffer.kb", 32) * 1024

case class MemoryBlock(blockId: BlockId, blockData: ManagedBuffer)

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

private val dep = handle.dependency
Expand All @@ -65,81 +64,94 @@ private[spark] class SortShuffleReader[K, C](
private val ser = Serializer.getSerializer(dep.serializer)
private val shuffleMemoryManager = SparkEnv.get.shuffleMemoryManager

private val memoryBlocks = new ArrayBuffer[MemoryBlock]()
private val fileBufferSize = conf.getInt("spark.shuffle.file.buffer.kb", 32) * 1024

private val tieredMerger = new TieredDiskMerger(conf, dep, context)
/** ArrayBuffer to store in-memory shuffle blocks */
private val inMemoryBlocks = new ArrayBuffer[MemoryShuffleBlock]()

/** 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) = {
val h1 = if (a == null) 0 else a.hashCode()
val h2 = if (b == null) 0 else b.hashCode()
h1 - h2
if (h1 < h2) -1 else if (h1 == h2) 0 else 1
}
})

override def read(): Iterator[Product2[K, C]] = {
if (!dep.mapSideCombine && dep.aggregator.isDefined) {
val iter = BlockStoreShuffleFetcher.fetch(handle.shuffleId, startPartition, context, ser)
new InterruptibleIterator(context,
dep.aggregator.get.combineValuesByKey(iter, context))
} else {
sortShuffleRead()
}
}
/** A merge thread to merge on-disk blocks */
private val tieredMerger = new TieredDiskMerger(conf, dep, keyComparator, context)

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

for ((blockId, blockData) <- fetchRawBlocks()) {
if (blockData.isEmpty) {
throw new IllegalStateException(s"block $blockId is empty for unknown reason")
}

memoryBlocks += MemoryBlock(blockId, blockData.get)
inMemoryBlocks += MemoryShuffleBlock(blockId, blockData.get)

// Try to fit block in memory. If this fails, merge in-memory blocks to disk.
val blockSize = blockData.get.size
val granted = shuffleMemoryManager.tryToAcquire(blockData.get.size)
logInfo(s"Granted $granted memory for shuffle block")

if (granted < blockSize) {
shuffleMemoryManager.release(granted)
logInfo(s"Granted $granted memory is not enough to store shuffle block ($blockSize), " +
s"try to consolidate in-memory blocks to release the memory")

val itrGroup = memoryBlocksToIterators()
val partialMergedIter =
MergeUtil.mergeSort(itrGroup, keyComparator, dep.keyOrdering, dep.aggregator)
shuffleMemoryManager.release(granted)

// Write merged blocks to disk
val (tmpBlockId, file) = blockManager.diskBlockManager.createTempShuffleBlock()
val fos = new BufferedOutputStream(new FileOutputStream(file), fileBufferSize)
blockManager.dataSerializeStream(tmpBlockId, fos, partialMergedIter, ser)
val fos = new FileOutputStream(file)
val bos = new BufferedOutputStream(fos, fileBufferSize)

if (inMemoryBlocks.size > 1) {
val itrGroup = inMemoryBlocksToIterators()
val partialMergedItr =
MergeUtil.mergeSort(itrGroup, keyComparator, dep.keyOrdering, dep.aggregator)
blockManager.dataSerializeStream(tmpBlockId, bos, partialMergedItr, ser)
} else {
val buffer = inMemoryBlocks.map(_.blockData.nioByteBuffer()).head
val channel = fos.getChannel
while (buffer.hasRemaining) {
channel.write(buffer)
}
channel.close()
}

tieredMerger.registerOnDiskBlock(tmpBlockId, file)

for (block <- memoryBlocks) {
for (block <- inMemoryBlocks) {
shuffleMemoryManager.release(block.blockData.size)
}
memoryBlocks.clear()
inMemoryBlocks.clear()
}

shuffleRawBlockFetcherItr.currentResult = null
}

tieredMerger.doneRegisteringOnDiskBlocks()

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

// Release the in-memory block and on-disk file when iteration is completed.
// Release the in-memory block when iteration is completed.
val completionItr = CompletionIterator[Product2[K, C], Iterator[Product2[K, C]]](
mergedItr, () => {
memoryBlocks.foreach(block => shuffleMemoryManager.release(block.blockData.size))
memoryBlocks.clear()
inMemoryBlocks.foreach(block => shuffleMemoryManager.release(block.blockData.size))
inMemoryBlocks.clear()
})

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

def memoryBlocksToIterators(): Seq[Iterator[Product2[K, C]]] = {
memoryBlocks.map{ case MemoryBlock(id, buf) =>
private def inMemoryBlocksToIterators(): Seq[Iterator[Product2[K, C]]] = {
inMemoryBlocks.map{ case MemoryShuffleBlock(id, buf) =>
blockManager.dataDeserialize(id, buf.nioByteBuffer(), ser)
.asInstanceOf[Iterator[Product2[K, C]]]
}
Expand All @@ -149,27 +161,25 @@ private[spark] class SortShuffleReader[K, C](

private def fetchRawBlocks(): Iterator[(BlockId, Option[ManagedBuffer])] = {
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))
}
val blocksByAddress: Seq[(BlockManagerId, Seq[(BlockId, Long)])] = splitsByAddress.toSeq.map {
case (address, splits) =>
(address, splits.map(s => (ShuffleBlockId(handle.shuffleId, s._1, startPartition), s._2)))
}
var numMapBlocks = 0
blocksByAddress.foreach { case (_, blocks) =>
blocks.foreach { case (_, len) => if (len > 0) numMapBlocks += 1 }

val blocksByAddress = splitsByAddress.toSeq.map { case (address, splits) =>
val blocks = splits.map { s =>
(ShuffleBlockId(handle.shuffleId, s._1, startPartition), s._2)
}
(address, blocks.toSeq)
}
val threadId = Thread.currentThread.getId
logInfo(s"Fetching $numMapBlocks blocks for $threadId")

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

val completionItr = CompletionIterator[
(BlockId, Option[ManagedBuffer]),
Expand Down
Loading