From 3d51b00b5724aa94bf5ed7967f881de5988c5b52 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Thu, 3 Mar 2016 17:07:59 -0800 Subject: [PATCH 01/42] Delete the BlockStore interface. --- .../apache/spark/storage/BlockManager.scala | 1 - .../org/apache/spark/storage/BlockStore.scala | 63 ------------------- .../org/apache/spark/storage/DiskStore.scala | 16 ++--- .../apache/spark/storage/MemoryStore.scala | 20 +++--- 4 files changed, 18 insertions(+), 82 deletions(-) delete mode 100644 core/src/main/scala/org/apache/spark/storage/BlockStore.scala diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index b38e2ec57fe32..5fced4c9a2f1e 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -1229,7 +1229,6 @@ private[spark] class BlockManager( rpcEnv.stop(slaveEndpoint) blockInfoManager.clear() memoryStore.clear() - diskStore.clear() futureExecutionContext.shutdownNow() logInfo("BlockManager stopped") } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockStore.scala b/core/src/main/scala/org/apache/spark/storage/BlockStore.scala deleted file mode 100644 index b069918b16106..0000000000000 --- a/core/src/main/scala/org/apache/spark/storage/BlockStore.scala +++ /dev/null @@ -1,63 +0,0 @@ -/* - * 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.storage - -import java.nio.ByteBuffer - -import org.apache.spark.Logging - -/** - * Abstract class to store blocks. - */ -private[spark] abstract class BlockStore(val blockManager: BlockManager) extends Logging { - - def putBytes(blockId: BlockId, bytes: ByteBuffer, level: StorageLevel): Unit - - /** - * Attempt to store an iterator of values. - * - * @return an iterator of values (in case the put failed), or the estimated size of the stored - * values if the put succeeded. - */ - def putIterator( - blockId: BlockId, - values: Iterator[Any], - level: StorageLevel): Either[Iterator[Any], Long] - - /** - * Return the size of a block in bytes. - */ - def getSize(blockId: BlockId): Long - - def getBytes(blockId: BlockId): Option[ByteBuffer] - - def getValues(blockId: BlockId): Option[Iterator[Any]] - - /** - * Remove a block, if it exists. - * - * @param blockId the block to remove. - * @return True if the block was found and removed, False otherwise. - * @throws IllegalStateException if the block is pinned by a task. - */ - def remove(blockId: BlockId): Boolean - - def contains(blockId: BlockId): Boolean - - def clear() { } -} diff --git a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala index e35aa1b0684da..e09325d5b5afa 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala @@ -28,15 +28,15 @@ import org.apache.spark.util.Utils * Stores BlockManager blocks on disk. */ private[spark] class DiskStore(blockManager: BlockManager, diskManager: DiskBlockManager) - extends BlockStore(blockManager) with Logging { + extends Logging { val minMemoryMapBytes = blockManager.conf.getSizeAsBytes("spark.storage.memoryMapThreshold", "2m") - override def getSize(blockId: BlockId): Long = { + def getSize(blockId: BlockId): Long = { diskManager.getFile(blockId.name).length } - override def putBytes(blockId: BlockId, _bytes: ByteBuffer, level: StorageLevel): Unit = { + def putBytes(blockId: BlockId, _bytes: ByteBuffer, level: StorageLevel): Unit = { // So that we do not modify the input offsets ! // duplicate does not copy buffer, so inexpensive val bytes = _bytes.duplicate() @@ -56,7 +56,7 @@ private[spark] class DiskStore(blockManager: BlockManager, diskManager: DiskBloc file.getName, Utils.bytesToString(bytes.limit), finishTime - startTime)) } - override def putIterator( + def putIterator( blockId: BlockId, values: Iterator[Any], level: StorageLevel): Right[Iterator[Any], Long] = { @@ -113,16 +113,16 @@ private[spark] class DiskStore(blockManager: BlockManager, diskManager: DiskBloc } } - override def getBytes(blockId: BlockId): Option[ByteBuffer] = { + def getBytes(blockId: BlockId): Option[ByteBuffer] = { val file = diskManager.getFile(blockId.name) getBytes(file, 0, file.length) } - override def getValues(blockId: BlockId): Option[Iterator[Any]] = { + def getValues(blockId: BlockId): Option[Iterator[Any]] = { getBytes(blockId).map(buffer => blockManager.dataDeserialize(blockId, buffer)) } - override def remove(blockId: BlockId): Boolean = { + def remove(blockId: BlockId): Boolean = { val file = diskManager.getFile(blockId.name) if (file.exists()) { val ret = file.delete() @@ -135,7 +135,7 @@ private[spark] class DiskStore(blockManager: BlockManager, diskManager: DiskBloc } } - override def contains(blockId: BlockId): Boolean = { + def contains(blockId: BlockId): Boolean = { val file = diskManager.getFile(blockId.name) file.exists() } diff --git a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala index bb72fe4bcafbe..c06ca9a1a879e 100644 --- a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala @@ -23,7 +23,7 @@ import java.util.LinkedHashMap import scala.collection.mutable import scala.collection.mutable.ArrayBuffer -import org.apache.spark.TaskContext +import org.apache.spark.{Logging, TaskContext} import org.apache.spark.memory.MemoryManager import org.apache.spark.util.{SizeEstimator, Utils} import org.apache.spark.util.collection.SizeTrackingVector @@ -35,7 +35,7 @@ private case class MemoryEntry(value: Any, size: Long, deserialized: Boolean) * serialized ByteBuffers. */ private[spark] class MemoryStore(blockManager: BlockManager, memoryManager: MemoryManager) - extends BlockStore(blockManager) { + extends Logging { // Note: all changes to memory allocations, notably putting blocks, evicting blocks, and // acquiring or releasing unroll memory, must be synchronized on `memoryManager`! @@ -81,13 +81,13 @@ private[spark] class MemoryStore(blockManager: BlockManager, memoryManager: Memo memoryUsed - currentUnrollMemory } - override def getSize(blockId: BlockId): Long = { + def getSize(blockId: BlockId): Long = { entries.synchronized { entries.get(blockId).size } } - override def putBytes(blockId: BlockId, _bytes: ByteBuffer, level: StorageLevel): Unit = { + def putBytes(blockId: BlockId, _bytes: ByteBuffer, level: StorageLevel): Unit = { require(!contains(blockId), s"Block $blockId is already present in the MemoryStore") // Work on a duplicate - since the original input might be used elsewhere. val bytes = _bytes.duplicate() @@ -116,7 +116,7 @@ private[spark] class MemoryStore(blockManager: BlockManager, memoryManager: Memo } } - override def putIterator( + def putIterator( blockId: BlockId, values: Iterator[Any], level: StorageLevel): Either[Iterator[Any], Long] = { @@ -169,7 +169,7 @@ private[spark] class MemoryStore(blockManager: BlockManager, memoryManager: Memo } } - override def getBytes(blockId: BlockId): Option[ByteBuffer] = { + def getBytes(blockId: BlockId): Option[ByteBuffer] = { val entry = entries.synchronized { entries.get(blockId) } @@ -182,7 +182,7 @@ private[spark] class MemoryStore(blockManager: BlockManager, memoryManager: Memo } } - override def getValues(blockId: BlockId): Option[Iterator[Any]] = { + def getValues(blockId: BlockId): Option[Iterator[Any]] = { val entry = entries.synchronized { entries.get(blockId) } @@ -196,7 +196,7 @@ private[spark] class MemoryStore(blockManager: BlockManager, memoryManager: Memo } } - override def remove(blockId: BlockId): Boolean = memoryManager.synchronized { + def remove(blockId: BlockId): Boolean = memoryManager.synchronized { val entry = entries.synchronized { entries.remove(blockId) } @@ -210,7 +210,7 @@ private[spark] class MemoryStore(blockManager: BlockManager, memoryManager: Memo } } - override def clear(): Unit = memoryManager.synchronized { + def clear(): Unit = memoryManager.synchronized { entries.synchronized { entries.clear() } @@ -455,7 +455,7 @@ private[spark] class MemoryStore(blockManager: BlockManager, memoryManager: Memo } } - override def contains(blockId: BlockId): Boolean = { + def contains(blockId: BlockId): Boolean = { entries.synchronized { entries.containsKey(blockId) } } From 14d003ffe06ac763e128a1b3fbee8fb22fb2cc17 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Thu, 3 Mar 2016 17:13:14 -0800 Subject: [PATCH 02/42] Remove unused DiskStore.getBytes() overloads. --- .../org/apache/spark/storage/DiskStore.scala | 24 +++++++------------ 1 file changed, 8 insertions(+), 16 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala index e09325d5b5afa..bb01c4e31c8a4 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala @@ -17,7 +17,7 @@ package org.apache.spark.storage -import java.io.{File, FileOutputStream, IOException, RandomAccessFile} +import java.io.{FileOutputStream, IOException, RandomAccessFile} import java.nio.ByteBuffer import java.nio.channels.FileChannel.MapMode @@ -90,38 +90,30 @@ private[spark] class DiskStore(blockManager: BlockManager, diskManager: DiskBloc Right(length) } - private def getBytes(file: File, offset: Long, length: Long): Option[ByteBuffer] = { + def getBytes(blockId: BlockId): Option[ByteBuffer] = { + val file = diskManager.getFile(blockId.name) val channel = new RandomAccessFile(file, "r").getChannel Utils.tryWithSafeFinally { // For small files, directly read rather than memory map - if (length < minMemoryMapBytes) { - val buf = ByteBuffer.allocate(length.toInt) - channel.position(offset) + if (file.length < minMemoryMapBytes) { + val buf = ByteBuffer.allocate(file.length.toInt) + channel.position(0) while (buf.remaining() != 0) { if (channel.read(buf) == -1) { throw new IOException("Reached EOF before filling buffer\n" + - s"offset=$offset\nfile=${file.getAbsolutePath}\nbuf.remaining=${buf.remaining}") + s"offset=0\nfile=${file.getAbsolutePath}\nbuf.remaining=${buf.remaining}") } } buf.flip() Some(buf) } else { - Some(channel.map(MapMode.READ_ONLY, offset, length)) + Some(channel.map(MapMode.READ_ONLY, 0, file.length)) } } { channel.close() } } - def getBytes(blockId: BlockId): Option[ByteBuffer] = { - val file = diskManager.getFile(blockId.name) - getBytes(file, 0, file.length) - } - - def getValues(blockId: BlockId): Option[Iterator[Any]] = { - getBytes(blockId).map(buffer => blockManager.dataDeserialize(blockId, buffer)) - } - def remove(blockId: BlockId): Boolean = { val file = diskManager.getFile(blockId.name) if (file.exists()) { From 1a764c05a58c5ef299508569ac4096b5fa0468d9 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Thu, 3 Mar 2016 17:14:36 -0800 Subject: [PATCH 03/42] DiskStore.getBytes() never returns None, so it shouldn't return an Option. --- .../scala/org/apache/spark/storage/BlockManager.scala | 8 +------- .../main/scala/org/apache/spark/storage/DiskStore.scala | 6 +++--- .../org/apache/spark/storage/BlockManagerSuite.scala | 4 ++-- 3 files changed, 6 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 5fced4c9a2f1e..a1506a5a4ee93 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -475,13 +475,7 @@ private[spark] class BlockManager( // Look for block on disk, potentially storing it back in memory if required if (level.useDisk) { logDebug(s"Getting block $blockId from disk") - val bytes: ByteBuffer = diskStore.getBytes(blockId) match { - case Some(b) => b - case None => - releaseLock(blockId) - throw new BlockException( - blockId, s"Block $blockId not found on disk, though it should be") - } + val bytes: ByteBuffer = diskStore.getBytes(blockId) assert(0 == bytes.position()) if (!level.useMemory) { diff --git a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala index bb01c4e31c8a4..062bee7c27cf7 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala @@ -90,7 +90,7 @@ private[spark] class DiskStore(blockManager: BlockManager, diskManager: DiskBloc Right(length) } - def getBytes(blockId: BlockId): Option[ByteBuffer] = { + def getBytes(blockId: BlockId): ByteBuffer = { val file = diskManager.getFile(blockId.name) val channel = new RandomAccessFile(file, "r").getChannel Utils.tryWithSafeFinally { @@ -105,9 +105,9 @@ private[spark] class DiskStore(blockManager: BlockManager, diskManager: DiskBloc } } buf.flip() - Some(buf) + buf } else { - Some(channel.map(MapMode.READ_ONLY, 0, file.length)) + channel.map(MapMode.READ_ONLY, 0, file.length) } } { channel.close() diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index 0485b0501c030..60b7fd74cf827 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -849,12 +849,12 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE val diskStoreMapped = new DiskStore(blockManager, diskBlockManager) diskStoreMapped.putBytes(blockId, byteBuffer, StorageLevel.DISK_ONLY) - val mapped = diskStoreMapped.getBytes(blockId).get + val mapped = diskStoreMapped.getBytes(blockId) when(blockManager.conf).thenReturn(conf.clone.set(confKey, "1m")) val diskStoreNotMapped = new DiskStore(blockManager, diskBlockManager) diskStoreNotMapped.putBytes(blockId, byteBuffer, StorageLevel.DISK_ONLY) - val notMapped = diskStoreNotMapped.getBytes(blockId).get + val notMapped = diskStoreNotMapped.getBytes(blockId) // Not possible to do isInstanceOf due to visibility of HeapByteBuffer assert(notMapped.getClass.getName.endsWith("HeapByteBuffer"), From 9814ba6b479850ca9c198af1556e16017a769280 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Thu, 3 Mar 2016 17:31:02 -0800 Subject: [PATCH 04/42] Simplify DiskStore.putIterator's return type. --- .../main/scala/org/apache/spark/storage/BlockManager.scala | 6 +----- .../src/main/scala/org/apache/spark/storage/DiskStore.scala | 4 ++-- .../main/scala/org/apache/spark/storage/MemoryStore.scala | 2 +- 3 files changed, 4 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index a1506a5a4ee93..0c17c426b81e8 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -836,11 +836,7 @@ private[spark] class BlockManager( } else if (putLevel.useDisk) { data match { case IteratorValues(iterator) => - diskStore.putIterator(blockId, iterator(), putLevel) match { - case Right(s) => - size = s - // putIterator() will never return Left (see its return type). - } + size = diskStore.putIterator(blockId, iterator(), putLevel) case ByteBufferValues(bytes) => bytes.rewind() size = bytes.limit() diff --git a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala index 062bee7c27cf7..3db29b485d5f2 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala @@ -59,7 +59,7 @@ private[spark] class DiskStore(blockManager: BlockManager, diskManager: DiskBloc def putIterator( blockId: BlockId, values: Iterator[Any], - level: StorageLevel): Right[Iterator[Any], Long] = { + level: StorageLevel): Long = { logDebug(s"Attempting to write values for block $blockId") val startTime = System.currentTimeMillis val file = diskManager.getFile(blockId) @@ -87,7 +87,7 @@ private[spark] class DiskStore(blockManager: BlockManager, diskManager: DiskBloc logDebug("Block %s stored as %s file on disk in %d ms".format( file.getName, Utils.bytesToString(length), timeTaken)) - Right(length) + length } def getBytes(blockId: BlockId): ByteBuffer = { diff --git a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala index c06ca9a1a879e..4f05a5939ca79 100644 --- a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala @@ -162,7 +162,7 @@ private[spark] class MemoryStore(blockManager: BlockManager, memoryManager: Memo // Not enough space to unroll this block; drop to disk if applicable if (level.useDisk && allowPersistToDisk) { logWarning(s"Persisting block $blockId to disk instead.") - blockManager.diskStore.putIterator(blockId, iteratorValues, level) + Right(blockManager.diskStore.putIterator(blockId, iteratorValues, level)) } else { Left(iteratorValues) } From 14d5652107cdfc8c32e94231cfc2919339d2a2a5 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Thu, 3 Mar 2016 17:33:03 -0800 Subject: [PATCH 05/42] Remove MemoryStore.putIterator() overload. --- .../scala/org/apache/spark/storage/MemoryStore.scala | 10 +--------- 1 file changed, 1 insertion(+), 9 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala index 4f05a5939ca79..2d98d1b4efc57 100644 --- a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala @@ -116,14 +116,6 @@ private[spark] class MemoryStore(blockManager: BlockManager, memoryManager: Memo } } - def putIterator( - blockId: BlockId, - values: Iterator[Any], - level: StorageLevel): Either[Iterator[Any], Long] = { - require(!contains(blockId), s"Block $blockId is already present in the MemoryStore") - putIterator(blockId, values, level, allowPersistToDisk = true) - } - /** * Attempt to put the given block in memory store. * @@ -140,7 +132,7 @@ private[spark] class MemoryStore(blockManager: BlockManager, memoryManager: Memo blockId: BlockId, values: Iterator[Any], level: StorageLevel, - allowPersistToDisk: Boolean): Either[Iterator[Any], Long] = { + allowPersistToDisk: Boolean = true): Either[Iterator[Any], Long] = { require(!contains(blockId), s"Block $blockId is already present in the MemoryStore") val unrolledValues = unrollSafely(blockId, values) unrolledValues match { From 5c294ace7dcdabf4e325367866eac18aa6e16efb Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Thu, 3 Mar 2016 17:39:06 -0800 Subject: [PATCH 06/42] DiskStore put() methods don't need to take a StorageLevel. --- .../scala/org/apache/spark/storage/BlockManager.scala | 8 ++++---- .../main/scala/org/apache/spark/storage/DiskStore.scala | 7 ++----- .../main/scala/org/apache/spark/storage/MemoryStore.scala | 2 +- .../org/apache/spark/storage/BlockManagerSuite.scala | 4 ++-- 4 files changed, 9 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 0c17c426b81e8..9b404e5614ee8 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -836,11 +836,11 @@ private[spark] class BlockManager( } else if (putLevel.useDisk) { data match { case IteratorValues(iterator) => - size = diskStore.putIterator(blockId, iterator(), putLevel) + size = diskStore.putIterator(blockId, iterator()) case ByteBufferValues(bytes) => bytes.rewind() size = bytes.limit() - diskStore.putBytes(blockId, bytes, putLevel) + diskStore.putBytes(blockId, bytes) } } else { assert(putLevel == StorageLevel.NONE) @@ -1067,9 +1067,9 @@ private[spark] class BlockManager( logInfo(s"Writing block $blockId to disk") data() match { case Left(elements) => - diskStore.putIterator(blockId, elements.toIterator, level) + diskStore.putIterator(blockId, elements.toIterator) case Right(bytes) => - diskStore.putBytes(blockId, bytes, level) + diskStore.putBytes(blockId, bytes) } blockIsUpdated = true } diff --git a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala index 3db29b485d5f2..251ca2aa1696c 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala @@ -36,7 +36,7 @@ private[spark] class DiskStore(blockManager: BlockManager, diskManager: DiskBloc diskManager.getFile(blockId.name).length } - def putBytes(blockId: BlockId, _bytes: ByteBuffer, level: StorageLevel): Unit = { + def putBytes(blockId: BlockId, _bytes: ByteBuffer): Unit = { // So that we do not modify the input offsets ! // duplicate does not copy buffer, so inexpensive val bytes = _bytes.duplicate() @@ -56,10 +56,7 @@ private[spark] class DiskStore(blockManager: BlockManager, diskManager: DiskBloc file.getName, Utils.bytesToString(bytes.limit), finishTime - startTime)) } - def putIterator( - blockId: BlockId, - values: Iterator[Any], - level: StorageLevel): Long = { + def putIterator(blockId: BlockId, values: Iterator[Any]): Long = { logDebug(s"Attempting to write values for block $blockId") val startTime = System.currentTimeMillis val file = diskManager.getFile(blockId) diff --git a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala index 2d98d1b4efc57..ea5090072778d 100644 --- a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala @@ -154,7 +154,7 @@ private[spark] class MemoryStore(blockManager: BlockManager, memoryManager: Memo // Not enough space to unroll this block; drop to disk if applicable if (level.useDisk && allowPersistToDisk) { logWarning(s"Persisting block $blockId to disk instead.") - Right(blockManager.diskStore.putIterator(blockId, iteratorValues, level)) + Right(blockManager.diskStore.putIterator(blockId, iteratorValues)) } else { Left(iteratorValues) } diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index 60b7fd74cf827..b9e36ba870cb8 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -848,12 +848,12 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE val diskBlockManager = new DiskBlockManager(blockManager, conf) val diskStoreMapped = new DiskStore(blockManager, diskBlockManager) - diskStoreMapped.putBytes(blockId, byteBuffer, StorageLevel.DISK_ONLY) + diskStoreMapped.putBytes(blockId, byteBuffer) val mapped = diskStoreMapped.getBytes(blockId) when(blockManager.conf).thenReturn(conf.clone.set(confKey, "1m")) val diskStoreNotMapped = new DiskStore(blockManager, diskBlockManager) - diskStoreNotMapped.putBytes(blockId, byteBuffer, StorageLevel.DISK_ONLY) + diskStoreNotMapped.putBytes(blockId, byteBuffer) val notMapped = diskStoreNotMapped.getBytes(blockId) // Not possible to do isInstanceOf due to visibility of HeapByteBuffer From c8d0e695f473590bed1cc318f98c8445834c00a6 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Thu, 3 Mar 2016 18:05:23 -0800 Subject: [PATCH 07/42] Factor common error-handling code in DiskStore.put*() into helper function. --- .../org/apache/spark/storage/DiskStore.scala | 77 ++++++++++--------- .../spark/storage/BlockManagerSuite.scala | 2 + 2 files changed, 44 insertions(+), 35 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala index 251ca2aa1696c..ab18f75a2f617 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala @@ -21,6 +21,8 @@ import java.io.{FileOutputStream, IOException, RandomAccessFile} import java.nio.ByteBuffer import java.nio.channels.FileChannel.MapMode +import com.google.common.io.Closeables + import org.apache.spark.Logging import org.apache.spark.util.Utils @@ -36,55 +38,60 @@ private[spark] class DiskStore(blockManager: BlockManager, diskManager: DiskBloc diskManager.getFile(blockId.name).length } - def putBytes(blockId: BlockId, _bytes: ByteBuffer): Unit = { - // So that we do not modify the input offsets ! - // duplicate does not copy buffer, so inexpensive - val bytes = _bytes.duplicate() + /** + * Invokes the provided callback function to write the specific block. + * + * @throws IllegalStateException if the block already exists in the disk store. + */ + def put(blockId: BlockId)(writeFunc: FileOutputStream => Unit): Unit = { + if (contains(blockId)) { + throw new IllegalStateException(s"Block $blockId is already present in the disk store") + } logDebug(s"Attempting to put block $blockId") val startTime = System.currentTimeMillis val file = diskManager.getFile(blockId) - val channel = new FileOutputStream(file).getChannel - Utils.tryWithSafeFinally { - while (bytes.remaining > 0) { - channel.write(bytes) + val fileOutputStream = new FileOutputStream(file) + var threwException: Boolean = true + try { + writeFunc(fileOutputStream) + threwException = false + } finally { + try { + Closeables.close(fileOutputStream, threwException) + } finally { + if (threwException) { + remove(blockId) + } } - } { - channel.close() } val finishTime = System.currentTimeMillis logDebug("Block %s stored as %s file on disk in %d ms".format( - file.getName, Utils.bytesToString(bytes.limit), finishTime - startTime)) + file.getName, + Utils.bytesToString(file.length()), + finishTime - startTime)) } - def putIterator(blockId: BlockId, values: Iterator[Any]): Long = { - logDebug(s"Attempting to write values for block $blockId") - val startTime = System.currentTimeMillis - val file = diskManager.getFile(blockId) - val outputStream = new FileOutputStream(file) - try { + def putBytes(blockId: BlockId, _bytes: ByteBuffer): Unit = { + // So that we do not modify the input offsets ! + // duplicate does not copy buffer, so inexpensive + val bytes = _bytes.duplicate() + put(blockId) { fileOutputStream => + val channel = fileOutputStream.getChannel Utils.tryWithSafeFinally { - blockManager.dataSerializeStream(blockId, outputStream, values) + while (bytes.remaining > 0) { + channel.write(bytes) + } } { - // Close outputStream here because it should be closed before file is deleted. - outputStream.close() + channel.close() } - } catch { - case e: Throwable => - if (file.exists()) { - if (!file.delete()) { - logWarning(s"Error deleting ${file}") - } - } - throw e } + } - val length = file.length - - val timeTaken = System.currentTimeMillis - startTime - logDebug("Block %s stored as %s file on disk in %d ms".format( - file.getName, Utils.bytesToString(length), timeTaken)) - - length + def putIterator(blockId: BlockId, values: Iterator[Any]): Long = { + put(blockId) { fileOutputStream => + blockManager.dataSerializeStream(blockId, fileOutputStream, values) + } + getSize(blockId) } def getBytes(blockId: BlockId): ByteBuffer = { diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index b9e36ba870cb8..9f398cbd2f167 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -851,6 +851,8 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE diskStoreMapped.putBytes(blockId, byteBuffer) val mapped = diskStoreMapped.getBytes(blockId) + assert(diskStoreMapped.remove(blockId)) + when(blockManager.conf).thenReturn(conf.clone.set(confKey, "1m")) val diskStoreNotMapped = new DiskStore(blockManager, diskBlockManager) diskStoreNotMapped.putBytes(blockId, byteBuffer) From 46b3877a462a19b9284990e05830f9782629235d Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Thu, 3 Mar 2016 18:09:45 -0800 Subject: [PATCH 08/42] Remove DiskStore.putIterator(). --- .../scala/org/apache/spark/storage/BlockManager.scala | 9 +++++++-- .../main/scala/org/apache/spark/storage/DiskStore.scala | 7 ------- .../scala/org/apache/spark/storage/MemoryStore.scala | 5 ++++- 3 files changed, 11 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 9b404e5614ee8..16c21c9df31db 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -836,7 +836,10 @@ private[spark] class BlockManager( } else if (putLevel.useDisk) { data match { case IteratorValues(iterator) => - size = diskStore.putIterator(blockId, iterator()) + diskStore.put(blockId) { fileOutputStream => + dataSerializeStream(blockId, fileOutputStream, iterator()) + } + size = diskStore.getSize(blockId) case ByteBufferValues(bytes) => bytes.rewind() size = bytes.limit() @@ -1067,7 +1070,9 @@ private[spark] class BlockManager( logInfo(s"Writing block $blockId to disk") data() match { case Left(elements) => - diskStore.putIterator(blockId, elements.toIterator) + diskStore.put(blockId) { fileOutputStream => + dataSerializeStream(blockId, fileOutputStream, elements.toIterator) + } case Right(bytes) => diskStore.putBytes(blockId, bytes) } diff --git a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala index ab18f75a2f617..94240491a419f 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala @@ -87,13 +87,6 @@ private[spark] class DiskStore(blockManager: BlockManager, diskManager: DiskBloc } } - def putIterator(blockId: BlockId, values: Iterator[Any]): Long = { - put(blockId) { fileOutputStream => - blockManager.dataSerializeStream(blockId, fileOutputStream, values) - } - getSize(blockId) - } - def getBytes(blockId: BlockId): ByteBuffer = { val file = diskManager.getFile(blockId.name) val channel = new RandomAccessFile(file, "r").getChannel diff --git a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala index ea5090072778d..e1ee38de31fb4 100644 --- a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala @@ -154,7 +154,10 @@ private[spark] class MemoryStore(blockManager: BlockManager, memoryManager: Memo // Not enough space to unroll this block; drop to disk if applicable if (level.useDisk && allowPersistToDisk) { logWarning(s"Persisting block $blockId to disk instead.") - Right(blockManager.diskStore.putIterator(blockId, iteratorValues)) + blockManager.diskStore.put(blockId) { fileOutputStream => + blockManager.dataSerializeStream(blockId, fileOutputStream, iteratorValues) + } + Right(blockManager.diskStore.getSize(blockId)) } else { Left(iteratorValues) } From 27cee47a6228e470ab54891fd1d2dec676a49938 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Thu, 3 Mar 2016 18:21:57 -0800 Subject: [PATCH 09/42] Remove DiskStore's dependency on BlockManager. --- .../apache/spark/storage/BlockManager.scala | 2 +- .../spark/storage/DiskBlockManager.scala | 2 +- .../org/apache/spark/storage/DiskStore.scala | 7 +- .../spark/storage/BlockManagerSuite.scala | 49 +------------ .../apache/spark/storage/DiskStoreSuite.scala | 70 +++++++++++++++++++ 5 files changed, 76 insertions(+), 54 deletions(-) create mode 100644 core/src/test/scala/org/apache/spark/storage/DiskStoreSuite.scala diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 16c21c9df31db..fa680ac4da3ab 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -87,7 +87,7 @@ private[spark] class BlockManager( // Actual storage of where blocks are kept private[spark] val memoryStore = new MemoryStore(this, memoryManager) - private[spark] val diskStore = new DiskStore(this, diskBlockManager) + private[spark] val diskStore = new DiskStore(conf, diskBlockManager) memoryManager.setMemoryStore(memoryStore) // Note: depending on the memory manager, `maxStorageMemory` may actually vary over time. diff --git a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala index 4daf22f71415e..3b2cf177c0ee5 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala @@ -37,7 +37,7 @@ private[spark] class DiskBlockManager(blockManager: BlockManager, conf: SparkCon extends Logging { private[spark] - val subDirsPerLocalDir = blockManager.conf.getInt("spark.diskStore.subDirectories", 64) + val subDirsPerLocalDir = conf.getInt("spark.diskStore.subDirectories", 64) /* Create one local directory for each path mentioned in spark.local.dir; then, inside this * directory, create multiple subdirectories that we will hash files into, in order to avoid diff --git a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala index 94240491a419f..caecd97a0b722 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala @@ -23,16 +23,15 @@ import java.nio.channels.FileChannel.MapMode import com.google.common.io.Closeables -import org.apache.spark.Logging +import org.apache.spark.{Logging, SparkConf} import org.apache.spark.util.Utils /** * Stores BlockManager blocks on disk. */ -private[spark] class DiskStore(blockManager: BlockManager, diskManager: DiskBlockManager) - extends Logging { +private[spark] class DiskStore(conf: SparkConf, diskManager: DiskBlockManager) extends Logging { - val minMemoryMapBytes = blockManager.conf.getSizeAsBytes("spark.storage.memoryMapThreshold", "2m") + private val minMemoryMapBytes = conf.getSizeAsBytes("spark.storage.memoryMapThreshold", "2m") def getSize(blockId: BlockId): Long = { diskManager.getFile(blockId.name).length diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index 9f398cbd2f167..1c00f5fb7c9ce 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -17,8 +17,7 @@ package org.apache.spark.storage -import java.nio.{ByteBuffer, MappedByteBuffer} -import java.util.Arrays +import java.nio.ByteBuffer import scala.collection.mutable.ArrayBuffer import scala.concurrent.duration._ @@ -829,52 +828,6 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE } } - test("reads of memory-mapped and non memory-mapped files are equivalent") { - val confKey = "spark.storage.memoryMapThreshold" - - // Create a non-trivial (not all zeros) byte array - var counter = 0.toByte - def incr: Byte = {counter = (counter + 1).toByte; counter;} - val bytes = Array.fill[Byte](1000)(incr) - val byteBuffer = ByteBuffer.wrap(bytes) - - val blockId = BlockId("rdd_1_2") - - // This sequence of mocks makes these tests fairly brittle. It would - // be nice to refactor classes involved in disk storage in a way that - // allows for easier testing. - val blockManager = mock(classOf[BlockManager]) - when(blockManager.conf).thenReturn(conf.clone.set(confKey, "0")) - val diskBlockManager = new DiskBlockManager(blockManager, conf) - - val diskStoreMapped = new DiskStore(blockManager, diskBlockManager) - diskStoreMapped.putBytes(blockId, byteBuffer) - val mapped = diskStoreMapped.getBytes(blockId) - - assert(diskStoreMapped.remove(blockId)) - - when(blockManager.conf).thenReturn(conf.clone.set(confKey, "1m")) - val diskStoreNotMapped = new DiskStore(blockManager, diskBlockManager) - diskStoreNotMapped.putBytes(blockId, byteBuffer) - val notMapped = diskStoreNotMapped.getBytes(blockId) - - // Not possible to do isInstanceOf due to visibility of HeapByteBuffer - assert(notMapped.getClass.getName.endsWith("HeapByteBuffer"), - "Expected HeapByteBuffer for un-mapped read") - assert(mapped.isInstanceOf[MappedByteBuffer], "Expected MappedByteBuffer for mapped read") - - def arrayFromByteBuffer(in: ByteBuffer): Array[Byte] = { - val array = new Array[Byte](in.remaining()) - in.get(array) - array - } - - val mappedAsArray = arrayFromByteBuffer(mapped) - val notMappedAsArray = arrayFromByteBuffer(notMapped) - assert(Arrays.equals(mappedAsArray, bytes)) - assert(Arrays.equals(notMappedAsArray, bytes)) - } - test("updated block statuses") { store = makeBlockManager(12000) store.registerTask(0) diff --git a/core/src/test/scala/org/apache/spark/storage/DiskStoreSuite.scala b/core/src/test/scala/org/apache/spark/storage/DiskStoreSuite.scala new file mode 100644 index 0000000000000..fcefb4ff1d8bf --- /dev/null +++ b/core/src/test/scala/org/apache/spark/storage/DiskStoreSuite.scala @@ -0,0 +1,70 @@ +/* + * 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.storage + +import java.nio.{ByteBuffer, MappedByteBuffer} +import java.util.Arrays + +import org.mockito.Mockito.{mock, RETURNS_SMART_NULLS} + +import org.apache.spark.{SparkConf, SparkFunSuite} + +class DiskStoreSuite extends SparkFunSuite { + test("reads of memory-mapped and non memory-mapped files are equivalent") { + val confKey = "spark.storage.memoryMapThreshold" + + // Create a non-trivial (not all zeros) byte array + var counter = 0.toByte + def incr: Byte = {counter = (counter + 1).toByte; counter;} + val bytes = Array.fill[Byte](1000)(incr) + val byteBuffer = ByteBuffer.wrap(bytes) + + val blockId = BlockId("rdd_1_2") + val blockManager = mock(classOf[BlockManager], RETURNS_SMART_NULLS) + val diskBlockManager = new DiskBlockManager(blockManager, new SparkConf()) + + // This sequence of mocks makes these tests fairly brittle. It would + // be nice to refactor classes involved in disk storage in a way that + // allows for easier testing. + + val diskStoreMapped = new DiskStore(new SparkConf().set(confKey, "0"), diskBlockManager) + diskStoreMapped.putBytes(blockId, byteBuffer) + val mapped = diskStoreMapped.getBytes(blockId) + assert(diskStoreMapped.remove(blockId)) + + val diskStoreNotMapped = new DiskStore(new SparkConf().set(confKey, "1m"), diskBlockManager) + diskStoreNotMapped.putBytes(blockId, byteBuffer) + val notMapped = diskStoreNotMapped.getBytes(blockId) + + // Not possible to do isInstanceOf due to visibility of HeapByteBuffer + assert(notMapped.getClass.getName.endsWith("HeapByteBuffer"), + "Expected HeapByteBuffer for un-mapped read") + assert(mapped.isInstanceOf[MappedByteBuffer], "Expected MappedByteBuffer for mapped read") + + def arrayFromByteBuffer(in: ByteBuffer): Array[Byte] = { + val array = new Array[Byte](in.remaining()) + in.get(array) + array + } + + val mappedAsArray = arrayFromByteBuffer(mapped) + val notMappedAsArray = arrayFromByteBuffer(notMapped) + assert(Arrays.equals(mappedAsArray, bytes)) + assert(Arrays.equals(notMappedAsArray, bytes)) + } +} From 1a50c8115f31a1acd3b0a4dd5e1b3d28e81d05f5 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Thu, 3 Mar 2016 18:25:31 -0800 Subject: [PATCH 10/42] Minor simplifications in DiskStoreSuite test. --- .../scala/org/apache/spark/storage/DiskStoreSuite.scala | 9 ++------- 1 file changed, 2 insertions(+), 7 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/storage/DiskStoreSuite.scala b/core/src/test/scala/org/apache/spark/storage/DiskStoreSuite.scala index fcefb4ff1d8bf..7c98548ab5299 100644 --- a/core/src/test/scala/org/apache/spark/storage/DiskStoreSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/DiskStoreSuite.scala @@ -25,23 +25,18 @@ import org.mockito.Mockito.{mock, RETURNS_SMART_NULLS} import org.apache.spark.{SparkConf, SparkFunSuite} class DiskStoreSuite extends SparkFunSuite { + test("reads of memory-mapped and non memory-mapped files are equivalent") { val confKey = "spark.storage.memoryMapThreshold" // Create a non-trivial (not all zeros) byte array - var counter = 0.toByte - def incr: Byte = {counter = (counter + 1).toByte; counter;} - val bytes = Array.fill[Byte](1000)(incr) + val bytes = Array.tabulate[Byte](1000)(_.toByte) val byteBuffer = ByteBuffer.wrap(bytes) val blockId = BlockId("rdd_1_2") val blockManager = mock(classOf[BlockManager], RETURNS_SMART_NULLS) val diskBlockManager = new DiskBlockManager(blockManager, new SparkConf()) - // This sequence of mocks makes these tests fairly brittle. It would - // be nice to refactor classes involved in disk storage in a way that - // allows for easier testing. - val diskStoreMapped = new DiskStore(new SparkConf().set(confKey, "0"), diskBlockManager) diskStoreMapped.putBytes(blockId, byteBuffer) val mapped = diskStoreMapped.getBytes(blockId) From f3b60052c42215d1440cc5660bdea443bed5598e Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Thu, 3 Mar 2016 18:29:18 -0800 Subject: [PATCH 11/42] Remove outdated comment in DiskBlockManager. --- .../scala/org/apache/spark/storage/DiskBlockManager.scala | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala index 3b2cf177c0ee5..83d640e4a4a8d 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala @@ -26,9 +26,7 @@ import org.apache.spark.util.{ShutdownHookManager, Utils} /** * Creates and maintains the logical mapping between logical blocks and physical on-disk - * locations. By default, one block is mapped to one file with a name given by its BlockId. - * However, it is also possible to have a block map to only a segment of a file, by calling - * mapBlockToFileSegment(). + * locations. One block is mapped to one file with a name given by its BlockId. * * Block files are hashed among the directories listed in spark.local.dir (or in * SPARK_LOCAL_DIRS, if it's set). From 2d86e290f77c15aa8ba6e6e46f6c39987ee351a5 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Thu, 3 Mar 2016 18:39:25 -0800 Subject: [PATCH 12/42] Remove DiskStore's dependency on BlockManager. --- .../org/apache/spark/storage/BlockManager.scala | 13 +++++++++---- .../org/apache/spark/storage/DiskBlockManager.scala | 11 +++-------- .../spark/storage/DiskBlockManagerSuite.scala | 5 +---- .../org/apache/spark/storage/DiskStoreSuite.scala | 5 +---- 4 files changed, 14 insertions(+), 20 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index fa680ac4da3ab..6ffb62a1581ed 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -78,7 +78,15 @@ private[spark] class BlockManager( numUsableCores: Int) extends BlockDataManager with Logging { - val diskBlockManager = new DiskBlockManager(this, conf) + private[spark] val externalShuffleServiceEnabled = + conf.getBoolean("spark.shuffle.service.enabled", false) + + val diskBlockManager = { + // Only perform cleanup if an external service is not serving our shuffle files. + val deleteFilesOnStop = + !externalShuffleServiceEnabled || executorId == SparkContext.DRIVER_IDENTIFIER + new DiskBlockManager(conf, deleteFilesOnStop) + } private[storage] val blockInfoManager = new BlockInfoManager @@ -96,9 +104,6 @@ private[spark] class BlockManager( // to revisit whether reporting this value as the "max" is intuitive to the user. private val maxMemory = memoryManager.maxStorageMemory - private[spark] - val externalShuffleServiceEnabled = conf.getBoolean("spark.shuffle.service.enabled", false) - // Port used by the external shuffle service. In Yarn mode, this may be already be // set through the Hadoop configuration as the server is launched in the Yarn NM. private val externalShuffleServicePort = { diff --git a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala index 83d640e4a4a8d..e51d96e57bc6f 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala @@ -31,11 +31,9 @@ import org.apache.spark.util.{ShutdownHookManager, Utils} * Block files are hashed among the directories listed in spark.local.dir (or in * SPARK_LOCAL_DIRS, if it's set). */ -private[spark] class DiskBlockManager(blockManager: BlockManager, conf: SparkConf) - extends Logging { +private[spark] class DiskBlockManager(conf: SparkConf, deleteFilesOnStop: Boolean) extends Logging { - private[spark] - val subDirsPerLocalDir = conf.getInt("spark.diskStore.subDirectories", 64) + private[spark] val subDirsPerLocalDir = conf.getInt("spark.diskStore.subDirectories", 64) /* Create one local directory for each path mentioned in spark.local.dir; then, inside this * directory, create multiple subdirectories that we will hash files into, in order to avoid @@ -161,10 +159,7 @@ private[spark] class DiskBlockManager(blockManager: BlockManager, conf: SparkCon } private def doStop(): Unit = { - // Only perform cleanup if an external service is not serving our shuffle files. - // Also blockManagerId could be null if block manager is not initialized properly. - if (!blockManager.externalShuffleServiceEnabled || - (blockManager.blockManagerId != null && blockManager.blockManagerId.isDriver)) { + if (deleteFilesOnStop) { localDirs.foreach { localDir => if (localDir.isDirectory() && localDir.exists()) { try { diff --git a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala index 69e17461df755..bbfd6df3b6990 100644 --- a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala @@ -21,7 +21,6 @@ import java.io.{File, FileWriter} import scala.language.reflectiveCalls -import org.mockito.Mockito.{mock, when} import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach} import org.apache.spark.{SparkConf, SparkFunSuite} @@ -33,8 +32,6 @@ class DiskBlockManagerSuite extends SparkFunSuite with BeforeAndAfterEach with B private var rootDir1: File = _ private var rootDirs: String = _ - val blockManager = mock(classOf[BlockManager]) - when(blockManager.conf).thenReturn(testConf) var diskBlockManager: DiskBlockManager = _ override def beforeAll() { @@ -57,7 +54,7 @@ class DiskBlockManagerSuite extends SparkFunSuite with BeforeAndAfterEach with B super.beforeEach() val conf = testConf.clone conf.set("spark.local.dir", rootDirs) - diskBlockManager = new DiskBlockManager(blockManager, conf) + diskBlockManager = new DiskBlockManager(conf, deleteFilesOnStop = true) } override def afterEach() { diff --git a/core/src/test/scala/org/apache/spark/storage/DiskStoreSuite.scala b/core/src/test/scala/org/apache/spark/storage/DiskStoreSuite.scala index 7c98548ab5299..97e74fe706002 100644 --- a/core/src/test/scala/org/apache/spark/storage/DiskStoreSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/DiskStoreSuite.scala @@ -20,8 +20,6 @@ package org.apache.spark.storage import java.nio.{ByteBuffer, MappedByteBuffer} import java.util.Arrays -import org.mockito.Mockito.{mock, RETURNS_SMART_NULLS} - import org.apache.spark.{SparkConf, SparkFunSuite} class DiskStoreSuite extends SparkFunSuite { @@ -34,8 +32,7 @@ class DiskStoreSuite extends SparkFunSuite { val byteBuffer = ByteBuffer.wrap(bytes) val blockId = BlockId("rdd_1_2") - val blockManager = mock(classOf[BlockManager], RETURNS_SMART_NULLS) - val diskBlockManager = new DiskBlockManager(blockManager, new SparkConf()) + val diskBlockManager = new DiskBlockManager(new SparkConf(), deleteFilesOnStop = true) val diskStoreMapped = new DiskStore(new SparkConf().set(confKey, "0"), diskBlockManager) diskStoreMapped.putBytes(blockId, byteBuffer) From 9e3ae78f62310aa291833842113c9832ac520bfa Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Thu, 3 Mar 2016 19:28:49 -0800 Subject: [PATCH 13/42] Shorten period of holding memoryManager lock. --- .../apache/spark/storage/MemoryStore.scala | 69 +++++++------------ 1 file changed, 26 insertions(+), 43 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala index e1ee38de31fb4..51172c261d181 100644 --- a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala @@ -313,19 +313,26 @@ private[spark] class MemoryStore(blockManager: BlockManager, memoryManager: Memo blockId.asRDDId.map(_.rddId) } + private def acquireStorageMemory(blockId: BlockId, size: Long): Boolean = { + // Synchronize on memoryManager so that the pending unroll memory isn't stolen by another + // task. + memoryManager.synchronized { + // Note: if we have previously unrolled this block successfully, then pending unroll + // memory should be non-zero. This is the amount that we already reserved during the + // unrolling process. In this case, we can just reuse this space to cache our block. + // The synchronization on `memoryManager` here guarantees that the release and acquire + // happen atomically. This relies on the assumption that all memory acquisitions are + // synchronized on the same lock. + releasePendingUnrollMemoryForThisTask() + memoryManager.acquireStorageMemory(blockId, size) + } + } + /** * Try to put in a set of values, if we can free up enough space. The value should either be * an Array if deserialized is true or a ByteBuffer otherwise. Its (possibly estimated) size * must also be passed by the caller. * - * `value` will be lazily created. If it cannot be put into MemoryStore or disk, `value` won't be - * created to avoid OOM since it may be a big ByteBuffer. - * - * Synchronize on `memoryManager` to ensure that all the put requests and its associated block - * dropping is done by only on thread at a time. Otherwise while one thread is dropping - * blocks to free memory for one block, another thread may use up the freed space for - * another block. - * * @return whether put was successful. */ private def tryToPut( @@ -333,42 +340,18 @@ private[spark] class MemoryStore(blockManager: BlockManager, memoryManager: Memo value: () => Any, size: Long, deserialized: Boolean): Boolean = { - - /* TODO: Its possible to optimize the locking by locking entries only when selecting blocks - * to be dropped. Once the to-be-dropped blocks have been selected, and lock on entries has - * been released, it must be ensured that those to-be-dropped blocks are not double counted - * for freeing up more space for another block that needs to be put. Only then the actually - * dropping of blocks (and writing to disk if necessary) can proceed in parallel. */ - - memoryManager.synchronized { - // Note: if we have previously unrolled this block successfully, then pending unroll - // memory should be non-zero. This is the amount that we already reserved during the - // unrolling process. In this case, we can just reuse this space to cache our block. - // The synchronization on `memoryManager` here guarantees that the release and acquire - // happen atomically. This relies on the assumption that all memory acquisitions are - // synchronized on the same lock. - releasePendingUnrollMemoryForThisTask() - val enoughMemory = memoryManager.acquireStorageMemory(blockId, size) - if (enoughMemory) { - // We acquired enough memory for the block, so go ahead and put it - val entry = new MemoryEntry(value(), size, deserialized) - entries.synchronized { - entries.put(blockId, entry) - } - val valuesOrBytes = if (deserialized) "values" else "bytes" - logInfo("Block %s stored as %s in memory (estimated size %s, free %s)".format( - blockId, valuesOrBytes, Utils.bytesToString(size), Utils.bytesToString(blocksMemoryUsed))) - } else { - // Tell the block manager that we couldn't put it in memory so that it can drop it to - // disk if the block allows disk storage. - lazy val data = if (deserialized) { - Left(value().asInstanceOf[Array[Any]]) - } else { - Right(value().asInstanceOf[ByteBuffer].duplicate()) - } - blockManager.dropFromMemory(blockId, () => data) + if (acquireStorageMemory(blockId, size)) { + // We acquired enough memory for the block, so go ahead and put it + val entry = new MemoryEntry(value(), size, deserialized) + entries.synchronized { + entries.put(blockId, entry) } - enoughMemory + val valuesOrBytes = if (deserialized) "values" else "bytes" + logInfo("Block %s stored as %s in memory (estimated size %s, free %s)".format( + blockId, valuesOrBytes, Utils.bytesToString(size), Utils.bytesToString(blocksMemoryUsed))) + true + } else { + false } } From d8487d4e8ee5bb5b64d60f1158fc7420ac6a2a54 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Thu, 3 Mar 2016 19:48:35 -0800 Subject: [PATCH 14/42] MemoryStore.put() no longer handles dropping to disk. This is now handled by the caller. --- .../apache/spark/storage/BlockManager.scala | 19 +++++++++-- .../apache/spark/storage/MemoryStore.scala | 34 +++++++------------ .../spark/storage/BlockManagerSuite.scala | 13 +++---- 3 files changed, 33 insertions(+), 33 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 6ffb62a1581ed..22d8f2bf2bcf3 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -514,7 +514,7 @@ private[spark] class BlockManager( val valuesToReturn: Iterator[Any] = { if (level.deserialized) { // Cache the values before returning them - memoryStore.putIterator(blockId, values, level, allowPersistToDisk = false) match { + memoryStore.putIterator(blockId, values, level) match { case Left(iter) => // The memory store put() failed, so it returned the iterator back to us: iter @@ -831,12 +831,25 @@ private[spark] class BlockManager( case Right(s) => size = s case Left(iter) => - iteratorFromFailedMemoryStorePut = Some(iter) + // Not enough space to unroll this block; drop to disk if applicable + if (level.useDisk) { + logWarning(s"Persisting block $blockId to disk instead.") + diskStore.put(blockId) { fileOutputStream => + dataSerializeStream(blockId, fileOutputStream, iter) + } + Right(diskStore.getSize(blockId)) + } else { + iteratorFromFailedMemoryStorePut = Some(iter) + } } case ByteBufferValues(bytes) => bytes.rewind() size = bytes.limit() - memoryStore.putBytes(blockId, bytes, putLevel) + val putSucceeded = memoryStore.putBytes(blockId, bytes, putLevel) + if (!putSucceeded && level.useDisk) { + logWarning(s"Persisting block $blockId to disk instead.") + diskStore.putBytes(blockId, bytes) + } } } else if (putLevel.useDisk) { data match { diff --git a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala index 51172c261d181..0b74b01a1ad25 100644 --- a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala @@ -87,14 +87,19 @@ private[spark] class MemoryStore(blockManager: BlockManager, memoryManager: Memo } } - def putBytes(blockId: BlockId, _bytes: ByteBuffer, level: StorageLevel): Unit = { + /** + * Attempt to put the given block into the memory store. + * + * @return true if the put() succeeded, false otherwise. + */ + def putBytes(blockId: BlockId, _bytes: ByteBuffer, level: StorageLevel): Boolean = { require(!contains(blockId), s"Block $blockId is already present in the MemoryStore") // Work on a duplicate - since the original input might be used elsewhere. val bytes = _bytes.duplicate() bytes.rewind() if (level.deserialized) { val values = blockManager.dataDeserialize(blockId, bytes) - putIterator(blockId, values, level) + putIterator(blockId, values, level).isRight } else { tryToPut(blockId, () => bytes, bytes.limit, deserialized = false) } @@ -119,20 +124,14 @@ private[spark] class MemoryStore(blockManager: BlockManager, memoryManager: Memo /** * Attempt to put the given block in memory store. * - * There may not be enough space to fully unroll the iterator in memory, in which case we - * optionally drop the values to disk if - * (1) the block's storage level specifies useDisk, and - * (2) `allowPersistToDisk` is true. - * - * One scenario in which `allowPersistToDisk` is false is when the BlockManager reads a block - * back from disk and attempts to cache it in memory. In this case, we should not persist the - * block back on disk again, as it is already in disk store. + * @return the estimated size of the stored data if the put() succeeded, or an iterator + * in case the put() failed (the returned iterator lets callers fall back to the disk + * store if desired). */ private[storage] def putIterator( blockId: BlockId, values: Iterator[Any], - level: StorageLevel, - allowPersistToDisk: Boolean = true): Either[Iterator[Any], Long] = { + level: StorageLevel): Either[Iterator[Any], Long] = { require(!contains(blockId), s"Block $blockId is already present in the MemoryStore") val unrolledValues = unrollSafely(blockId, values) unrolledValues match { @@ -151,16 +150,7 @@ private[spark] class MemoryStore(blockManager: BlockManager, memoryManager: Memo } Right(size) case Right(iteratorValues) => - // Not enough space to unroll this block; drop to disk if applicable - if (level.useDisk && allowPersistToDisk) { - logWarning(s"Persisting block $blockId to disk instead.") - blockManager.diskStore.put(blockId) { fileOutputStream => - blockManager.dataSerializeStream(blockId, fileOutputStream, iteratorValues) - } - Right(blockManager.diskStore.getSize(blockId)) - } else { - Left(iteratorValues) - } + Left(iteratorValues) } } diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index 1c00f5fb7c9ce..8d56fa48b0a6d 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -1187,19 +1187,16 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE store.putIterator("b3", smallIterator, StorageLevel.MEMORY_ONLY) assert(memoryStore.currentUnrollMemoryForThisTask === 0) - // Unroll huge block with not enough space. This should fail and drop the new block to disk - // directly in addition to kicking out b2 in the process. Memory store should contain only - // b3, while disk store should contain b1, b2 and b4. + // Unroll huge block with not enough space. This should fail and return an iterator so that + // the block may be stored to disk. During the unrolling process, block "b2" should be kicked + // out, so the memory store should contain only b3, while the disk store should contain + // b1, b2 and b4. val result4 = memoryStore.putIterator("b4", bigIterator, memAndDisk) - assert(result4.isRight) + assert(result4.isLeft) assert(!memoryStore.contains("b1")) assert(!memoryStore.contains("b2")) assert(memoryStore.contains("b3")) assert(!memoryStore.contains("b4")) - assert(diskStore.contains("b1")) - assert(diskStore.contains("b2")) - assert(!diskStore.contains("b3")) - assert(diskStore.contains("b4")) assert(memoryStore.currentUnrollMemoryForThisTask > 0) // we returned an iterator } From 10a667d62642ab478ab00b5e0267be93d6b01417 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Thu, 3 Mar 2016 20:06:36 -0800 Subject: [PATCH 15/42] MemoryStore.putBytes() shouldn't perform deserialization. --- .../apache/spark/storage/BlockManager.scala | 7 +++++- .../apache/spark/storage/MemoryStore.scala | 23 ++++--------------- 2 files changed, 10 insertions(+), 20 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 22d8f2bf2bcf3..b4635301b6ec0 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -845,7 +845,12 @@ private[spark] class BlockManager( case ByteBufferValues(bytes) => bytes.rewind() size = bytes.limit() - val putSucceeded = memoryStore.putBytes(blockId, bytes, putLevel) + val putSucceeded = if (level.deserialized) { + val values = dataDeserialize(blockId, bytes.duplicate()) + memoryStore.putIterator(blockId, values, level).isRight + } else { + memoryStore.putBytes(blockId, size, () => bytes) + } if (!putSucceeded && level.useDisk) { logWarning(s"Persisting block $blockId to disk instead.") diskStore.putBytes(blockId, bytes) diff --git a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala index 0b74b01a1ad25..ee85a284de77a 100644 --- a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala @@ -87,31 +87,15 @@ private[spark] class MemoryStore(blockManager: BlockManager, memoryManager: Memo } } - /** - * Attempt to put the given block into the memory store. - * - * @return true if the put() succeeded, false otherwise. - */ - def putBytes(blockId: BlockId, _bytes: ByteBuffer, level: StorageLevel): Boolean = { - require(!contains(blockId), s"Block $blockId is already present in the MemoryStore") - // Work on a duplicate - since the original input might be used elsewhere. - val bytes = _bytes.duplicate() - bytes.rewind() - if (level.deserialized) { - val values = blockManager.dataDeserialize(blockId, bytes) - putIterator(blockId, values, level).isRight - } else { - tryToPut(blockId, () => bytes, bytes.limit, deserialized = false) - } - } - /** * Use `size` to test if there is enough space in MemoryStore. If so, create the ByteBuffer and * put it into MemoryStore. Otherwise, the ByteBuffer won't be created. * * The caller should guarantee that `size` is correct. + * + * @return true if the put() succeeded, false otherwise. */ - def putBytes(blockId: BlockId, size: Long, _bytes: () => ByteBuffer): Unit = { + def putBytes(blockId: BlockId, size: Long, _bytes: () => ByteBuffer): Boolean = { require(!contains(blockId), s"Block $blockId is already present in the MemoryStore") // Work on a duplicate - since the original input might be used elsewhere. lazy val bytes = _bytes().duplicate().rewind().asInstanceOf[ByteBuffer] @@ -119,6 +103,7 @@ private[spark] class MemoryStore(blockManager: BlockManager, memoryManager: Memo if (putSuccess) { assert(bytes.limit == size) } + putSuccess } /** From 87e775d585d2db7c91af9c2587df2eb395040248 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Fri, 4 Mar 2016 11:31:46 -0800 Subject: [PATCH 16/42] MemoryStore should take its own conf, not obtain it from BlockManager. --- .../scala/org/apache/spark/storage/BlockManager.scala | 2 +- .../main/scala/org/apache/spark/storage/MemoryStore.scala | 8 +++++--- 2 files changed, 6 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index b4635301b6ec0..638e8a785f540 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -94,7 +94,7 @@ private[spark] class BlockManager( ThreadUtils.newDaemonCachedThreadPool("block-manager-future", 128)) // Actual storage of where blocks are kept - private[spark] val memoryStore = new MemoryStore(this, memoryManager) + private[spark] val memoryStore = new MemoryStore(conf, this, memoryManager) private[spark] val diskStore = new DiskStore(conf, diskBlockManager) memoryManager.setMemoryStore(memoryStore) diff --git a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala index ee85a284de77a..e5143ddd97871 100644 --- a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala @@ -23,7 +23,7 @@ import java.util.LinkedHashMap import scala.collection.mutable import scala.collection.mutable.ArrayBuffer -import org.apache.spark.{Logging, TaskContext} +import org.apache.spark.{Logging, SparkConf, TaskContext} import org.apache.spark.memory.MemoryManager import org.apache.spark.util.{SizeEstimator, Utils} import org.apache.spark.util.collection.SizeTrackingVector @@ -34,13 +34,15 @@ private case class MemoryEntry(value: Any, size: Long, deserialized: Boolean) * Stores blocks in memory, either as Arrays of deserialized Java objects or as * serialized ByteBuffers. */ -private[spark] class MemoryStore(blockManager: BlockManager, memoryManager: MemoryManager) +private[spark] class MemoryStore( + conf: SparkConf, + blockManager: BlockManager, + memoryManager: MemoryManager) extends Logging { // Note: all changes to memory allocations, notably putting blocks, evicting blocks, and // acquiring or releasing unroll memory, must be synchronized on `memoryManager`! - private val conf = blockManager.conf private val entries = new LinkedHashMap[BlockId, MemoryEntry](32, 0.75f, true) // A mapping from taskAttemptId to amount of memory used for unrolling a block (in bytes) From 2923850c27931cd8efb49449b19438e82763c39e Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Fri, 4 Mar 2016 11:53:47 -0800 Subject: [PATCH 17/42] Move MemoryManager into new o.a.s.storage.memory package --- .../main/scala/org/apache/spark/memory/MemoryManager.scala | 4 +++- .../scala/org/apache/spark/memory/StorageMemoryPool.scala | 4 +++- .../main/scala/org/apache/spark/storage/BlockManager.scala | 1 + .../org/apache/spark/storage/{ => memory}/MemoryStore.scala | 3 ++- .../scala/org/apache/spark/memory/MemoryManagerSuite.scala | 3 ++- .../org/apache/spark/memory/StaticMemoryManagerSuite.scala | 3 ++- .../org/apache/spark/memory/UnifiedMemoryManagerSuite.scala | 3 ++- 7 files changed, 15 insertions(+), 6 deletions(-) rename core/src/main/scala/org/apache/spark/storage/{ => memory}/MemoryStore.scala (99%) diff --git a/core/src/main/scala/org/apache/spark/memory/MemoryManager.scala b/core/src/main/scala/org/apache/spark/memory/MemoryManager.scala index b5adbd88a2c23..e89b03e38b468 100644 --- a/core/src/main/scala/org/apache/spark/memory/MemoryManager.scala +++ b/core/src/main/scala/org/apache/spark/memory/MemoryManager.scala @@ -20,7 +20,8 @@ package org.apache.spark.memory import javax.annotation.concurrent.GuardedBy import org.apache.spark.{Logging, SparkConf} -import org.apache.spark.storage.{BlockId, MemoryStore} +import org.apache.spark.storage.BlockId +import org.apache.spark.storage.memory.MemoryStore import org.apache.spark.unsafe.array.ByteArrayMethods import org.apache.spark.unsafe.memory.MemoryAllocator @@ -113,6 +114,7 @@ private[spark] abstract class MemoryManager( /** * Release all memory for the given task and mark it as inactive (e.g. when a task ends). + * * @return the number of bytes freed. */ private[memory] def releaseAllExecutionMemoryForTask(taskAttemptId: Long): Long = synchronized { diff --git a/core/src/main/scala/org/apache/spark/memory/StorageMemoryPool.scala b/core/src/main/scala/org/apache/spark/memory/StorageMemoryPool.scala index 6a88966f60d23..1d376adf1a12e 100644 --- a/core/src/main/scala/org/apache/spark/memory/StorageMemoryPool.scala +++ b/core/src/main/scala/org/apache/spark/memory/StorageMemoryPool.scala @@ -20,7 +20,8 @@ package org.apache.spark.memory import javax.annotation.concurrent.GuardedBy import org.apache.spark.Logging -import org.apache.spark.storage.{BlockId, MemoryStore} +import org.apache.spark.storage.BlockId +import org.apache.spark.storage.memory.MemoryStore /** * Performs bookkeeping for managing an adjustable-size pool of memory that is used for storage @@ -55,6 +56,7 @@ private[memory] class StorageMemoryPool(lock: Object) extends MemoryPool(lock) w /** * Acquire N bytes of memory to cache the given block, evicting existing ones if necessary. + * * @return whether all N bytes were successfully granted. */ def acquireMemory(blockId: BlockId, numBytes: Long): Boolean = lock.synchronized { diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 638e8a785f540..33e1969da1056 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -40,6 +40,7 @@ import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo import org.apache.spark.rpc.RpcEnv import org.apache.spark.serializer.{Serializer, SerializerInstance} import org.apache.spark.shuffle.ShuffleManager +import org.apache.spark.storage.memory._ import org.apache.spark.util._ private[spark] sealed trait BlockValues diff --git a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala similarity index 99% rename from core/src/main/scala/org/apache/spark/storage/MemoryStore.scala rename to core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala index e5143ddd97871..38a69c66c96bc 100644 --- a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.storage +package org.apache.spark.storage.memory import java.nio.ByteBuffer import java.util.LinkedHashMap @@ -25,6 +25,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.{Logging, SparkConf, TaskContext} import org.apache.spark.memory.MemoryManager +import org.apache.spark.storage.{BlockId, BlockManager, StorageLevel} import org.apache.spark.util.{SizeEstimator, Utils} import org.apache.spark.util.collection.SizeTrackingVector diff --git a/core/src/test/scala/org/apache/spark/memory/MemoryManagerSuite.scala b/core/src/test/scala/org/apache/spark/memory/MemoryManagerSuite.scala index d9764c7c10983..686e948b5df70 100644 --- a/core/src/test/scala/org/apache/spark/memory/MemoryManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/memory/MemoryManagerSuite.scala @@ -31,7 +31,8 @@ import org.scalatest.BeforeAndAfterEach import org.scalatest.time.SpanSugar._ import org.apache.spark.SparkFunSuite -import org.apache.spark.storage.{BlockId, BlockStatus, MemoryStore, StorageLevel} +import org.apache.spark.storage.{BlockId, BlockStatus, StorageLevel} +import org.apache.spark.storage.memory.MemoryStore /** diff --git a/core/src/test/scala/org/apache/spark/memory/StaticMemoryManagerSuite.scala b/core/src/test/scala/org/apache/spark/memory/StaticMemoryManagerSuite.scala index eee78d396e147..741d4fdf78197 100644 --- a/core/src/test/scala/org/apache/spark/memory/StaticMemoryManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/memory/StaticMemoryManagerSuite.scala @@ -20,7 +20,8 @@ package org.apache.spark.memory import org.mockito.Mockito.when import org.apache.spark.SparkConf -import org.apache.spark.storage.{MemoryStore, TestBlockId} +import org.apache.spark.storage.TestBlockId +import org.apache.spark.storage.memory.MemoryStore class StaticMemoryManagerSuite extends MemoryManagerSuite { private val conf = new SparkConf().set("spark.storage.unrollFraction", "0.4") diff --git a/core/src/test/scala/org/apache/spark/memory/UnifiedMemoryManagerSuite.scala b/core/src/test/scala/org/apache/spark/memory/UnifiedMemoryManagerSuite.scala index 9686c6621b465..9001a26652c92 100644 --- a/core/src/test/scala/org/apache/spark/memory/UnifiedMemoryManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/memory/UnifiedMemoryManagerSuite.scala @@ -20,7 +20,8 @@ package org.apache.spark.memory import org.scalatest.PrivateMethodTester import org.apache.spark.SparkConf -import org.apache.spark.storage.{MemoryStore, TestBlockId} +import org.apache.spark.storage.TestBlockId +import org.apache.spark.storage.memory.MemoryStore class UnifiedMemoryManagerSuite extends MemoryManagerSuite with PrivateMethodTester { private val dummyBlock = TestBlockId("--") From 40f4e436e2d99eebc41b5f8703936f8497b9443c Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Fri, 4 Mar 2016 14:33:28 -0800 Subject: [PATCH 18/42] getBytes() and getValues() no longer implicitly serialize / deserialize. --- .../org/apache/spark/storage/BlockManager.scala | 15 +++++++++++++-- .../apache/spark/storage/memory/MemoryStore.scala | 9 +++------ .../apache/spark/storage/BlockManagerSuite.scala | 3 +-- 3 files changed, 17 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 33e1969da1056..074cc8a80bb9b 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -463,12 +463,23 @@ private[spark] class BlockManager( if (level.useMemory) { logDebug(s"Getting block $blockId from memory") val result = if (asBlockResult) { - memoryStore.getValues(blockId).map { iter => + val maybeIter = if (level.deserialized) { + memoryStore.getValues(blockId) + } else { + memoryStore.getBytes(blockId).map { bytes => + dataDeserialize(blockId, bytes) + } + } + maybeIter.map { iter => val ci = CompletionIterator[Any, Iterator[Any]](iter, releaseLock(blockId)) new BlockResult(ci, DataReadMethod.Memory, info.size) } } else { - memoryStore.getBytes(blockId) + if (level.deserialized) { + memoryStore.getValues(blockId).map(iter => dataSerialize(blockId, iter)) + } else { + memoryStore.getBytes(blockId) + } } result match { case Some(values) => diff --git a/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala index 38a69c66c96bc..c1ea543b432f7 100644 --- a/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala @@ -148,9 +148,8 @@ private[spark] class MemoryStore( } if (entry == null) { None - } else if (entry.deserialized) { - Some(blockManager.dataSerialize(blockId, entry.value.asInstanceOf[Array[Any]].iterator)) } else { + require(!entry.deserialized, "should only call getBytes on blocks stored in serialized form") Some(entry.value.asInstanceOf[ByteBuffer].duplicate()) // Doesn't actually copy the data } } @@ -161,11 +160,9 @@ private[spark] class MemoryStore( } if (entry == null) { None - } else if (entry.deserialized) { - Some(entry.value.asInstanceOf[Array[Any]].iterator) } else { - val buffer = entry.value.asInstanceOf[ByteBuffer].duplicate() // Doesn't actually copy data - Some(blockManager.dataDeserialize(blockId, buffer)) + require(entry.deserialized, "should only call getValues on deserialized blocks") + Some(entry.value.asInstanceOf[Array[Any]].iterator) } } diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index 8d56fa48b0a6d..66b07f19a73ae 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -613,7 +613,6 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE store.putSingle("a3", a3, storageLevel) assert(accessMethod("a2").isDefined, "a2 was not in store") assert(accessMethod("a3").isDefined, "a3 was not in store") - assert(!store.memoryStore.contains("a1"), "a1 was in memory store") assert(accessMethod("a1").isDefined, "a1 was not in store") val dataShouldHaveBeenCachedBackIntoMemory = { if (storageLevel.deserialized) !getAsBytes @@ -734,7 +733,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE store.putSingle("a1", new Array[Byte](10000), StorageLevel.MEMORY_ONLY) assert(store.getSingleAndReleaseLock("a1") === None, "a1 was in store") store.putSingle("a2", new Array[Byte](10000), StorageLevel.MEMORY_AND_DISK) - assert(store.memoryStore.getValues("a2") === None, "a2 was in memory store") + assert(!store.memoryStore.contains("a2"), "a2 was in memory store") assert(store.getSingleAndReleaseLock("a2").isDefined, "a2 was not in store") } From 495ad976699ab05a8b452c39c65ebcc13c1718db Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Fri, 4 Mar 2016 16:00:00 -0800 Subject: [PATCH 19/42] Split doGetLocal() and getLocal() into smaller, simpler methods. --- .../spark/broadcast/TorrentBroadcast.scala | 2 +- .../apache/spark/storage/BlockManager.scala | 192 +++++++----------- .../spark/broadcast/BroadcastSuite.scala | 2 +- .../BlockManagerReplicationSuite.scala | 2 +- .../spark/storage/BlockManagerSuite.scala | 2 +- .../streaming/ReceivedBlockHandlerSuite.scala | 4 +- 6 files changed, 81 insertions(+), 123 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala index dabc81001834b..550e1ba6d3de0 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala @@ -173,7 +173,7 @@ private[spark] class TorrentBroadcast[T: ClassTag](obj: T, id: Long) TorrentBroadcast.synchronized { setConf(SparkEnv.get.conf) val blockManager = SparkEnv.get.blockManager - blockManager.getLocal(broadcastId).map(_.data.next()) match { + blockManager.getLocalValues(broadcastId).map(_.data.next()) match { case Some(x) => releaseLock(broadcastId) x.asInstanceOf[T] diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 074cc8a80bb9b..8aa2152fa0d3f 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -291,13 +291,9 @@ private[spark] class BlockManager( if (blockId.isShuffle) { shuffleManager.shuffleBlockResolver.getBlockData(blockId.asInstanceOf[ShuffleBlockId]) } else { - val blockBytesOpt = doGetLocal(blockId, asBlockResult = false) - .asInstanceOf[Option[ByteBuffer]] - if (blockBytesOpt.isDefined) { - val buffer = blockBytesOpt.get - new BlockManagerManagedBuffer(this, blockId, buffer) - } else { - throw new BlockNotFoundException(blockId.toString) + getLocalBytes(blockId) match { + case Some(buffer) => new BlockManagerManagedBuffer(this, blockId, buffer) + case None => throw new BlockNotFoundException(blockId.toString) } } } @@ -413,11 +409,49 @@ private[spark] class BlockManager( } /** - * Get block from local block manager. + * Get block from local block manager as an iterator of Java objects. */ - def getLocal(blockId: BlockId): Option[BlockResult] = { + def getLocalValues(blockId: BlockId): Option[BlockResult] = { logDebug(s"Getting local block $blockId") - doGetLocal(blockId, asBlockResult = true).asInstanceOf[Option[BlockResult]] + blockInfoManager.lockForReading(blockId) match { + case None => + logDebug(s"Block $blockId was not found") + None + case Some(info) => + val level = info.level + logDebug(s"Level for block $blockId is $level") + if (level.useMemory && memoryStore.contains(blockId)) { + val iter: Iterator[Any] = if (level.deserialized) { + memoryStore.getValues(blockId).get + } else { + dataDeserialize(blockId, memoryStore.getBytes(blockId).get) + } + val ci = CompletionIterator[Any, Iterator[Any]](iter, releaseLock(blockId)) + Some(new BlockResult(ci, DataReadMethod.Memory, info.size)) + } else if (level.useDisk && diskStore.contains(blockId)) { + val valuesFromDisk = dataDeserialize(blockId, diskStore.getBytes(blockId)) + val iterToReturn: Iterator[Any] = { + if (level.deserialized) { + // Cache the values before returning them + memoryStore.putIterator(blockId, valuesFromDisk, level) match { + case Left(iter) => + // The memory store put() failed, so it returned the iterator back to us: + iter + case Right(_) => + // The put() succeeded, so we can read the values back: + memoryStore.getValues(blockId).get + } + } else { + valuesFromDisk + } + } + val ci = CompletionIterator[Any, Iterator[Any]](iterToReturn, releaseLock(blockId)) + Some(new BlockResult(ci, DataReadMethod.Disk, info.size)) + } else { + releaseLock(blockId) + throw new SparkException(s"Block $blockId was not found even though it's read-locked") + } + } } /** @@ -434,82 +468,38 @@ private[spark] class BlockManager( Option( shuffleBlockResolver.getBlockData(blockId.asInstanceOf[ShuffleBlockId]).nioByteBuffer()) } else { - doGetLocal(blockId, asBlockResult = false).asInstanceOf[Option[ByteBuffer]] - } - } - - private def doGetLocal(blockId: BlockId, asBlockResult: Boolean): Option[Any] = { - blockInfoManager.lockForReading(blockId) match { - case None => - logDebug(s"Block $blockId was not found") - None - case Some(info) => - doGetLocal(blockId, info, asBlockResult) + blockInfoManager.lockForReading(blockId) match { + case None => + logDebug(s"Block $blockId was not found") + None + case Some(info) => + Some(doGetLocalBytes(blockId, info)) + } } } - /** - * Get a local block from the block manager. - * Assumes that the caller holds a read lock on the block. - */ - private def doGetLocal( - blockId: BlockId, - info: BlockInfo, - asBlockResult: Boolean): Option[Any] = { + private def doGetLocalBytes(blockId: BlockId, info: BlockInfo): ByteBuffer = { val level = info.level logDebug(s"Level for block $blockId is $level") - - // Look for the block in memory - if (level.useMemory) { - logDebug(s"Getting block $blockId from memory") - val result = if (asBlockResult) { - val maybeIter = if (level.deserialized) { - memoryStore.getValues(blockId) - } else { - memoryStore.getBytes(blockId).map { bytes => - dataDeserialize(blockId, bytes) - } - } - maybeIter.map { iter => - val ci = CompletionIterator[Any, Iterator[Any]](iter, releaseLock(blockId)) - new BlockResult(ci, DataReadMethod.Memory, info.size) - } + if (level.deserialized) { + // Try to avoid expensive serialization by reading a pre-serialized copy from disk: + if (level.useDisk && diskStore.contains(blockId)) { + diskStore.getBytes(blockId) + } else if (level.useMemory && memoryStore.contains(blockId)) { + // The block was not found on disk, so serialize an in-memory copy: + dataSerialize(blockId, memoryStore.getValues(blockId).get) } else { - if (level.deserialized) { - memoryStore.getValues(blockId).map(iter => dataSerialize(blockId, iter)) - } else { - memoryStore.getBytes(blockId) - } + releaseLock(blockId) + throw new SparkException(s"Block $blockId was not found even though it's read-locked") } - result match { - case Some(values) => - return result - case None => - logDebug(s"Block $blockId not found in memory") - } - } - - // Look for block on disk, potentially storing it back in memory if required - if (level.useDisk) { - logDebug(s"Getting block $blockId from disk") - val bytes: ByteBuffer = diskStore.getBytes(blockId) - assert(0 == bytes.position()) - - if (!level.useMemory) { - // If the block shouldn't be stored in memory, we can just return it - if (asBlockResult) { - val iter = CompletionIterator[Any, Iterator[Any]]( - dataDeserialize(blockId, bytes), releaseLock(blockId)) - return Some(new BlockResult(iter, DataReadMethod.Disk, info.size)) - } else { - return Some(bytes) - } - } else { - // Otherwise, we also have to store something in the memory store - if (!level.deserialized && !asBlockResult) { - /* We'll store the bytes in memory if the block's storage level includes - * "memory serialized" and we requested its serialized bytes. */ - memoryStore.putBytes(blockId, bytes.limit, () => { + } else { // storage level is serialized + if (level.useMemory && memoryStore.contains(blockId)) { + memoryStore.getBytes(blockId).get + } else if (level.useDisk && diskStore.contains(blockId)) { + val bytes = diskStore.getBytes(blockId) + if (level.useMemory) { + // Cache the bytes back into memory to speed up subsequent reads. + memoryStore.putBytes(blockId, bytes.limit(), () => { // https://issues.apache.org/jira/browse/SPARK-6076 // If the file size is bigger than the free memory, OOM will happen. So if we cannot // put it into MemoryStore, copyForMemory should not be created. That's why this @@ -519,37 +509,11 @@ private[spark] class BlockManager( }) bytes.rewind() } - if (!asBlockResult) { - return Some(bytes) - } else { - val values = dataDeserialize(blockId, bytes) - val valuesToReturn: Iterator[Any] = { - if (level.deserialized) { - // Cache the values before returning them - memoryStore.putIterator(blockId, values, level) match { - case Left(iter) => - // The memory store put() failed, so it returned the iterator back to us: - iter - case Right(_) => - // The put() succeeded, so we can read the values back: - memoryStore.getValues(blockId).get - } - } else { - values - } - } - val ci = CompletionIterator[Any, Iterator[Any]](valuesToReturn, releaseLock(blockId)) - return Some(new BlockResult(ci, DataReadMethod.Disk, info.size)) - } + bytes + } else { + releaseLock(blockId) + throw new SparkException(s"Block $blockId was not found even though it's read-locked") } - } else { - // This branch represents a case where the BlockInfoManager contained an entry for - // the block but the block could not be found in any of the block stores. This case - // should never occur, but for completeness's sake we address it here. - logError( - s"Block $blockId is supposedly stored locally but was not found in any block store") - releaseLock(blockId) - None } } @@ -629,7 +593,7 @@ private[spark] class BlockManager( * automatically be freed once the result's `data` iterator is fully consumed. */ def get(blockId: BlockId): Option[BlockResult] = { - val local = getLocal(blockId) + val local = getLocalValues(blockId) if (local.isDefined) { logInfo(s"Found block $blockId locally") return local @@ -919,13 +883,7 @@ private[spark] class BlockManager( Await.ready(replicationFuture, Duration.Inf) } else if (putLevel.replication > 1 && blockWasSuccessfullyStored) { val remoteStartTime = System.currentTimeMillis - val bytesToReplicate: ByteBuffer = { - doGetLocal(blockId, putBlockInfo, asBlockResult = false) - .map(_.asInstanceOf[ByteBuffer]) - .getOrElse { - throw new SparkException(s"Block $blockId was not found even though it was just stored") - } - } + val bytesToReplicate = doGetLocalBytes(blockId, putBlockInfo) try { replicate(blockId, bytesToReplicate, putLevel) } finally { diff --git a/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala b/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala index 88fdbbdaec902..f97cfbba3265b 100644 --- a/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala +++ b/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala @@ -37,7 +37,7 @@ class DummyBroadcastClass(rdd: RDD[Int]) extends Serializable { rdd.map { x => val bm = SparkEnv.get.blockManager // Check if broadcast block was fetched - val isFound = bm.getLocal(BroadcastBlockId(bid)).isDefined + val isFound = bm.getLocalValues(BroadcastBlockId(bid)).isDefined (x, isFound) }.collect().toSet } diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala index ae1faf5d98f38..aa47562ebfd90 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala @@ -366,7 +366,7 @@ class BlockManagerReplicationSuite extends SparkFunSuite with Matchers with Befo testStore => blockLocations.contains(testStore.blockManagerId.executorId) }.foreach { testStore => val testStoreName = testStore.blockManagerId.executorId - assert(testStore.getLocal(blockId).isDefined, s"$blockId was not found in $testStoreName") + assert(testStore.getLocalValues(blockId).isDefined, s"$blockId was not found in $testStoreName") testStore.releaseLock(blockId) assert(master.getLocations(blockId).map(_.executorId).toSet.contains(testStoreName), s"master does not have status for ${blockId.name} in $testStoreName") diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index 66b07f19a73ae..ef980449fe11f 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -1317,7 +1317,7 @@ private object BlockManagerSuite { getLocalAndReleaseLock(blockId).isDefined } - val getLocalAndReleaseLock: (BlockId) => Option[BlockResult] = wrapGet(store.getLocal) + val getLocalAndReleaseLock: (BlockId) => Option[BlockResult] = wrapGet(store.getLocalValues) val getAndReleaseLock: (BlockId) => Option[BlockResult] = wrapGet(store.get) val getSingleAndReleaseLock: (BlockId) => Option[Any] = wrapGet(store.getSingle) val getLocalBytesAndReleaseLock: (BlockId) => Option[ByteBuffer] = wrapGet(store.getLocalBytes) diff --git a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala index dd16fc3ecaf5d..61c74da58b66e 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala @@ -106,7 +106,7 @@ class ReceivedBlockHandlerSuite testBlockStoring(handler) { case (data, blockIds, storeResults) => // Verify the data in block manager is correct val storedData = blockIds.flatMap { blockId => - blockManager.getLocal(blockId).map(_.data.map(_.toString).toList).getOrElse(List.empty) + blockManager.getLocalValues(blockId).map(_.data.map(_.toString).toList).getOrElse(List.empty) }.toList storedData shouldEqual data @@ -130,7 +130,7 @@ class ReceivedBlockHandlerSuite testBlockStoring(handler) { case (data, blockIds, storeResults) => // Verify the data in block manager is correct val storedData = blockIds.flatMap { blockId => - blockManager.getLocal(blockId).map(_.data.map(_.toString).toList).getOrElse(List.empty) + blockManager.getLocalValues(blockId).map(_.data.map(_.toString).toList).getOrElse(List.empty) }.toList storedData shouldEqual data From 032e3a3b62e70b653a97bb2353c85087f9e4f843 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sat, 5 Mar 2016 11:05:50 -0800 Subject: [PATCH 20/42] Fix scalastyle violations. --- .../spark/storage/BlockManagerReplicationSuite.scala | 3 ++- .../spark/streaming/ReceivedBlockHandlerSuite.scala | 10 ++++++++-- 2 files changed, 10 insertions(+), 3 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala index aa47562ebfd90..b78a3648cd8bc 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala @@ -366,7 +366,8 @@ class BlockManagerReplicationSuite extends SparkFunSuite with Matchers with Befo testStore => blockLocations.contains(testStore.blockManagerId.executorId) }.foreach { testStore => val testStoreName = testStore.blockManagerId.executorId - assert(testStore.getLocalValues(blockId).isDefined, s"$blockId was not found in $testStoreName") + assert( + testStore.getLocalValues(blockId).isDefined, s"$blockId was not found in $testStoreName") testStore.releaseLock(blockId) assert(master.getLocations(blockId).map(_.executorId).toSet.contains(testStoreName), s"master does not have status for ${blockId.name} in $testStoreName") diff --git a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala index 61c74da58b66e..45424f9bac05a 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala @@ -106,7 +106,10 @@ class ReceivedBlockHandlerSuite testBlockStoring(handler) { case (data, blockIds, storeResults) => // Verify the data in block manager is correct val storedData = blockIds.flatMap { blockId => - blockManager.getLocalValues(blockId).map(_.data.map(_.toString).toList).getOrElse(List.empty) + blockManager + .getLocalValues(blockId) + .map(_.data.map(_.toString).toList) + .getOrElse(List.empty) }.toList storedData shouldEqual data @@ -130,7 +133,10 @@ class ReceivedBlockHandlerSuite testBlockStoring(handler) { case (data, blockIds, storeResults) => // Verify the data in block manager is correct val storedData = blockIds.flatMap { blockId => - blockManager.getLocalValues(blockId).map(_.data.map(_.toString).toList).getOrElse(List.empty) + blockManager + .getLocalValues(blockId) + .map(_.data.map(_.toString).toList) + .getOrElse(List.empty) }.toList storedData shouldEqual data From 988f00393676eabfc11e665f20f9ce26388e4c11 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sat, 5 Mar 2016 13:02:38 -0800 Subject: [PATCH 21/42] Fix leaked lock in getOrElseUpdate() when block already exists. --- .../main/scala/org/apache/spark/storage/BlockManager.scala | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 8aa2152fa0d3f..a1857628a79aa 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -652,12 +652,16 @@ private[spark] class BlockManager( case DoPutSucceeded => // doPut() didn't hand work back to us, so the block already existed or was successfully // stored. Therefore, we now hold a read lock on the block. - val blockResult = get(blockId).getOrElse { + val blockResult = getLocalValues(blockId).getOrElse { // Since we held a read lock between the doPut() and get() calls, the block should not // have been evicted, so get() not returning the block indicates some internal error. releaseLock(blockId) throw new SparkException(s"get() failed for block $blockId even though we held a lock") } + // We already hold a read lock on the block from the doPut() call and getLocalValues() + // acquires the lock again, so we need to call releaseLock() here so that the net number + // of lock acquisitions is 1 (since the caller will only call release() once). + releaseLock(blockId) Left(blockResult) case DoPutIteratorFailed(iter) => // The put failed, likely because the data was too large to fit in memory and could not be From 31a500834bab30d3a162885bfc88b08d2c7ffb0f Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 8 Mar 2016 11:13:44 -0800 Subject: [PATCH 22/42] Document lock requirements of doGetLocalBytes --- .../main/scala/org/apache/spark/storage/BlockManager.scala | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index a1857628a79aa..a9def9e7c595f 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -478,6 +478,12 @@ private[spark] class BlockManager( } } + /** + * Get block from the local block manager as serialized bytes. + * + * Must be called while holding a read lock on the block. + * Releases the read lock upon exception; keeps the read lock upon successful return. + */ private def doGetLocalBytes(blockId: BlockId, info: BlockInfo): ByteBuffer = { val level = info.level logDebug(s"Level for block $blockId is $level") From ca5a3f30fdf74694fa9bf5e1352133df3051257e Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 8 Mar 2016 11:15:26 -0800 Subject: [PATCH 23/42] Add clarifying comment to doGetLocalBytes() --- core/src/main/scala/org/apache/spark/storage/BlockManager.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index a9def9e7c595f..82671d700d673 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -487,6 +487,8 @@ private[spark] class BlockManager( private def doGetLocalBytes(blockId: BlockId, info: BlockInfo): ByteBuffer = { val level = info.level logDebug(s"Level for block $blockId is $level") + // In order, try to read the serialized bytes from memory, then from disk, then fall back to + // serializing in-memory objects, and, finally, throw an exception if the block does not exist. if (level.deserialized) { // Try to avoid expensive serialization by reading a pre-serialized copy from disk: if (level.useDisk && diskStore.contains(blockId)) { From 14857b3e53d1b496b7ff07099c53ab2d775f950a Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 8 Mar 2016 11:17:51 -0800 Subject: [PATCH 24/42] Remove unnecessary putBlockInfo.synchronized call --- .../apache/spark/storage/BlockManager.scala | 143 +++++++++--------- 1 file changed, 69 insertions(+), 74 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 82671d700d673..30c3cf0b052c2 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -805,87 +805,82 @@ private[spark] class BlockManager( var blockWasSuccessfullyStored = false var iteratorFromFailedMemoryStorePut: Option[Iterator[Any]] = None - putBlockInfo.synchronized { - logTrace("Put for block %s took %s to get into synchronized block" - .format(blockId, Utils.getUsedTimeMs(startTimeMs))) - - try { - if (putLevel.useMemory) { - // Put it in memory first, even if it also has useDisk set to true; - // We will drop it to disk later if the memory store can't hold it. - data match { - case IteratorValues(iterator) => - memoryStore.putIterator(blockId, iterator(), putLevel) match { - case Right(s) => - size = s - case Left(iter) => - // Not enough space to unroll this block; drop to disk if applicable - if (level.useDisk) { - logWarning(s"Persisting block $blockId to disk instead.") - diskStore.put(blockId) { fileOutputStream => - dataSerializeStream(blockId, fileOutputStream, iter) - } - Right(diskStore.getSize(blockId)) - } else { - iteratorFromFailedMemoryStorePut = Some(iter) + try { + if (putLevel.useMemory) { + // Put it in memory first, even if it also has useDisk set to true; + // We will drop it to disk later if the memory store can't hold it. + data match { + case IteratorValues(iterator) => + memoryStore.putIterator(blockId, iterator(), putLevel) match { + case Right(s) => + size = s + case Left(iter) => + // Not enough space to unroll this block; drop to disk if applicable + if (level.useDisk) { + logWarning(s"Persisting block $blockId to disk instead.") + diskStore.put(blockId) { fileOutputStream => + dataSerializeStream(blockId, fileOutputStream, iter) } - } - case ByteBufferValues(bytes) => - bytes.rewind() - size = bytes.limit() - val putSucceeded = if (level.deserialized) { - val values = dataDeserialize(blockId, bytes.duplicate()) - memoryStore.putIterator(blockId, values, level).isRight - } else { - memoryStore.putBytes(blockId, size, () => bytes) - } - if (!putSucceeded && level.useDisk) { - logWarning(s"Persisting block $blockId to disk instead.") - diskStore.putBytes(blockId, bytes) - } - } - } else if (putLevel.useDisk) { - data match { - case IteratorValues(iterator) => - diskStore.put(blockId) { fileOutputStream => - dataSerializeStream(blockId, fileOutputStream, iterator()) - } - size = diskStore.getSize(blockId) - case ByteBufferValues(bytes) => - bytes.rewind() - size = bytes.limit() + Right(diskStore.getSize(blockId)) + } else { + iteratorFromFailedMemoryStorePut = Some(iter) + } + } + case ByteBufferValues(bytes) => + bytes.rewind() + size = bytes.limit() + val putSucceeded = if (level.deserialized) { + val values = dataDeserialize(blockId, bytes.duplicate()) + memoryStore.putIterator(blockId, values, level).isRight + } else { + memoryStore.putBytes(blockId, size, () => bytes) + } + if (!putSucceeded && level.useDisk) { + logWarning(s"Persisting block $blockId to disk instead.") diskStore.putBytes(blockId, bytes) - } - } else { - assert(putLevel == StorageLevel.NONE) - throw new BlockException( - blockId, s"Attempted to put block $blockId without specifying storage level!") + } + } + } else if (putLevel.useDisk) { + data match { + case IteratorValues(iterator) => + diskStore.put(blockId) { fileOutputStream => + dataSerializeStream(blockId, fileOutputStream, iterator()) + } + size = diskStore.getSize(blockId) + case ByteBufferValues(bytes) => + bytes.rewind() + size = bytes.limit() + diskStore.putBytes(blockId, bytes) } + } else { + assert(putLevel == StorageLevel.NONE) + throw new BlockException( + blockId, s"Attempted to put block $blockId without specifying storage level!") + } - val putBlockStatus = getCurrentBlockStatus(blockId, putBlockInfo) - blockWasSuccessfullyStored = putBlockStatus.storageLevel.isValid - if (blockWasSuccessfullyStored) { - // Now that the block is in either the memory, externalBlockStore, or disk store, - // let other threads read it, and tell the master about it. - putBlockInfo.size = size - if (tellMaster) { - reportBlockStatus(blockId, putBlockInfo, putBlockStatus) - } - Option(TaskContext.get()).foreach { c => - c.taskMetrics().incUpdatedBlockStatuses(Seq((blockId, putBlockStatus))) - } + val putBlockStatus = getCurrentBlockStatus(blockId, putBlockInfo) + blockWasSuccessfullyStored = putBlockStatus.storageLevel.isValid + if (blockWasSuccessfullyStored) { + // Now that the block is in either the memory, externalBlockStore, or disk store, + // let other threads read it, and tell the master about it. + putBlockInfo.size = size + if (tellMaster) { + reportBlockStatus(blockId, putBlockInfo, putBlockStatus) } - } finally { - if (blockWasSuccessfullyStored) { - if (keepReadLock) { - blockInfoManager.downgradeLock(blockId) - } else { - blockInfoManager.unlock(blockId) - } + Option(TaskContext.get()).foreach { c => + c.taskMetrics().incUpdatedBlockStatuses(Seq((blockId, putBlockStatus))) + } + } + } finally { + if (blockWasSuccessfullyStored) { + if (keepReadLock) { + blockInfoManager.downgradeLock(blockId) } else { - blockInfoManager.removeBlock(blockId) - logWarning(s"Putting block $blockId failed") + blockInfoManager.unlock(blockId) } + } else { + blockInfoManager.removeBlock(blockId) + logWarning(s"Putting block $blockId failed") } } logDebug("Put block %s locally took %s".format(blockId, Utils.getUsedTimeMs(startTimeMs))) From 92c5125f2f4736335971e779fc39e9fa74f8c310 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 8 Mar 2016 15:34:05 -0800 Subject: [PATCH 25/42] Remove effectiveStorageLevel from put() APIs. --- .../apache/spark/storage/BlockManager.scala | 39 +++++++------------ 1 file changed, 13 insertions(+), 26 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 30c3cf0b052c2..2ac520c841047 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -688,15 +688,13 @@ private[spark] class BlockManager( blockId: BlockId, values: Iterator[Any], level: StorageLevel, - tellMaster: Boolean = true, - effectiveStorageLevel: Option[StorageLevel] = None): Boolean = { + tellMaster: Boolean = true): Boolean = { require(values != null, "Values is null") val result = doPut( blockId, IteratorValues(() => values), level, - tellMaster, - effectiveStorageLevel) + tellMaster) result == DoPutSucceeded } @@ -726,10 +724,9 @@ private[spark] class BlockManager( blockId: BlockId, bytes: ByteBuffer, level: StorageLevel, - tellMaster: Boolean = true, - effectiveStorageLevel: Option[StorageLevel] = None): Boolean = { + tellMaster: Boolean = true): Boolean = { require(bytes != null, "Bytes is null") - val result = doPut(blockId, ByteBufferValues(bytes), level, tellMaster, effectiveStorageLevel) + val result = doPut(blockId, ByteBufferValues(bytes), level, tellMaster) result == DoPutSucceeded } @@ -739,9 +736,6 @@ private[spark] class BlockManager( * * If the block already exists, this method will not overwrite it. * - * @param effectiveStorageLevel the level according to which the block will actually be handled. - * This allows the caller to specify an alternate behavior of doPut - * while preserving the original level specified by the user. * @param keepReadLock if true, this method will hold the read lock when it returns (even if the * block already exists). If false, this method will hold no locks when it * returns. @@ -754,14 +748,10 @@ private[spark] class BlockManager( data: BlockValues, level: StorageLevel, tellMaster: Boolean = true, - effectiveStorageLevel: Option[StorageLevel] = None, keepReadLock: Boolean = false): DoPutResult = { require(blockId != null, "BlockId is null") require(level != null && level.isValid, "StorageLevel is null or invalid") - effectiveStorageLevel.foreach { level => - require(level != null && level.isValid, "Effective StorageLevel is null or invalid") - } /* Remember the block's storage level so that we can correctly drop it to disk if it needs * to be dropped right after it got put into memory. Note, however, that other threads will @@ -785,19 +775,16 @@ private[spark] class BlockManager( // Size of the block in bytes var size = 0L - // The level we actually use to put the block - val putLevel = effectiveStorageLevel.getOrElse(level) - // If we're storing bytes, then initiate the replication before storing them locally. // This is faster as data is already serialized and ready to send. val replicationFuture = data match { - case b: ByteBufferValues if putLevel.replication > 1 => + case b: ByteBufferValues if level.replication > 1 => // Duplicate doesn't copy the bytes, but just creates a wrapper val bufferView = b.buffer.duplicate() Future { // This is a blocking action and should run in futureExecutionContext which is a cached // thread pool - replicate(blockId, bufferView, putLevel) + replicate(blockId, bufferView, level) }(futureExecutionContext) case _ => null } @@ -806,12 +793,12 @@ private[spark] class BlockManager( var iteratorFromFailedMemoryStorePut: Option[Iterator[Any]] = None try { - if (putLevel.useMemory) { + if (level.useMemory) { // Put it in memory first, even if it also has useDisk set to true; // We will drop it to disk later if the memory store can't hold it. data match { case IteratorValues(iterator) => - memoryStore.putIterator(blockId, iterator(), putLevel) match { + memoryStore.putIterator(blockId, iterator(), level) match { case Right(s) => size = s case Left(iter) => @@ -840,7 +827,7 @@ private[spark] class BlockManager( diskStore.putBytes(blockId, bytes) } } - } else if (putLevel.useDisk) { + } else if (level.useDisk) { data match { case IteratorValues(iterator) => diskStore.put(blockId) { fileOutputStream => @@ -853,7 +840,7 @@ private[spark] class BlockManager( diskStore.putBytes(blockId, bytes) } } else { - assert(putLevel == StorageLevel.NONE) + assert(level == StorageLevel.NONE) throw new BlockException( blockId, s"Attempted to put block $blockId without specifying storage level!") } @@ -888,11 +875,11 @@ private[spark] class BlockManager( if (replicationFuture != null) { // Wait for asynchronous replication to finish Await.ready(replicationFuture, Duration.Inf) - } else if (putLevel.replication > 1 && blockWasSuccessfullyStored) { + } else if (level.replication > 1 && blockWasSuccessfullyStored) { val remoteStartTime = System.currentTimeMillis val bytesToReplicate = doGetLocalBytes(blockId, putBlockInfo) try { - replicate(blockId, bytesToReplicate, putLevel) + replicate(blockId, bytesToReplicate, level) } finally { BlockManager.dispose(bytesToReplicate) } @@ -900,7 +887,7 @@ private[spark] class BlockManager( .format(blockId, Utils.getUsedTimeMs(remoteStartTime))) } - if (putLevel.replication > 1) { + if (level.replication > 1) { logDebug("Putting block %s with replication took %s" .format(blockId, Utils.getUsedTimeMs(startTimeMs))) } else { From 7a08a179f8951abbdcb7e70f6bfb53821fbc7352 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 8 Mar 2016 15:59:34 -0800 Subject: [PATCH 26/42] Split doPut() into doPutBytes() and doPutIterator(). --- .../apache/spark/storage/BlockManager.scala | 242 +++++++++++------- .../spark/storage/memory/MemoryStore.scala | 21 +- 2 files changed, 166 insertions(+), 97 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 2ac520c841047..4b297a0391290 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -43,22 +43,12 @@ import org.apache.spark.shuffle.ShuffleManager import org.apache.spark.storage.memory._ import org.apache.spark.util._ -private[spark] sealed trait BlockValues -private[spark] case class ByteBufferValues(buffer: ByteBuffer) extends BlockValues -private[spark] case class IteratorValues(iterator: () => Iterator[Any]) extends BlockValues - /* Class for returning a fetched block and associated metrics. */ private[spark] class BlockResult( val data: Iterator[Any], val readMethod: DataReadMethod.Value, val bytes: Long) -// Class for representing return value of doPut() -private sealed trait DoPutResult -private case object DoPutSucceeded extends DoPutResult -private case object DoPutBytesFailed extends DoPutResult -private case class DoPutIteratorFailed(iter: Iterator[Any]) extends DoPutResult - /** * Manager running on every node (driver and executors) which provides interfaces for putting and * retrieving blocks both locally and remotely into various stores (memory, disk, and off-heap). @@ -656,8 +646,8 @@ private[spark] class BlockManager( level: StorageLevel, makeIterator: () => Iterator[Any]): Either[BlockResult, Iterator[Any]] = { // Initially we hold no locks on this block. - doPut(blockId, IteratorValues(makeIterator), level, keepReadLock = true) match { - case DoPutSucceeded => + doPutIterator(blockId, makeIterator, level, keepReadLock = true) match { + case None => // doPut() didn't hand work back to us, so the block already existed or was successfully // stored. Therefore, we now hold a read lock on the block. val blockResult = getLocalValues(blockId).getOrElse { @@ -671,13 +661,11 @@ private[spark] class BlockManager( // of lock acquisitions is 1 (since the caller will only call release() once). releaseLock(blockId) Left(blockResult) - case DoPutIteratorFailed(iter) => + case Some(iter) => // The put failed, likely because the data was too large to fit in memory and could not be // dropped to disk. Therefore, we need to pass the input iterator back to the caller so // that they can decide what to do with the values (e.g. process them without caching). Right(iter) - case DoPutBytesFailed => - throw new SparkException("doPut returned an invalid failure response") } } @@ -690,12 +678,13 @@ private[spark] class BlockManager( level: StorageLevel, tellMaster: Boolean = true): Boolean = { require(values != null, "Values is null") - val result = doPut( + val maybeIterator = doPutIterator( blockId, - IteratorValues(() => values), + () => values, level, tellMaster) - result == DoPutSucceeded + // If doPut() didn't hand work back to us, then block already existed or was successfully stored + maybeIterator.isEmpty } /** @@ -726,8 +715,129 @@ private[spark] class BlockManager( level: StorageLevel, tellMaster: Boolean = true): Boolean = { require(bytes != null, "Bytes is null") - val result = doPut(blockId, ByteBufferValues(bytes), level, tellMaster) - result == DoPutSucceeded + doPutBytes(blockId, bytes, level, tellMaster) + } + + /** + * Put the given bytes according to the given level in one of the block stores, replicating + * the values if necessary. + * + * If the block already exists, this method will not overwrite it. + * + * @param keepReadLock if true, this method will hold the read lock when it returns (even if the + * block already exists). If false, this method will hold no locks when it + * returns. + * @return true if the block was already present or if the put succeeded, false otherwise. + */ + private def doPutBytes( + blockId: BlockId, + bytes: ByteBuffer, + level: StorageLevel, + tellMaster: Boolean = true, + keepReadLock: Boolean = false): Boolean = { + + require(blockId != null, "BlockId is null") + require(level != null && level.isValid, "StorageLevel is null or invalid") + + /* Remember the block's storage level so that we can correctly drop it to disk if it needs + * to be dropped right after it got put into memory. Note, however, that other threads will + * not be able to get() this block until we call markReady on its BlockInfo. */ + val putBlockInfo = { + val newInfo = new BlockInfo(level, tellMaster) + if (blockInfoManager.lockNewBlockForWriting(blockId, newInfo)) { + newInfo + } else { + logWarning(s"Block $blockId already exists on this machine; not re-adding it") + if (!keepReadLock) { + // lockNewBlockForWriting returned a read lock on the existing block, so we must free it: + releaseLock(blockId) + } + return true + } + } + + val startTimeMs = System.currentTimeMillis + + // Since we're storing bytes, initiate the replication before storing them locally. + // This is faster as data is already serialized and ready to send. + val replicationFuture = if (level.replication > 1) { + // Duplicate doesn't copy the bytes, but just creates a wrapper + val bufferView = bytes.duplicate() + Future { + // This is a blocking action and should run in futureExecutionContext which is a cached + // thread pool + replicate(blockId, bufferView, level) + }(futureExecutionContext) + } else { + null + } + + var blockWasSuccessfullyStored = false + + bytes.rewind() + val size = bytes.limit() + + try { + if (level.useMemory) { + // Put it in memory first, even if it also has useDisk set to true; + // We will drop it to disk later if the memory store can't hold it. + + val putSucceeded = if (level.deserialized) { + val values = dataDeserialize(blockId, bytes.duplicate()) + memoryStore.putIterator(blockId, values, level).isRight + } else { + memoryStore.putBytes(blockId, size, () => bytes) + } + if (!putSucceeded && level.useDisk) { + logWarning(s"Persisting block $blockId to disk instead.") + diskStore.putBytes(blockId, bytes) + } + } else if (level.useDisk) { + diskStore.putBytes(blockId, bytes) + } else { + assert(level == StorageLevel.NONE) + throw new BlockException( + blockId, s"Attempted to put block $blockId without specifying storage level!") + } + + val putBlockStatus = getCurrentBlockStatus(blockId, putBlockInfo) + blockWasSuccessfullyStored = putBlockStatus.storageLevel.isValid + if (blockWasSuccessfullyStored) { + // Now that the block is in either the memory, externalBlockStore, or disk store, + // let other threads read it, and tell the master about it. + putBlockInfo.size = size + if (tellMaster) { + reportBlockStatus(blockId, putBlockInfo, putBlockStatus) + } + Option(TaskContext.get()).foreach { c => + c.taskMetrics().incUpdatedBlockStatuses(Seq((blockId, putBlockStatus))) + } + } + } finally { + if (blockWasSuccessfullyStored) { + if (keepReadLock) { + blockInfoManager.downgradeLock(blockId) + } else { + blockInfoManager.unlock(blockId) + } + } else { + blockInfoManager.removeBlock(blockId) + logWarning(s"Putting block $blockId failed") + } + } + logDebug("Put block %s locally took %s".format(blockId, Utils.getUsedTimeMs(startTimeMs))) + + if (level.replication > 1) { + // Wait for asynchronous replication to finish + Await.ready(replicationFuture, Duration.Inf) + logDebug("Putting block %s with replication took %s" + .format(blockId, Utils.getUsedTimeMs(startTimeMs))) + } else { + logDebug("Putting block %s without replication took %s" + .format(blockId, Utils.getUsedTimeMs(startTimeMs))) + } + + blockWasSuccessfullyStored } /** @@ -739,16 +849,15 @@ private[spark] class BlockManager( * @param keepReadLock if true, this method will hold the read lock when it returns (even if the * block already exists). If false, this method will hold no locks when it * returns. - * @return [[DoPutSucceeded]] if the block was already present or if the put succeeded, or - * [[DoPutBytesFailed]] if the put failed and we were storing bytes, or - * [[DoPutIteratorFailed]] if the put failed and we were storing an iterator. + * @return None if the block was already present or if the put succeeded, or Some(iterator) + * if the put failed. */ - private def doPut( + private def doPutIterator( blockId: BlockId, - data: BlockValues, + iterator: () => Iterator[Any], level: StorageLevel, tellMaster: Boolean = true, - keepReadLock: Boolean = false): DoPutResult = { + keepReadLock: Boolean = false): Option[Iterator[Any]] = { require(blockId != null, "BlockId is null") require(level != null && level.isValid, "StorageLevel is null or invalid") @@ -766,7 +875,7 @@ private[spark] class BlockManager( // lockNewBlockForWriting returned a read lock on the existing block, so we must free it: releaseLock(blockId) } - return DoPutSucceeded + return None } } @@ -775,20 +884,6 @@ private[spark] class BlockManager( // Size of the block in bytes var size = 0L - // If we're storing bytes, then initiate the replication before storing them locally. - // This is faster as data is already serialized and ready to send. - val replicationFuture = data match { - case b: ByteBufferValues if level.replication > 1 => - // Duplicate doesn't copy the bytes, but just creates a wrapper - val bufferView = b.buffer.duplicate() - Future { - // This is a blocking action and should run in futureExecutionContext which is a cached - // thread pool - replicate(blockId, bufferView, level) - }(futureExecutionContext) - case _ => null - } - var blockWasSuccessfullyStored = false var iteratorFromFailedMemoryStorePut: Option[Iterator[Any]] = None @@ -796,49 +891,26 @@ private[spark] class BlockManager( if (level.useMemory) { // Put it in memory first, even if it also has useDisk set to true; // We will drop it to disk later if the memory store can't hold it. - data match { - case IteratorValues(iterator) => - memoryStore.putIterator(blockId, iterator(), level) match { - case Right(s) => - size = s - case Left(iter) => - // Not enough space to unroll this block; drop to disk if applicable - if (level.useDisk) { - logWarning(s"Persisting block $blockId to disk instead.") - diskStore.put(blockId) { fileOutputStream => - dataSerializeStream(blockId, fileOutputStream, iter) - } - Right(diskStore.getSize(blockId)) - } else { - iteratorFromFailedMemoryStorePut = Some(iter) - } - } - case ByteBufferValues(bytes) => - bytes.rewind() - size = bytes.limit() - val putSucceeded = if (level.deserialized) { - val values = dataDeserialize(blockId, bytes.duplicate()) - memoryStore.putIterator(blockId, values, level).isRight - } else { - memoryStore.putBytes(blockId, size, () => bytes) - } - if (!putSucceeded && level.useDisk) { + memoryStore.putIterator(blockId, iterator(), level) match { + case Right(s) => + size = s + case Left(iter) => + // Not enough space to unroll this block; drop to disk if applicable + if (level.useDisk) { logWarning(s"Persisting block $blockId to disk instead.") - diskStore.putBytes(blockId, bytes) + diskStore.put(blockId) { fileOutputStream => + dataSerializeStream(blockId, fileOutputStream, iter) + } + Right(diskStore.getSize(blockId)) + } else { + iteratorFromFailedMemoryStorePut = Some(iter) } } } else if (level.useDisk) { - data match { - case IteratorValues(iterator) => - diskStore.put(blockId) { fileOutputStream => - dataSerializeStream(blockId, fileOutputStream, iterator()) - } - size = diskStore.getSize(blockId) - case ByteBufferValues(bytes) => - bytes.rewind() - size = bytes.limit() - diskStore.putBytes(blockId, bytes) + diskStore.put(blockId) { fileOutputStream => + dataSerializeStream(blockId, fileOutputStream, iterator()) } + size = diskStore.getSize(blockId) } else { assert(level == StorageLevel.NONE) throw new BlockException( @@ -871,11 +943,7 @@ private[spark] class BlockManager( } } logDebug("Put block %s locally took %s".format(blockId, Utils.getUsedTimeMs(startTimeMs))) - - if (replicationFuture != null) { - // Wait for asynchronous replication to finish - Await.ready(replicationFuture, Duration.Inf) - } else if (level.replication > 1 && blockWasSuccessfullyStored) { + if (level.replication > 1 && blockWasSuccessfullyStored) { val remoteStartTime = System.currentTimeMillis val bytesToReplicate = doGetLocalBytes(blockId, putBlockInfo) try { @@ -896,11 +964,9 @@ private[spark] class BlockManager( } if (blockWasSuccessfullyStored) { - DoPutSucceeded - } else if (iteratorFromFailedMemoryStorePut.isDefined) { - DoPutIteratorFailed(iteratorFromFailedMemoryStorePut.get) + None } else { - DoPutBytesFailed + Some(iteratorFromFailedMemoryStorePut.get) } } diff --git a/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala index c1ea543b432f7..0462e729abf95 100644 --- a/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala @@ -125,18 +125,21 @@ private[spark] class MemoryStore( unrolledValues match { case Left(arrayValues) => // Values are fully unrolled in memory, so store them as an array - val size = { - if (level.deserialized) { - val sizeEstimate = SizeEstimator.estimate(arrayValues.asInstanceOf[AnyRef]) - tryToPut(blockId, () => arrayValues, sizeEstimate, deserialized = true) - sizeEstimate + if (level.deserialized) { + val sizeEstimate = SizeEstimator.estimate(arrayValues.asInstanceOf[AnyRef]) + if (tryToPut(blockId, () => arrayValues, sizeEstimate, deserialized = true)) { + Right(sizeEstimate) } else { - val bytes = blockManager.dataSerialize(blockId, arrayValues.iterator) - tryToPut(blockId, () => bytes, bytes.limit, deserialized = false) - bytes.limit() + Left(arrayValues.toIterator) + } + } else { + val bytes = blockManager.dataSerialize(blockId, arrayValues.iterator) + if (tryToPut(blockId, () => bytes, bytes.limit, deserialized = false)) { + Right(bytes.limit()) + } else { + Left(arrayValues.toIterator) } } - Right(size) case Right(iteratorValues) => Left(iteratorValues) } From a16276e1cfb92fba88f2963b0d918aa14f2500a4 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 8 Mar 2016 16:01:46 -0800 Subject: [PATCH 27/42] Remove unreachable level == StorageLevel.NONE case. This is unreachable because we check whether level.isValid earlier in the same method. --- .../scala/org/apache/spark/storage/BlockManager.scala | 9 --------- 1 file changed, 9 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 4b297a0391290..60eaaaf29467b 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -781,7 +781,6 @@ private[spark] class BlockManager( if (level.useMemory) { // Put it in memory first, even if it also has useDisk set to true; // We will drop it to disk later if the memory store can't hold it. - val putSucceeded = if (level.deserialized) { val values = dataDeserialize(blockId, bytes.duplicate()) memoryStore.putIterator(blockId, values, level).isRight @@ -794,10 +793,6 @@ private[spark] class BlockManager( } } else if (level.useDisk) { diskStore.putBytes(blockId, bytes) - } else { - assert(level == StorageLevel.NONE) - throw new BlockException( - blockId, s"Attempted to put block $blockId without specifying storage level!") } val putBlockStatus = getCurrentBlockStatus(blockId, putBlockInfo) @@ -911,10 +906,6 @@ private[spark] class BlockManager( dataSerializeStream(blockId, fileOutputStream, iterator()) } size = diskStore.getSize(blockId) - } else { - assert(level == StorageLevel.NONE) - throw new BlockException( - blockId, s"Attempted to put block $blockId without specifying storage level!") } val putBlockStatus = getCurrentBlockStatus(blockId, putBlockInfo) From 82886e03f60e2b4b0b97b0c6ae640ca10dede145 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 8 Mar 2016 16:03:43 -0800 Subject: [PATCH 28/42] Fix statement without side-effects. --- core/src/main/scala/org/apache/spark/storage/BlockManager.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 60eaaaf29467b..413096be20ee4 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -896,7 +896,7 @@ private[spark] class BlockManager( diskStore.put(blockId) { fileOutputStream => dataSerializeStream(blockId, fileOutputStream, iter) } - Right(diskStore.getSize(blockId)) + size = diskStore.getSize(blockId) } else { iteratorFromFailedMemoryStorePut = Some(iter) } From 66796b5bf89ddedc9644b9f5692441293c0c0aaa Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 8 Mar 2016 16:15:11 -0800 Subject: [PATCH 29/42] Minor comment reword. --- .../main/scala/org/apache/spark/storage/BlockManager.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 413096be20ee4..e361c1a4dbb36 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -799,7 +799,7 @@ private[spark] class BlockManager( blockWasSuccessfullyStored = putBlockStatus.storageLevel.isValid if (blockWasSuccessfullyStored) { // Now that the block is in either the memory, externalBlockStore, or disk store, - // let other threads read it, and tell the master about it. + // tell the master about it. putBlockInfo.size = size if (tellMaster) { reportBlockStatus(blockId, putBlockInfo, putBlockStatus) @@ -912,7 +912,7 @@ private[spark] class BlockManager( blockWasSuccessfullyStored = putBlockStatus.storageLevel.isValid if (blockWasSuccessfullyStored) { // Now that the block is in either the memory, externalBlockStore, or disk store, - // let other threads read it, and tell the master about it. + // tell the master about it. putBlockInfo.size = size if (tellMaster) { reportBlockStatus(blockId, putBlockInfo, putBlockStatus) From 94d2bd65f4fc70e75ddff4fe70c8fc4c68b481af Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Wed, 9 Mar 2016 15:41:26 -0800 Subject: [PATCH 30/42] Merge doGetRemote() with getRemoteBytes() --- .../apache/spark/storage/BlockManager.scala | 28 ++++++------------- 1 file changed, 9 insertions(+), 19 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index e361c1a4dbb36..4c817fdf73dcc 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -521,16 +521,9 @@ private[spark] class BlockManager( * This does not acquire a lock on this block in this JVM. */ def getRemote(blockId: BlockId): Option[BlockResult] = { - logDebug(s"Getting remote block $blockId") - doGetRemote(blockId, asBlockResult = true).asInstanceOf[Option[BlockResult]] - } - - /** - * Get block from remote block managers as serialized bytes. - */ - def getRemoteBytes(blockId: BlockId): Option[ByteBuffer] = { - logDebug(s"Getting remote block $blockId as bytes") - doGetRemote(blockId, asBlockResult = false).asInstanceOf[Option[ByteBuffer]] + getRemoteBytes(blockId).map { data => + new BlockResult(dataDeserialize(blockId, data), DataReadMethod.Network, data.limit()) + } } /** @@ -543,7 +536,11 @@ private[spark] class BlockManager( preferredLocs ++ otherLocs } - private def doGetRemote(blockId: BlockId, asBlockResult: Boolean): Option[Any] = { + /** + * Get block from remote block managers as serialized bytes. + */ + def getRemoteBytes(blockId: BlockId): Option[ByteBuffer] = { + logDebug(s"Getting remote block $blockId") require(blockId != null, "BlockId is null") val locations = getLocations(blockId) var numFetchFailures = 0 @@ -568,14 +565,7 @@ private[spark] class BlockManager( } if (data != null) { - if (asBlockResult) { - return Some(new BlockResult( - dataDeserialize(blockId, data), - DataReadMethod.Network, - data.limit())) - } else { - return Some(data) - } + return Some(data) } logDebug(s"The value of block $blockId is null") } From e3a4c223fcfd943266590a479f7e43d78bafabd5 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Wed, 9 Mar 2016 16:46:57 -0800 Subject: [PATCH 31/42] getRemote -> getRemoteValues() --- .../main/scala/org/apache/spark/storage/BlockManager.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 4c817fdf73dcc..94cc260fcce69 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -520,7 +520,7 @@ private[spark] class BlockManager( * * This does not acquire a lock on this block in this JVM. */ - def getRemote(blockId: BlockId): Option[BlockResult] = { + def getRemoteValues(blockId: BlockId): Option[BlockResult] = { getRemoteBytes(blockId).map { data => new BlockResult(dataDeserialize(blockId, data), DataReadMethod.Network, data.limit()) } @@ -586,7 +586,7 @@ private[spark] class BlockManager( logInfo(s"Found block $blockId locally") return local } - val remote = getRemote(blockId) + val remote = getRemoteValues(blockId) if (remote.isDefined) { logInfo(s"Found block $blockId remotely") return remote From 5cc4a860dedddf26debae1ded0fd35d75d815de7 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Wed, 9 Mar 2016 16:51:33 -0800 Subject: [PATCH 32/42] Move putIterator() block replication code into try block so that we hold read lock. --- .../apache/spark/storage/BlockManager.scala | 24 +++++++++---------- 1 file changed, 12 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 94cc260fcce69..71f8816c03ea7 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -910,6 +910,18 @@ private[spark] class BlockManager( Option(TaskContext.get()).foreach { c => c.taskMetrics().incUpdatedBlockStatuses(Seq((blockId, putBlockStatus))) } + logDebug("Put block %s locally took %s".format(blockId, Utils.getUsedTimeMs(startTimeMs))) + if (level.replication > 1 && blockWasSuccessfullyStored) { + val remoteStartTime = System.currentTimeMillis + val bytesToReplicate = doGetLocalBytes(blockId, putBlockInfo) + try { + replicate(blockId, bytesToReplicate, level) + } finally { + BlockManager.dispose(bytesToReplicate) + } + logDebug("Put block %s remotely took %s" + .format(blockId, Utils.getUsedTimeMs(remoteStartTime))) + } } } finally { if (blockWasSuccessfullyStored) { @@ -923,18 +935,6 @@ private[spark] class BlockManager( logWarning(s"Putting block $blockId failed") } } - logDebug("Put block %s locally took %s".format(blockId, Utils.getUsedTimeMs(startTimeMs))) - if (level.replication > 1 && blockWasSuccessfullyStored) { - val remoteStartTime = System.currentTimeMillis - val bytesToReplicate = doGetLocalBytes(blockId, putBlockInfo) - try { - replicate(blockId, bytesToReplicate, level) - } finally { - BlockManager.dispose(bytesToReplicate) - } - logDebug("Put block %s remotely took %s" - .format(blockId, Utils.getUsedTimeMs(remoteStartTime))) - } if (level.replication > 1) { logDebug("Putting block %s with replication took %s" From 01474717ac30d472e24ebdb35e0ea5f8b9570e3c Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Wed, 9 Mar 2016 16:53:07 -0800 Subject: [PATCH 33/42] Simplify return at end of doPutIterator() --- .../main/scala/org/apache/spark/storage/BlockManager.scala | 7 ++----- 1 file changed, 2 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 71f8816c03ea7..bb0c1592600bb 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -944,11 +944,8 @@ private[spark] class BlockManager( .format(blockId, Utils.getUsedTimeMs(startTimeMs))) } - if (blockWasSuccessfullyStored) { - None - } else { - Some(iteratorFromFailedMemoryStorePut.get) - } + assert(blockWasSuccessfullyStored == iteratorFromFailedMemoryStorePut.isEmpty) + iteratorFromFailedMemoryStorePut } /** From 84e4d768dfc52a7a610cb935c838e76eb28fbebf Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Wed, 9 Mar 2016 17:00:57 -0800 Subject: [PATCH 34/42] Move acquireStorageMemory back to old call site. --- .../spark/storage/memory/MemoryStore.scala | 32 +++++++++---------- 1 file changed, 16 insertions(+), 16 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala index 0462e729abf95..a80b2357ff911 100644 --- a/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala @@ -291,21 +291,6 @@ private[spark] class MemoryStore( blockId.asRDDId.map(_.rddId) } - private def acquireStorageMemory(blockId: BlockId, size: Long): Boolean = { - // Synchronize on memoryManager so that the pending unroll memory isn't stolen by another - // task. - memoryManager.synchronized { - // Note: if we have previously unrolled this block successfully, then pending unroll - // memory should be non-zero. This is the amount that we already reserved during the - // unrolling process. In this case, we can just reuse this space to cache our block. - // The synchronization on `memoryManager` here guarantees that the release and acquire - // happen atomically. This relies on the assumption that all memory acquisitions are - // synchronized on the same lock. - releasePendingUnrollMemoryForThisTask() - memoryManager.acquireStorageMemory(blockId, size) - } - } - /** * Try to put in a set of values, if we can free up enough space. The value should either be * an Array if deserialized is true or a ByteBuffer otherwise. Its (possibly estimated) size @@ -318,7 +303,22 @@ private[spark] class MemoryStore( value: () => Any, size: Long, deserialized: Boolean): Boolean = { - if (acquireStorageMemory(blockId, size)) { + val acquiredEnoughStorageMemory = { + // Synchronize on memoryManager so that the pending unroll memory isn't stolen by another + // task. + memoryManager.synchronized { + // Note: if we have previously unrolled this block successfully, then pending unroll + // memory should be non-zero. This is the amount that we already reserved during the + // unrolling process. In this case, we can just reuse this space to cache our block. + // The synchronization on `memoryManager` here guarantees that the release and acquire + // happen atomically. This relies on the assumption that all memory acquisitions are + // synchronized on the same lock. + releasePendingUnrollMemoryForThisTask() + memoryManager.acquireStorageMemory(blockId, size) + } + } + + if (acquiredEnoughStorageMemory) { // We acquired enough memory for the block, so go ahead and put it val entry = new MemoryEntry(value(), size, deserialized) entries.synchronized { From e2ee2e2af503f759a1b152d1dc9e9c7be3616d94 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Wed, 9 Mar 2016 17:02:30 -0800 Subject: [PATCH 35/42] Fix minor formatting issue in putIterator(). --- .../main/scala/org/apache/spark/storage/BlockManager.scala | 7 +------ 1 file changed, 1 insertion(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index bb0c1592600bb..5a099a01d1e69 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -668,13 +668,8 @@ private[spark] class BlockManager( level: StorageLevel, tellMaster: Boolean = true): Boolean = { require(values != null, "Values is null") - val maybeIterator = doPutIterator( - blockId, - () => values, - level, - tellMaster) // If doPut() didn't hand work back to us, then block already existed or was successfully stored - maybeIterator.isEmpty + doPutIterator(blockId, () => values, level, tellMaster).isEmpty } /** From 1d0cd0d6fa10142d1c6a5b7c98026be7e93e8dde Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Wed, 9 Mar 2016 17:08:20 -0800 Subject: [PATCH 36/42] Add comment to explain why we don't cache deserialized blocks' bytes in memory. --- .../main/scala/org/apache/spark/storage/BlockManager.scala | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 5a099a01d1e69..9b4db2a069fd2 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -482,6 +482,10 @@ private[spark] class BlockManager( if (level.deserialized) { // Try to avoid expensive serialization by reading a pre-serialized copy from disk: if (level.useDisk && diskStore.contains(blockId)) { + // Note: we purposely do not try to put the block back into memory here. Since this branch + // handles deserialized blocks, this block may only be cached in memory as objects, not + // serialized bytes. Because the caller only requested bytes, it doesn't make sense to + // cache the block's deserialized objects since that caching may not have a payoff. diskStore.getBytes(blockId) } else if (level.useMemory && memoryStore.contains(blockId)) { // The block was not found on disk, so serialize an in-memory copy: From 08bc93b4f0734c7739534a47388238beab39efe8 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Wed, 9 Mar 2016 17:11:27 -0800 Subject: [PATCH 37/42] Simplify code in getLocalBytes --- .../scala/org/apache/spark/storage/BlockManager.scala | 8 +------- 1 file changed, 1 insertion(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 9b4db2a069fd2..cb026d667b544 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -458,13 +458,7 @@ private[spark] class BlockManager( Option( shuffleBlockResolver.getBlockData(blockId.asInstanceOf[ShuffleBlockId]).nioByteBuffer()) } else { - blockInfoManager.lockForReading(blockId) match { - case None => - logDebug(s"Block $blockId was not found") - None - case Some(info) => - Some(doGetLocalBytes(blockId, info)) - } + blockInfoManager.lockForReading(blockId).map { info => doGetLocalBytes(blockId, info) } } } From c6f1956c3ecc4698e5a174803e0165ae66017311 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Wed, 9 Mar 2016 17:24:12 -0800 Subject: [PATCH 38/42] Tautology --- core/src/main/scala/org/apache/spark/storage/BlockManager.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index cb026d667b544..4591e5c638dc0 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -904,7 +904,7 @@ private[spark] class BlockManager( c.taskMetrics().incUpdatedBlockStatuses(Seq((blockId, putBlockStatus))) } logDebug("Put block %s locally took %s".format(blockId, Utils.getUsedTimeMs(startTimeMs))) - if (level.replication > 1 && blockWasSuccessfullyStored) { + if (level.replication > 1) { val remoteStartTime = System.currentTimeMillis val bytesToReplicate = doGetLocalBytes(blockId, putBlockInfo) try { From a9a528b859c0b294bf6c3d00fb331c8595a2d1d6 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Wed, 9 Mar 2016 18:50:42 -0800 Subject: [PATCH 39/42] Move async replication wait code into try block --- .../scala/org/apache/spark/storage/BlockManager.scala | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 4591e5c638dc0..1909e07211fae 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -791,6 +791,11 @@ private[spark] class BlockManager( c.taskMetrics().incUpdatedBlockStatuses(Seq((blockId, putBlockStatus))) } } + logDebug("Put block %s locally took %s".format(blockId, Utils.getUsedTimeMs(startTimeMs))) + if (level.replication > 1) { + // Wait for asynchronous replication to finish + Await.ready(replicationFuture, Duration.Inf) + } } finally { if (blockWasSuccessfullyStored) { if (keepReadLock) { @@ -803,11 +808,8 @@ private[spark] class BlockManager( logWarning(s"Putting block $blockId failed") } } - logDebug("Put block %s locally took %s".format(blockId, Utils.getUsedTimeMs(startTimeMs))) if (level.replication > 1) { - // Wait for asynchronous replication to finish - Await.ready(replicationFuture, Duration.Inf) logDebug("Putting block %s with replication took %s" .format(blockId, Utils.getUsedTimeMs(startTimeMs))) } else { From 2e8b531ff6393b6d022e36b53f11befd3576f0b5 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Thu, 10 Mar 2016 11:13:26 -0800 Subject: [PATCH 40/42] Factor out duplicated code from the doPut() methods. --- .../apache/spark/storage/BlockManager.scala | 164 ++++++++---------- 1 file changed, 68 insertions(+), 96 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 1909e07211fae..4947f218b682f 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -718,49 +718,25 @@ private[spark] class BlockManager( level: StorageLevel, tellMaster: Boolean = true, keepReadLock: Boolean = false): Boolean = { - - require(blockId != null, "BlockId is null") - require(level != null && level.isValid, "StorageLevel is null or invalid") - - /* Remember the block's storage level so that we can correctly drop it to disk if it needs - * to be dropped right after it got put into memory. Note, however, that other threads will - * not be able to get() this block until we call markReady on its BlockInfo. */ - val putBlockInfo = { - val newInfo = new BlockInfo(level, tellMaster) - if (blockInfoManager.lockNewBlockForWriting(blockId, newInfo)) { - newInfo + doPut(blockId, level, tellMaster = tellMaster, keepReadLock = keepReadLock) { putBlockInfo => + val startTimeMs = System.currentTimeMillis + // Since we're storing bytes, initiate the replication before storing them locally. + // This is faster as data is already serialized and ready to send. + val replicationFuture = if (level.replication > 1) { + // Duplicate doesn't copy the bytes, but just creates a wrapper + val bufferView = bytes.duplicate() + Future { + // This is a blocking action and should run in futureExecutionContext which is a cached + // thread pool + replicate(blockId, bufferView, level) + }(futureExecutionContext) } else { - logWarning(s"Block $blockId already exists on this machine; not re-adding it") - if (!keepReadLock) { - // lockNewBlockForWriting returned a read lock on the existing block, so we must free it: - releaseLock(blockId) - } - return true + null } - } - val startTimeMs = System.currentTimeMillis + bytes.rewind() + val size = bytes.limit() - // Since we're storing bytes, initiate the replication before storing them locally. - // This is faster as data is already serialized and ready to send. - val replicationFuture = if (level.replication > 1) { - // Duplicate doesn't copy the bytes, but just creates a wrapper - val bufferView = bytes.duplicate() - Future { - // This is a blocking action and should run in futureExecutionContext which is a cached - // thread pool - replicate(blockId, bufferView, level) - }(futureExecutionContext) - } else { - null - } - - var blockWasSuccessfullyStored = false - - bytes.rewind() - val size = bytes.limit() - - try { if (level.useMemory) { // Put it in memory first, even if it also has useDisk set to true; // We will drop it to disk later if the memory store can't hold it. @@ -779,7 +755,7 @@ private[spark] class BlockManager( } val putBlockStatus = getCurrentBlockStatus(blockId, putBlockInfo) - blockWasSuccessfullyStored = putBlockStatus.storageLevel.isValid + val blockWasSuccessfullyStored = putBlockStatus.storageLevel.isValid if (blockWasSuccessfullyStored) { // Now that the block is in either the memory, externalBlockStore, or disk store, // tell the master about it. @@ -796,6 +772,49 @@ private[spark] class BlockManager( // Wait for asynchronous replication to finish Await.ready(replicationFuture, Duration.Inf) } + if (blockWasSuccessfullyStored) { + None + } else { + Some(bytes) + } + }.isEmpty + } + + /** + * Helper method used to abstract common code from [[doPutBytes()]] and [[doPutIterator()]]. + * + * @param putBody a function which attempts the actual put() and returns None on success + * or Some on failure. + */ + private def doPut[T]( + blockId: BlockId, + level: StorageLevel, + tellMaster: Boolean, + keepReadLock: Boolean)(putBody: BlockInfo => Option[T]): Option[T] = { + + require(blockId != null, "BlockId is null") + require(level != null && level.isValid, "StorageLevel is null or invalid") + + val putBlockInfo = { + val newInfo = new BlockInfo(level, tellMaster) + if (blockInfoManager.lockNewBlockForWriting(blockId, newInfo)) { + newInfo + } else { + logWarning(s"Block $blockId already exists on this machine; not re-adding it") + if (!keepReadLock) { + // lockNewBlockForWriting returned a read lock on the existing block, so we must free it: + releaseLock(blockId) + } + return None + } + } + + val startTimeMs = System.currentTimeMillis + var blockWasSuccessfullyStored: Boolean = false + var result: Option[T] = None + try { + result = putBody(putBlockInfo) + blockWasSuccessfullyStored = result.isEmpty } finally { if (blockWasSuccessfullyStored) { if (keepReadLock) { @@ -808,7 +827,6 @@ private[spark] class BlockManager( logWarning(s"Putting block $blockId failed") } } - if (level.replication > 1) { logDebug("Putting block %s with replication took %s" .format(blockId, Utils.getUsedTimeMs(startTimeMs))) @@ -816,8 +834,7 @@ private[spark] class BlockManager( logDebug("Putting block %s without replication took %s" .format(blockId, Utils.getUsedTimeMs(startTimeMs))) } - - blockWasSuccessfullyStored + result } /** @@ -838,36 +855,11 @@ private[spark] class BlockManager( level: StorageLevel, tellMaster: Boolean = true, keepReadLock: Boolean = false): Option[Iterator[Any]] = { - - require(blockId != null, "BlockId is null") - require(level != null && level.isValid, "StorageLevel is null or invalid") - - /* Remember the block's storage level so that we can correctly drop it to disk if it needs - * to be dropped right after it got put into memory. Note, however, that other threads will - * not be able to get() this block until we call markReady on its BlockInfo. */ - val putBlockInfo = { - val newInfo = new BlockInfo(level, tellMaster) - if (blockInfoManager.lockNewBlockForWriting(blockId, newInfo)) { - newInfo - } else { - logWarning(s"Block $blockId already exists on this machine; not re-adding it") - if (!keepReadLock) { - // lockNewBlockForWriting returned a read lock on the existing block, so we must free it: - releaseLock(blockId) - } - return None - } - } - - val startTimeMs = System.currentTimeMillis - - // Size of the block in bytes - var size = 0L - - var blockWasSuccessfullyStored = false - var iteratorFromFailedMemoryStorePut: Option[Iterator[Any]] = None - - try { + doPut(blockId, level, tellMaster = tellMaster, keepReadLock = keepReadLock) { putBlockInfo => + val startTimeMs = System.currentTimeMillis + var iteratorFromFailedMemoryStorePut: Option[Iterator[Any]] = None + // Size of the block in bytes + var size = 0L if (level.useMemory) { // Put it in memory first, even if it also has useDisk set to true; // We will drop it to disk later if the memory store can't hold it. @@ -894,7 +886,7 @@ private[spark] class BlockManager( } val putBlockStatus = getCurrentBlockStatus(blockId, putBlockInfo) - blockWasSuccessfullyStored = putBlockStatus.storageLevel.isValid + val blockWasSuccessfullyStored = putBlockStatus.storageLevel.isValid if (blockWasSuccessfullyStored) { // Now that the block is in either the memory, externalBlockStore, or disk store, // tell the master about it. @@ -918,29 +910,9 @@ private[spark] class BlockManager( .format(blockId, Utils.getUsedTimeMs(remoteStartTime))) } } - } finally { - if (blockWasSuccessfullyStored) { - if (keepReadLock) { - blockInfoManager.downgradeLock(blockId) - } else { - blockInfoManager.unlock(blockId) - } - } else { - blockInfoManager.removeBlock(blockId) - logWarning(s"Putting block $blockId failed") - } - } - - if (level.replication > 1) { - logDebug("Putting block %s with replication took %s" - .format(blockId, Utils.getUsedTimeMs(startTimeMs))) - } else { - logDebug("Putting block %s without replication took %s" - .format(blockId, Utils.getUsedTimeMs(startTimeMs))) + assert(blockWasSuccessfullyStored == iteratorFromFailedMemoryStorePut.isEmpty) + iteratorFromFailedMemoryStorePut } - - assert(blockWasSuccessfullyStored == iteratorFromFailedMemoryStorePut.isEmpty) - iteratorFromFailedMemoryStorePut } /** From 7a9cbc6278d59ecef64419101213ddd81163fece Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Thu, 10 Mar 2016 11:58:44 -0800 Subject: [PATCH 41/42] Always try to cache bytes in memory when reading serialized blocks from disk. --- .../apache/spark/storage/BlockManager.scala | 56 ++++++++++++++----- .../spark/storage/BlockManagerSuite.scala | 9 ++- 2 files changed, 49 insertions(+), 16 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 4947f218b682f..973486e8c51ba 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -419,20 +419,42 @@ private[spark] class BlockManager( val ci = CompletionIterator[Any, Iterator[Any]](iter, releaseLock(blockId)) Some(new BlockResult(ci, DataReadMethod.Memory, info.size)) } else if (level.useDisk && diskStore.contains(blockId)) { - val valuesFromDisk = dataDeserialize(blockId, diskStore.getBytes(blockId)) val iterToReturn: Iterator[Any] = { + val diskBytes = diskStore.getBytes(blockId) if (level.deserialized) { - // Cache the values before returning them - memoryStore.putIterator(blockId, valuesFromDisk, level) match { - case Left(iter) => - // The memory store put() failed, so it returned the iterator back to us: - iter - case Right(_) => - // The put() succeeded, so we can read the values back: - memoryStore.getValues(blockId).get + val diskIterator = dataDeserialize(blockId, diskBytes) + if (level.useMemory) { + // Cache the values before returning them + memoryStore.putIterator(blockId, diskIterator, level) match { + case Left(iter) => + // The memory store put() failed, so it returned the iterator back to us: + iter + case Right(_) => + // The put() succeeded, so we can read the values back: + memoryStore.getValues(blockId).get + } + } else { + diskIterator + } + } else { // storage level is serialized + if (level.useMemory) { + // Cache the bytes back into memory to speed up subsequent reads. + val putSucceeded = memoryStore.putBytes(blockId, diskBytes.limit(), () => { + // https://issues.apache.org/jira/browse/SPARK-6076 + // If the file size is bigger than the free memory, OOM will happen. So if we + // cannot put it into MemoryStore, copyForMemory should not be created. That's why + // this action is put into a `() => ByteBuffer` and created lazily. + val copyForMemory = ByteBuffer.allocate(diskBytes.limit) + copyForMemory.put(diskBytes) + }) + if (putSucceeded) { + dataDeserialize(blockId, memoryStore.getBytes(blockId).get) + } else { + dataDeserialize(blockId, diskBytes) + } + } else { + dataDeserialize(blockId, diskBytes) } - } else { - valuesFromDisk } } val ci = CompletionIterator[Any, Iterator[Any]](iterToReturn, releaseLock(blockId)) @@ -495,7 +517,7 @@ private[spark] class BlockManager( val bytes = diskStore.getBytes(blockId) if (level.useMemory) { // Cache the bytes back into memory to speed up subsequent reads. - memoryStore.putBytes(blockId, bytes.limit(), () => { + val memoryStorePutSucceeded = memoryStore.putBytes(blockId, bytes.limit(), () => { // https://issues.apache.org/jira/browse/SPARK-6076 // If the file size is bigger than the free memory, OOM will happen. So if we cannot // put it into MemoryStore, copyForMemory should not be created. That's why this @@ -503,9 +525,15 @@ private[spark] class BlockManager( val copyForMemory = ByteBuffer.allocate(bytes.limit) copyForMemory.put(bytes) }) - bytes.rewind() + if (memoryStorePutSucceeded) { + memoryStore.getBytes(blockId).get + } else { + bytes.rewind() + bytes + } + } else { + bytes } - bytes } else { releaseLock(blockId) throw new SparkException(s"Block $blockId was not found even though it's read-locked") diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index ef980449fe11f..42595c8cf2daa 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -615,8 +615,13 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE assert(accessMethod("a3").isDefined, "a3 was not in store") assert(accessMethod("a1").isDefined, "a1 was not in store") val dataShouldHaveBeenCachedBackIntoMemory = { - if (storageLevel.deserialized) !getAsBytes - else getAsBytes + if (storageLevel.deserialized) { + !getAsBytes + } else { + // If the block's storage level is serialized, then always cache the bytes in memory, even + // if the caller requested values. + true + } } if (dataShouldHaveBeenCachedBackIntoMemory) { assert(store.memoryStore.contains("a1"), "a1 was not in memory store") From 2752655518c9907dd2c9195f7da8cff28824d71e Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Thu, 10 Mar 2016 12:00:49 -0800 Subject: [PATCH 42/42] Style cleanup. --- .../scala/org/apache/spark/storage/BlockManager.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 973486e8c51ba..873330e136e22 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -839,10 +839,10 @@ private[spark] class BlockManager( val startTimeMs = System.currentTimeMillis var blockWasSuccessfullyStored: Boolean = false - var result: Option[T] = None - try { - result = putBody(putBlockInfo) - blockWasSuccessfullyStored = result.isEmpty + val result: Option[T] = try { + val res = putBody(putBlockInfo) + blockWasSuccessfullyStored = res.isEmpty + res } finally { if (blockWasSuccessfullyStored) { if (keepReadLock) {