From 779c0f95f66f3487adeccaf67f8f3136d2b56c68 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Fri, 5 Sep 2014 16:46:12 +0800 Subject: [PATCH 01/27] initial commit of sort-merge shuffle reader Conflicts: core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala core/src/main/scala/org/apache/spark/storage/BlockManager.scala core/src/test/scala/org/apache/spark/storage/BlockFetcherIteratorSuite.scala Conflicts: core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala --- .../spark/shuffle/ShuffleMemoryManager.scala | 10 +- .../hash/BlockStoreShuffleFetcher.scala | 2 +- .../shuffle/sort/SortShuffleManager.scala | 3 +- .../shuffle/sort/SortShuffleReader.scala | 298 ++++++++++++++++++ .../shuffle/sort/SortShuffleWriter.scala | 4 + .../storage/ShuffleBlockFetcherIterator.scala | 71 +++-- .../util/collection/ExternalSorter.scala | 226 ++++++------- 7 files changed, 471 insertions(+), 143 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleMemoryManager.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleMemoryManager.scala index 3bcc7178a3d8b..017081ef42783 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleMemoryManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleMemoryManager.scala @@ -93,14 +93,16 @@ private[spark] class ShuffleMemoryManager(maxMemory: Long) extends Logging { } /** Release numBytes bytes for the current thread. */ - def release(numBytes: Long): Unit = synchronized { - val threadId = Thread.currentThread().getId - val curMem = threadMemory.getOrElse(threadId, 0L) + def release(numBytes: Long): Unit = release(numBytes, Thread.currentThread().getId) + + /** Release numBytes bytes for the specific thread. */ + def release(numBytes: Long, tid: Long): Unit = synchronized { + val curMem = threadMemory.getOrElse(tid, 0L) if (curMem < numBytes) { throw new SparkException( s"Internal error: release called on ${numBytes} bytes but thread only has ${curMem}") } - threadMemory(threadId) -= numBytes + threadMemory(tid) -= numBytes notifyAll() // Notify waiters who locked "this" in tryToAcquire that memory has been freed } diff --git a/core/src/main/scala/org/apache/spark/shuffle/hash/BlockStoreShuffleFetcher.scala b/core/src/main/scala/org/apache/spark/shuffle/hash/BlockStoreShuffleFetcher.scala index 7a2c5ae32d98b..b3b0471732be3 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/hash/BlockStoreShuffleFetcher.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/hash/BlockStoreShuffleFetcher.scala @@ -27,7 +27,7 @@ import org.apache.spark.shuffle.FetchFailedException import org.apache.spark.storage.{BlockId, BlockManagerId, ShuffleBlockFetcherIterator, ShuffleBlockId} import org.apache.spark.util.CompletionIterator -private[hash] object BlockStoreShuffleFetcher extends Logging { +private[shuffle] object BlockStoreShuffleFetcher extends Logging { def fetch[T]( shuffleId: Int, reduceId: Int, diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala index 0497036192154..0c0a358cdaa26 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala @@ -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 { @@ -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 SortShuffleReader( handle.asInstanceOf[BaseShuffleHandle[K, _, C]], startPartition, endPartition, context) } diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala new file mode 100644 index 0000000000000..e1b2fab95cd13 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala @@ -0,0 +1,298 @@ +/* + * 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.{BufferedOutputStream, FileOutputStream, File} +import java.nio.ByteBuffer +import java.util.Comparator +import java.util.concurrent.{CountDownLatch, TimeUnit, LinkedBlockingQueue} + +import org.apache.spark.network.ManagedBuffer + +import scala.collection.mutable.{ArrayBuffer, HashMap} + +import org.apache.spark.{Logging, InterruptibleIterator, SparkEnv, TaskContext} +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.ExternalSorter + +private[spark] class SortShuffleReader[K, C]( + handle: BaseShuffleHandle[K, _, C], + startPartition: Int, + endPartition: Int, + context: TaskContext) + extends ShuffleReader[K, C] with Logging { + + require(endPartition == startPartition + 1, + "Sort shuffle currently only supports fetching one partition") + + sealed trait ShufflePartition + case class MemoryPartition(blockId: BlockId, blockData: ManagedBuffer) extends ShufflePartition + case class FilePartition(blockId: BlockId, mappedFile: File) extends ShufflePartition + + private val mergingGroup = new LinkedBlockingQueue[ShufflePartition]() + private val mergedGroup = new LinkedBlockingQueue[ShufflePartition]() + private var numSplits: Int = 0 + private val mergeFinished = new CountDownLatch(1) + private val mergingThread = new MergingThread() + private val tid = Thread.currentThread().getId + private var shuffleRawBlockFetcherItr: ShuffleRawBlockFetcherIterator = null + + 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 ioSortFactor = conf.getInt("spark.shuffle.ioSortFactor", 100) + private val fileBufferSize = conf.getInt("spark.shuffle.file.buffer.kb", 32) * 1024 + + 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 + } + }) + + 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() + } + } + + private def sortShuffleRead(): Iterator[Product2[K, C]] = { + val rawBlockIterator = fetchRawBlock() + + mergingThread.setNumSplits(numSplits) + mergingThread.setDaemon(true) + mergingThread.start() + + for ((blockId, blockData) <- rawBlockIterator) { + if (blockData.isEmpty) { + throw new IllegalStateException(s"block $blockId is empty for unknown reason") + } + + val amountToRequest = blockData.get.size + val granted = shuffleMemoryManager.tryToAcquire(amountToRequest) + val shouldSpill = if (granted < amountToRequest) { + shuffleMemoryManager.release(granted) + logInfo(s"Grant memory $granted less than the amount to request $amountToRequest, " + + s"spilling data to file") + true + } else { + false + } + + if (!shouldSpill) { + mergingGroup.offer(MemoryPartition(blockId, blockData.get)) + } else { + val (tmpBlockId, file) = blockManager.diskBlockManager.createTempBlock() + val channel = new FileOutputStream(file).getChannel() + val byteBuffer = blockData.get.nioByteBuffer() + while (byteBuffer.remaining() > 0) { + channel.write(byteBuffer) + } + channel.close() + mergingGroup.offer(FilePartition(tmpBlockId, file)) + } + + shuffleRawBlockFetcherItr.currentResult = null + } + + mergeFinished.await() + + // Merge the final group for combiner to directly feed to the reducer + val finalMergedPartArray = mergedGroup.toArray(new Array[ShufflePartition](mergedGroup.size())) + val finalItrGroup = getIteratorGroup(finalMergedPartArray) + val mergedItr = if (dep.aggregator.isDefined) { + ExternalSorter.mergeWithAggregation(finalItrGroup, dep.aggregator.get.mergeCombiners, + keyComparator, dep.keyOrdering.isDefined) + } else { + ExternalSorter.mergeSort(finalItrGroup, keyComparator) + } + + mergedGroup.clear() + + // Release the shuffle used memory of this thread + shuffleMemoryManager.releaseMemoryForThisThread() + + // Release the in-memory block and on-disk file when iteration is completed. + val completionItr = CompletionIterator[Product2[K, C], Iterator[Product2[K, C]]]( + mergedItr, releaseUnusedShufflePartition(finalMergedPartArray)) + + new InterruptibleIterator(context, completionItr.map(p => (p._1, p._2))) + } + + override def stop(): Unit = ??? + + private def fetchRawBlock(): 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))) + } + blocksByAddress.foreach { case (_, blocks) => + blocks.foreach { case (_, len) => if (len > 0) numSplits += 1 } + } + logInfo(s"Fetch $numSplits partitions for $tid") + + shuffleRawBlockFetcherItr = new ShuffleRawBlockFetcherIterator( + context, + SparkEnv.get.blockTransferService, + blockManager, + blocksByAddress, + SparkEnv.get.conf.getLong("spark.reducer.maxMbInFlight", 48) * 1024 * 1024) + + val completionItr = CompletionIterator[ + (BlockId, Option[ManagedBuffer]), + Iterator[(BlockId, Option[ManagedBuffer])]](shuffleRawBlockFetcherItr, { + context.taskMetrics.updateShuffleReadMetrics() + }) + + new InterruptibleIterator[(BlockId, Option[ManagedBuffer])](context, completionItr) + } + + private def getIteratorGroup(shufflePartGroup: Array[ShufflePartition]) + : Seq[Iterator[Product2[K, C]]] = { + shufflePartGroup.map { part => + val itr = part match { + case MemoryPartition(id, buf) => + // Release memory usage + shuffleMemoryManager.release(buf.size, tid) + blockManager.dataDeserialize(id, buf.nioByteBuffer(), ser) + case FilePartition(id, file) => + val blockData = blockManager.diskStore.getBytes(id).getOrElse( + throw new IllegalStateException(s"cannot get data from block $id")) + blockManager.dataDeserialize(id, blockData, ser) + } + itr.asInstanceOf[Iterator[Product2[K, C]]] + }.toSeq + } + + + /** + * Release the left in-memory buffer or on-disk file after merged. + */ + private def releaseUnusedShufflePartition(shufflePartGroup: Array[ShufflePartition]): Unit = { + shufflePartGroup.map { part => + part match { + case MemoryPartition(id, buf) => buf.release() + case FilePartition(id, file) => + try { + file.delete() + } catch { + // Swallow the exception + case e: Throwable => logWarning(s"Unexpected errors when deleting file: ${ + file.getAbsolutePath}", e) + } + } + } + } + + private class MergingThread extends Thread { + private var isLooped = true + private var leftTobeMerged = 0 + + def setNumSplits(numSplits: Int) { + leftTobeMerged = numSplits + } + + override def run() { + while (isLooped) { + if (leftTobeMerged < ioSortFactor && leftTobeMerged > 0) { + var count = leftTobeMerged + while (count > 0) { + val part = mergingGroup.poll(100, TimeUnit.MILLISECONDS) + if (part != null) { + mergedGroup.offer(part) + count -= 1 + leftTobeMerged -= 1 + } + } + } else if (leftTobeMerged >= ioSortFactor) { + val mergingPartArray = ArrayBuffer[ShufflePartition]() + var count = if (numSplits / ioSortFactor > ioSortFactor) { + ioSortFactor + } else { + val mergedSize = mergedGroup.size() + val left = leftTobeMerged - (ioSortFactor - mergedSize - 1) + if (left <= ioSortFactor) { + left + } else { + ioSortFactor + } + } + val countCopy = count + + while (count > 0) { + val part = mergingGroup.poll(100, TimeUnit.MILLISECONDS) + if (part != null) { + mergingPartArray += part + count -= 1 + leftTobeMerged -= 1 + } + } + + // Merge the partitions + val itrGroup = getIteratorGroup(mergingPartArray.toArray) + val partialMergedIter = if (dep.aggregator.isDefined) { + ExternalSorter.mergeWithAggregation(itrGroup, dep.aggregator.get.mergeCombiners, + keyComparator, dep.keyOrdering.isDefined) + } else { + ExternalSorter.mergeSort(itrGroup, keyComparator) + } + // Write merged partitions to disk + val (tmpBlockId, file) = blockManager.diskBlockManager.createTempBlock() + val fos = new BufferedOutputStream(new FileOutputStream(file), fileBufferSize) + blockManager.dataSerializeStream(tmpBlockId, fos, partialMergedIter, ser) + logInfo(s"Merge $countCopy partitions and write into file ${file.getName}") + + releaseUnusedShufflePartition(mergingPartArray.toArray) + mergedGroup.add(FilePartition(tmpBlockId, file)) + } else { + val mergedSize = mergedGroup.size() + if (mergedSize > ioSortFactor) { + leftTobeMerged = mergedSize + + // Swap the merged group and merging group and do merge again, + // since file number is still larger than ioSortFactor + assert(mergingGroup.size() == 0) + mergingGroup.addAll(mergedGroup) + mergedGroup.clear() + } else { + assert(mergingGroup.size() == 0) + isLooped = false + mergeFinished.countDown() + } + } + } + } + } +} diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala index a066435df6fb0..f3d15d261e782 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala @@ -54,6 +54,10 @@ private[spark] class SortShuffleWriter[K, V, C]( sorter = new ExternalSorter[K, V, C]( dep.aggregator, Some(dep.partitioner), dep.keyOrdering, dep.serializer) sorter.insertAll(records) + } else if (dep.keyOrdering.isDefined) { + sorter = new ExternalSorter[K, V, V]( + None, Some(dep.partitioner), dep.keyOrdering, dep.serializer) + sorter.insertAll(records) } else { // In this case we pass neither an aggregator nor an ordering to the sorter, because we don't // care whether the keys get sorted in each partition; that will be done on the reduce side diff --git a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala index 8f28ef49a8a6f..b3df9e7104d7e 100644 --- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala @@ -30,6 +30,41 @@ import org.apache.spark.network.buffer.ManagedBuffer import org.apache.spark.serializer.Serializer import org.apache.spark.util.{CompletionIterator, Utils} +private[spark] +final class ShuffleBlockFetcherIterator( + context: TaskContext, + blockTransferService: BlockTransferService, + blockManager: BlockManager, + blocksByAddress: Seq[(BlockManagerId, Seq[(BlockId, Long)])], + serializer: Serializer, + maxBytesInFlight: Long) + extends Iterator[(BlockId, Try[Iterator[Any]])] { + + val shuffleRawBlockFetcherItr = new ShuffleRawBlockFetcherIterator( + context, + blockTransferService, + blockManager, + blocksByAddress, + maxBytesInFlight) + + def hasNext: Boolean = shuffleRawBlockFetcherItr.hasNext + + def next(): (BlockId, Try[Iterator[Any]]) = { + val (blockId, block) = shuffleRawBlockFetcherItr.next() + val completedItr = block.map { buf => + val is = blockManager.wrapForCompression(blockId, buf.createInputStream()) + val itr = serializer.newInstance().deserializeStream(is).asIterator + CompletionIterator[Any, Iterator[Any]](itr, { + // Once the iterator is exhausted, release the buffer and set currentResult to null + // so we don't release it again in cleanup. + buf.release() + shuffleRawBlockFetcherItr.currentResult = null + }) + } + (blockId, completedItr) + } +} + /** * An iterator that fetches multiple blocks. For local blocks, it fetches from the local block * manager. For remote blocks, it fetches them using the provided BlockTransferService. @@ -46,20 +81,18 @@ import org.apache.spark.util.{CompletionIterator, Utils} * @param blocksByAddress list of blocks to fetch grouped by the [[BlockManagerId]]. * For each block we also require the size (in bytes as a long field) in * order to throttle the memory usage. - * @param serializer serializer used to deserialize the data. * @param maxBytesInFlight max size (in bytes) of remote blocks to fetch at any given point. */ private[spark] -final class ShuffleBlockFetcherIterator( +final class ShuffleRawBlockFetcherIterator( context: TaskContext, shuffleClient: ShuffleClient, blockManager: BlockManager, blocksByAddress: Seq[(BlockManagerId, Seq[(BlockId, Long)])], - serializer: Serializer, maxBytesInFlight: Long) - extends Iterator[(BlockId, Try[Iterator[Any]])] with Logging { + extends Iterator[(BlockId, Try[ManagedBuffer])] with Logging { - import ShuffleBlockFetcherIterator._ + import ShuffleRawBlockFetcherIterator._ /** * Total number of blocks to fetch. This can be smaller than the total number of blocks @@ -93,7 +126,7 @@ final class ShuffleBlockFetcherIterator( * Current [[FetchResult]] being processed. We track this so we can release the current buffer * in case of a runtime exception when processing the current buffer. */ - @volatile private[this] var currentResult: FetchResult = null + private[spark] var currentResult: FetchResult = null /** * Queue of fetch requests to issue; we'll pull requests off this gradually to make sure that @@ -272,7 +305,7 @@ final class ShuffleBlockFetcherIterator( override def hasNext: Boolean = numBlocksProcessed < numBlocksToFetch - override def next(): (BlockId, Try[Iterator[Any]]) = { + override def next(): (BlockId, Try[ManagedBuffer]) = { numBlocksProcessed += 1 val startFetchWait = System.currentTimeMillis() currentResult = results.take() @@ -290,32 +323,18 @@ final class ShuffleBlockFetcherIterator( sendRequest(fetchRequests.dequeue()) } - val iteratorTry: Try[Iterator[Any]] = result match { - case FailureFetchResult(_, e) => - Failure(e) - case SuccessFetchResult(blockId, _, buf) => - // There is a chance that createInputStream can fail (e.g. fetching a local file that does - // not exist, SPARK-4085). In that case, we should propagate the right exception so - // the scheduler gets a FetchFailedException. - Try(buf.createInputStream()).map { is0 => - val is = blockManager.wrapForCompression(blockId, is0) - val iter = serializer.newInstance().deserializeStream(is).asIterator - CompletionIterator[Any, Iterator[Any]](iter, { - // Once the iterator is exhausted, release the buffer and set currentResult to null - // so we don't release it again in cleanup. - currentResult = null - buf.release() - }) - } + val bufferTry: Try[ManagedBuffer] = result match { + case FailureFetchResult(_, e) => Failure(e) + case SuccessFetchResult(blockId, _, buf) => Success(buf) } - (result.blockId, iteratorTry) + (result.blockId, bufferTry) } } private[storage] -object ShuffleBlockFetcherIterator { +object ShuffleRawBlockFetcherIterator { /** * A request to fetch blocks from a remote BlockManager. diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala index 035f3767ff554..6210214724978 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala @@ -87,6 +87,8 @@ private[spark] class ExternalSorter[K, V, C]( serializer: Option[Serializer] = None) extends Logging with Spillable[SizeTrackingPairCollection[(Int, K), C]] { + import ExternalSorter._ + private val numPartitions = partitioner.map(_.numPartitions).getOrElse(1) private val shouldPartition = numPartitions > 1 @@ -407,116 +409,6 @@ private[spark] class ExternalSorter[K, V, C]( } } - /** - * Merge-sort a sequence of (K, C) iterators using a given a comparator for the keys. - */ - private def mergeSort(iterators: Seq[Iterator[Product2[K, C]]], comparator: Comparator[K]) - : Iterator[Product2[K, C]] = - { - val bufferedIters = iterators.filter(_.hasNext).map(_.buffered) - type Iter = BufferedIterator[Product2[K, C]] - val heap = new mutable.PriorityQueue[Iter]()(new Ordering[Iter] { - // Use the reverse of comparator.compare because PriorityQueue dequeues the max - override def compare(x: Iter, y: Iter): Int = -comparator.compare(x.head._1, y.head._1) - }) - heap.enqueue(bufferedIters: _*) // Will contain only the iterators with hasNext = true - new Iterator[Product2[K, C]] { - override def hasNext: Boolean = !heap.isEmpty - - override def next(): Product2[K, C] = { - if (!hasNext) { - throw new NoSuchElementException - } - val firstBuf = heap.dequeue() - val firstPair = firstBuf.next() - if (firstBuf.hasNext) { - heap.enqueue(firstBuf) - } - firstPair - } - } - } - - /** - * Merge a sequence of (K, C) iterators by aggregating values for each key, assuming that each - * iterator is sorted by key with a given comparator. If the comparator is not a total ordering - * (e.g. when we sort objects by hash code and different keys may compare as equal although - * they're not), we still merge them by doing equality tests for all keys that compare as equal. - */ - private def mergeWithAggregation( - iterators: Seq[Iterator[Product2[K, C]]], - mergeCombiners: (C, C) => C, - comparator: Comparator[K], - totalOrder: Boolean) - : Iterator[Product2[K, C]] = - { - if (!totalOrder) { - // We only have a partial ordering, e.g. comparing the keys by hash code, which means that - // multiple distinct keys might be treated as equal by the ordering. To deal with this, we - // need to read all keys considered equal by the ordering at once and compare them. - new Iterator[Iterator[Product2[K, C]]] { - val sorted = mergeSort(iterators, comparator).buffered - - // Buffers reused across elements to decrease memory allocation - val keys = new ArrayBuffer[K] - val combiners = new ArrayBuffer[C] - - override def hasNext: Boolean = sorted.hasNext - - override def next(): Iterator[Product2[K, C]] = { - if (!hasNext) { - throw new NoSuchElementException - } - keys.clear() - combiners.clear() - val firstPair = sorted.next() - keys += firstPair._1 - combiners += firstPair._2 - val key = firstPair._1 - while (sorted.hasNext && comparator.compare(sorted.head._1, key) == 0) { - val pair = sorted.next() - var i = 0 - var foundKey = false - while (i < keys.size && !foundKey) { - if (keys(i) == pair._1) { - combiners(i) = mergeCombiners(combiners(i), pair._2) - foundKey = true - } - i += 1 - } - if (!foundKey) { - keys += pair._1 - combiners += pair._2 - } - } - - // Note that we return an iterator of elements since we could've had many keys marked - // equal by the partial order; we flatten this below to get a flat iterator of (K, C). - keys.iterator.zip(combiners.iterator) - } - }.flatMap(i => i) - } else { - // We have a total ordering, so the objects with the same key are sequential. - new Iterator[Product2[K, C]] { - val sorted = mergeSort(iterators, comparator).buffered - - override def hasNext: Boolean = sorted.hasNext - - override def next(): Product2[K, C] = { - if (!hasNext) { - throw new NoSuchElementException - } - val elem = sorted.next() - val k = elem._1 - var c = elem._2 - while (sorted.hasNext && sorted.head._1 == k) { - c = mergeCombiners(c, sorted.head._2) - } - (k, c) - } - } - } - } /** * An internal class for reading a spilled file partition by partition. Expects all the @@ -830,3 +722,117 @@ private[spark] class ExternalSorter[K, V, C]( } } } + +private[spark] object ExternalSorter { + + /** + * Merge-sort a sequence of (K, C) iterators using a given a comparator for the keys. + */ + def mergeSort[K, C](iterators: Seq[Iterator[Product2[K, C]]], comparator: Comparator[K]) + : Iterator[Product2[K, C]] = + { + val bufferedIters = iterators.filter(_.hasNext).map(_.buffered) + type Iter = BufferedIterator[Product2[K, C]] + val heap = new mutable.PriorityQueue[Iter]()(new Ordering[Iter] { + // Use the reverse of comparator.compare because PriorityQueue dequeues the max + override def compare(x: Iter, y: Iter): Int = -comparator.compare(x.head._1, y.head._1) + }) + heap.enqueue(bufferedIters: _*) // Will contain only the iterators with hasNext = true + new Iterator[Product2[K, C]] { + override def hasNext: Boolean = !heap.isEmpty + + override def next(): Product2[K, C] = { + if (!hasNext) { + throw new NoSuchElementException + } + val firstBuf = heap.dequeue() + val firstPair = firstBuf.next() + if (firstBuf.hasNext) { + heap.enqueue(firstBuf) + } + firstPair + } + } + } + + /** + * Merge a sequence of (K, C) iterators by aggregating values for each key, assuming that each + * iterator is sorted by key with a given comparator. If the comparator is not a total ordering + * (e.g. when we sort objects by hash code and different keys may compare as equal although + * they're not), we still merge them by doing equality tests for all keys that compare as equal. + */ + def mergeWithAggregation[K, C]( + iterators: Seq[Iterator[Product2[K, C]]], + mergeCombiners: (C, C) => C, + comparator: Comparator[K], + totalOrder: Boolean) + : Iterator[Product2[K, C]] = + { + if (!totalOrder) { + // We only have a partial ordering, e.g. comparing the keys by hash code, which means that + // multiple distinct keys might be treated as equal by the ordering. To deal with this, we + // need to read all keys considered equal by the ordering at once and compare them. + new Iterator[Iterator[Product2[K, C]]] { + val sorted = mergeSort(iterators, comparator).buffered + + // Buffers reused across elements to decrease memory allocation + val keys = new ArrayBuffer[K] + val combiners = new ArrayBuffer[C] + + override def hasNext: Boolean = sorted.hasNext + + override def next(): Iterator[Product2[K, C]] = { + if (!hasNext) { + throw new NoSuchElementException + } + keys.clear() + combiners.clear() + val firstPair = sorted.next() + keys += firstPair._1 + combiners += firstPair._2 + val key = firstPair._1 + while (sorted.hasNext && comparator.compare(sorted.head._1, key) == 0) { + val pair = sorted.next() + var i = 0 + var foundKey = false + while (i < keys.size && !foundKey) { + if (keys(i) == pair._1) { + combiners(i) = mergeCombiners(combiners(i), pair._2) + foundKey = true + } + i += 1 + } + if (!foundKey) { + keys += pair._1 + combiners += pair._2 + } + } + + // Note that we return an iterator of elements since we could've had many keys marked + // equal by the partial order; we flatten this below to get a flat iterator of (K, C). + keys.iterator.zip(combiners.iterator) + } + }.flatMap(i => i) + } else { + // We have a total ordering, so the objects with the same key are sequential. + new Iterator[Product2[K, C]] { + val sorted = mergeSort(iterators, comparator).buffered + + override def hasNext: Boolean = sorted.hasNext + + override def next(): Product2[K, C] = { + if (!hasNext) { + throw new NoSuchElementException + } + val elem = sorted.next() + val k = elem._1 + var c = elem._2 + while (sorted.hasNext && sorted.head._1 == k) { + c = mergeCombiners(c, sorted.head._2) + } + (k, c) + } + } + } + } +} From 4f46dc0cd51be849d364ddc9916735438f3686af Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Wed, 22 Oct 2014 12:49:35 -0700 Subject: [PATCH 02/27] Readability improvements to SortShuffleReader --- .../shuffle/sort/SortShuffleReader.scala | 202 +++++++++--------- 1 file changed, 103 insertions(+), 99 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala index e1b2fab95cd13..fbf391a6284a6 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala @@ -18,15 +18,13 @@ package org.apache.spark.shuffle.sort import java.io.{BufferedOutputStream, FileOutputStream, File} -import java.nio.ByteBuffer import java.util.Comparator import java.util.concurrent.{CountDownLatch, TimeUnit, LinkedBlockingQueue} -import org.apache.spark.network.ManagedBuffer - import scala.collection.mutable.{ArrayBuffer, HashMap} import org.apache.spark.{Logging, InterruptibleIterator, SparkEnv, TaskContext} +import org.apache.spark.network.ManagedBuffer import org.apache.spark.serializer.Serializer import org.apache.spark.shuffle.{ShuffleReader, BaseShuffleHandle} import org.apache.spark.shuffle.hash.BlockStoreShuffleFetcher @@ -34,6 +32,21 @@ import org.apache.spark.storage._ import org.apache.spark.util.CompletionIterator import org.apache.spark.util.collection.ExternalSorter +/** + * SortShuffleReader assumes that the records within each block are sorted by key, and essentially + * performs a huge tiered merge between all the map output blocks for a partition. + * + * As blocks are fetched, stores them on disk or in memory depending on their size. A single + * background thread merges these blocks and writes the results to disk. + * + * The shape of the tiered merge is controlled by a single parameter, maxMergeWidth, which limits + * the number of blocks merged at once. At any point during its operation, we can think of the + * shuffle reader being at a merge "level". The first level merges the fetched map output blocks, + * at most maxMergeWidth at a time. If the merged blocks created from the first level merge exceed + * maxMergeWidth, a second level merges the results of the first, and so on. When at last the total + * blocks fall beneath maxMergeWidth, the final merge feeds into the iterator that read(...) + * returns. + */ private[spark] class SortShuffleReader[K, C]( handle: BaseShuffleHandle[K, _, C], startPartition: Int, @@ -44,17 +57,25 @@ private[spark] class SortShuffleReader[K, C]( require(endPartition == startPartition + 1, "Sort shuffle currently only supports fetching one partition") - sealed trait ShufflePartition - case class MemoryPartition(blockId: BlockId, blockData: ManagedBuffer) extends ShufflePartition - case class FilePartition(blockId: BlockId, mappedFile: File) extends ShufflePartition + sealed trait ShuffleBlock + case class MemoryBlock(blockId: BlockId, blockData: ManagedBuffer) extends ShuffleBlock + case class FileBlock(blockId: BlockId, mappedFile: File) extends ShuffleBlock - private val mergingGroup = new LinkedBlockingQueue[ShufflePartition]() - private val mergedGroup = new LinkedBlockingQueue[ShufflePartition]() - private var numSplits: Int = 0 + /** + * Blocks awaiting merge at the current level. All fetched blocks go here immediately. After the + * first level merge of fetched blocks has completed, more levels of merge may be required in + * order to not overwhelm the final merge. In those cases, the already-merged blocks awaiting + * further merges will go here as well. + */ + private val blocksAwaitingMerge = new LinkedBlockingQueue[ShuffleBlock]() + /** Blocks already merged at the current level. */ + private val mergedBlocks = new LinkedBlockingQueue[ShuffleBlock]() + /** The total number of map output blocks to be fetched. */ + private var numMapBlocks: Int = 0 private val mergeFinished = new CountDownLatch(1) private val mergingThread = new MergingThread() - private val tid = Thread.currentThread().getId - private var shuffleRawBlockFetcherItr: ShuffleRawBlockFetcherIterator = null + private val threadId = Thread.currentThread().getId + private var shuffleRawBlockFetcherItr: ShuffleRawBlockFetcherIterator = _ private val dep = handle.dependency private val conf = SparkEnv.get.conf @@ -62,7 +83,7 @@ private[spark] class SortShuffleReader[K, C]( private val ser = Serializer.getSerializer(dep.serializer) private val shuffleMemoryManager = SparkEnv.get.shuffleMemoryManager - private val ioSortFactor = conf.getInt("spark.shuffle.ioSortFactor", 100) + private val maxMergeWidth = conf.getInt("spark.shuffle.maxMergeWidth", 100) private val fileBufferSize = conf.getInt("spark.shuffle.file.buffer.kb", 32) * 1024 private val keyComparator: Comparator[K] = dep.keyOrdering.getOrElse(new Comparator[K] { @@ -84,31 +105,21 @@ private[spark] class SortShuffleReader[K, C]( } private def sortShuffleRead(): Iterator[Product2[K, C]] = { - val rawBlockIterator = fetchRawBlock() - - mergingThread.setNumSplits(numSplits) mergingThread.setDaemon(true) mergingThread.start() - for ((blockId, blockData) <- rawBlockIterator) { + for ((blockId, blockData) <- fetchRawBlocks()) { if (blockData.isEmpty) { throw new IllegalStateException(s"block $blockId is empty for unknown reason") } - val amountToRequest = blockData.get.size - val granted = shuffleMemoryManager.tryToAcquire(amountToRequest) - val shouldSpill = if (granted < amountToRequest) { + val blockSize = blockData.get.size + // Try to fit block in memory. If this fails, spill it to disk. + val granted = shuffleMemoryManager.tryToAcquire(blockSize) + if (granted < blockSize) { shuffleMemoryManager.release(granted) - logInfo(s"Grant memory $granted less than the amount to request $amountToRequest, " + + logInfo(s"Granted memory $granted for block less than its size $blockSize, " + s"spilling data to file") - true - } else { - false - } - - if (!shouldSpill) { - mergingGroup.offer(MemoryPartition(blockId, blockData.get)) - } else { val (tmpBlockId, file) = blockManager.diskBlockManager.createTempBlock() val channel = new FileOutputStream(file).getChannel() val byteBuffer = blockData.get.nioByteBuffer() @@ -116,7 +127,9 @@ private[spark] class SortShuffleReader[K, C]( channel.write(byteBuffer) } channel.close() - mergingGroup.offer(FilePartition(tmpBlockId, file)) + blocksAwaitingMerge.offer(FileBlock(tmpBlockId, file)) + } else { + blocksAwaitingMerge.offer(MemoryBlock(blockId, blockData.get)) } shuffleRawBlockFetcherItr.currentResult = null @@ -125,8 +138,8 @@ private[spark] class SortShuffleReader[K, C]( mergeFinished.await() // Merge the final group for combiner to directly feed to the reducer - val finalMergedPartArray = mergedGroup.toArray(new Array[ShufflePartition](mergedGroup.size())) - val finalItrGroup = getIteratorGroup(finalMergedPartArray) + val finalMergedPartArray = mergedBlocks.toArray(new Array[ShuffleBlock](mergedBlocks.size())) + val finalItrGroup = blocksToRecordIterators(finalMergedPartArray) val mergedItr = if (dep.aggregator.isDefined) { ExternalSorter.mergeWithAggregation(finalItrGroup, dep.aggregator.get.mergeCombiners, keyComparator, dep.keyOrdering.isDefined) @@ -134,21 +147,21 @@ private[spark] class SortShuffleReader[K, C]( ExternalSorter.mergeSort(finalItrGroup, keyComparator) } - mergedGroup.clear() + mergedBlocks.clear() // Release the shuffle used memory of this thread shuffleMemoryManager.releaseMemoryForThisThread() // Release the in-memory block and on-disk file when iteration is completed. val completionItr = CompletionIterator[Product2[K, C], Iterator[Product2[K, C]]]( - mergedItr, releaseUnusedShufflePartition(finalMergedPartArray)) + mergedItr, releaseUnusedShuffleBlock(finalMergedPartArray)) new InterruptibleIterator(context, completionItr.map(p => (p._1, p._2))) } override def stop(): Unit = ??? - private def fetchRawBlock(): Iterator[(BlockId, Option[ManagedBuffer])] = { + 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) { @@ -159,9 +172,9 @@ private[spark] class SortShuffleReader[K, C]( (address, splits.map(s => (ShuffleBlockId(handle.shuffleId, s._1, startPartition), s._2))) } blocksByAddress.foreach { case (_, blocks) => - blocks.foreach { case (_, len) => if (len > 0) numSplits += 1 } + blocks.foreach { case (_, len) => if (len > 0) numMapBlocks += 1 } } - logInfo(s"Fetch $numSplits partitions for $tid") + logInfo(s"Fetching $numMapBlocks blocks for $threadId") shuffleRawBlockFetcherItr = new ShuffleRawBlockFetcherIterator( context, @@ -172,22 +185,21 @@ private[spark] class SortShuffleReader[K, C]( val completionItr = CompletionIterator[ (BlockId, Option[ManagedBuffer]), - Iterator[(BlockId, Option[ManagedBuffer])]](shuffleRawBlockFetcherItr, { - context.taskMetrics.updateShuffleReadMetrics() - }) + Iterator[(BlockId, Option[ManagedBuffer])]](shuffleRawBlockFetcherItr, + () => context.taskMetrics.updateShuffleReadMetrics()) new InterruptibleIterator[(BlockId, Option[ManagedBuffer])](context, completionItr) } - private def getIteratorGroup(shufflePartGroup: Array[ShufflePartition]) + private def blocksToRecordIterators(shufflePartGroup: Seq[ShuffleBlock]) : Seq[Iterator[Product2[K, C]]] = { shufflePartGroup.map { part => val itr = part match { - case MemoryPartition(id, buf) => + case MemoryBlock(id, buf) => // Release memory usage - shuffleMemoryManager.release(buf.size, tid) + shuffleMemoryManager.release(buf.size, threadId) blockManager.dataDeserialize(id, buf.nioByteBuffer(), ser) - case FilePartition(id, file) => + case FileBlock(id, file) => val blockData = blockManager.diskStore.getBytes(id).getOrElse( throw new IllegalStateException(s"cannot get data from block $id")) blockManager.dataDeserialize(id, blockData, ser) @@ -200,11 +212,11 @@ private[spark] class SortShuffleReader[K, C]( /** * Release the left in-memory buffer or on-disk file after merged. */ - private def releaseUnusedShufflePartition(shufflePartGroup: Array[ShufflePartition]): Unit = { + private def releaseUnusedShuffleBlock(shufflePartGroup: Array[ShuffleBlock]): Unit = { shufflePartGroup.map { part => part match { - case MemoryPartition(id, buf) => buf.release() - case FilePartition(id, file) => + case MemoryBlock(id, buf) => buf.release() + case FileBlock(id, file) => try { file.delete() } catch { @@ -217,80 +229,72 @@ private[spark] class SortShuffleReader[K, C]( } private class MergingThread extends Thread { - private var isLooped = true - private var leftTobeMerged = 0 - - def setNumSplits(numSplits: Int) { - leftTobeMerged = numSplits + override def run() { + while (blocksAwaitingMerge.size() > 0) { + mergeLevel() + if (mergedBlocks.size() > maxMergeWidth) { + // End of the current merge level, but not yet ready to proceed to the final merge. + // Swap the merged group and merging group to proceed to the next merge level, + assert(blocksAwaitingMerge.size() == 0) + blocksAwaitingMerge.addAll(mergedBlocks) + mergedBlocks.clear() + } + } + mergeFinished.countDown() } - override def run() { - while (isLooped) { - if (leftTobeMerged < ioSortFactor && leftTobeMerged > 0) { - var count = leftTobeMerged - while (count > 0) { - val part = mergingGroup.poll(100, TimeUnit.MILLISECONDS) + /** + * Carry out the current merge level. I.e. move all blocks out of blocksAwaitingMerge, either by + * merging them or placing them directly in mergedBlocks.. + */ + private def mergeLevel() { + while (blocksAwaitingMerge.size() > 0) { + if (blocksAwaitingMerge.size() < maxMergeWidth) { + // If the remaining blocks awaiting merge at this level don't exceed the maxMergeWidth, + // pass them all on to the next level. + while (blocksAwaitingMerge.size() > 0) { + val part = blocksAwaitingMerge.poll(100, TimeUnit.MILLISECONDS) if (part != null) { - mergedGroup.offer(part) - count -= 1 - leftTobeMerged -= 1 + mergedBlocks.offer(part) } } - } else if (leftTobeMerged >= ioSortFactor) { - val mergingPartArray = ArrayBuffer[ShufflePartition]() - var count = if (numSplits / ioSortFactor > ioSortFactor) { - ioSortFactor + } else if (blocksAwaitingMerge.size() >= maxMergeWidth) { + // Because the remaining blocks awaiting merge at this level exceed the maxMergeWidth, a + // merge is required. + // TODO: is numMapBlocks right here? + var numToMerge = if (numMapBlocks / maxMergeWidth > maxMergeWidth) { + maxMergeWidth } else { - val mergedSize = mergedGroup.size() - val left = leftTobeMerged - (ioSortFactor - mergedSize - 1) - if (left <= ioSortFactor) { - left - } else { - ioSortFactor - } + val mergedSize = mergedBlocks.size() + val left = blocksAwaitingMerge.size() - (maxMergeWidth - mergedSize - 1) + math.min(left, maxMergeWidth) } - val countCopy = count - while (count > 0) { - val part = mergingGroup.poll(100, TimeUnit.MILLISECONDS) + val blocksToMerge = ArrayBuffer[ShuffleBlock]() + while (numToMerge > 0) { + val part = blocksAwaitingMerge.poll(100, TimeUnit.MILLISECONDS) if (part != null) { - mergingPartArray += part - count -= 1 - leftTobeMerged -= 1 + blocksToMerge += part + numToMerge -= 1 } } - // Merge the partitions - val itrGroup = getIteratorGroup(mergingPartArray.toArray) + // Merge the blocks + val itrGroup = blocksToRecordIterators(blocksToMerge) val partialMergedIter = if (dep.aggregator.isDefined) { ExternalSorter.mergeWithAggregation(itrGroup, dep.aggregator.get.mergeCombiners, keyComparator, dep.keyOrdering.isDefined) } else { ExternalSorter.mergeSort(itrGroup, keyComparator) } - // Write merged partitions to disk + // Write merged blocks to disk val (tmpBlockId, file) = blockManager.diskBlockManager.createTempBlock() val fos = new BufferedOutputStream(new FileOutputStream(file), fileBufferSize) blockManager.dataSerializeStream(tmpBlockId, fos, partialMergedIter, ser) - logInfo(s"Merge $countCopy partitions and write into file ${file.getName}") - - releaseUnusedShufflePartition(mergingPartArray.toArray) - mergedGroup.add(FilePartition(tmpBlockId, file)) - } else { - val mergedSize = mergedGroup.size() - if (mergedSize > ioSortFactor) { - leftTobeMerged = mergedSize - - // Swap the merged group and merging group and do merge again, - // since file number is still larger than ioSortFactor - assert(mergingGroup.size() == 0) - mergingGroup.addAll(mergedGroup) - mergedGroup.clear() - } else { - assert(mergingGroup.size() == 0) - isLooped = false - mergeFinished.countDown() - } + logInfo(s"Merged ${blocksToMerge.size} blocks into file ${file.getName}") + + releaseUnusedShuffleBlock(blocksToMerge.toArray) + mergedBlocks.add(FileBlock(tmpBlockId, file)) } } } From 0861cf90067172ab3530a3e7e16166385e9c2e1e Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Wed, 22 Oct 2014 22:12:53 -0700 Subject: [PATCH 03/27] Clarify mergeWidth logic --- .../shuffle/sort/SortShuffleReader.scala | 21 ++++++++++--------- 1 file changed, 11 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala index fbf391a6284a6..ceb77046c3e8c 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala @@ -261,21 +261,22 @@ private[spark] class SortShuffleReader[K, C]( } else if (blocksAwaitingMerge.size() >= maxMergeWidth) { // Because the remaining blocks awaiting merge at this level exceed the maxMergeWidth, a // merge is required. - // TODO: is numMapBlocks right here? - var numToMerge = if (numMapBlocks / maxMergeWidth > maxMergeWidth) { - maxMergeWidth - } else { - val mergedSize = mergedBlocks.size() - val left = blocksAwaitingMerge.size() - (maxMergeWidth - mergedSize - 1) - math.min(left, maxMergeWidth) - } + + // Pick a number of blocks to merge that (1) is <= maxMergeWidth, (2) if possible, + // ensures that the number of blocks in the next level be <= maxMergeWidth so that it can + // be the final merge, and (3) is as small as possible. + val mergedBlocksHeadroomAfterMerge = (maxMergeWidth - mergedBlocks.size() - 1) + // Ideal in the sense that we would move on to the next level after this merge and that + // level would have exactly maxMergeWidth blocks to merge. This can never be negative, + // because, at this point, we know blocksAwaitingMerge is at least maxMergeWidth. + val idealWidth = blocksAwaitingMerge.size() - mergedBlocksHeadroomAfterMerge + val mergeWidth = math.min(idealWidth, maxMergeWidth) val blocksToMerge = ArrayBuffer[ShuffleBlock]() - while (numToMerge > 0) { + while (blocksToMerge.size < mergeWidth) { val part = blocksAwaitingMerge.poll(100, TimeUnit.MILLISECONDS) if (part != null) { blocksToMerge += part - numToMerge -= 1 } } From 8f49b7879bff2d14ce1d0254b09bc9791a3418b5 Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Thu, 23 Oct 2014 10:09:49 -0700 Subject: [PATCH 04/27] Add blocks remaining at level counter back in --- .../shuffle/sort/SortShuffleReader.scala | 21 ++++++++++++------- 1 file changed, 14 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala index ceb77046c3e8c..ef578f27d6a24 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala @@ -229,14 +229,19 @@ private[spark] class SortShuffleReader[K, C]( } private class MergingThread extends Thread { + var remainingToMergeAtLevel: Int = _ + override def run() { - while (blocksAwaitingMerge.size() > 0) { + remainingToMergeAtLevel = numMapBlocks + while (remainingToMergeAtLevel > 0) { mergeLevel() + assert(blocksAwaitingMerge.size() == 0) + assert(remainingToMergeAtLevel == 0) if (mergedBlocks.size() > maxMergeWidth) { // End of the current merge level, but not yet ready to proceed to the final merge. // Swap the merged group and merging group to proceed to the next merge level, - assert(blocksAwaitingMerge.size() == 0) blocksAwaitingMerge.addAll(mergedBlocks) + remainingToMergeAtLevel = blocksAwaitingMerge.size() mergedBlocks.clear() } } @@ -245,20 +250,21 @@ private[spark] class SortShuffleReader[K, C]( /** * Carry out the current merge level. I.e. move all blocks out of blocksAwaitingMerge, either by - * merging them or placing them directly in mergedBlocks.. + * merging them or placing them directly in mergedBlocks. */ private def mergeLevel() { - while (blocksAwaitingMerge.size() > 0) { - if (blocksAwaitingMerge.size() < maxMergeWidth) { + while (remainingToMergeAtLevel > 0) { + if (remainingToMergeAtLevel < maxMergeWidth) { // If the remaining blocks awaiting merge at this level don't exceed the maxMergeWidth, // pass them all on to the next level. - while (blocksAwaitingMerge.size() > 0) { + while (remainingToMergeAtLevel > 0) { val part = blocksAwaitingMerge.poll(100, TimeUnit.MILLISECONDS) if (part != null) { mergedBlocks.offer(part) + remainingToMergeAtLevel -= 1 } } - } else if (blocksAwaitingMerge.size() >= maxMergeWidth) { + } else { // Because the remaining blocks awaiting merge at this level exceed the maxMergeWidth, a // merge is required. @@ -277,6 +283,7 @@ private[spark] class SortShuffleReader[K, C]( val part = blocksAwaitingMerge.poll(100, TimeUnit.MILLISECONDS) if (part != null) { blocksToMerge += part + remainingToMergeAtLevel -= 1 } } From fcafa168e83452ef844fe99b2548d3302d269b40 Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Fri, 24 Oct 2014 15:30:07 -0700 Subject: [PATCH 05/27] Small fix --- .../scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala index ef578f27d6a24..d16c673d6ab72 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala @@ -275,7 +275,7 @@ private[spark] class SortShuffleReader[K, C]( // Ideal in the sense that we would move on to the next level after this merge and that // level would have exactly maxMergeWidth blocks to merge. This can never be negative, // because, at this point, we know blocksAwaitingMerge is at least maxMergeWidth. - val idealWidth = blocksAwaitingMerge.size() - mergedBlocksHeadroomAfterMerge + val idealWidth = remainingToMergeAtLevel - mergedBlocksHeadroomAfterMerge val mergeWidth = math.min(idealWidth, maxMergeWidth) val blocksToMerge = ArrayBuffer[ShuffleBlock]() From 21dae69d11fd6ca09a775f8c83d865fff13aab55 Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Sat, 25 Oct 2014 15:45:20 -0700 Subject: [PATCH 06/27] Move merge to a separate class and use a priority queue instead of levels --- .../shuffle/sort/SortShuffleReader.scala | 233 ++++-------------- .../util/collection/ExternalSorter.scala | 135 ++-------- .../spark/util/collection/MergeUtil.scala | 150 +++++++++++ .../util/collection/TieredDiskMerger.scala | 206 ++++++++++++++++ 4 files changed, 425 insertions(+), 299 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/util/collection/MergeUtil.scala create mode 100644 core/src/main/scala/org/apache/spark/util/collection/TieredDiskMerger.scala diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala index d16c673d6ab72..ae1d4ff33b929 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala @@ -17,9 +17,8 @@ package org.apache.spark.shuffle.sort -import java.io.{BufferedOutputStream, FileOutputStream, File} +import java.io.{BufferedOutputStream, FileOutputStream} import java.util.Comparator -import java.util.concurrent.{CountDownLatch, TimeUnit, LinkedBlockingQueue} import scala.collection.mutable.{ArrayBuffer, HashMap} @@ -30,22 +29,19 @@ 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.ExternalSorter +import org.apache.spark.util.collection.{MergeUtil, TieredDiskMerger} /** - * SortShuffleReader assumes that the records within each block are sorted by key, and essentially - * performs a huge tiered merge between all the map output blocks for a partition. + * SortShuffleReader merges and aggregates shuffle data that has already been sorted within each + * map output block. * - * As blocks are fetched, stores them on disk or in memory depending on their size. A single - * background thread merges these blocks and writes the results to disk. + * As blocks are fetched, we store them in memory until we fail to acquire space frm the + * ShuffleMemoryManager. When this occurs, we merge the in-memory blocks to disk and go back to + * fetching. * - * The shape of the tiered merge is controlled by a single parameter, maxMergeWidth, which limits - * the number of blocks merged at once. At any point during its operation, we can think of the - * shuffle reader being at a merge "level". The first level merges the fetched map output blocks, - * at most maxMergeWidth at a time. If the merged blocks created from the first level merge exceed - * maxMergeWidth, a second level merges the results of the first, and so on. When at last the total - * blocks fall beneath maxMergeWidth, the final merge feeds into the iterator that read(...) - * returns. + * 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], @@ -57,24 +53,10 @@ private[spark] class SortShuffleReader[K, C]( require(endPartition == startPartition + 1, "Sort shuffle currently only supports fetching one partition") - sealed trait ShuffleBlock - case class MemoryBlock(blockId: BlockId, blockData: ManagedBuffer) extends ShuffleBlock - case class FileBlock(blockId: BlockId, mappedFile: File) extends ShuffleBlock + private val fileBufferSize = conf.getInt("spark.shuffle.file.buffer.kb", 32) * 1024 + + case class MemoryBlock(blockId: BlockId, blockData: ManagedBuffer) - /** - * Blocks awaiting merge at the current level. All fetched blocks go here immediately. After the - * first level merge of fetched blocks has completed, more levels of merge may be required in - * order to not overwhelm the final merge. In those cases, the already-merged blocks awaiting - * further merges will go here as well. - */ - private val blocksAwaitingMerge = new LinkedBlockingQueue[ShuffleBlock]() - /** Blocks already merged at the current level. */ - private val mergedBlocks = new LinkedBlockingQueue[ShuffleBlock]() - /** The total number of map output blocks to be fetched. */ - private var numMapBlocks: Int = 0 - private val mergeFinished = new CountDownLatch(1) - private val mergingThread = new MergingThread() - private val threadId = Thread.currentThread().getId private var shuffleRawBlockFetcherItr: ShuffleRawBlockFetcherIterator = _ private val dep = handle.dependency @@ -83,8 +65,9 @@ private[spark] class SortShuffleReader[K, C]( private val ser = Serializer.getSerializer(dep.serializer) private val shuffleMemoryManager = SparkEnv.get.shuffleMemoryManager - private val maxMergeWidth = conf.getInt("spark.shuffle.maxMergeWidth", 100) - private val fileBufferSize = conf.getInt("spark.shuffle.file.buffer.kb", 32) * 1024 + private val memoryBlocks = new ArrayBuffer[MemoryBlock]() + + private val tieredMerger = new TieredDiskMerger(conf, dep, context) private val keyComparator: Comparator[K] = dep.keyOrdering.getOrElse(new Comparator[K] { override def compare(a: K, b: K) = { @@ -105,60 +88,63 @@ private[spark] class SortShuffleReader[K, C]( } private def sortShuffleRead(): Iterator[Product2[K, C]] = { - mergingThread.setDaemon(true) - mergingThread.start() + 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) + + // Try to fit block in memory. If this fails, merge in-memory blocks to disk. val blockSize = blockData.get.size - // Try to fit block in memory. If this fails, spill it to disk. - val granted = shuffleMemoryManager.tryToAcquire(blockSize) + val granted = shuffleMemoryManager.tryToAcquire(blockData.get.size) if (granted < blockSize) { shuffleMemoryManager.release(granted) - logInfo(s"Granted memory $granted for block less than its size $blockSize, " + - s"spilling data to file") + + val itrGroup = memoryBlocksToIterators() + val partialMergedIter = + MergeUtil.mergeSort(itrGroup, keyComparator, dep.keyOrdering, dep.aggregator) + + // Write merged blocks to disk val (tmpBlockId, file) = blockManager.diskBlockManager.createTempBlock() - val channel = new FileOutputStream(file).getChannel() - val byteBuffer = blockData.get.nioByteBuffer() - while (byteBuffer.remaining() > 0) { - channel.write(byteBuffer) + val fos = new BufferedOutputStream(new FileOutputStream(file), fileBufferSize) + blockManager.dataSerializeStream(tmpBlockId, fos, partialMergedIter, ser) + tieredMerger.registerOnDiskBlock(tmpBlockId, file) + + for (block <- memoryBlocks) { + shuffleMemoryManager.release(block.blockData.size) } - channel.close() - blocksAwaitingMerge.offer(FileBlock(tmpBlockId, file)) - } else { - blocksAwaitingMerge.offer(MemoryBlock(blockId, blockData.get)) + memoryBlocks.clear() } shuffleRawBlockFetcherItr.currentResult = null } + tieredMerger.doneRegisteringOnDiskBlocks() - mergeFinished.await() - - // Merge the final group for combiner to directly feed to the reducer - val finalMergedPartArray = mergedBlocks.toArray(new Array[ShuffleBlock](mergedBlocks.size())) - val finalItrGroup = blocksToRecordIterators(finalMergedPartArray) - val mergedItr = if (dep.aggregator.isDefined) { - ExternalSorter.mergeWithAggregation(finalItrGroup, dep.aggregator.get.mergeCombiners, - keyComparator, dep.keyOrdering.isDefined) - } else { - ExternalSorter.mergeSort(finalItrGroup, keyComparator) - } - - mergedBlocks.clear() - - // Release the shuffle used memory of this thread - shuffleMemoryManager.releaseMemoryForThisThread() + // Merge on-disk blocks with in-memory blocks to directly feed to the reducer. + val finalItrGroup = memoryBlocksToIterators() ++ 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. val completionItr = CompletionIterator[Product2[K, C], Iterator[Product2[K, C]]]( - mergedItr, releaseUnusedShuffleBlock(finalMergedPartArray)) + mergedItr, () => { + memoryBlocks.foreach(block => shuffleMemoryManager.release(block.blockData.size)) + memoryBlocks.clear() + }) new InterruptibleIterator(context, completionItr.map(p => (p._1, p._2))) } + def memoryBlocksToIterators(): Seq[Iterator[Product2[K, C]]] = { + memoryBlocks.map{ case MemoryBlock(id, buf) => + blockManager.dataDeserialize(id, buf.nioByteBuffer(), ser) + .asInstanceOf[Iterator[Product2[K, C]]] + } + } + override def stop(): Unit = ??? private def fetchRawBlocks(): Iterator[(BlockId, Option[ManagedBuffer])] = { @@ -171,9 +157,11 @@ private[spark] class SortShuffleReader[K, C]( 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 threadId = Thread.currentThread.getId logInfo(s"Fetching $numMapBlocks blocks for $threadId") shuffleRawBlockFetcherItr = new ShuffleRawBlockFetcherIterator( @@ -190,121 +178,4 @@ private[spark] class SortShuffleReader[K, C]( new InterruptibleIterator[(BlockId, Option[ManagedBuffer])](context, completionItr) } - - private def blocksToRecordIterators(shufflePartGroup: Seq[ShuffleBlock]) - : Seq[Iterator[Product2[K, C]]] = { - shufflePartGroup.map { part => - val itr = part match { - case MemoryBlock(id, buf) => - // Release memory usage - shuffleMemoryManager.release(buf.size, threadId) - blockManager.dataDeserialize(id, buf.nioByteBuffer(), ser) - case FileBlock(id, file) => - val blockData = blockManager.diskStore.getBytes(id).getOrElse( - throw new IllegalStateException(s"cannot get data from block $id")) - blockManager.dataDeserialize(id, blockData, ser) - } - itr.asInstanceOf[Iterator[Product2[K, C]]] - }.toSeq - } - - - /** - * Release the left in-memory buffer or on-disk file after merged. - */ - private def releaseUnusedShuffleBlock(shufflePartGroup: Array[ShuffleBlock]): Unit = { - shufflePartGroup.map { part => - part match { - case MemoryBlock(id, buf) => buf.release() - case FileBlock(id, file) => - try { - file.delete() - } catch { - // Swallow the exception - case e: Throwable => logWarning(s"Unexpected errors when deleting file: ${ - file.getAbsolutePath}", e) - } - } - } - } - - private class MergingThread extends Thread { - var remainingToMergeAtLevel: Int = _ - - override def run() { - remainingToMergeAtLevel = numMapBlocks - while (remainingToMergeAtLevel > 0) { - mergeLevel() - assert(blocksAwaitingMerge.size() == 0) - assert(remainingToMergeAtLevel == 0) - if (mergedBlocks.size() > maxMergeWidth) { - // End of the current merge level, but not yet ready to proceed to the final merge. - // Swap the merged group and merging group to proceed to the next merge level, - blocksAwaitingMerge.addAll(mergedBlocks) - remainingToMergeAtLevel = blocksAwaitingMerge.size() - mergedBlocks.clear() - } - } - mergeFinished.countDown() - } - - /** - * Carry out the current merge level. I.e. move all blocks out of blocksAwaitingMerge, either by - * merging them or placing them directly in mergedBlocks. - */ - private def mergeLevel() { - while (remainingToMergeAtLevel > 0) { - if (remainingToMergeAtLevel < maxMergeWidth) { - // If the remaining blocks awaiting merge at this level don't exceed the maxMergeWidth, - // pass them all on to the next level. - while (remainingToMergeAtLevel > 0) { - val part = blocksAwaitingMerge.poll(100, TimeUnit.MILLISECONDS) - if (part != null) { - mergedBlocks.offer(part) - remainingToMergeAtLevel -= 1 - } - } - } else { - // Because the remaining blocks awaiting merge at this level exceed the maxMergeWidth, a - // merge is required. - - // Pick a number of blocks to merge that (1) is <= maxMergeWidth, (2) if possible, - // ensures that the number of blocks in the next level be <= maxMergeWidth so that it can - // be the final merge, and (3) is as small as possible. - val mergedBlocksHeadroomAfterMerge = (maxMergeWidth - mergedBlocks.size() - 1) - // Ideal in the sense that we would move on to the next level after this merge and that - // level would have exactly maxMergeWidth blocks to merge. This can never be negative, - // because, at this point, we know blocksAwaitingMerge is at least maxMergeWidth. - val idealWidth = remainingToMergeAtLevel - mergedBlocksHeadroomAfterMerge - val mergeWidth = math.min(idealWidth, maxMergeWidth) - - val blocksToMerge = ArrayBuffer[ShuffleBlock]() - while (blocksToMerge.size < mergeWidth) { - val part = blocksAwaitingMerge.poll(100, TimeUnit.MILLISECONDS) - if (part != null) { - blocksToMerge += part - remainingToMergeAtLevel -= 1 - } - } - - // Merge the blocks - val itrGroup = blocksToRecordIterators(blocksToMerge) - val partialMergedIter = if (dep.aggregator.isDefined) { - ExternalSorter.mergeWithAggregation(itrGroup, dep.aggregator.get.mergeCombiners, - keyComparator, dep.keyOrdering.isDefined) - } else { - ExternalSorter.mergeSort(itrGroup, keyComparator) - } - // Write merged blocks to disk - val (tmpBlockId, file) = blockManager.diskBlockManager.createTempBlock() - val fos = new BufferedOutputStream(new FileOutputStream(file), fileBufferSize) - blockManager.dataSerializeStream(tmpBlockId, fos, partialMergedIter, ser) - logInfo(s"Merged ${blocksToMerge.size} blocks into file ${file.getName}") - - releaseUnusedShuffleBlock(blocksToMerge.toArray) - mergedBlocks.add(FileBlock(tmpBlockId, file)) - } - } - } - } } diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala index 6210214724978..fca4fb2e08674 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala @@ -29,6 +29,7 @@ import org.apache.spark._ import org.apache.spark.serializer.{DeserializationStream, Serializer} import org.apache.spark.executor.ShuffleWriteMetrics import org.apache.spark.storage.{BlockObjectWriter, BlockId} +import org.apache.spark.network.ManagedBuffer /** * Sorts and potentially merges a number of key-value pairs of type (K, V) to produce key-combiner @@ -87,8 +88,6 @@ private[spark] class ExternalSorter[K, V, C]( serializer: Option[Serializer] = None) extends Logging with Spillable[SizeTrackingPairCollection[(Int, K), C]] { - import ExternalSorter._ - private val numPartitions = partitioner.map(_.numPartitions).getOrElse(1) private val shouldPartition = numPartitions > 1 @@ -189,6 +188,20 @@ private[spark] class ExternalSorter[K, V, C]( elementsPerPartition: Array[Long]) private val spills = new ArrayBuffer[SpilledFile] + def insertInMemoryBlock(buffer: ManagedBuffer) { + // try to acquire memory + val granted = shuffleMemoryManager.tryToAcquire(buffer.size) + if (granted < buffer.size) { + if (!spillingEnabled) { + throw new IllegalStateException() + } + // spill in-memory partitions + } + // maybe spill + + // spilling should probably tell our background merger thread to try stuff + } + def insertAll(records: Iterator[_ <: Product2[K, V]]): Unit = { // TODO: stop combining if we find that the reduction factor isn't high val shouldCombine = aggregator.isDefined @@ -397,12 +410,12 @@ private[spark] class ExternalSorter[K, V, C]( val iterators = readers.map(_.readNextPartition()) ++ Seq(inMemIterator) if (aggregator.isDefined) { // Perform partial aggregation across partitions - (p, mergeWithAggregation( + (p, MergeUtil.mergeWithAggregation( iterators, aggregator.get.mergeCombiners, keyComparator, ordering.isDefined)) } else if (ordering.isDefined) { // No aggregator given, but we have an ordering (e.g. used by reduce tasks in sortByKey); // sort the elements without trying to merge them - (p, mergeSort(iterators, ordering.get)) + (p, MergeUtil.mergeSort(iterators, ordering.get)) } else { (p, iterators.iterator.flatten) } @@ -722,117 +735,3 @@ private[spark] class ExternalSorter[K, V, C]( } } } - -private[spark] object ExternalSorter { - - /** - * Merge-sort a sequence of (K, C) iterators using a given a comparator for the keys. - */ - def mergeSort[K, C](iterators: Seq[Iterator[Product2[K, C]]], comparator: Comparator[K]) - : Iterator[Product2[K, C]] = - { - val bufferedIters = iterators.filter(_.hasNext).map(_.buffered) - type Iter = BufferedIterator[Product2[K, C]] - val heap = new mutable.PriorityQueue[Iter]()(new Ordering[Iter] { - // Use the reverse of comparator.compare because PriorityQueue dequeues the max - override def compare(x: Iter, y: Iter): Int = -comparator.compare(x.head._1, y.head._1) - }) - heap.enqueue(bufferedIters: _*) // Will contain only the iterators with hasNext = true - new Iterator[Product2[K, C]] { - override def hasNext: Boolean = !heap.isEmpty - - override def next(): Product2[K, C] = { - if (!hasNext) { - throw new NoSuchElementException - } - val firstBuf = heap.dequeue() - val firstPair = firstBuf.next() - if (firstBuf.hasNext) { - heap.enqueue(firstBuf) - } - firstPair - } - } - } - - /** - * Merge a sequence of (K, C) iterators by aggregating values for each key, assuming that each - * iterator is sorted by key with a given comparator. If the comparator is not a total ordering - * (e.g. when we sort objects by hash code and different keys may compare as equal although - * they're not), we still merge them by doing equality tests for all keys that compare as equal. - */ - def mergeWithAggregation[K, C]( - iterators: Seq[Iterator[Product2[K, C]]], - mergeCombiners: (C, C) => C, - comparator: Comparator[K], - totalOrder: Boolean) - : Iterator[Product2[K, C]] = - { - if (!totalOrder) { - // We only have a partial ordering, e.g. comparing the keys by hash code, which means that - // multiple distinct keys might be treated as equal by the ordering. To deal with this, we - // need to read all keys considered equal by the ordering at once and compare them. - new Iterator[Iterator[Product2[K, C]]] { - val sorted = mergeSort(iterators, comparator).buffered - - // Buffers reused across elements to decrease memory allocation - val keys = new ArrayBuffer[K] - val combiners = new ArrayBuffer[C] - - override def hasNext: Boolean = sorted.hasNext - - override def next(): Iterator[Product2[K, C]] = { - if (!hasNext) { - throw new NoSuchElementException - } - keys.clear() - combiners.clear() - val firstPair = sorted.next() - keys += firstPair._1 - combiners += firstPair._2 - val key = firstPair._1 - while (sorted.hasNext && comparator.compare(sorted.head._1, key) == 0) { - val pair = sorted.next() - var i = 0 - var foundKey = false - while (i < keys.size && !foundKey) { - if (keys(i) == pair._1) { - combiners(i) = mergeCombiners(combiners(i), pair._2) - foundKey = true - } - i += 1 - } - if (!foundKey) { - keys += pair._1 - combiners += pair._2 - } - } - - // Note that we return an iterator of elements since we could've had many keys marked - // equal by the partial order; we flatten this below to get a flat iterator of (K, C). - keys.iterator.zip(combiners.iterator) - } - }.flatMap(i => i) - } else { - // We have a total ordering, so the objects with the same key are sequential. - new Iterator[Product2[K, C]] { - val sorted = mergeSort(iterators, comparator).buffered - - override def hasNext: Boolean = sorted.hasNext - - override def next(): Product2[K, C] = { - if (!hasNext) { - throw new NoSuchElementException - } - val elem = sorted.next() - val k = elem._1 - var c = elem._2 - while (sorted.hasNext && sorted.head._1 == k) { - c = mergeCombiners(c, sorted.head._2) - } - (k, c) - } - } - } - } -} diff --git a/core/src/main/scala/org/apache/spark/util/collection/MergeUtil.scala b/core/src/main/scala/org/apache/spark/util/collection/MergeUtil.scala new file mode 100644 index 0000000000000..246e36b15ab82 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/collection/MergeUtil.scala @@ -0,0 +1,150 @@ +/* + * 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.util.collection + +import java.util.Comparator + +import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer +import org.apache.spark.Aggregator + +private[spark] object MergeUtil { + def mergeSort[K, C]( + iterators: Seq[Iterator[Product2[K, C]]], + comparator: Comparator[K], + keyOrdering: Option[Ordering[K]], + aggregator: Option[Aggregator[K, _, C]]) + : Iterator[Product2[K, C]] = { + if (aggregator.isDefined) { + mergeWithAggregation(iterators, aggregator.get.mergeCombiners, + comparator, keyOrdering.isDefined) + } else { + mergeSort(iterators, comparator) + } + } + + /** + * Merge-sort a sequence of (K, C) iterators using a given a comparator for the keys. + */ + def mergeSort[K, C](iterators: Seq[Iterator[Product2[K, C]]], comparator: Comparator[K]) + : Iterator[Product2[K, C]] = { + val bufferedIters = iterators.filter(_.hasNext).map(_.buffered) + type Iter = BufferedIterator[Product2[K, C]] + val heap = new mutable.PriorityQueue[Iter]()(new Ordering[Iter] { + // Use the reverse of comparator.compare because PriorityQueue dequeues the max + override def compare(x: Iter, y: Iter): Int = -comparator.compare(x.head._1, y.head._1) + }) + heap.enqueue(bufferedIters: _*) // Will contain only the iterators with hasNext = true + new Iterator[Product2[K, C]] { + override def hasNext: Boolean = !heap.isEmpty + + override def next(): Product2[K, C] = { + if (!hasNext) { + throw new NoSuchElementException + } + val firstBuf = heap.dequeue() + val firstPair = firstBuf.next() + if (firstBuf.hasNext) { + heap.enqueue(firstBuf) + } + firstPair + } + } + } + + /** + * Merge a sequence of (K, C) iterators by aggregating values for each key, assuming that each + * iterator is sorted by key with a given comparator. If the comparator is not a total ordering + * (e.g. when we sort objects by hash code and different keys may compare as equal although + * they're not), we still merge them by doing equality tests for all keys that compare as equal. + */ + def mergeWithAggregation[K, C]( + iterators: Seq[Iterator[Product2[K, C]]], + mergeCombiners: (C, C) => C, + comparator: Comparator[K], + totalOrder: Boolean) + : Iterator[Product2[K, C]] = { + if (!totalOrder) { + // We only have a partial ordering, e.g. comparing the keys by hash code, which means that + // multiple distinct keys might be treated as equal by the ordering. To deal with this, we + // need to read all keys considered equal by the ordering at once and compare them. + new Iterator[Iterator[Product2[K, C]]] { + val sorted = mergeSort(iterators, comparator).buffered + + // Buffers reused across elements to decrease memory allocation + val keys = new ArrayBuffer[K] + val combiners = new ArrayBuffer[C] + + override def hasNext: Boolean = sorted.hasNext + + override def next(): Iterator[Product2[K, C]] = { + if (!hasNext) { + throw new NoSuchElementException + } + keys.clear() + combiners.clear() + val firstPair = sorted.next() + keys += firstPair._1 + combiners += firstPair._2 + val key = firstPair._1 + while (sorted.hasNext && comparator.compare(sorted.head._1, key) == 0) { + val pair = sorted.next() + var i = 0 + var foundKey = false + while (i < keys.size && !foundKey) { + if (keys(i) == pair._1) { + combiners(i) = mergeCombiners(combiners(i), pair._2) + foundKey = true + } + i += 1 + } + if (!foundKey) { + keys += pair._1 + combiners += pair._2 + } + } + + // Note that we return an iterator of elements since we could've had many keys marked + // equal by the partial order; we flatten this below to get a flat iterator of (K, C). + keys.iterator.zip(combiners.iterator) + } + }.flatMap(i => i) + } else { + // We have a total ordering, so the objects with the same key are sequential. + new Iterator[Product2[K, C]] { + val sorted = mergeSort(iterators, comparator).buffered + + override def hasNext: Boolean = sorted.hasNext + + override def next(): Product2[K, C] = { + if (!hasNext) { + throw new NoSuchElementException + } + val elem = sorted.next() + val k = elem._1 + var c = elem._2 + while (sorted.hasNext && sorted.head._1 == k) { + c = mergeCombiners(c, sorted.head._2) + } + (k, c) + } + } + } + } + +} diff --git a/core/src/main/scala/org/apache/spark/util/collection/TieredDiskMerger.scala b/core/src/main/scala/org/apache/spark/util/collection/TieredDiskMerger.scala new file mode 100644 index 0000000000000..788401775d6b1 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/collection/TieredDiskMerger.scala @@ -0,0 +1,206 @@ +/* + * 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.util.collection + +import org.apache.spark._ +import org.apache.spark.storage.BlockId +import org.apache.spark.serializer.Serializer +import org.apache.spark.util.CompletionIterator + +import java.util.Comparator +import java.util.concurrent.{PriorityBlockingQueue, CountDownLatch} +import java.io.{File, FileOutputStream, BufferedOutputStream} + +import scala.collection.mutable.ArrayBuffer + +/** + * Explain the boundaries of where this starts and why we have second thread + * + * Manages blocks of sorted data on disk that need to be merged together. Carries out a tiered + * merge that will never merge more than spark.shuffle.maxMergeWidth segments at a time. Except for + * the final merge, which merges disk blocks to a returned iterator, TieredDiskMerger merges blocks + * from disk to disk. + * + * TieredDiskMerger carries out disk-to-disk merges in a background thread that can run concurrently + * with blocks being deposited on disk. + * + * When deciding which blocks to merge, it first tries to minimize the number of blocks, and then + * the size of the blocks chosen. + */ +private[spark] class TieredDiskMerger[K, C]( + conf: SparkConf, + dep: ShuffleDependency[K, _, C], + context: TaskContext) extends Logging { + + case class DiskShuffleBlock(blockId: BlockId, file: File, len: Long) + extends Comparable[DiskShuffleBlock] { + def compareTo(o: DiskShuffleBlock): Int = len.compare(o.len) + } + + 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 + } + }) + + private val maxMergeWidth = conf.getInt("spark.shuffle.maxMergeWidth", 10) + private val fileBufferSize = conf.getInt("spark.shuffle.file.buffer.kb", 32) * 1024 + private val blockManager = SparkEnv.get.blockManager + private val ser = Serializer.getSerializer(dep.serializer) + + private val blocks = new PriorityBlockingQueue[DiskShuffleBlock]() + + private val mergeReadyMonitor = new AnyRef() + private val mergeFinished = new CountDownLatch(1) + + @volatile private var doneRegistering = false + + def registerOnDiskBlock(blockId: BlockId, file: File): Unit = { + assert(!doneRegistering) + blocks.put(new DiskShuffleBlock(blockId, file, file.length())) + + mergeReadyMonitor.synchronized { + if (shouldMergeNow()) { + mergeReadyMonitor.notify() + } + } + } + + /** + * Notify the merger that no more on disk blocks will be registered. + */ + def doneRegisteringOnDiskBlocks(): Unit = { + doneRegistering = true + mergeReadyMonitor.synchronized { + mergeReadyMonitor.notify() + } + } + + def readMerged(): Iterator[Product2[K, C]] = { + mergeFinished.await() + + // Merge the final group for combiner to directly feed to the reducer + val finalMergedPartArray = blocks.toArray(new Array[DiskShuffleBlock](blocks.size())) + val finalItrGroup = blocksToRecordIterators(finalMergedPartArray) + val mergedItr = + MergeUtil.mergeSort(finalItrGroup, keyComparator, dep.keyOrdering, dep.aggregator) + + blocks.clear() + + // Release the in-memory block and on-disk file when iteration is completed. + val completionItr = CompletionIterator[Product2[K, C], Iterator[Product2[K, C]]]( + mergedItr, releaseShuffleBlocks(finalMergedPartArray)) + + new InterruptibleIterator(context, completionItr.map(p => (p._1, p._2))) + + } + + def start() { + new DiskToDiskMergingThread().start() + } + + /** + * Release the left in-memory buffer or on-disk file after merged. + */ + private def releaseShuffleBlocks(shufflePartGroup: Array[DiskShuffleBlock]): Unit = { + shufflePartGroup.map { case DiskShuffleBlock(id, file, length) => + try { + file.delete() + } catch { + // Swallow the exception + case e: Exception => logWarning(s"Unexpected errors when deleting file: ${ + file.getAbsolutePath}", e) + } + } + } + + private def blocksToRecordIterators(shufflePartGroup: Seq[DiskShuffleBlock]) + : Seq[Iterator[Product2[K, C]]] = { + shufflePartGroup.map { case DiskShuffleBlock(id, file, length) => + val blockData = blockManager.diskStore.getBytes(id).getOrElse( + throw new IllegalStateException(s"cannot get data from block $id")) + val itr = blockManager.dataDeserialize(id, blockData, ser) + itr.asInstanceOf[Iterator[Product2[K, C]]] + }.toSeq + } + + /** + * Whether we should carry out a disk-to-disk merge now or wait for more blocks or a done + * registering notification to come in. + * + * We want to avoid merging more blocks than we need to. Our last disk-to-disk merge may + * merge fewer than maxMergeWidth blocks, as its only requirement is that, after it has been + * carried out, <= maxMergeWidth blocks remain. E.g., if maxMergeWidth is 10, no more blocks + * will come in, and we have 13 on-disk blocks, the optimal number of blocks to include in the + * last disk-to-disk merge is 4. + * + * While blocks are still coming in, we don't know the optimal number, so we hold off until we + * either receive the notification that no more blocks are coming in, or until maxMergeWidth + * merge is required no matter what. + * + * E.g. if maxMergeWidth is 10 and we have 19 or more on-disk blocks, a 10-block merge will put us + * at 10 or more blocks, so we might as well carry it out now. + */ + private def shouldMergeNow(): Boolean = doneRegistering || blocks.size() >= maxMergeWidth * 2 - 1 + + private class DiskToDiskMergingThread extends Thread { + // TODO: there will be more than one of these so we need more unique names? + setName("tiered-merge-thread") + setDaemon(true) + + override def run() { + // Each iteration of this loop carries out a disk-to-disk merge. We remain in this loop until + // no more disk-to-disk merges need to be carried out, i.e. when no more blocks are coming in + // and the final merge won't need to merge more than maxMergeWidth blocks. + while (!doneRegistering || blocks.size() > maxMergeWidth) { + while (!shouldMergeNow()) { + mergeReadyMonitor.synchronized { + mergeReadyMonitor.wait() + } + } + + if (blocks.size() > maxMergeWidth) { + val blocksToMerge = new ArrayBuffer[DiskShuffleBlock]() + // Try to pick the smallest merge width that will result in the next merge being the final + // merge. + val mergeWidth = math.min(blocks.size - maxMergeWidth + 1, maxMergeWidth) + (0 until mergeWidth).foreach { + blocksToMerge += blocks.take() + } + + // Merge the blocks + val itrGroup = blocksToRecordIterators(blocksToMerge) + val partialMergedIter = + MergeUtil.mergeSort(itrGroup, keyComparator, dep.keyOrdering, dep.aggregator) + // Write merged blocks to disk + val (tmpBlockId, file) = blockManager.diskBlockManager.createTempBlock() + val fos = new BufferedOutputStream(new FileOutputStream(file), fileBufferSize) + blockManager.dataSerializeStream(tmpBlockId, fos, partialMergedIter, ser) + logInfo(s"Merged ${blocksToMerge.size} on-disk blocks into file ${file.getName}") + + releaseShuffleBlocks(blocksToMerge.toArray) + blocks.add(DiskShuffleBlock(tmpBlockId, file, file.length())) + } + } + + mergeFinished.countDown() + } + } +} From 8e3766aa06218d29259b618c25aa9d6958193452 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Thu, 30 Oct 2014 13:00:02 +0800 Subject: [PATCH 07/27] Rebase to the latest code and fix some conflicts --- .../spark/shuffle/sort/SortShuffleReader.scala | 6 +++--- .../spark/util/collection/ExternalSorter.scala | 16 ---------------- .../spark/util/collection/TieredDiskMerger.scala | 2 +- 3 files changed, 4 insertions(+), 20 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala index ae1d4ff33b929..1064f7d20dd6f 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala @@ -23,7 +23,7 @@ import java.util.Comparator import scala.collection.mutable.{ArrayBuffer, HashMap} import org.apache.spark.{Logging, InterruptibleIterator, SparkEnv, TaskContext} -import org.apache.spark.network.ManagedBuffer +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 @@ -35,7 +35,7 @@ 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 frm the + * As blocks are fetched, we store them in memory until we fail to acquire space from the * ShuffleMemoryManager. When this occurs, we merge the in-memory blocks to disk and go back to * fetching. * @@ -108,7 +108,7 @@ private[spark] class SortShuffleReader[K, C]( MergeUtil.mergeSort(itrGroup, keyComparator, dep.keyOrdering, dep.aggregator) // Write merged blocks to disk - val (tmpBlockId, file) = blockManager.diskBlockManager.createTempBlock() + val (tmpBlockId, file) = blockManager.diskBlockManager.createTempShuffleBlock() val fos = new BufferedOutputStream(new FileOutputStream(file), fileBufferSize) blockManager.dataSerializeStream(tmpBlockId, fos, partialMergedIter, ser) tieredMerger.registerOnDiskBlock(tmpBlockId, file) diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala index fca4fb2e08674..74475d489b5e7 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala @@ -21,7 +21,6 @@ import java.io._ import java.util.Comparator import scala.collection.mutable.ArrayBuffer -import scala.collection.mutable import com.google.common.io.ByteStreams @@ -29,7 +28,6 @@ import org.apache.spark._ import org.apache.spark.serializer.{DeserializationStream, Serializer} import org.apache.spark.executor.ShuffleWriteMetrics import org.apache.spark.storage.{BlockObjectWriter, BlockId} -import org.apache.spark.network.ManagedBuffer /** * Sorts and potentially merges a number of key-value pairs of type (K, V) to produce key-combiner @@ -188,20 +186,6 @@ private[spark] class ExternalSorter[K, V, C]( elementsPerPartition: Array[Long]) private val spills = new ArrayBuffer[SpilledFile] - def insertInMemoryBlock(buffer: ManagedBuffer) { - // try to acquire memory - val granted = shuffleMemoryManager.tryToAcquire(buffer.size) - if (granted < buffer.size) { - if (!spillingEnabled) { - throw new IllegalStateException() - } - // spill in-memory partitions - } - // maybe spill - - // spilling should probably tell our background merger thread to try stuff - } - def insertAll(records: Iterator[_ <: Product2[K, V]]): Unit = { // TODO: stop combining if we find that the reduction factor isn't high val shouldCombine = aggregator.isDefined diff --git a/core/src/main/scala/org/apache/spark/util/collection/TieredDiskMerger.scala b/core/src/main/scala/org/apache/spark/util/collection/TieredDiskMerger.scala index 788401775d6b1..b1487a519797f 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/TieredDiskMerger.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/TieredDiskMerger.scala @@ -190,7 +190,7 @@ private[spark] class TieredDiskMerger[K, C]( val partialMergedIter = MergeUtil.mergeSort(itrGroup, keyComparator, dep.keyOrdering, dep.aggregator) // Write merged blocks to disk - val (tmpBlockId, file) = blockManager.diskBlockManager.createTempBlock() + val (tmpBlockId, file) = blockManager.diskBlockManager.createTempShuffleBlock() val fos = new BufferedOutputStream(new FileOutputStream(file), fileBufferSize) blockManager.dataSerializeStream(tmpBlockId, fos, partialMergedIter, ser) logInfo(s"Merged ${blocksToMerge.size} on-disk blocks into file ${file.getName}") From 98c039b21a03fddc3076d35cddd18d9439e99587 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Tue, 4 Nov 2014 15:11:29 +0800 Subject: [PATCH 08/27] SortShuffleReader code improvement --- .../shuffle/sort/MixedShuffleReader.scala | 40 ++++++++ .../shuffle/sort/SortShuffleManager.scala | 2 +- .../shuffle/sort/SortShuffleReader.scala | 98 ++++++++++--------- .../util/collection/TieredDiskMerger.scala | 92 +++++++++-------- .../scala/org/apache/spark/rdd/RDDSuite.scala | 8 +- 5 files changed, 144 insertions(+), 96 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/shuffle/sort/MixedShuffleReader.scala diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/MixedShuffleReader.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/MixedShuffleReader.scala new file mode 100644 index 0000000000000..73255f59f5f1d --- /dev/null +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/MixedShuffleReader.scala @@ -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() +} diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala index 0c0a358cdaa26..5338cc9881482 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala @@ -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) } diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala index 1064f7d20dd6f..6590e638dc16c 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala @@ -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 @@ -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 @@ -65,29 +64,25 @@ 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()) { @@ -95,51 +90,68 @@ private[spark] class SortShuffleReader[K, C]( 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]]] } @@ -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]), diff --git a/core/src/main/scala/org/apache/spark/util/collection/TieredDiskMerger.scala b/core/src/main/scala/org/apache/spark/util/collection/TieredDiskMerger.scala index b1487a519797f..06abdb7787e22 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/TieredDiskMerger.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/TieredDiskMerger.scala @@ -17,22 +17,22 @@ package org.apache.spark.util.collection -import org.apache.spark._ -import org.apache.spark.storage.BlockId -import org.apache.spark.serializer.Serializer -import org.apache.spark.util.CompletionIterator - +import java.io.{File, FileOutputStream, BufferedOutputStream} import java.util.Comparator import java.util.concurrent.{PriorityBlockingQueue, CountDownLatch} -import java.io.{File, FileOutputStream, BufferedOutputStream} import scala.collection.mutable.ArrayBuffer +import org.apache.spark._ +import org.apache.spark.storage.BlockId +import org.apache.spark.serializer.Serializer +import org.apache.spark.util.CompletionIterator + /** * Explain the boundaries of where this starts and why we have second thread * * Manages blocks of sorted data on disk that need to be merged together. Carries out a tiered - * merge that will never merge more than spark.shuffle.maxMergeWidth segments at a time. Except for + * merge that will never merge more than spark.shuffle.maxMergeFactor segments at a time. Except for * the final merge, which merges disk blocks to a returned iterator, TieredDiskMerger merges blocks * from disk to disk. * @@ -45,36 +45,36 @@ import scala.collection.mutable.ArrayBuffer private[spark] class TieredDiskMerger[K, C]( conf: SparkConf, dep: ShuffleDependency[K, _, C], + keyComparator: Comparator[K], context: TaskContext) extends Logging { + /** Manage the on-disk shuffle block and related file, file size */ case class DiskShuffleBlock(blockId: BlockId, file: File, len: Long) extends Comparable[DiskShuffleBlock] { def compareTo(o: DiskShuffleBlock): Int = len.compare(o.len) } - 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 - } - }) - - private val maxMergeWidth = conf.getInt("spark.shuffle.maxMergeWidth", 10) + private val maxMergeFactor = conf.getInt("spark.shuffle.maxMergeFactor", 100) private val fileBufferSize = conf.getInt("spark.shuffle.file.buffer.kb", 32) * 1024 private val blockManager = SparkEnv.get.blockManager private val ser = Serializer.getSerializer(dep.serializer) - private val blocks = new PriorityBlockingQueue[DiskShuffleBlock]() + /** PriorityQueue to store the on-disk merging blocks, blocks are merged by size ordering */ + private val onDiskBlocks = new PriorityBlockingQueue[DiskShuffleBlock]() + /** A merging thread to merge on-disk blocks */ + private val diskToDiskMerger = new DiskToDiskMerger + + /** Signal to block/signal the merge action */ private val mergeReadyMonitor = new AnyRef() + private val mergeFinished = new CountDownLatch(1) @volatile private var doneRegistering = false def registerOnDiskBlock(blockId: BlockId, file: File): Unit = { assert(!doneRegistering) - blocks.put(new DiskShuffleBlock(blockId, file, file.length())) + onDiskBlocks.put(new DiskShuffleBlock(blockId, file, file.length())) mergeReadyMonitor.synchronized { if (shouldMergeNow()) { @@ -97,30 +97,29 @@ private[spark] class TieredDiskMerger[K, C]( mergeFinished.await() // Merge the final group for combiner to directly feed to the reducer - val finalMergedPartArray = blocks.toArray(new Array[DiskShuffleBlock](blocks.size())) + val finalMergedPartArray = onDiskBlocks.toArray(new Array[DiskShuffleBlock](onDiskBlocks.size())) val finalItrGroup = blocksToRecordIterators(finalMergedPartArray) val mergedItr = MergeUtil.mergeSort(finalItrGroup, keyComparator, dep.keyOrdering, dep.aggregator) - blocks.clear() + onDiskBlocks.clear() - // Release the in-memory block and on-disk file when iteration is completed. + // Release the on-disk file when iteration is completed. val completionItr = CompletionIterator[Product2[K, C], Iterator[Product2[K, C]]]( mergedItr, releaseShuffleBlocks(finalMergedPartArray)) - new InterruptibleIterator(context, completionItr.map(p => (p._1, p._2))) - + new InterruptibleIterator(context, completionItr) } def start() { - new DiskToDiskMergingThread().start() + diskToDiskMerger.start() } /** * Release the left in-memory buffer or on-disk file after merged. */ private def releaseShuffleBlocks(shufflePartGroup: Array[DiskShuffleBlock]): Unit = { - shufflePartGroup.map { case DiskShuffleBlock(id, file, length) => + shufflePartGroup.map { case DiskShuffleBlock(_, file, _) => try { file.delete() } catch { @@ -133,11 +132,10 @@ private[spark] class TieredDiskMerger[K, C]( private def blocksToRecordIterators(shufflePartGroup: Seq[DiskShuffleBlock]) : Seq[Iterator[Product2[K, C]]] = { - shufflePartGroup.map { case DiskShuffleBlock(id, file, length) => - val blockData = blockManager.diskStore.getBytes(id).getOrElse( - throw new IllegalStateException(s"cannot get data from block $id")) - val itr = blockManager.dataDeserialize(id, blockData, ser) - itr.asInstanceOf[Iterator[Product2[K, C]]] + shufflePartGroup.map { case DiskShuffleBlock(id, _, _) => + val blockData = blockManager.getBlockData(id) + blockManager.dataDeserialize(id, blockData.nioByteBuffer(), ser) + .asInstanceOf[Iterator[Product2[K, C]]] }.toSeq } @@ -146,57 +144,57 @@ private[spark] class TieredDiskMerger[K, C]( * registering notification to come in. * * We want to avoid merging more blocks than we need to. Our last disk-to-disk merge may - * merge fewer than maxMergeWidth blocks, as its only requirement is that, after it has been - * carried out, <= maxMergeWidth blocks remain. E.g., if maxMergeWidth is 10, no more blocks + * merge fewer than maxMergeFactor blocks, as its only requirement is that, after it has been + * carried out, <= maxMergeFactor blocks remain. E.g., if maxMergeFactor is 10, no more blocks * will come in, and we have 13 on-disk blocks, the optimal number of blocks to include in the * last disk-to-disk merge is 4. * * While blocks are still coming in, we don't know the optimal number, so we hold off until we - * either receive the notification that no more blocks are coming in, or until maxMergeWidth + * either receive the notification that no more blocks are coming in, or until maxMergeFactor * merge is required no matter what. * - * E.g. if maxMergeWidth is 10 and we have 19 or more on-disk blocks, a 10-block merge will put us + * E.g. if maxMergeFactor is 10 and we have 19 or more on-disk blocks, a 10-block merge will put us * at 10 or more blocks, so we might as well carry it out now. */ - private def shouldMergeNow(): Boolean = doneRegistering || blocks.size() >= maxMergeWidth * 2 - 1 + private def shouldMergeNow(): Boolean = doneRegistering || onDiskBlocks.size() >= maxMergeFactor * 2 - 1 - private class DiskToDiskMergingThread extends Thread { - // TODO: there will be more than one of these so we need more unique names? - setName("tiered-merge-thread") + private final class DiskToDiskMerger extends Thread { + setName(s"tiered-merge-thread-${Thread.currentThread().getId}") setDaemon(true) override def run() { // Each iteration of this loop carries out a disk-to-disk merge. We remain in this loop until // no more disk-to-disk merges need to be carried out, i.e. when no more blocks are coming in - // and the final merge won't need to merge more than maxMergeWidth blocks. - while (!doneRegistering || blocks.size() > maxMergeWidth) { + // and the final merge won't need to merge more than maxMergeFactor blocks. + while (!doneRegistering || onDiskBlocks.size() > maxMergeFactor) { while (!shouldMergeNow()) { mergeReadyMonitor.synchronized { mergeReadyMonitor.wait() } } - if (blocks.size() > maxMergeWidth) { + if (onDiskBlocks.size() > maxMergeFactor) { val blocksToMerge = new ArrayBuffer[DiskShuffleBlock]() // Try to pick the smallest merge width that will result in the next merge being the final // merge. - val mergeWidth = math.min(blocks.size - maxMergeWidth + 1, maxMergeWidth) - (0 until mergeWidth).foreach { - blocksToMerge += blocks.take() + val mergeFactor = math.min(onDiskBlocks.size - maxMergeFactor + 1, maxMergeFactor) + (0 until mergeFactor).foreach { + blocksToMerge += onDiskBlocks.take() } // Merge the blocks val itrGroup = blocksToRecordIterators(blocksToMerge) - val partialMergedIter = + val partialMergedItr = MergeUtil.mergeSort(itrGroup, keyComparator, dep.keyOrdering, dep.aggregator) // 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) + blockManager.dataSerializeStream(tmpBlockId, fos, partialMergedItr, ser) + logInfo(s"Merged ${blocksToMerge.size} on-disk blocks into file ${file.getName}") releaseShuffleBlocks(blocksToMerge.toArray) - blocks.add(DiskShuffleBlock(tmpBlockId, file, file.length())) + onDiskBlocks.add(DiskShuffleBlock(tmpBlockId, file, file.length())) } } diff --git a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala index df42faab64505..17e26724ae720 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -19,14 +19,14 @@ package org.apache.spark.rdd import java.io.{ObjectInputStream, ObjectOutputStream, IOException} -import com.esotericsoftware.kryo.KryoException - import scala.collection.mutable.{ArrayBuffer, HashMap} import scala.collection.JavaConverters._ import scala.reflect.ClassTag import org.scalatest.FunSuite +import com.esotericsoftware.kryo.KryoException + import org.apache.spark._ import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} import org.apache.spark.rdd.RDDSuiteUtils._ @@ -726,8 +726,8 @@ class RDDSuite extends FunSuite with SharedSparkContext { val repartitioned = data.repartitionAndSortWithinPartitions(partitioner) val partitions = repartitioned.glom().collect() - assert(partitions(0) === Seq((0, 5), (0, 8), (2, 6))) - assert(partitions(1) === Seq((1, 3), (3, 8), (3, 8))) + assert(partitions(0).toSet === Set((0, 5), (0, 8), (2, 6))) + assert(partitions(1).toSet === Set((1, 3), (3, 8), (3, 8))) } test("intersection") { From 7d999efe4eae6f07c82e15ed4f92328bf766e01d Mon Sep 17 00:00:00 2001 From: jerryshao Date: Wed, 5 Nov 2014 09:10:14 +0800 Subject: [PATCH 09/27] Changes to rebase to the latest master branch --- .../shuffle/sort/MixedShuffleReader.scala | 2 - .../shuffle/sort/SortShuffleReader.scala | 57 ++++++++++++------- .../storage/ShuffleBlockFetcherIterator.scala | 4 +- 3 files changed, 39 insertions(+), 24 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/MixedShuffleReader.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/MixedShuffleReader.scala index 73255f59f5f1d..2fe3cc0f31b4d 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/MixedShuffleReader.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/MixedShuffleReader.scala @@ -35,6 +35,4 @@ private[spark] class MixedShuffleReader[K, C]( } override def read(): Iterator[Product2[K, C]] = shuffleReader.read() - - override def stop(): Unit = shuffleReader.stop() } diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala index 6590e638dc16c..8772b9740e12c 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala @@ -21,14 +21,15 @@ import java.io.{BufferedOutputStream, FileOutputStream} import java.util.Comparator import scala.collection.mutable.{ArrayBuffer, HashMap} +import scala.util.{Failure, Success, Try} -import org.apache.spark.{Logging, InterruptibleIterator, SparkEnv, TaskContext} +import org.apache.spark._ 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.{BaseShuffleHandle, FetchFailedException, ShuffleReader} import org.apache.spark.storage._ -import org.apache.spark.util.CompletionIterator -import org.apache.spark.util.collection.{TieredDiskMerger, MergeUtil} +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 @@ -69,6 +70,9 @@ private[spark] class SortShuffleReader[K, C]( /** ArrayBuffer to store in-memory shuffle blocks */ private val inMemoryBlocks = new ArrayBuffer[MemoryShuffleBlock]() + /** Manage the BlockManagerId and related shuffle blocks */ + private var statuses: Array[(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] { @@ -85,17 +89,26 @@ private[spark] class SortShuffleReader[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") + for ((blockId, blockOption) <- fetchRawBlocks()) { + val blockData = blockOption match { + case Success(block) => block + case Failure(e) => + blockId match { + case ShuffleBlockId (shufId, mapId, _) => + val address = statuses(mapId.toInt)._1 + throw new FetchFailedException (address, shufId.toInt, mapId.toInt, startPartition, + Utils.exceptionString (e)) + case _ => + throw new SparkException ( + s"Failed to get block $blockId, which is not a shuffle block", e) + } } - inMemoryBlocks += MemoryShuffleBlock(blockId, blockData.get) + inMemoryBlocks += MemoryShuffleBlock(blockId, blockData) // 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") + val blockSize = blockData.size + val granted = shuffleMemoryManager.tryToAcquire(blockSize) if (granted < blockSize) { logInfo(s"Granted $granted memory is not enough to store shuffle block ($blockSize), " + @@ -124,7 +137,10 @@ private[spark] class SortShuffleReader[K, C]( tieredMerger.registerOnDiskBlock(tmpBlockId, file) + logInfo(s"Merge ${inMemoryBlocks.size} in-memory blocks into file ${file.getName}") + for (block <- inMemoryBlocks) { + block.blockData.release() shuffleMemoryManager.release(block.blockData.size) } inMemoryBlocks.clear() @@ -143,7 +159,10 @@ private[spark] class SortShuffleReader[K, C]( // Release the in-memory block when iteration is completed. val completionItr = CompletionIterator[Product2[K, C], Iterator[Product2[K, C]]]( mergedItr, () => { - inMemoryBlocks.foreach(block => shuffleMemoryManager.release(block.blockData.size)) + inMemoryBlocks.foreach { block => + block.blockData.release() + shuffleMemoryManager.release(block.blockData.size) + } inMemoryBlocks.clear() }) @@ -157,10 +176,8 @@ private[spark] class SortShuffleReader[K, C]( } } - override def stop(): Unit = ??? - - private def fetchRawBlocks(): Iterator[(BlockId, Option[ManagedBuffer])] = { - val statuses = SparkEnv.get.mapOutputTracker.getServerStatuses(handle.shuffleId, startPartition) + private def fetchRawBlocks(): Iterator[(BlockId, Try[ManagedBuffer])] = { + statuses = SparkEnv.get.mapOutputTracker.getServerStatuses(handle.shuffleId, startPartition) val splitsByAddress = new HashMap[BlockManagerId, ArrayBuffer[(Int, Long)]]() for (((address, size), index) <- statuses.zipWithIndex) { @@ -176,16 +193,16 @@ private[spark] class SortShuffleReader[K, C]( shuffleRawBlockFetcherItr = new ShuffleRawBlockFetcherIterator( context, - SparkEnv.get.blockTransferService, + SparkEnv.get.blockManager.shuffleClient, blockManager, blocksByAddress, conf.getLong("spark.reducer.maxMbInFlight", 48) * 1024 * 1024) val completionItr = CompletionIterator[ - (BlockId, Option[ManagedBuffer]), - Iterator[(BlockId, Option[ManagedBuffer])]](shuffleRawBlockFetcherItr, + (BlockId, Try[ManagedBuffer]), + Iterator[(BlockId, Try[ManagedBuffer])]](shuffleRawBlockFetcherItr, () => context.taskMetrics.updateShuffleReadMetrics()) - new InterruptibleIterator[(BlockId, Option[ManagedBuffer])](context, completionItr) + new InterruptibleIterator[(BlockId, Try[ManagedBuffer])](context, completionItr) } } diff --git a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala index b3df9e7104d7e..1d87d15d5b8bf 100644 --- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala @@ -33,7 +33,7 @@ import org.apache.spark.util.{CompletionIterator, Utils} private[spark] final class ShuffleBlockFetcherIterator( context: TaskContext, - blockTransferService: BlockTransferService, + shuffleClient: ShuffleClient, blockManager: BlockManager, blocksByAddress: Seq[(BlockManagerId, Seq[(BlockId, Long)])], serializer: Serializer, @@ -42,7 +42,7 @@ final class ShuffleBlockFetcherIterator( val shuffleRawBlockFetcherItr = new ShuffleRawBlockFetcherIterator( context, - blockTransferService, + shuffleClient, blockManager, blocksByAddress, maxBytesInFlight) From 319e6d10c9888cab7534ac7913fa2ad686b81619 Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Tue, 4 Nov 2014 23:30:46 -0800 Subject: [PATCH 10/27] Don't spill more blocks than we need to --- .../shuffle/sort/SortShuffleReader.scala | 96 +++++++++++-------- 1 file changed, 58 insertions(+), 38 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala index 8772b9740e12c..63c1dab1cd348 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala @@ -20,7 +20,7 @@ package org.apache.spark.shuffle.sort import java.io.{BufferedOutputStream, FileOutputStream} import java.util.Comparator -import scala.collection.mutable.{ArrayBuffer, HashMap} +import scala.collection.mutable.{ArrayBuffer, HashMap, Queue} import scala.util.{Failure, Success, Try} import org.apache.spark._ @@ -59,6 +59,9 @@ private[spark] class SortShuffleReader[K, C]( /** Shuffle block fetcher iterator */ private var shuffleRawBlockFetcherItr: ShuffleRawBlockFetcherIterator = _ + /** Number of bytes left to fetch */ + private var unfetchedBytes: Long = _ + private val dep = handle.dependency private val conf = SparkEnv.get.conf private val blockManager = SparkEnv.get.blockManager @@ -68,7 +71,7 @@ private[spark] class SortShuffleReader[K, C]( private val fileBufferSize = conf.getInt("spark.shuffle.file.buffer.kb", 32) * 1024 /** ArrayBuffer to store in-memory shuffle blocks */ - private val inMemoryBlocks = new ArrayBuffer[MemoryShuffleBlock]() + private val inMemoryBlocks = new Queue[MemoryShuffleBlock]() /** Manage the BlockManagerId and related shuffle blocks */ private var statuses: Array[(BlockManagerId, Long)] = _ @@ -104,55 +107,26 @@ private[spark] class SortShuffleReader[K, C]( } } - inMemoryBlocks += MemoryShuffleBlock(blockId, blockData) - // 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) { logInfo(s"Granted $granted memory is not enough to store shuffle block ($blockSize), " + - s"try to consolidate in-memory blocks to release the memory") + s"spilling in-memory blocks to release the memory") shuffleMemoryManager.release(granted) - - // Write merged blocks to disk - val (tmpBlockId, file) = blockManager.diskBlockManager.createTempShuffleBlock() - 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) - - logInfo(s"Merge ${inMemoryBlocks.size} in-memory blocks into file ${file.getName}") - - for (block <- inMemoryBlocks) { - block.blockData.release() - shuffleMemoryManager.release(block.blockData.size) - } - inMemoryBlocks.clear() + spillInMemoryBlocks(MemoryShuffleBlock(blockId, blockData)) } + unfetchedBytes -= blockData.size() shuffleRawBlockFetcherItr.currentResult = null } + assert(unfetchedBytes == 0) tieredMerger.doneRegisteringOnDiskBlocks() // Merge on-disk blocks with in-memory blocks to directly feed to the reducer. - val finalItrGroup = inMemoryBlocksToIterators() ++ Seq(tieredMerger.readMerged()) + val finalItrGroup = inMemoryBlocksToIterators(inMemoryBlocks) ++ Seq(tieredMerger.readMerged()) val mergedItr = MergeUtil.mergeSort(finalItrGroup, keyComparator, dep.keyOrdering, dep.aggregator) @@ -169,8 +143,53 @@ private[spark] class SortShuffleReader[K, C]( new InterruptibleIterator(context, completionItr.map(p => (p._1, p._2))) } - private def inMemoryBlocksToIterators(): Seq[Iterator[Product2[K, C]]] = { - inMemoryBlocks.map{ case MemoryShuffleBlock(id, buf) => + def spillInMemoryBlocks(tippingBlock: MemoryShuffleBlock): Unit = { + // Write merged blocks to disk + val (tmpBlockId, file) = blockManager.diskBlockManager.createTempShuffleBlock() + val fos = new FileOutputStream(file) + val bos = new BufferedOutputStream(fos, fileBufferSize) + + // 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. + 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 + } + + if (blocksToSpill.size > 1) { + val itrGroup = inMemoryBlocksToIterators(blocksToSpill) + val partialMergedItr = + MergeUtil.mergeSort(itrGroup, keyComparator, dep.keyOrdering, dep.aggregator) + blockManager.dataSerializeStream(tmpBlockId, bos, partialMergedItr, ser) + } else { + val buffer = blocksToSpill.map(_.blockData.nioByteBuffer()).head + val channel = fos.getChannel + while (buffer.hasRemaining) { + channel.write(buffer) + } + channel.close() + } + + tieredMerger.registerOnDiskBlock(tmpBlockId, file) + + logInfo(s"Merged ${blocksToSpill.size} in-memory blocks into file ${file.getName}") + + for (block <- blocksToSpill) { + block.blockData.release() + if (block != tippingBlock) { + shuffleMemoryManager.release(block.blockData.size) + } + } + } + + 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]]] } @@ -190,6 +209,7 @@ private[spark] class SortShuffleReader[K, C]( } (address, blocks.toSeq) } + unfetchedBytes = blocksByAddress.flatMap(a => a._2.map(b => b._2)).sum shuffleRawBlockFetcherItr = new ShuffleRawBlockFetcherIterator( context, From 96ef5c16b96e247a13e0613a27d01c0395f86f6e Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Wed, 5 Nov 2014 00:57:16 -0800 Subject: [PATCH 11/27] Fix bug: add to inMemoryBlocks --- .../org/apache/spark/shuffle/sort/SortShuffleReader.scala | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala index 63c1dab1cd348..6ab6da52c0151 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala @@ -110,12 +110,15 @@ 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) { logInfo(s"Granted $granted memory is not enough to store shuffle block ($blockSize), " + s"spilling in-memory blocks to release the memory") shuffleMemoryManager.release(granted) - spillInMemoryBlocks(MemoryShuffleBlock(blockId, blockData)) + spillInMemoryBlocks(block) + } else { + inMemoryBlocks += block } unfetchedBytes -= blockData.size() From d481c98f431eb6da2764235db38f84f4db8def83 Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Wed, 5 Nov 2014 10:39:12 -0800 Subject: [PATCH 12/27] Fix another bug --- .../scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala index 6ab6da52c0151..df1a3fbfc4b9f 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala @@ -158,7 +158,7 @@ private[spark] class SortShuffleReader[K, C]( val blocksToSpill = new ArrayBuffer[MemoryShuffleBlock]() blocksToSpill += tippingBlock bytesToSpill -= tippingBlock.blockData.size - while (bytesToSpill > 0 && inMemoryBlocks.isEmpty) { + while (bytesToSpill > 0 && !inMemoryBlocks.isEmpty) { val block = inMemoryBlocks.dequeue() blocksToSpill += block bytesToSpill -= block.blockData.size From bf6a49dcf4e1c5123cffa78e95aa25693b858d46 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Wed, 5 Nov 2014 17:22:20 +0800 Subject: [PATCH 13/27] Bug fix and revert ShuffleMemoryManager --- .../apache/spark/shuffle/ShuffleMemoryManager.scala | 10 ++++------ .../apache/spark/shuffle/sort/SortShuffleReader.scala | 4 ++-- 2 files changed, 6 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleMemoryManager.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleMemoryManager.scala index 017081ef42783..5e0a6f5eb6e14 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleMemoryManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleMemoryManager.scala @@ -93,16 +93,14 @@ private[spark] class ShuffleMemoryManager(maxMemory: Long) extends Logging { } /** Release numBytes bytes for the current thread. */ - def release(numBytes: Long): Unit = release(numBytes, Thread.currentThread().getId) - - /** Release numBytes bytes for the specific thread. */ - def release(numBytes: Long, tid: Long): Unit = synchronized { - val curMem = threadMemory.getOrElse(tid, 0L) + def release(numBytes: Long): Unit = synchronized { + val threadId = Thread.currentThread().getId + val curMem = threadMemory.getOrElse(threadId, 0L) if (curMem < numBytes) { throw new SparkException( s"Internal error: release called on ${numBytes} bytes but thread only has ${curMem}") } - threadMemory(tid) -= numBytes + threadMemory(threadId) -= numBytes notifyAll() // Notify waiters who locked "this" in tryToAcquire that memory has been freed } diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala index df1a3fbfc4b9f..ea388a531ee0e 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala @@ -135,7 +135,7 @@ private[spark] class SortShuffleReader[K, C]( // Release the in-memory block when iteration is completed. val completionItr = CompletionIterator[Product2[K, C], Iterator[Product2[K, C]]]( - mergedItr, () => { + mergedItr, { inMemoryBlocks.foreach { block => block.blockData.release() shuffleMemoryManager.release(block.blockData.size) @@ -224,7 +224,7 @@ private[spark] class SortShuffleReader[K, C]( val completionItr = CompletionIterator[ (BlockId, Try[ManagedBuffer]), Iterator[(BlockId, Try[ManagedBuffer])]](shuffleRawBlockFetcherItr, - () => context.taskMetrics.updateShuffleReadMetrics()) + context.taskMetrics.updateShuffleReadMetrics()) new InterruptibleIterator[(BlockId, Try[ManagedBuffer])](context, completionItr) } From 79dc8237fd25d984a69c1eb746cbe93977b86944 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Fri, 7 Nov 2014 11:02:14 +0800 Subject: [PATCH 14/27] Fix some bugs in spilling to disk --- .../shuffle/sort/SortShuffleReader.scala | 50 ++++++++++++++++--- .../util/collection/TieredDiskMerger.scala | 17 ++++--- 2 files changed, 51 insertions(+), 16 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala index ea388a531ee0e..61b0a4e5aeb5f 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala @@ -70,6 +70,10 @@ private[spark] class SortShuffleReader[K, C]( private val fileBufferSize = conf.getInt("spark.shuffle.file.buffer.kb", 32) * 1024 + /** Number of bytes spilled in memory and on disk */ + private var _diskBytesSpilled = 0L + private var _memoryBytesSpilled = 0L + /** ArrayBuffer to store in-memory shuffle blocks */ private val inMemoryBlocks = new Queue[MemoryShuffleBlock]() @@ -107,22 +111,52 @@ private[spark] class SortShuffleReader[K, C]( } } + inMemoryBlocks += MemoryShuffleBlock(blockId, blockData) + 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) val block = MemoryShuffleBlock(blockId, blockData) if (granted < blockSize) { - logInfo(s"Granted $granted memory is not enough to store shuffle block ($blockSize), " + - s"spilling in-memory blocks to release the memory") + logInfo(s"Granted $granted memory is not enough to store shuffle block id $blockId, " + + s"block size $blockSize, try to consolidate in-memory blocks to release the memory") shuffleMemoryManager.release(granted) - spillInMemoryBlocks(block) - } else { - inMemoryBlocks += block - } - unfetchedBytes -= blockData.size() - shuffleRawBlockFetcherItr.currentResult = null + // Write merged blocks to disk + val (tmpBlockId, file) = blockManager.diskBlockManager.createTempShuffleBlock() + 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) + // TODO. change this into objectWriter + // TODO. Track the memory and disk spill + 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) + + logInfo(s"Merge ${inMemoryBlocks.size} in-memory blocks into file ${file.getName}") + + for (block <- inMemoryBlocks) { + block.blockData.release() + if (block.blockId != blockId) { + shuffleMemoryManager.release(block.blockData.size) + } + } + inMemoryBlocks.clear() + } } assert(unfetchedBytes == 0) diff --git a/core/src/main/scala/org/apache/spark/util/collection/TieredDiskMerger.scala b/core/src/main/scala/org/apache/spark/util/collection/TieredDiskMerger.scala index 06abdb7787e22..93cb48cadc6f9 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/TieredDiskMerger.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/TieredDiskMerger.scala @@ -97,8 +97,8 @@ private[spark] class TieredDiskMerger[K, C]( mergeFinished.await() // Merge the final group for combiner to directly feed to the reducer - val finalMergedPartArray = onDiskBlocks.toArray(new Array[DiskShuffleBlock](onDiskBlocks.size())) - val finalItrGroup = blocksToRecordIterators(finalMergedPartArray) + val finalMergedBlocks = onDiskBlocks.toArray(new Array[DiskShuffleBlock](onDiskBlocks.size())) + val finalItrGroup = blocksToRecordIterators(finalMergedBlocks) val mergedItr = MergeUtil.mergeSort(finalItrGroup, keyComparator, dep.keyOrdering, dep.aggregator) @@ -106,7 +106,7 @@ private[spark] class TieredDiskMerger[K, C]( // Release the on-disk file when iteration is completed. val completionItr = CompletionIterator[Product2[K, C], Iterator[Product2[K, C]]]( - mergedItr, releaseShuffleBlocks(finalMergedPartArray)) + mergedItr, releaseShuffleBlocks(finalMergedBlocks)) new InterruptibleIterator(context, completionItr) } @@ -118,9 +118,10 @@ private[spark] class TieredDiskMerger[K, C]( /** * Release the left in-memory buffer or on-disk file after merged. */ - private def releaseShuffleBlocks(shufflePartGroup: Array[DiskShuffleBlock]): Unit = { - shufflePartGroup.map { case DiskShuffleBlock(_, file, _) => + private def releaseShuffleBlocks(onDiskShuffleGroup: Array[DiskShuffleBlock]): Unit = { + onDiskShuffleGroup.map { case DiskShuffleBlock(_, file, _) => try { + logDebug(s"Deleting the unused temp shuffle file: ${file.getName}") file.delete() } catch { // Swallow the exception @@ -133,9 +134,7 @@ private[spark] class TieredDiskMerger[K, C]( private def blocksToRecordIterators(shufflePartGroup: Seq[DiskShuffleBlock]) : Seq[Iterator[Product2[K, C]]] = { shufflePartGroup.map { case DiskShuffleBlock(id, _, _) => - val blockData = blockManager.getBlockData(id) - blockManager.dataDeserialize(id, blockData.nioByteBuffer(), ser) - .asInstanceOf[Iterator[Product2[K, C]]] + blockManager.diskStore.getValues(id, ser).get.asInstanceOf[Iterator[Product2[K, C]]] }.toSeq } @@ -188,6 +187,8 @@ private[spark] class TieredDiskMerger[K, C]( MergeUtil.mergeSort(itrGroup, keyComparator, dep.keyOrdering, dep.aggregator) // Write merged blocks to disk val (tmpBlockId, file) = blockManager.diskBlockManager.createTempShuffleBlock() + //TODO. change this into objectWriter + // TODO. Track the memory and disk spill val fos = new BufferedOutputStream(new FileOutputStream(file), fileBufferSize) blockManager.dataSerializeStream(tmpBlockId, fos, partialMergedItr, ser) From 2e04b8592f7efb364bd61b2e81793b227aa30e83 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Mon, 10 Nov 2014 13:10:21 +0800 Subject: [PATCH 15/27] Modify to use BlockObjectWriter to write data --- .../shuffle/sort/SortShuffleReader.scala | 141 +++++++++--------- .../util/collection/TieredDiskMerger.scala | 42 +++++- 2 files changed, 106 insertions(+), 77 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala index 61b0a4e5aeb5f..3f8c50571ac01 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala @@ -17,10 +17,12 @@ package org.apache.spark.shuffle.sort -import java.io.{BufferedOutputStream, FileOutputStream} +import java.io.FileOutputStream import java.util.Comparator -import scala.collection.mutable.{ArrayBuffer, HashMap, Queue} +import org.apache.spark.executor.ShuffleWriteMetrics + +import scala.collection.mutable.{ArrayBuffer, HashMap} import scala.util.{Failure, Success, Try} import org.apache.spark._ @@ -59,9 +61,6 @@ private[spark] class SortShuffleReader[K, C]( /** Shuffle block fetcher iterator */ private var shuffleRawBlockFetcherItr: ShuffleRawBlockFetcherIterator = _ - /** Number of bytes left to fetch */ - private var unfetchedBytes: Long = _ - private val dep = handle.dependency private val conf = SparkEnv.get.conf private val blockManager = SparkEnv.get.blockManager @@ -71,11 +70,11 @@ private[spark] class SortShuffleReader[K, C]( private val fileBufferSize = conf.getInt("spark.shuffle.file.buffer.kb", 32) * 1024 /** Number of bytes spilled in memory and on disk */ - private var _diskBytesSpilled = 0L - private var _memoryBytesSpilled = 0L + private var _memoryBytesSpilled: Long = 0L + private var _diskBytesSpilled: Long = 0L /** ArrayBuffer to store in-memory shuffle blocks */ - private val inMemoryBlocks = new Queue[MemoryShuffleBlock]() + private val inMemoryBlocks = new ArrayBuffer[MemoryShuffleBlock]() /** Manage the BlockManagerId and related shuffle blocks */ private var statuses: Array[(BlockManagerId, Long)] = _ @@ -93,6 +92,10 @@ private[spark] class SortShuffleReader[K, C]( /** A merge thread to merge on-disk blocks */ private val tieredMerger = new TieredDiskMerger(conf, dep, keyComparator, context) + def memoryBytesSpilled: Long = _memoryBytesSpilled + + def diskBytesSpilled: Long = _diskBytesSpilled + tieredMerger.diskBytesSpilled + override def read(): Iterator[Product2[K, C]] = { tieredMerger.start() @@ -120,45 +123,12 @@ private[spark] class SortShuffleReader[K, C]( val block = MemoryShuffleBlock(blockId, blockData) if (granted < blockSize) { logInfo(s"Granted $granted memory is not enough to store shuffle block id $blockId, " + - s"block size $blockSize, try to consolidate in-memory blocks to release the memory") + s"block size $blockSize, spilling in-memory blocks to release the memory") shuffleMemoryManager.release(granted) - - // Write merged blocks to disk - val (tmpBlockId, file) = blockManager.diskBlockManager.createTempShuffleBlock() - 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) - // TODO. change this into objectWriter - // TODO. Track the memory and disk spill - 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) - - logInfo(s"Merge ${inMemoryBlocks.size} in-memory blocks into file ${file.getName}") - - for (block <- inMemoryBlocks) { - block.blockData.release() - if (block.blockId != blockId) { - shuffleMemoryManager.release(block.blockData.size) - } - } - inMemoryBlocks.clear() + spillInMemoryBlocks(block) } } - assert(unfetchedBytes == 0) tieredMerger.doneRegisteringOnDiskBlocks() @@ -167,6 +137,10 @@ private[spark] class SortShuffleReader[K, C]( val mergedItr = MergeUtil.mergeSort(finalItrGroup, keyComparator, dep.keyOrdering, dep.aggregator) + // Update the spilled info. + context.taskMetrics().memoryBytesSpilled += memoryBytesSpilled + context.taskMetrics().diskBytesSpilled += diskBytesSpilled + // Release the in-memory block when iteration is completed. val completionItr = CompletionIterator[Product2[K, C], Iterator[Product2[K, C]]]( mergedItr, { @@ -180,48 +154,78 @@ private[spark] class SortShuffleReader[K, C]( new InterruptibleIterator(context, completionItr.map(p => (p._1, p._2))) } - def spillInMemoryBlocks(tippingBlock: MemoryShuffleBlock): Unit = { + private def spillInMemoryBlocks(tippingBlock: MemoryShuffleBlock): Unit = { // Write merged blocks to disk val (tmpBlockId, file) = blockManager.diskBlockManager.createTempShuffleBlock() - val fos = new FileOutputStream(file) - val bos = new BufferedOutputStream(fos, fileBufferSize) - - // 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. - 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 - } - if (blocksToSpill.size > 1) { - val itrGroup = inMemoryBlocksToIterators(blocksToSpill) + _memoryBytesSpilled += inMemoryBlocks.map(_.blockData.size()).sum + + if (inMemoryBlocks.size > 1) { + val itrGroup = inMemoryBlocksToIterators(inMemoryBlocks) val partialMergedItr = MergeUtil.mergeSort(itrGroup, keyComparator, dep.keyOrdering, dep.aggregator) - blockManager.dataSerializeStream(tmpBlockId, bos, partialMergedItr, ser) + val curWriteMetrics = new ShuffleWriteMetrics() + var writer = + blockManager.getDiskWriter(tmpBlockId, file, ser, fileBufferSize, curWriteMetrics) + var success = false + + try { + partialMergedItr.foreach(p => writer.write(p)) + success = true + } finally { + if (!success) { + if (writer != null) { + writer.revertPartialWritesAndClose() + writer = null + } + if (file.exists()) { + file.delete() + } + } else { + writer.commitAndClose() + writer = null + } + } + _diskBytesSpilled += curWriteMetrics.shuffleBytesWritten + } else { - val buffer = blocksToSpill.map(_.blockData.nioByteBuffer()).head - val channel = fos.getChannel - while (buffer.hasRemaining) { - channel.write(buffer) + val fos = new FileOutputStream(file) + val buffer = inMemoryBlocks.map(_.blockData.nioByteBuffer()).head + 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() + } } - channel.close() } tieredMerger.registerOnDiskBlock(tmpBlockId, file) - logInfo(s"Merged ${blocksToSpill.size} in-memory blocks into file ${file.getName}") + logInfo(s"Merged ${inMemoryBlocks.size} in-memory blocks into file ${file.getName}") - for (block <- blocksToSpill) { + for (block <- inMemoryBlocks) { block.blockData.release() if (block != tippingBlock) { shuffleMemoryManager.release(block.blockData.size) } } + + inMemoryBlocks.clear() } private def inMemoryBlocksToIterators(blocks: Seq[MemoryShuffleBlock]) @@ -246,7 +250,6 @@ private[spark] class SortShuffleReader[K, C]( } (address, blocks.toSeq) } - unfetchedBytes = blocksByAddress.flatMap(a => a._2.map(b => b._2)).sum shuffleRawBlockFetcherItr = new ShuffleRawBlockFetcherIterator( context, diff --git a/core/src/main/scala/org/apache/spark/util/collection/TieredDiskMerger.scala b/core/src/main/scala/org/apache/spark/util/collection/TieredDiskMerger.scala index 93cb48cadc6f9..392150f748502 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/TieredDiskMerger.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/TieredDiskMerger.scala @@ -24,6 +24,7 @@ import java.util.concurrent.{PriorityBlockingQueue, CountDownLatch} import scala.collection.mutable.ArrayBuffer import org.apache.spark._ +import org.apache.spark.executor.ShuffleWriteMetrics import org.apache.spark.storage.BlockId import org.apache.spark.serializer.Serializer import org.apache.spark.util.CompletionIterator @@ -56,9 +57,13 @@ private[spark] class TieredDiskMerger[K, C]( private val maxMergeFactor = conf.getInt("spark.shuffle.maxMergeFactor", 100) private val fileBufferSize = conf.getInt("spark.shuffle.file.buffer.kb", 32) * 1024 + 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]() @@ -83,6 +88,8 @@ private[spark] class TieredDiskMerger[K, C]( } } + def diskBytesSpilled: Long = _diskBytesSpilled + /** * Notify the merger that no more on disk blocks will be registered. */ @@ -98,7 +105,7 @@ private[spark] class TieredDiskMerger[K, C]( // Merge the final group for combiner to directly feed to the reducer val finalMergedBlocks = onDiskBlocks.toArray(new Array[DiskShuffleBlock](onDiskBlocks.size())) - val finalItrGroup = blocksToRecordIterators(finalMergedBlocks) + val finalItrGroup = onDiskBlocksToIterators(finalMergedBlocks) val mergedItr = MergeUtil.mergeSort(finalItrGroup, keyComparator, dep.keyOrdering, dep.aggregator) @@ -131,7 +138,7 @@ private[spark] class TieredDiskMerger[K, C]( } } - private def blocksToRecordIterators(shufflePartGroup: Seq[DiskShuffleBlock]) + private def onDiskBlocksToIterators(shufflePartGroup: Seq[DiskShuffleBlock]) : Seq[Iterator[Product2[K, C]]] = { shufflePartGroup.map { case DiskShuffleBlock(id, _, _) => blockManager.diskStore.getValues(id, ser).get.asInstanceOf[Iterator[Product2[K, C]]] @@ -182,19 +189,38 @@ private[spark] class TieredDiskMerger[K, C]( } // Merge the blocks - val itrGroup = blocksToRecordIterators(blocksToMerge) + val itrGroup = onDiskBlocksToIterators(blocksToMerge) val partialMergedItr = MergeUtil.mergeSort(itrGroup, keyComparator, dep.keyOrdering, dep.aggregator) // Write merged blocks to disk val (tmpBlockId, file) = blockManager.diskBlockManager.createTempShuffleBlock() - //TODO. change this into objectWriter - // TODO. Track the memory and disk spill - val fos = new BufferedOutputStream(new FileOutputStream(file), fileBufferSize) - blockManager.dataSerializeStream(tmpBlockId, fos, partialMergedItr, ser) + val curWriteMetrics = new ShuffleWriteMetrics() + var writer = + blockManager.getDiskWriter(tmpBlockId, file, ser, fileBufferSize, curWriteMetrics) + var success = false + + try { + partialMergedItr.foreach(p => writer.write(p)) + success = true + } finally { + if (!success) { + if (writer != null) { + writer.revertPartialWritesAndClose() + writer = null + } + if (file.exists()) { + file.delete() + } + } else { + writer.commitAndClose() + writer = null + } + } + _diskBytesSpilled += curWriteMetrics.shuffleBytesWritten + releaseShuffleBlocks(blocksToMerge.toArray) logInfo(s"Merged ${blocksToMerge.size} on-disk blocks into file ${file.getName}") - releaseShuffleBlocks(blocksToMerge.toArray) onDiskBlocks.add(DiskShuffleBlock(tmpBlockId, file, file.length())) } } From c1f97b608fd8ab329f72cd4a8cc39841b5e64b3a Mon Sep 17 00:00:00 2001 From: jerryshao Date: Tue, 11 Nov 2014 13:13:25 +0800 Subject: [PATCH 16/27] Fix incorrect block size introduced bugs --- .../shuffle/sort/SortShuffleReader.scala | 85 ++++++++++++++----- .../util/collection/TieredDiskMerger.scala | 13 +-- 2 files changed, 70 insertions(+), 28 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala index 3f8c50571ac01..0f558b5fa0dec 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala @@ -22,7 +22,7 @@ import java.util.Comparator import org.apache.spark.executor.ShuffleWriteMetrics -import scala.collection.mutable.{ArrayBuffer, HashMap} +import scala.collection.mutable import scala.util.{Failure, Success, Try} import org.apache.spark._ @@ -73,11 +73,19 @@ private[spark] class SortShuffleReader[K, C]( private var _memoryBytesSpilled: Long = 0L private var _diskBytesSpilled: Long = 0L - /** ArrayBuffer to store in-memory shuffle blocks */ - private val inMemoryBlocks = new ArrayBuffer[MemoryShuffleBlock]() + /** Queue to store in-memory shuffle blocks */ + private val inMemoryBlocks = new mutable.Queue[MemoryShuffleBlock]() - /** Manage the BlockManagerId and related shuffle blocks */ - private var statuses: Array[(BlockManagerId, Long)] = _ + /** number of bytes left to fetch */ + private var unfetchedBytes: Long = 0L + + /** + * Maintain the relation between shuffle block and its size. The reason we should maintain this + * is that the request shuffle block size is not equal to the result size because of + * compression of size. So here we should maintain this make sure the correctness of our + * algorithm. + */ + private val shuffleBlockMap = new mutable.HashMap[ShuffleBlockId, (BlockManagerId, Long)]() /** keyComparator for mergeSort, id keyOrdering is not available, * using hashcode of key to compare */ @@ -99,14 +107,16 @@ private[spark] class SortShuffleReader[K, C]( override def read(): Iterator[Product2[K, C]] = { tieredMerger.start() + computeShuffleBlocks() + for ((blockId, blockOption) <- fetchRawBlocks()) { val blockData = blockOption match { case Success(block) => block case Failure(e) => blockId match { - case ShuffleBlockId (shufId, mapId, _) => - val address = statuses(mapId.toInt)._1 - throw new FetchFailedException (address, shufId.toInt, mapId.toInt, startPartition, + 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 ( @@ -114,22 +124,28 @@ private[spark] class SortShuffleReader[K, C]( } } - inMemoryBlocks += MemoryShuffleBlock(blockId, blockData) 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) val block = MemoryShuffleBlock(blockId, blockData) - if (granted < blockSize) { + if (granted >= blockSize) { + inMemoryBlocks += MemoryShuffleBlock(blockId, blockData) + } else { logInfo(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) } + + 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. @@ -158,10 +174,25 @@ private[spark] class SortShuffleReader[K, C]( // Write merged blocks to disk val (tmpBlockId, file) = blockManager.diskBlockManager.createTempShuffleBlock() - _memoryBytesSpilled += inMemoryBlocks.map(_.blockData.size()).sum + // 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. + // We use the request size to calculate the remaining spilled size to make sure the + // correctness, since the request size is slightly different from result block size because + // of size compression. + var bytesToSpill = unfetchedBytes + val blocksToSpill = new mutable.ArrayBuffer[MemoryShuffleBlock]() + blocksToSpill += tippingBlock + bytesToSpill -= shuffleBlockMap(tippingBlock.blockId.asInstanceOf[ShuffleBlockId])._2 + while (bytesToSpill > 0 && !inMemoryBlocks.isEmpty) { + val block = inMemoryBlocks.dequeue() + blocksToSpill += block + bytesToSpill -= shuffleBlockMap(block.blockId.asInstanceOf[ShuffleBlockId])._2 + } + + _memoryBytesSpilled += blocksToSpill.map(_.blockData.size()).sum if (inMemoryBlocks.size > 1) { - val itrGroup = inMemoryBlocksToIterators(inMemoryBlocks) + val itrGroup = inMemoryBlocksToIterators(blocksToSpill) val partialMergedItr = MergeUtil.mergeSort(itrGroup, keyComparator, dep.keyOrdering, dep.aggregator) val curWriteMetrics = new ShuffleWriteMetrics() @@ -190,7 +221,7 @@ private[spark] class SortShuffleReader[K, C]( } else { val fos = new FileOutputStream(file) - val buffer = inMemoryBlocks.map(_.blockData.nioByteBuffer()).head + val buffer = blocksToSpill.map(_.blockData.nioByteBuffer()).head var channel = fos.getChannel var success = false @@ -236,26 +267,36 @@ private[spark] class SortShuffleReader[K, C]( } } - private def fetchRawBlocks(): Iterator[(BlockId, Try[ManagedBuffer])] = { - statuses = SparkEnv.get.mapOutputTracker.getServerStatuses(handle.shuffleId, startPartition) + private def computeShuffleBlocks(): Unit = { + val statuses = SparkEnv.get.mapOutputTracker.getServerStatuses(handle.shuffleId, startPartition) - val splitsByAddress = new HashMap[BlockManagerId, ArrayBuffer[(Int, Long)]]() + val splitsByAddress = new mutable.HashMap[BlockManagerId, mutable.ArrayBuffer[(Int, Long)]]() for (((address, size), index) <- statuses.zipWithIndex) { - splitsByAddress.getOrElseUpdate(address, ArrayBuffer()) += ((index, size)) + splitsByAddress.getOrElseUpdate(address, mutable.ArrayBuffer()) += ((index, size)) } - val blocksByAddress = splitsByAddress.toSeq.map { case (address, splits) => - val blocks = splits.map { s => - (ShuffleBlockId(handle.shuffleId, s._1, startPartition), s._2) + splitsByAddress.foreach { case (id, blocks) => + blocks.foreach { case (idx, len) => + shuffleBlockMap.put(ShuffleBlockId(handle.shuffleId, idx, startPartition), (id, len)) + unfetchedBytes += len } - (address, blocks.toSeq) + } + } + + private def fetchRawBlocks(): Iterator[(BlockId, Try[ManagedBuffer])] = { + val blocksByAddress = new mutable.HashMap[BlockManagerId, + mutable.ArrayBuffer[(ShuffleBlockId, Long)]]() + + shuffleBlockMap.foreach { case (block, (id, len)) => + blocksByAddress.getOrElseUpdate(id, + mutable.ArrayBuffer[(ShuffleBlockId, Long)]()) += ((block, len)) } shuffleRawBlockFetcherItr = new ShuffleRawBlockFetcherIterator( context, SparkEnv.get.blockManager.shuffleClient, blockManager, - blocksByAddress, + blocksByAddress.toSeq, conf.getLong("spark.reducer.maxMbInFlight", 48) * 1024 * 1024) val completionItr = CompletionIterator[ diff --git a/core/src/main/scala/org/apache/spark/util/collection/TieredDiskMerger.scala b/core/src/main/scala/org/apache/spark/util/collection/TieredDiskMerger.scala index 392150f748502..b3fb56016ace4 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/TieredDiskMerger.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/TieredDiskMerger.scala @@ -17,11 +17,11 @@ package org.apache.spark.util.collection -import java.io.{File, FileOutputStream, BufferedOutputStream} +import java.io.File import java.util.Comparator import java.util.concurrent.{PriorityBlockingQueue, CountDownLatch} -import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable import org.apache.spark._ import org.apache.spark.executor.ShuffleWriteMetrics @@ -159,10 +159,11 @@ private[spark] class TieredDiskMerger[K, C]( * either receive the notification that no more blocks are coming in, or until maxMergeFactor * merge is required no matter what. * - * E.g. if maxMergeFactor is 10 and we have 19 or more on-disk blocks, a 10-block merge will put us - * at 10 or more blocks, so we might as well carry it out now. + * E.g. if maxMergeFactor is 10 and we have 19 or more on-disk blocks, a 10-block merge will put + * us at 10 or more blocks, so we might as well carry it out now. */ - private def shouldMergeNow(): Boolean = doneRegistering || onDiskBlocks.size() >= maxMergeFactor * 2 - 1 + private def shouldMergeNow(): Boolean = doneRegistering || + onDiskBlocks.size() >= maxMergeFactor * 2 - 1 private final class DiskToDiskMerger extends Thread { setName(s"tiered-merge-thread-${Thread.currentThread().getId}") @@ -180,7 +181,7 @@ private[spark] class TieredDiskMerger[K, C]( } if (onDiskBlocks.size() > maxMergeFactor) { - val blocksToMerge = new ArrayBuffer[DiskShuffleBlock]() + val blocksToMerge = new mutable.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) From b5e472d0a17f9c1a0ea96bae1eea6f7c8e80b18a Mon Sep 17 00:00:00 2001 From: jerryshao Date: Wed, 12 Nov 2014 16:23:47 +0800 Subject: [PATCH 17/27] Address the comments --- .../shuffle/sort/SortShuffleReader.scala | 29 ++++++++++--------- 1 file changed, 16 insertions(+), 13 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala index 0f558b5fa0dec..44dc976621f2f 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala @@ -22,7 +22,7 @@ import java.util.Comparator import org.apache.spark.executor.ShuffleWriteMetrics -import scala.collection.mutable +import scala.collection.mutable.{ArrayBuffer, HashMap, Queue} import scala.util.{Failure, Success, Try} import org.apache.spark._ @@ -74,7 +74,7 @@ private[spark] class SortShuffleReader[K, C]( private var _diskBytesSpilled: Long = 0L /** Queue to store in-memory shuffle blocks */ - private val inMemoryBlocks = new mutable.Queue[MemoryShuffleBlock]() + private val inMemoryBlocks = new Queue[MemoryShuffleBlock]() /** number of bytes left to fetch */ private var unfetchedBytes: Long = 0L @@ -85,7 +85,7 @@ private[spark] class SortShuffleReader[K, C]( * compression of size. So here we should maintain this make sure the correctness of our * algorithm. */ - private val shuffleBlockMap = new mutable.HashMap[ShuffleBlockId, (BlockManagerId, Long)]() + private val shuffleBlockMap = new HashMap[ShuffleBlockId, (BlockManagerId, Long)]() /** keyComparator for mergeSort, id keyOrdering is not available, * using hashcode of key to compare */ @@ -109,9 +109,9 @@ private[spark] class SortShuffleReader[K, C]( computeShuffleBlocks() - for ((blockId, blockOption) <- fetchRawBlocks()) { + for ((blockId, blockOption) <- fetchShuffleBlocks()) { val blockData = blockOption match { - case Success(block) => block + case Success(b) => b case Failure(e) => blockId match { case b @ ShuffleBlockId(shuffleId, mapId, _) => @@ -180,7 +180,7 @@ private[spark] class SortShuffleReader[K, C]( // correctness, since the request size is slightly different from result block size because // of size compression. var bytesToSpill = unfetchedBytes - val blocksToSpill = new mutable.ArrayBuffer[MemoryShuffleBlock]() + val blocksToSpill = new ArrayBuffer[MemoryShuffleBlock]() blocksToSpill += tippingBlock bytesToSpill -= shuffleBlockMap(tippingBlock.blockId.asInstanceOf[ShuffleBlockId])._2 while (bytesToSpill > 0 && !inMemoryBlocks.isEmpty) { @@ -247,7 +247,7 @@ private[spark] class SortShuffleReader[K, C]( tieredMerger.registerOnDiskBlock(tmpBlockId, file) - logInfo(s"Merged ${inMemoryBlocks.size} in-memory blocks into file ${file.getName}") + logInfo(s"Merged ${blocksToSpill.size} in-memory blocks into file ${file.getName}") for (block <- inMemoryBlocks) { block.blockData.release() @@ -267,12 +267,16 @@ private[spark] class SortShuffleReader[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 mutable.HashMap[BlockManagerId, mutable.ArrayBuffer[(Int, Long)]]() + val splitsByAddress = new HashMap[BlockManagerId, ArrayBuffer[(Int, Long)]]() for (((address, size), index) <- statuses.zipWithIndex) { - splitsByAddress.getOrElseUpdate(address, mutable.ArrayBuffer()) += ((index, size)) + splitsByAddress.getOrElseUpdate(address, ArrayBuffer()) += ((index, size)) } splitsByAddress.foreach { case (id, blocks) => @@ -283,13 +287,12 @@ private[spark] class SortShuffleReader[K, C]( } } - private def fetchRawBlocks(): Iterator[(BlockId, Try[ManagedBuffer])] = { - val blocksByAddress = new mutable.HashMap[BlockManagerId, - mutable.ArrayBuffer[(ShuffleBlockId, Long)]]() + private def fetchShuffleBlocks(): Iterator[(BlockId, Try[ManagedBuffer])] = { + val blocksByAddress = new HashMap[BlockManagerId, ArrayBuffer[(ShuffleBlockId, Long)]]() shuffleBlockMap.foreach { case (block, (id, len)) => blocksByAddress.getOrElseUpdate(id, - mutable.ArrayBuffer[(ShuffleBlockId, Long)]()) += ((block, len)) + ArrayBuffer[(ShuffleBlockId, Long)]()) += ((block, len)) } shuffleRawBlockFetcherItr = new ShuffleRawBlockFetcherIterator( From 40c59df313480df12c35f4851269925b47bae0be Mon Sep 17 00:00:00 2001 From: jerryshao Date: Wed, 12 Nov 2014 21:17:04 +0800 Subject: [PATCH 18/27] Fix some bugs --- .../org/apache/spark/shuffle/sort/SortShuffleReader.scala | 6 ++---- .../org/apache/spark/util/collection/TieredDiskMerger.scala | 4 ++-- 2 files changed, 4 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala index 44dc976621f2f..9510bc4babf9f 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala @@ -191,7 +191,7 @@ private[spark] class SortShuffleReader[K, C]( _memoryBytesSpilled += blocksToSpill.map(_.blockData.size()).sum - if (inMemoryBlocks.size > 1) { + if (blocksToSpill.size > 1) { val itrGroup = inMemoryBlocksToIterators(blocksToSpill) val partialMergedItr = MergeUtil.mergeSort(itrGroup, keyComparator, dep.keyOrdering, dep.aggregator) @@ -249,14 +249,12 @@ private[spark] class SortShuffleReader[K, C]( logInfo(s"Merged ${blocksToSpill.size} in-memory blocks into file ${file.getName}") - for (block <- inMemoryBlocks) { + for (block <- blocksToSpill) { block.blockData.release() if (block != tippingBlock) { shuffleMemoryManager.release(block.blockData.size) } } - - inMemoryBlocks.clear() } private def inMemoryBlocksToIterators(blocks: Seq[MemoryShuffleBlock]) diff --git a/core/src/main/scala/org/apache/spark/util/collection/TieredDiskMerger.scala b/core/src/main/scala/org/apache/spark/util/collection/TieredDiskMerger.scala index b3fb56016ace4..239b306eeca08 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/TieredDiskMerger.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/TieredDiskMerger.scala @@ -21,7 +21,7 @@ import java.io.File import java.util.Comparator import java.util.concurrent.{PriorityBlockingQueue, CountDownLatch} -import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer import org.apache.spark._ import org.apache.spark.executor.ShuffleWriteMetrics @@ -181,7 +181,7 @@ private[spark] class TieredDiskMerger[K, C]( } if (onDiskBlocks.size() > maxMergeFactor) { - val blocksToMerge = new mutable.ArrayBuffer[DiskShuffleBlock]() + 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) From 42bf77df5aa92463c3d7b13549849c5c69edf1a4 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Fri, 14 Nov 2014 22:11:16 +0800 Subject: [PATCH 19/27] Improve the failure process and expand ManagedBuffer --- .../shuffle/sort/SortShuffleReader.scala | 58 +++++++++++-------- .../util/collection/TieredDiskMerger.scala | 2 +- .../buffer/FileSegmentManagedBuffer.java | 5 ++ .../spark/network/buffer/ManagedBuffer.java | 6 ++ .../network/buffer/NettyManagedBuffer.java | 5 ++ .../network/buffer/NioManagedBuffer.java | 5 ++ 6 files changed, 55 insertions(+), 26 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala index 9510bc4babf9f..677a68f6900be 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala @@ -58,9 +58,6 @@ private[spark] class SortShuffleReader[K, C]( require(endPartition == startPartition + 1, "Sort shuffle currently only supports fetching one partition") - /** Shuffle block fetcher iterator */ - private var shuffleRawBlockFetcherItr: ShuffleRawBlockFetcherIterator = _ - private val dep = handle.dependency private val conf = SparkEnv.get.conf private val blockManager = SparkEnv.get.blockManager @@ -69,16 +66,9 @@ private[spark] class SortShuffleReader[K, C]( private val fileBufferSize = conf.getInt("spark.shuffle.file.buffer.kb", 32) * 1024 - /** Number of bytes spilled in memory and on disk */ - private var _memoryBytesSpilled: Long = 0L - private var _diskBytesSpilled: Long = 0L - /** Queue to store in-memory shuffle blocks */ private val inMemoryBlocks = new Queue[MemoryShuffleBlock]() - /** number of bytes left to fetch */ - private var unfetchedBytes: Long = 0L - /** * Maintain the relation between shuffle block and its size. The reason we should maintain this * is that the request shuffle block size is not equal to the result size because of @@ -100,6 +90,16 @@ private[spark] class SortShuffleReader[K, C]( /** 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 @@ -153,19 +153,23 @@ private[spark] class SortShuffleReader[K, C]( val mergedItr = MergeUtil.mergeSort(finalItrGroup, keyComparator, dep.keyOrdering, dep.aggregator) - // Update the spilled info. + // Update the spilled info and do cleanup work when task is finished. context.taskMetrics().memoryBytesSpilled += memoryBytesSpilled context.taskMetrics().diskBytesSpilled += 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, { - inMemoryBlocks.foreach { block => - block.blockData.release() - shuffleMemoryManager.release(block.blockData.size) - } - inMemoryBlocks.clear() - }) + mergedItr, releaseFinalShuffleMemory()) new InterruptibleIterator(context, completionItr.map(p => (p._1, p._2))) } @@ -174,6 +178,15 @@ private[spark] class SortShuffleReader[K, C]( // Write merged blocks to disk val (tmpBlockId, file) = blockManager.diskBlockManager.createTempShuffleBlock() + def releaseTempShuffleMemory(blocks: ArrayBuffer[MemoryShuffleBlock]): Unit = { + for (block <- blocks) { + block.blockData.release() + if (block != tippingBlock) { + shuffleMemoryManager.release(block.blockData.size) + } + } + } + // 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. // We use the request size to calculate the remaining spilled size to make sure the @@ -216,6 +229,7 @@ private[spark] class SortShuffleReader[K, C]( writer.commitAndClose() writer = null } + releaseTempShuffleMemory(blocksToSpill) } _diskBytesSpilled += curWriteMetrics.shuffleBytesWritten @@ -242,19 +256,13 @@ private[spark] class SortShuffleReader[K, C]( } else { _diskBytesSpilled = file.length() } + releaseTempShuffleMemory(blocksToSpill) } } tieredMerger.registerOnDiskBlock(tmpBlockId, file) logInfo(s"Merged ${blocksToSpill.size} in-memory blocks into file ${file.getName}") - - for (block <- blocksToSpill) { - block.blockData.release() - if (block != tippingBlock) { - shuffleMemoryManager.release(block.blockData.size) - } - } } private def inMemoryBlocksToIterators(blocks: Seq[MemoryShuffleBlock]) diff --git a/core/src/main/scala/org/apache/spark/util/collection/TieredDiskMerger.scala b/core/src/main/scala/org/apache/spark/util/collection/TieredDiskMerger.scala index 239b306eeca08..e25925b1b77c6 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/TieredDiskMerger.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/TieredDiskMerger.scala @@ -216,9 +216,9 @@ private[spark] class TieredDiskMerger[K, C]( writer.commitAndClose() writer = null } + releaseShuffleBlocks(blocksToMerge.toArray) } _diskBytesSpilled += curWriteMetrics.shuffleBytesWritten - releaseShuffleBlocks(blocksToMerge.toArray) logInfo(s"Merged ${blocksToMerge.size} on-disk blocks into file ${file.getName}") diff --git a/network/common/src/main/java/org/apache/spark/network/buffer/FileSegmentManagedBuffer.java b/network/common/src/main/java/org/apache/spark/network/buffer/FileSegmentManagedBuffer.java index 844eff4f4c701..08d02896e8ca9 100644 --- a/network/common/src/main/java/org/apache/spark/network/buffer/FileSegmentManagedBuffer.java +++ b/network/common/src/main/java/org/apache/spark/network/buffer/FileSegmentManagedBuffer.java @@ -151,4 +151,9 @@ public String toString() { .add("length", length) .toString(); } + + @Override + public boolean isDirect() { + return length >= MIN_MEMORY_MAP_BYTES; + } } diff --git a/network/common/src/main/java/org/apache/spark/network/buffer/ManagedBuffer.java b/network/common/src/main/java/org/apache/spark/network/buffer/ManagedBuffer.java index a415db593a788..41f86d4e2a711 100644 --- a/network/common/src/main/java/org/apache/spark/network/buffer/ManagedBuffer.java +++ b/network/common/src/main/java/org/apache/spark/network/buffer/ManagedBuffer.java @@ -68,4 +68,10 @@ public abstract class ManagedBuffer { * Convert the buffer into an Netty object, used to write the data out. */ public abstract Object convertToNetty() throws IOException; + + /** + * Tell whether to not this byte buffer is direct + * @return + */ + public abstract boolean isDirect(); } diff --git a/network/common/src/main/java/org/apache/spark/network/buffer/NettyManagedBuffer.java b/network/common/src/main/java/org/apache/spark/network/buffer/NettyManagedBuffer.java index c806bfa45bef3..909942c64f93b 100644 --- a/network/common/src/main/java/org/apache/spark/network/buffer/NettyManagedBuffer.java +++ b/network/common/src/main/java/org/apache/spark/network/buffer/NettyManagedBuffer.java @@ -73,4 +73,9 @@ public String toString() { .add("buf", buf) .toString(); } + + @Override + public boolean isDirect() { + return buf.isDirect(); + } } diff --git a/network/common/src/main/java/org/apache/spark/network/buffer/NioManagedBuffer.java b/network/common/src/main/java/org/apache/spark/network/buffer/NioManagedBuffer.java index f55b884bc45ce..4727abe0e4157 100644 --- a/network/common/src/main/java/org/apache/spark/network/buffer/NioManagedBuffer.java +++ b/network/common/src/main/java/org/apache/spark/network/buffer/NioManagedBuffer.java @@ -71,5 +71,10 @@ public String toString() { .add("buf", buf) .toString(); } + + @Override + public boolean isDirect() { + return buf.isDirect(); + } } From a9eaef8a695d85b4a80f2c4b3835d60352fab271 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Mon, 17 Nov 2014 13:07:32 +0800 Subject: [PATCH 20/27] Copy the memory from off-heap to on-heap and some code style modification --- .../shuffle/sort/SortShuffleReader.scala | 26 ++++++++++++------- .../util/collection/TieredDiskMerger.scala | 17 ++++++------ .../spark/network/buffer/ManagedBuffer.java | 8 +++--- .../spark/network/TestManagedBuffer.java | 5 ++++ 4 files changed, 35 insertions(+), 21 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala index 677a68f6900be..000a1b0c3664c 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala @@ -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._ @@ -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 @@ -164,7 +173,6 @@ private[spark] class SortShuffleReader[K, C]( } inMemoryBlocks.clear() } - context.addTaskCompletionListener(_ => releaseFinalShuffleMemory()) // Release the in-memory block when iteration is completed. @@ -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) { diff --git a/core/src/main/scala/org/apache/spark/util/collection/TieredDiskMerger.scala b/core/src/main/scala/org/apache/spark/util/collection/TieredDiskMerger.scala index e25925b1b77c6..98954c45a69d3 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/TieredDiskMerger.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/TieredDiskMerger.scala @@ -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]() @@ -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())) @@ -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. */ @@ -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 @@ -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) { diff --git a/network/common/src/main/java/org/apache/spark/network/buffer/ManagedBuffer.java b/network/common/src/main/java/org/apache/spark/network/buffer/ManagedBuffer.java index 41f86d4e2a711..4bbc852e5c446 100644 --- a/network/common/src/main/java/org/apache/spark/network/buffer/ManagedBuffer.java +++ b/network/common/src/main/java/org/apache/spark/network/buffer/ManagedBuffer.java @@ -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(); } diff --git a/network/common/src/test/java/org/apache/spark/network/TestManagedBuffer.java b/network/common/src/test/java/org/apache/spark/network/TestManagedBuffer.java index 38113a918f795..02a994a04a2a9 100644 --- a/network/common/src/test/java/org/apache/spark/network/TestManagedBuffer.java +++ b/network/common/src/test/java/org/apache/spark/network/TestManagedBuffer.java @@ -101,4 +101,9 @@ public boolean equals(Object other) { } return false; } + + @Override + public boolean isDirect() { + return underlying.isDirect(); + } } From 6f48c5c522aaeb4a3789f6e0527c7697858d06ea Mon Sep 17 00:00:00 2001 From: jerryshao Date: Tue, 18 Nov 2014 15:19:30 +0800 Subject: [PATCH 21/27] Fix rebase introduced issue --- .../apache/spark/network/buffer/FileSegmentManagedBuffer.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/network/common/src/main/java/org/apache/spark/network/buffer/FileSegmentManagedBuffer.java b/network/common/src/main/java/org/apache/spark/network/buffer/FileSegmentManagedBuffer.java index 08d02896e8ca9..aab8d0fa73671 100644 --- a/network/common/src/main/java/org/apache/spark/network/buffer/FileSegmentManagedBuffer.java +++ b/network/common/src/main/java/org/apache/spark/network/buffer/FileSegmentManagedBuffer.java @@ -154,6 +154,6 @@ public String toString() { @Override public boolean isDirect() { - return length >= MIN_MEMORY_MAP_BYTES; + return length >= conf.memoryMapBytes(); } } From c2ddcce152a1d15f9cb2676887f9abd56506c7e1 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Tue, 18 Nov 2014 15:27:10 +0800 Subject: [PATCH 22/27] Revert some unwanted changes --- .../scala/org/apache/spark/shuffle/ShuffleMemoryManager.scala | 2 +- .../apache/spark/shuffle/hash/BlockStoreShuffleFetcher.scala | 2 +- .../org/apache/spark/storage/ShuffleBlockFetcherIterator.scala | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleMemoryManager.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleMemoryManager.scala index 5e0a6f5eb6e14..3bcc7178a3d8b 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleMemoryManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleMemoryManager.scala @@ -93,7 +93,7 @@ private[spark] class ShuffleMemoryManager(maxMemory: Long) extends Logging { } /** Release numBytes bytes for the current thread. */ - def release(numBytes: Long): Unit = synchronized { + def release(numBytes: Long): Unit = synchronized { val threadId = Thread.currentThread().getId val curMem = threadMemory.getOrElse(threadId, 0L) if (curMem < numBytes) { diff --git a/core/src/main/scala/org/apache/spark/shuffle/hash/BlockStoreShuffleFetcher.scala b/core/src/main/scala/org/apache/spark/shuffle/hash/BlockStoreShuffleFetcher.scala index b3b0471732be3..7a2c5ae32d98b 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/hash/BlockStoreShuffleFetcher.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/hash/BlockStoreShuffleFetcher.scala @@ -27,7 +27,7 @@ import org.apache.spark.shuffle.FetchFailedException import org.apache.spark.storage.{BlockId, BlockManagerId, ShuffleBlockFetcherIterator, ShuffleBlockId} import org.apache.spark.util.CompletionIterator -private[shuffle] object BlockStoreShuffleFetcher extends Logging { +private[hash] object BlockStoreShuffleFetcher extends Logging { def fetch[T]( shuffleId: Int, reduceId: Int, diff --git a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala index 1d87d15d5b8bf..12a01429e565d 100644 --- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala @@ -126,7 +126,7 @@ final class ShuffleRawBlockFetcherIterator( * Current [[FetchResult]] being processed. We track this so we can release the current buffer * in case of a runtime exception when processing the current buffer. */ - private[spark] var currentResult: FetchResult = null + @volatile private[spark] var currentResult: FetchResult = null /** * Queue of fetch requests to issue; we'll pull requests off this gradually to make sure that From f170db35e0cd9959be79bfbe3092b09d80758959 Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Mon, 24 Nov 2014 00:23:03 -0800 Subject: [PATCH 23/27] Clean up comments, break up large methods, spill based on actual block size, and properly increment _diskBytesSpilled --- .../shuffle/sort/MixedShuffleReader.scala | 4 + .../shuffle/sort/SortShuffleReader.scala | 159 ++++++++++-------- .../util/collection/TieredDiskMerger.scala | 3 +- 3 files changed, 90 insertions(+), 76 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/MixedShuffleReader.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/MixedShuffleReader.scala index 2fe3cc0f31b4d..1a98c4e1f7f20 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/MixedShuffleReader.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/MixedShuffleReader.scala @@ -21,6 +21,10 @@ 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, diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala index 000a1b0c3664c..c4a6ffcb67116 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala @@ -17,6 +17,7 @@ package org.apache.spark.shuffle.sort +import java.io.File import java.io.FileOutputStream import java.nio.ByteBuffer import java.util.Comparator @@ -38,7 +39,7 @@ import org.apache.spark.util.collection.{MergeUtil, TieredDiskMerger} * 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 the in-memory blocks to disk and go back to + * 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 @@ -70,10 +71,10 @@ private[spark] class SortShuffleReader[K, C]( private val inMemoryBlocks = new Queue[MemoryShuffleBlock]() /** - * Maintain the relation between shuffle block and its size. The reason we should maintain this - * is that the request shuffle block size is not equal to the result size because of - * compression of size. So here we should maintain this make sure the correctness of our - * algorithm. + * 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)]() @@ -97,7 +98,7 @@ private[spark] class SortShuffleReader[K, C]( private var _memoryBytesSpilled: Long = 0L private var _diskBytesSpilled: Long = 0L - /** number of bytes left to fetch */ + /** Number of bytes left to fetch */ private var unfetchedBytes: Long = 0L def memoryBytesSpilled: Long = _memoryBytesSpilled @@ -131,7 +132,7 @@ private[spark] class SortShuffleReader[K, C]( val granted = shuffleMemoryManager.tryToAcquire(blockSize) if (granted >= blockSize) { if (blockData.isDirect) { - // If the memory shuffle block is allocated on direct buffer, copy it on heap, + // If the shuffle block is allocated on a direct buffer, copy it to an on-heap buffer, // otherwise off heap memory will be increased out of control. val onHeapBuffer = ByteBuffer.allocate(blockSize.toInt) onHeapBuffer.put(blockData.nioByteBuffer) @@ -142,7 +143,7 @@ private[spark] class SortShuffleReader[K, C]( inMemoryBlocks += MemoryShuffleBlock(blockId, blockData) } } else { - logDebug(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 $blockId, " + s"block size $blockSize, spilling in-memory blocks to release the memory") shuffleMemoryManager.release(granted) @@ -162,7 +163,7 @@ private[spark] class SortShuffleReader[K, C]( val mergedItr = MergeUtil.mergeSort(finalItrGroup, keyComparator, dep.keyOrdering, dep.aggregator) - // Update the spilled info and do cleanup work when task is finished. + // Update the spill metrics and do cleanup work when task is finished. context.taskMetrics().memoryBytesSpilled += memoryBytesSpilled context.taskMetrics().diskBytesSpilled += diskBytesSpilled @@ -182,95 +183,105 @@ private[spark] class SortShuffleReader[K, C]( 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 = { - // Write merged blocks to disk val (tmpBlockId, file) = blockManager.diskBlockManager.createTempShuffleBlock() - def releaseTempShuffleMemory(blocks: ArrayBuffer[MemoryShuffleBlock]): Unit = { - for (block <- blocks) { - block.blockData.release() - if (block != tippingBlock) { - shuffleMemoryManager.release(block.blockData.size) - } - } - } - // 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. - // We use the request size to calculate the remaining spilled size to make sure the - // correctness, since the request size is slightly different from result block size because - // of size compression. + // 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 -= shuffleBlockMap(tippingBlock.blockId.asInstanceOf[ShuffleBlockId])._2 + bytesToSpill -= tippingBlock.blockData.size while (bytesToSpill > 0 && !inMemoryBlocks.isEmpty) { val block = inMemoryBlocks.dequeue() blocksToSpill += block - bytesToSpill -= shuffleBlockMap(block.blockId.asInstanceOf[ShuffleBlockId])._2 + bytesToSpill -= block.blockData.size } _memoryBytesSpilled += blocksToSpill.map(_.blockData.size()).sum if (blocksToSpill.size > 1) { - 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) - 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 + 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() } - releaseTempShuffleMemory(blocksToSpill) + } else { + _diskBytesSpilled += file.length() } - _diskBytesSpilled += curWriteMetrics.shuffleBytesWritten + // 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() + } + } - } else { - val fos = new FileOutputStream(file) - val buffer = blocksToSpill.map(_.blockData.nioByteBuffer()).head - var channel = fos.getChannel - var success = false - - try { - while (buffer.hasRemaining) { - channel.write(buffer) + /** + * 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) + var success = false + + try { + partialMergedItr.foreach(writer.write) + success = true + } finally { + if (!success) { + if (writer != null) { + writer.revertPartialWritesAndClose() + writer = null } - success = true - } finally { - if (channel != null) { - channel.close() - channel = null + if (file.exists()) { + file.delete() } - if (!success) { - if (file.exists()) { - file.delete() - } - } else { - _diskBytesSpilled = file.length() + } else { + writer.commitAndClose() + writer = null + } + for (block <- blocksToSpill) { + block.blockData.release() + if (block != tippingBlock) { + shuffleMemoryManager.release(block.blockData.size) } - releaseTempShuffleMemory(blocksToSpill) } } - - tieredMerger.registerOnDiskBlock(tmpBlockId, file) - - logInfo(s"Merged ${blocksToSpill.size} in-memory blocks into file ${file.getName}") + _diskBytesSpilled += curWriteMetrics.shuffleBytesWritten } private def inMemoryBlocksToIterators(blocks: Seq[MemoryShuffleBlock]) diff --git a/core/src/main/scala/org/apache/spark/util/collection/TieredDiskMerger.scala b/core/src/main/scala/org/apache/spark/util/collection/TieredDiskMerger.scala index 98954c45a69d3..8628c3c16e0a4 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/TieredDiskMerger.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/TieredDiskMerger.scala @@ -30,8 +30,6 @@ import org.apache.spark.serializer.Serializer import org.apache.spark.util.CompletionIterator /** - * Explain the boundaries of where this starts and why we have second thread - * * Manages blocks of sorted data on disk that need to be merged together. Carries out a tiered * merge that will never merge more than spark.shuffle.maxMergeFactor segments at a time. Except for * the final merge, which merges disk blocks to a returned iterator, TieredDiskMerger merges blocks @@ -72,6 +70,7 @@ private[spark] class TieredDiskMerger[K, C]( private val mergeFinished = new CountDownLatch(1) + /** Whether more on-disk blocks may come in */ @volatile private var doneRegistering = false /** Number of bytes spilled on disk */ From 123aea18e09a025ba5aad19100743580b96dbc02 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Tue, 25 Nov 2014 09:15:58 +0800 Subject: [PATCH 24/27] Log improve --- .../org/apache/spark/shuffle/sort/SortShuffleReader.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala index c4a6ffcb67116..64a9979662c6d 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala @@ -133,7 +133,7 @@ private[spark] class SortShuffleReader[K, C]( 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 out of control. + // otherwise off heap memory will be increased to the shuffle memory size. val onHeapBuffer = ByteBuffer.allocate(blockSize.toInt) onHeapBuffer.put(blockData.nioByteBuffer) @@ -143,8 +143,8 @@ private[spark] class SortShuffleReader[K, C]( inMemoryBlocks += MemoryShuffleBlock(blockId, blockData) } } else { - logDebug(s"Granted $granted memory is not enough to store shuffle block $blockId, " + - s"block size $blockSize, spilling in-memory blocks to release the memory") + 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)) From e0351057681b132c0b898a1a6e79f738906b8fe3 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Tue, 25 Nov 2014 10:07:55 +0800 Subject: [PATCH 25/27] Fix scala style issue --- .../org/apache/spark/shuffle/sort/SortShuffleReader.scala | 2 +- .../org/apache/spark/util/collection/TieredDiskMerger.scala | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala index 64a9979662c6d..87c7a67cd6c1c 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala @@ -53,7 +53,7 @@ private[spark] class SortShuffleReader[K, C]( context: TaskContext) extends ShuffleReader[K, C] with Logging { - /** Manage the fetched in-memory shuffle block and related buffer*/ + /** Manage the fetched in-memory shuffle block and related buffer */ case class MemoryShuffleBlock(blockId: BlockId, blockData: ManagedBuffer) require(endPartition == startPartition + 1, diff --git a/core/src/main/scala/org/apache/spark/util/collection/TieredDiskMerger.scala b/core/src/main/scala/org/apache/spark/util/collection/TieredDiskMerger.scala index 8628c3c16e0a4..1b33042abb92c 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/TieredDiskMerger.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/TieredDiskMerger.scala @@ -31,9 +31,9 @@ import org.apache.spark.util.CompletionIterator /** * Manages blocks of sorted data on disk that need to be merged together. Carries out a tiered - * merge that will never merge more than spark.shuffle.maxMergeFactor segments at a time. Except for - * the final merge, which merges disk blocks to a returned iterator, TieredDiskMerger merges blocks - * from disk to disk. + * merge that will never merge more than spark.shuffle.maxMergeFactor segments at a time. + * Except for the final merge, which merges disk blocks to a returned iterator, TieredDiskMerger + * merges blocks from disk to disk. * * TieredDiskMerger carries out disk-to-disk merges in a background thread that can run concurrently * with blocks being deposited on disk. From 8b7370125395950942257764c4ae792c6c20cd6a Mon Sep 17 00:00:00 2001 From: Saisai Shao Date: Sat, 21 Feb 2015 22:07:45 -0800 Subject: [PATCH 26/27] Fix rebase issues --- .../org/apache/spark/shuffle/sort/SortShuffleReader.scala | 4 ++-- core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala | 3 +-- 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala index 87c7a67cd6c1c..53d48536fe050 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleReader.scala @@ -164,8 +164,8 @@ private[spark] class SortShuffleReader[K, C]( MergeUtil.mergeSort(finalItrGroup, keyComparator, dep.keyOrdering, dep.aggregator) // Update the spill metrics and do cleanup work when task is finished. - context.taskMetrics().memoryBytesSpilled += memoryBytesSpilled - context.taskMetrics().diskBytesSpilled += diskBytesSpilled + context.taskMetrics().incMemoryBytesSpilled(memoryBytesSpilled) + context.taskMetrics().incDiskBytesSpilled(diskBytesSpilled) def releaseFinalShuffleMemory(): Unit = { inMemoryBlocks.foreach { block => diff --git a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala index 17e26724ae720..a675c5b9008f8 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -23,9 +23,8 @@ import scala.collection.mutable.{ArrayBuffer, HashMap} import scala.collection.JavaConverters._ import scala.reflect.ClassTag -import org.scalatest.FunSuite - import com.esotericsoftware.kryo.KryoException +import org.scalatest.FunSuite import org.apache.spark._ import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} From d6c94da3e67b01855d4dc5d42c9068c77cba7453 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Mon, 13 Apr 2015 12:52:58 +0800 Subject: [PATCH 27/27] Fix dead lock --- .../org/apache/spark/util/collection/TieredDiskMerger.scala | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/collection/TieredDiskMerger.scala b/core/src/main/scala/org/apache/spark/util/collection/TieredDiskMerger.scala index 1b33042abb92c..d55413a8d31be 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/TieredDiskMerger.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/TieredDiskMerger.scala @@ -93,8 +93,8 @@ private[spark] class TieredDiskMerger[K, C]( * Notify the merger that no more on disk blocks will be registered. */ def doneRegisteringOnDiskBlocks(): Unit = { - doneRegistering = true mergeReadyMonitor.synchronized { + doneRegistering = true mergeReadyMonitor.notify() } } @@ -175,7 +175,9 @@ private[spark] class TieredDiskMerger[K, C]( while (!doneRegistering || onDiskBlocks.size() > maxMergeFactor) { while (!shouldMergeNow()) { mergeReadyMonitor.synchronized { - mergeReadyMonitor.wait() + if (!shouldMergeNow()) { + mergeReadyMonitor.wait() + } } }