Skip to content
Closed
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -133,8 +133,6 @@ private[spark] class BlockManager(

private[spark] val externalShuffleServiceEnabled =
conf.get(config.SHUFFLE_SERVICE_ENABLED)
private val chunkSize =
conf.getSizeAsBytes("spark.storage.memoryMapLimitForTests", Int.MaxValue.toString).toInt
private val remoteReadNioBufferConversion =
conf.getBoolean("spark.network.remoteReadNioBufferConversion", false)

Expand Down Expand Up @@ -451,7 +449,7 @@ private[spark] class BlockManager(
new EncryptedBlockData(tmpFile, blockSize, conf, key).toChunkedByteBuffer(allocator)

case None =>
ChunkedByteBuffer.fromFile(tmpFile, conf.get(config.MEMORY_MAP_LIMIT_FOR_TESTS).toInt)
ChunkedByteBuffer.fromFile(tmpFile)
}
putBytes(blockId, buffer, level)(classTag)
tmpFile.delete()
Expand Down Expand Up @@ -797,7 +795,7 @@ private[spark] class BlockManager(
if (remoteReadNioBufferConversion) {
return Some(new ChunkedByteBuffer(data.nioByteBuffer()))
} else {
return Some(ChunkedByteBuffer.fromManagedBuffer(data, chunkSize))
return Some(ChunkedByteBuffer.fromManagedBuffer(data))
}
}
logDebug(s"The value of block $blockId is null")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -169,24 +169,25 @@ private[spark] class ChunkedByteBuffer(var chunks: Array[ByteBuffer]) {

}

object ChunkedByteBuffer {
private[spark] object ChunkedByteBuffer {


// TODO eliminate this method if we switch BlockManager to getting InputStreams
def fromManagedBuffer(data: ManagedBuffer, maxChunkSize: Int): ChunkedByteBuffer = {
def fromManagedBuffer(data: ManagedBuffer): ChunkedByteBuffer = {
data match {
case f: FileSegmentManagedBuffer =>
fromFile(f.getFile, maxChunkSize, f.getOffset, f.getLength)
fromFile(f.getFile, f.getOffset, f.getLength)
case other =>
new ChunkedByteBuffer(other.nioByteBuffer())
}
}

def fromFile(file: File, maxChunkSize: Int): ChunkedByteBuffer = {
fromFile(file, maxChunkSize, 0, file.length())
def fromFile(file: File): ChunkedByteBuffer = {
fromFile(file, 0, file.length())
}

private def fromFile(
file: File,
maxChunkSize: Int,
offset: Long,
length: Long): ChunkedByteBuffer = {
// We do *not* memory map the file, because we may end up putting this into the memory store,
Expand All @@ -195,7 +196,11 @@ object ChunkedByteBuffer {
val is = new FileInputStream(file)
ByteStreams.skipFully(is, offset)
val in = new LimitedInputStream(is, length)
val chunkSize = math.min(maxChunkSize, length).toInt
// Though in theory you should be able to index into an array of size Int.MaxValue, in practice
// jvms don't let you go up to limit. It seems you may only need - 2, but we leave a little
// extra room.
val maxArraySize = Int.MaxValue - 512
Copy link
Member

Choose a reason for hiding this comment

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

Looks like the max is safely more like Int.MaxValue - 5 from things like https://www.quora.com/What-is-the-maximum-size-of-the-array-in-Java You could probably push it further, but whatever.

Copy link
Member

Choose a reason for hiding this comment

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

We already had seen the similar problem in other places. Can we use this value here, too?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

great suggestion, thanks, just updated

val chunkSize = math.min(maxArraySize, length).toInt
val out = new ChunkedByteBufferOutputStream(chunkSize, ByteBuffer.allocate _)
Utils.tryWithSafeFinally {
IOUtils.copy(in, out)
Expand Down