From 5cdcd4246e586346a8e1ac2242dd795fdb1ae068 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Fri, 20 Feb 2015 16:35:12 -0600 Subject: [PATCH 01/97] add some failing tests, though these probably shouldnt actually get merged --- .../spark/broadcast/BroadcastSuite.scala | 8 +++++ .../rdd/LargePartitionCachingSuite.scala | 36 +++++++++++++++++++ 2 files changed, 44 insertions(+) create mode 100644 core/src/test/scala/org/apache/spark/rdd/LargePartitionCachingSuite.scala 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 af3272692d7a1..49815890623df 100644 --- a/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala +++ b/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala @@ -179,6 +179,14 @@ class BroadcastSuite extends FunSuite with LocalSparkContext { assert(thrown.getMessage.toLowerCase.contains("stopped")) } + test("large broadcast variable") { + //Note this currently fails by killing the whole test runner + sc = new SparkContext("local", "test", httpConf) + val bigArr = new Array[Long]((2.3e9 / 8).toInt) + val bcArr = sc.broadcast(bigArr) + sc.parallelize(1 to 1).map{x => bcArr.value.size}.count() + } + /** * Verify the persistence of state associated with an HttpBroadcast in either local mode or * local-cluster mode (when distributed = true). diff --git a/core/src/test/scala/org/apache/spark/rdd/LargePartitionCachingSuite.scala b/core/src/test/scala/org/apache/spark/rdd/LargePartitionCachingSuite.scala new file mode 100644 index 0000000000000..21c6e5fe3ab50 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/rdd/LargePartitionCachingSuite.scala @@ -0,0 +1,36 @@ +/* + * 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.rdd + +import org.apache.spark.SharedSparkContext +import org.apache.spark.storage.StorageLevel +import org.scalatest.FunSuite + +class LargePartitionCachingSuite extends FunSuite with SharedSparkContext { + + def largePartitionRdd = sc.parallelize(1 to 1e6.toInt, 1).map{i => new Array[Byte](2.2e3.toInt)} + + test("memory serialized cache large partitions") { + //this test doesn't actually work, b/c we'll just think we don't have enough memory, + // and so it won't get persisted :( + largePartitionRdd.persist(StorageLevel.MEMORY_ONLY_SER).count() + } + + test("disk cache large partitions") { + largePartitionRdd.persist(StorageLevel.DISK_ONLY).count() + } +} From 03db862833f3c4feef2d72620bc5c9a893dab2f5 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Mon, 23 Feb 2015 14:28:22 -0600 Subject: [PATCH 02/97] steal some code from earlier work of @mridulm --- .../org/apache/spark/io/LargeByteBuffer.scala | 1505 +++++++++++++++++ .../io/WrappedByteArrayOutputStream.scala | 121 ++ .../org/apache/spark/storage/BlockStore.scala | 22 +- 3 files changed, 1640 insertions(+), 8 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/io/LargeByteBuffer.scala create mode 100644 core/src/main/scala/org/apache/spark/io/WrappedByteArrayOutputStream.scala diff --git a/core/src/main/scala/org/apache/spark/io/LargeByteBuffer.scala b/core/src/main/scala/org/apache/spark/io/LargeByteBuffer.scala new file mode 100644 index 0000000000000..509fe186793b6 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/io/LargeByteBuffer.scala @@ -0,0 +1,1505 @@ +/* + * 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.io + +import java.io.{RandomAccessFile, DataInput, InputStream, OutputStream} +import java.nio.{ByteBuffer, BufferUnderflowException, BufferOverflowException} +import java.nio.channels.{WritableByteChannel, ReadableByteChannel} + +import scala.collection.mutable.{ArrayBuffer, HashSet} + +import org.apache.spark.Logging +import org.apache.spark.io.IOConfig.BufferType +import org.apache.spark.storage.{FileSegment, BlockManager} + +/** + * This is used to control chaining of cleaners. + * For some usecases, invocation of clean on LargeByteBuffer must not immediately clean + * but be tied to clean of other buffers. + * This allows a way to override how/when clean is invoked + * + * TODO: We should revisit this design and see if we can use ref tracking : might be more general + * at higher cost ? Punting on it for now + */ +trait BufferCleaner extends Logging { + @volatile private var invoked = false + + protected def doClean(buffer: LargeByteBuffer) + + final def clean(buffer: LargeByteBuffer) { + if (invoked) return + invoked = true + doClean(buffer) + } +} + +/** + * This is a variant of ByteBuffer to be used internally in spark, which is not limited to 2G + * which ByteBuffers are limited to. + * Externally, it exposes all the api which java.nio.ByteBuffer exposes. + * Internally, it maintains a sequence of Containers which manage the ByteBuffer data. + * Not all the data might be loaded into memory (like disk or tachyon data) - so actual + * memory footprint - heap and vm could be much lower than capacity. + * + * TODO: Currently we are slightly fast and loose in terms of concurrent modifications to this + * buffer, maybe revisit this later ? Note: this is not much different from earlier though ! + * + * TODO: Explore if (at all) we can leverage zero copy transfers. The issue (currently) is that this + * will require the file to be kept open (repeatedly opening/closing file is not good + * for each transfer) and this has an impact on ulimit. Not to mention writing of mmap'ed buffer is + * pretty quick (it is the first failover in case direct transfer is not possible in file zero copy) + * + * TODO: After redesign to containers, we got rid of parent containers to free - the side effect is + * that if there are direct ByteBuffers, we are not handling explicit cleanup of those in some + * cases (when we duplicate/slice them). Currently spark does not need this, but might in future + * so relook at it later. + */ +// We should make this constructor private: but for now, +// leaving it public since TachyonStore needs it +class LargeByteBuffer private[spark](private val inputContainers: ArrayBuffer[ByteBufferContainer], + private val needDuplicate: Boolean, val ephemeralDiskBacked: Boolean) extends Logging { + + // TODO: TEMP code: to flush out potential resource leaks. REMOVE ME + private val allocateLocationThrowable: Throwable = { + if (inputContainers.exists(c => c.requireRelease() || c.requireFree())) { + new Throwable("blockId = " + BlockManager.getLookupBlockId) + } else { + null + } + } + private var disposeLocationThrowable: Throwable = null + + @volatile private var allowCleanerOverride = true + @volatile private var cleaner: BufferCleaner = new BufferCleaner { + override def doClean(buffer: LargeByteBuffer) = { + assert (LargeByteBuffer.this == buffer) + doDispose(needRelease = false) + } + } + + // should not be empty + assert (null != inputContainers && ! inputContainers.isEmpty) + // should not have any null's + assert (inputContainers.find(_ == null).isEmpty) + + // println("Num containers = " + inputContainers.size) + + // Position, limit and capacity relevant over the engire LargeByteBuffer + @volatile private var globalPosition = 0L + @volatile private var globalLimit = 0L + @volatile private var currentContainerIndex = 0 + + // The buffers in which the actual data is held. + private var containers: Array[ByteBufferContainer] = null + + // aggregate capacities of the individual buffers. + // bufferPositionStart(0) will be capacity of 1st buffer, bufferPositionStart(1) will be + // sum of capacity of 0th and 1st block buffer + private var bufferPositionStart: Array[Long] = null + + // Contains the indices of a containers which requires release before subsequent invocation of + // read/write should be serviced. This is required since current read/write might have moved the + // position but since we are returning bytebuffers which depend on the validity of the existing + // bytebuffer, we cant release them yet. + private var needReleaseIndices = new HashSet[Int]() + + private val readable = ! inputContainers.exists(! _.isReadable) + private val writable = ! inputContainers.exists(! _.isWritable) + + + // initialize + @volatile private var globalCapacity = { + + // Ensure that there are no empty buffers : messes up with our code : unless it + // is a single buffer (for empty buffer for marker case) + assert (inputContainers.find(0 == _.capacity()).isEmpty || 1 == inputContainers.length) + + containers = { + if (needDuplicate) inputContainers.map(_.duplicate()).toArray else inputContainers.toArray + } + containers.foreach(_.validate()) + + def initializeBufferPositionStart(arr: Array[ByteBufferContainer]) { + val buff = new ArrayBuffer[Long](arr.length + 1) + buff += 0L + + buff ++= arr.map(_.capacity().asInstanceOf[Long]).scanLeft(0L)(_ + _).slice(1, arr.length + 1) + assert (buff.length == arr.length + 1) + bufferPositionStart = buff.toArray + } + + initializeBufferPositionStart(containers) + + // remove references from inputBuffers + inputContainers.clear() + + globalLimit = bufferPositionStart(containers.length) + globalPosition = 0L + currentContainerIndex = 0 + + assert (globalLimit == containers.map(_.capacity().asInstanceOf[Long]).sum) + + globalLimit + } + + final def position(): Long = globalPosition + + final def limit(): Long = globalLimit + + final def capacity(): Long = globalCapacity + + final def limit(newLimit: Long) { + if ((newLimit > capacity()) || (newLimit < 0)) { + throw new IllegalArgumentException("newLimit = " + newLimit + ", capacity = " + capacity()) + } + + globalLimit = newLimit + if (position() > newLimit) position(newLimit) + } + + def skip(skipBy: Long) = position(position() + skipBy) + + private def releasePendingContainers() { + if (! needReleaseIndices.isEmpty) { + val iter = needReleaseIndices.iterator + while (iter.hasNext) { + val index = iter.next() + assert (index >= 0 && index < containers.length) + // It is possible to move from one container to next before the previous + // container was acquired. For example, get forcing move to next container + // since current was exhausted immediatelly followed by a position() + // so the container we moved to was never acquired. + + // assert (containers(index).isAcquired) + // will this always be satisfied ? + // assert (index != currentContainerIndex) + if (containers(index).isAcquired) containers(index).release() + } + needReleaseIndices.clear() + } + } + + private def toNewContainer(newIndex: Int) { + if (newIndex != currentContainerIndex && currentContainerIndex < containers.length) { + + assert (currentContainerIndex >= 0) + needReleaseIndices += currentContainerIndex + } + currentContainerIndex = newIndex + } + + // expensive method, sigh ... optimize it later ? + final def position(newPosition: Long) { + + if ((newPosition > globalLimit) || (newPosition < 0)) throw new IllegalArgumentException() + + if (currentContainerIndex < bufferPositionStart.length - 1 && + newPosition >= bufferPositionStart(currentContainerIndex) && + newPosition < bufferPositionStart(currentContainerIndex + 1)) { + // Same buffer - easy method ... + globalPosition = newPosition + // Changed position - free previously returned buffers. + releasePendingContainers() + return + } + + // Find appropriate currentContainerIndex + // Since bufferPositionStart is sorted, can be replaced with binary search if required. + // For now, not in the perf critical path since buffers size is very low typically. + var index = 0 + val cLen = containers.length + while (index < cLen) { + if (newPosition >= bufferPositionStart(index) && + newPosition < bufferPositionStart(index + 1)) { + globalPosition = newPosition + toNewContainer(index) + // Changed position - free earlier and previously returned buffers. + releasePendingContainers() + return + } + index += 1 + } + + if (newPosition == globalLimit && newPosition == bufferPositionStart(cLen)) { + // boundary. + globalPosition = newPosition + toNewContainer(cLen) + // Changed position - free earlier and previously returned buffers. + releasePendingContainers() + return + } + + assert (assertion = false, "Unexpected to come here .... newPosition = " + newPosition + + ", bufferPositionStart = " + bufferPositionStart.mkString("[", ", ", "]")) + } + + + /** + * Clears this buffer. The position is set to zero, the limit is set to + * the capacity, and the mark is discarded. + * + *

Invoke this method before using a sequence of channel-read or + * put operations to fill this buffer. + * + *

This method does not actually erase the data in the buffer, but it + * is named as if it did because it will most often be used in situations + * in which that might as well be the case.

+ */ + final def clear() { + // if (0 == globalCapacity) return + + needReleaseIndices += 0 + globalPosition = 0L + toNewContainer(0) + globalLimit = globalCapacity + + // Now free all pending containers + releasePendingContainers() + } + + /** + * Flips this buffer. The limit is set to the current position and then + * the position is set to zero. If the mark is defined then it is + * discarded. + * + *

After a sequence of channel-read or put operations, invoke + * this method to prepare for a sequence of channel-write or relative + * get operations. + */ + final def flip() { + needReleaseIndices += 0 + globalLimit = globalPosition + globalPosition = 0L + toNewContainer(0) + + // Now free all pending containers + releasePendingContainers() + } + + /** + * Rewinds this buffer. The position is set to zero and the mark is + * discarded. + * + *

Invoke this method before a sequence of channel-write or get + * operations, assuming that the limit has already been set + * appropriately. + */ + final def rewind() { + needReleaseIndices += 0 + globalPosition = 0L + toNewContainer(0) + + // Now free all pending containers + releasePendingContainers() + } + + /** + * Returns the number of elements between the current position and the + * limit.

+ * + * @return The number of elements remaining in this buffer + */ + final def remaining(): Long = { + globalLimit - globalPosition + } + + /** + * Tells whether there are any elements between the current position and + * the limit.

+ * + * @return true if, and only if, there is at least one element + * remaining in this buffer + */ + final def hasRemaining() = { + globalPosition < globalLimit + } + + // private def currentBuffer(): ByteBuffer = buffers(currentContainerIndex) + + // number of bytes remaining in currently active underlying buffer + private def currentRemaining(): Int = { + if (hasRemaining()) { + // validate currentContainerIndex is valid + assert (globalPosition >= bufferPositionStart(currentContainerIndex) && + globalPosition < bufferPositionStart(currentContainerIndex + 1), + "globalPosition = " + globalPosition + + ", currentContainerIndex = " + currentContainerIndex + + ", bufferPositionStart = " + bufferPositionStart.mkString("[", ", ", " ]")) + + currentRemaining0(currentContainerIndex) + } else 0 + } + + // Without any validation : required when we are bumping the index (when validation will fail) ... + private def currentRemaining0(which: Int): Int = { + // currentBuffer().remaining() + math.max(0, math.min(bufferPositionStart(which + 1), + globalLimit) - globalPosition).asInstanceOf[Int] + } + + // Set the approppriate position/limit for the current underlying buffer to mirror our + // the LargeByteBuffer's state. + private def fetchCurrentBuffer(): ByteBuffer = { + releasePendingContainers() + + assert (currentContainerIndex < containers.length) + + val container = containers(currentContainerIndex) + if (! container.isAcquired) { + container.acquire() + } + + assert (container.isAcquired) + if (LargeByteBuffer.enableExpensiveAssert) { + assert (! containers.exists( b => (b ne container) && b.isAcquired)) + } + + assert (currentContainerIndex < bufferPositionStart.length && + globalPosition < bufferPositionStart(currentContainerIndex + 1), + "currentContainerIndex = " + currentContainerIndex + ", bufferPositionStart = " + + bufferPositionStart.mkString("[", ", ", "]") + ", this = " + this) + + val buffPosition = (globalPosition - bufferPositionStart(currentContainerIndex)). + asInstanceOf[Int] + + val buffer = container.getByteBuffer + buffer.position(buffPosition) + val diff = buffer.capacity - buffPosition + val left = remaining() + if (diff <= left) { + buffer.limit(buffer.capacity()) + } else { + // Can happen if limit() was called. + buffer.limit(buffPosition + left.asInstanceOf[Int]) + } + + buffer + } + + // To be used ONLY to test in suites. + private[spark] def fetchCurrentBufferForTesting(): ByteBuffer = { + if ("1" != System.getProperty("SPARK_TESTING")) { + throw new IllegalStateException("This method is to be used ONLY within spark test suites") + } + + fetchCurrentBuffer() + } + + // Expects that the invoker has ensured that this can be safely invoked. + // That is, it wont be invoked when the loop wont terminate. + private def toNonEmptyBuffer() { + + if (! hasRemaining()) { + var newIndex = currentContainerIndex + // Ensure we are in the right block or not. + while (newIndex < containers.length && globalPosition >= bufferPositionStart(newIndex + 1)) { + newIndex += 1 + } + toNewContainer(newIndex) + // Do not do this - since we might not yet have consumed the buffer which caused EOF right now + /* + // Add last one also, and release it too - since we are at the end of the buffer with nothing + // more pending. + if (newIndex >= 0 && currentContainerIndex < containers.length) { + needReleaseIndices += newIndex + } + */ + assert (currentContainerIndex >= 0) + // releasePendingContainers() + return + } + + var index = currentContainerIndex + while (0 == currentRemaining0(index) && index < containers.length) { + index += 1 + } + assert (currentContainerIndex < containers.length) + toNewContainer(index) + assert (0 != currentRemaining()) + } + + private def assertPreconditions(containerIndex: Int) { + assert (globalPosition >= bufferPositionStart(containerIndex), + "globalPosition = " + globalPosition + ", containerIndex = " + containerIndex + + ", bufferPositionStart = " + bufferPositionStart.mkString("[", ", ", " ]")) + assert (globalPosition < bufferPositionStart(containerIndex + 1), + "globalPosition = " + globalPosition + ", containerIndex = " + containerIndex + + ", bufferPositionStart = " + bufferPositionStart.mkString("[", ", ", " ]")) + + assert (globalLimit <= globalCapacity) + assert (containerIndex < containers.length) + } + + + /** + * Attempts to return a ByteBuffer of the requested size. + * It is possible to return a buffer of size smaller than requested + * even though hasRemaining == true + * + * On return, position would have been moved 'ahead' by the size of the buffer returned : + * that is, we treat that the returned buffer has been already 'read' from this LargeByteBuffer + * + * + * This is used to primarily retrieve content of this buffer to expose via ByteBuffer + * to some other api which is deemed too cumbersome to move to LargeByteBuffer (like the + * chunked sending of contents via ConnectionManager) Note that the lifecycle of the ByteBuffer + * returned is inherently tied to the state of this LargeByteBuffer. For example,if the underlying + * container is a disk backed container, and we make subsequent calls to get(), the returned + * ByteBuffer can be dispose'ed off + * + * @param maxChunkSize Max size of the ByteBuffer to retrieve. + * @return + */ + + private def fetchBufferOfSize(maxChunkSize: Int): ByteBuffer = { + fetchBufferOfSizeImpl(maxChunkSize, canReleaseContainers = true) + } + + private def fetchBufferOfSizeImpl(maxChunkSize: Int, + canReleaseContainers: Boolean): ByteBuffer = { + if (canReleaseContainers) releasePendingContainers() + assert (maxChunkSize > 0) + + // not checking for degenerate case of maxChunkSize == 0 + if (globalPosition >= globalLimit) { + // throw exception + throw new BufferUnderflowException() + } + + // Check preconditions : disable these later, since they might be expensive to + // evaluate for every IO op + assertPreconditions(currentContainerIndex) + + val currentBufferRemaining = currentRemaining() + + assert (currentBufferRemaining > 0) + + val size = math.min(currentBufferRemaining, maxChunkSize) + + val newBuffer = if (currentBufferRemaining > maxChunkSize) { + val currentBuffer = fetchCurrentBuffer() + val buff = ByteBufferContainer.createSlice(currentBuffer, + currentBuffer.position(), maxChunkSize) + assert (buff.remaining() == maxChunkSize) + buff + } else { + val currentBuffer = fetchCurrentBuffer() + val buff = currentBuffer.slice() + assert (buff.remaining() == currentBufferRemaining) + buff + } + + assert (size == newBuffer.remaining()) + assert (0 == newBuffer.position()) + assert (size == newBuffer.limit()) + assert (newBuffer.capacity() == newBuffer.limit()) + + globalPosition += newBuffer.remaining + toNonEmptyBuffer() + + newBuffer + } + + // Can we service the read/write from the currently active (underlying) bytebuffer or not. + // For almost all cases, this will return true allowing us to optimize away the more expensive + // computations. + private def localReadWritePossible(size: Int) = + size >= 0 && globalPosition + size <= bufferPositionStart(currentContainerIndex + 1) + + + def getLong(): Long = { + assert (readable) + releasePendingContainers() + + if (remaining() < 8) throw new BufferUnderflowException + + if (localReadWritePossible(8)) { + val buff = fetchCurrentBuffer() + assert (buff.remaining() >= 8) + val retval = buff.getLong + globalPosition += 8 + toNonEmptyBuffer() + return retval + } + + val buff = readFully(8) + buff.getLong + } + + def getInt(): Int = { + assert (readable) + releasePendingContainers() + + if (remaining() < 4) throw new BufferUnderflowException + + if (localReadWritePossible(4)) { + val buff = fetchCurrentBuffer() + assert (buff.remaining() >= 4) + val retval = buff.getInt + globalPosition += 4 + toNonEmptyBuffer() + return retval + } + + val buff = readFully(4) + buff.getInt + } + + def getChar(): Char = { + assert (readable) + releasePendingContainers() + + if (remaining() < 2) throw new BufferUnderflowException + + if (localReadWritePossible(2)) { + val buff = fetchCurrentBuffer() + assert (buff.remaining() >= 2) + val retval = buff.getChar + globalPosition += 2 + toNonEmptyBuffer() + return retval + } + + // if slice is becoming too expensive, revisit this ... + val buff = readFully(2) + buff.getChar + } + + def get(): Byte = { + assert (readable) + releasePendingContainers() + + if (! hasRemaining()) throw new BufferUnderflowException + + // If we have remaining bytes, previous invocations MUST have ensured that we are at + // a buffer which has data to be read. + assert (localReadWritePossible(1)) + + val buff = fetchCurrentBuffer() + assert (buff.remaining() >= 1, "buff.remaining = " + buff.remaining()) + val retval = buff.get() + globalPosition += 1 + toNonEmptyBuffer() + + retval + } + + def get(arr: Array[Byte], offset: Int, size: Int): Int = { + assert (readable) + releasePendingContainers() + + LargeByteBuffer.checkOffsets(arr, offset, size) + + // kyro depends on this it seems ? + // assert (size > 0) + if (0 == size) return 0 + + if (! hasRemaining()) return -1 + + if (localReadWritePossible(size)) { + val buff = fetchCurrentBuffer() + assert (buff.remaining() >= size) + buff.get(arr, offset, size) + globalPosition += size + toNonEmptyBuffer() + return size + } + + var remainingSize = math.min(size, remaining()).asInstanceOf[Int] + var currentOffset = offset + + while (remainingSize > 0) { + val buff = fetchBufferOfSize(remainingSize) + val toCopy = math.min(buff.remaining(), remainingSize) + + buff.get(arr, currentOffset, toCopy) + currentOffset += toCopy + remainingSize -= toCopy + } + + currentOffset - offset + } + + + private def createSlice(size: Long): LargeByteBuffer = { + + releasePendingContainers() + + if (remaining() < size) { + // logInfo("createSlice. remaining = " + remaining() + ", size " + size + ", this = " + this) + throw new BufferOverflowException + } + + // kyro depends on this it seems ? + // assert (size > 0) + if (0 == size) return LargeByteBuffer.EMPTY_BUFFER + + val arr = new ArrayBuffer[ByteBufferContainer](2) + var totalLeft = size + + // assert (currentRemaining() < totalLeft || totalLeft != size || currentAsByteBuffer) + + var containerIndex = currentContainerIndex + while (totalLeft > 0 && hasRemaining()) { + assertPreconditions(containerIndex) + val container = containers(containerIndex) + val currentLeft = currentRemaining0(containerIndex) + + assert (globalPosition + currentLeft <= globalLimit) + assert (globalPosition >= bufferPositionStart(containerIndex) && + (globalPosition < bufferPositionStart(containerIndex + 1))) + + val from = (globalPosition - bufferPositionStart(containerIndex)).asInstanceOf[Int] + val sliceSize = math.min(totalLeft, currentLeft) + assert (from >= 0) + assert (sliceSize > 0 && sliceSize <= Int.MaxValue) + + val slice = container.createSlice(from, sliceSize.asInstanceOf[Int]) + arr += slice + + globalPosition += sliceSize + totalLeft -= sliceSize + if (currentLeft == sliceSize) containerIndex += 1 + } + + // Using toNonEmptyBuffer instead of directly moving to next here so that + // other checks can be performed there. + toNonEmptyBuffer() + // force cleanup - this is fine since we are not using the buffers directly + // which are actively needed (the returned value is on containers which can + // recreate) + releasePendingContainers() + // free current container if acquired. + if (currentContainerIndex < containers.length) { + containers(currentContainerIndex).release() + } + assert (currentContainerIndex == containerIndex) + + val retval = new LargeByteBuffer(arr, false, ephemeralDiskBacked) + retval.overrideCleaner(LargeByteBuffer.noopDisposeFunction) + retval + } + + // Get a composite sequence of ByteBuffer which might straddle one or more underlying buffers + // This is to be used only for writes : and ensures that writes are done into the appropriate + // underlying bytebuffers. + def getCompositeWriteBuffer(size: Long): LargeByteBuffer = { + assert(writable) + assert(size >= 0) + + createSlice(size) + } + + // get a buffer which is of the specified size and contains data from the underlying buffers + // Note, the actual data might be spread across the underlying buffers. + // This MUST BE used only for specific usecases like getInt, etc. Not for bulk copy ! + private def readFully(size: Int): ByteBuffer = { + assert (readable) + + if (remaining() < size) { + // throw exception + throw new BufferUnderflowException() + } + + // kyro depends on this it seems ? + // assert (size > 0) + if (0 == size) return LargeByteBuffer.EMPTY_BYTEBUFFER + + // Expected to be handled elsewhere. + assert (! localReadWritePossible(size)) + + val localBuff = { + val buff = fetchBufferOfSize(size) + // assert(buff.remaining() <= size) + // if (buff.remaining() == size) return buff + assert(buff.remaining() < size) + ByteBuffer.allocate(size).put(buff) + } + + // assert (localBuff.hasRemaining) + + while (localBuff.hasRemaining) { + val buff = fetchBufferOfSize(localBuff.remaining()) + localBuff.put(buff) + } + + localBuff.flip() + localBuff + } + + + + def put(b: Byte) { + assert (writable) + if (remaining() < 1) { + // logInfo("put byte. remaining = " + remaining() + ", this = " + this) + throw new BufferOverflowException + } + + assert (currentRemaining() > 0) + + fetchCurrentBuffer().put(b) + globalPosition += 1 + // Check to need to bump the index ? + toNonEmptyBuffer() + } + + + def put(buffer: ByteBuffer) { + assert (writable) + if (remaining() < buffer.remaining()) { + throw new BufferOverflowException + } + + val bufferRemaining = buffer.remaining() + if (localReadWritePossible(bufferRemaining)) { + + assert (currentRemaining() >= bufferRemaining) + + fetchCurrentBuffer().put(buffer) + + globalPosition += bufferRemaining + toNonEmptyBuffer() + return + } + + while (buffer.hasRemaining) { + val currentBufferRemaining = currentRemaining() + val bufferRemaining = buffer.remaining() + + if (currentBufferRemaining >= bufferRemaining) { + fetchCurrentBuffer().put(buffer) + globalPosition += bufferRemaining + } else { + // Split across buffers. + val currentBuffer = fetchCurrentBuffer() + assert (currentBuffer.remaining() >= currentBufferRemaining) + val sliced = ByteBufferContainer.createSlice(buffer, buffer.position(), + currentBufferRemaining) + assert (sliced.remaining() == currentBufferRemaining) + currentBuffer.put(sliced) + // move buffer pos + buffer.position(buffer.position() + currentBufferRemaining) + + globalPosition += currentBufferRemaining + } + toNonEmptyBuffer() + } + + assert (! hasRemaining() || currentRemaining() > 0) + } + + def put(other: LargeByteBuffer) { + assert (writable) + if (this.remaining() < other.remaining()) { + throw new BufferOverflowException + } + + while (other.hasRemaining()) { + val buffer = other.fetchBufferOfSize(other.currentRemaining()) + this.put(buffer) + } + } + + + def duplicate(): LargeByteBuffer = { + val containersCopy = new ArrayBuffer[ByteBufferContainer](containers.size) + // We do a duplicate as part of construction - so avoid double duplicate. + // containersCopy ++= containers.map(_.duplicate()) + containersCopy ++= containers + val retval = new LargeByteBuffer(containersCopy, true, ephemeralDiskBacked) + + // set limit and position (in that order) ... + retval.limit(this.limit()) + retval.position(this.position()) + + // Now release our containers - if any had been acquired + releasePendingContainers() + + retval + } + + + /** + * 'read' a LargeByteBuffer of size specified and return that. + * Position will be incremented by size + * + * The name might be slightly confusing : rename ? + * + * @param size Amount of data to be read from this buffer and returned + * @return + */ + def readLargeBuffer(size: Long, partialReadAllowed: Boolean): LargeByteBuffer = { + if (! hasRemaining() && ! partialReadAllowed) throw new BufferUnderflowException + if (remaining() < size && ! partialReadAllowed) throw new BufferUnderflowException + + + assert (readable) + assert (size >= 0) + + releasePendingContainers() + + if (0 == size) return LargeByteBuffer.EMPTY_BUFFER + + createSlice(size) + } + + + // This is essentially a workaround to exposing underlying buffers + def readFrom(channel: ReadableByteChannel): Long = { + + assert (writable) + releasePendingContainers() + + // this also allows us to avoid nasty corner cases in the loop. + if (! hasRemaining()) { + // logInfo("readFrom channel. remaining = " + remaining() + ", this = " + this) + throw new BufferOverflowException + } + + var totalBytesRead = 0L + + while (hasRemaining()) { + // read what we can ... + val buffer = fetchCurrentBuffer() + val bufferRemaining = currentRemaining() + val bytesRead = channel.read(buffer) + + if (bytesRead > 0) { + totalBytesRead += bytesRead + // bump position too .. + globalPosition += bytesRead + if (bytesRead >= bufferRemaining) toNonEmptyBuffer() + } + else if (-1 == bytesRead) { + // if we had already read some data in the loop, return that. + if (totalBytesRead > 0) return totalBytesRead + return -1 + } // nothing available to read, retry later. return + else if (0 == bytesRead) { + return totalBytesRead + } + + // toNonEmptyBuffer() + } + + // Cleanup last buffer ? + toNonEmptyBuffer() + totalBytesRead + } + + // This is essentially a workaround to exposing underlying buffers + def readFrom(inStrm: InputStream): Long = { + + assert (writable) + releasePendingContainers() + + // this also allows us to avoid nasty corner cases in the loop. + // if (! hasRemaining()) throw new BufferOverflowException + if (! hasRemaining()) return 0 + + var totalBytesRead = 0L + + val buff = new Array[Byte](LargeByteBuffer.TEMP_ARRAY_SIZE) + + while (hasRemaining()) { + // read what we can ... note, since there is no gaurantee that underlying buffer might + // expose array() method, we do double copy - from stream to buff and from buff to bytearray. + // see if we can optimize this later ... + val buffer = fetchCurrentBuffer() + val bufferRemaining = buffer.remaining() + val max = math.min(buff.length, bufferRemaining) + val bytesRead = inStrm.read(buff, 0, max) + + if (bytesRead > 0) { + buffer.put(buff, 0, bytesRead) + totalBytesRead += bytesRead + // bump position too .. + globalPosition += bytesRead + // buffer.position(buffer.position + bytesRead) + if (bytesRead >= bufferRemaining) toNonEmptyBuffer() + } + else if (-1 == bytesRead) { + // if we had already read some data in the loop, return that. + if (totalBytesRead > 0) return totalBytesRead + return -1 + } // nothing available to read, retry later. return + else if (0 == bytesRead) { + return totalBytesRead + } + + // toNonEmptyBuffer() + } + + totalBytesRead + } + + // This is essentially a workaround to exposing underlying buffers + // Note: very similar to readFrom(InputStream) : not trying anything fancy to reduce + // code for performance reasons. + def readFrom(inStrm: DataInput): Long = { + + assert (writable) + releasePendingContainers() + + // this also allows us to avoid nasty corner cases in the loop. + // if (! hasRemaining()) throw new BufferOverflowException + if (! hasRemaining()) return 0 + + var totalBytesRead = 0L + + val buff = new Array[Byte](LargeByteBuffer.TEMP_ARRAY_SIZE) + + while (hasRemaining()) { + // read what we can ... note, since there is no gaurantee that underlying buffer might + // expose array() method, we do double copy - from stream to buff and from buff to bytearray. + // see if we can optimize this later ... + val buffer = fetchCurrentBuffer() + val bufferRemaining = buffer.remaining() + val max = math.min(buff.length, bufferRemaining) + inStrm.readFully(buff, 0, max) + val bytesRead = max + + if (bytesRead > 0) { + buffer.put(buff, 0, bytesRead) + totalBytesRead += bytesRead + // bump position too .. + globalPosition += bytesRead + // buffer.position(buffer.position() + bytesRead) + if (bytesRead >= bufferRemaining) toNonEmptyBuffer() + } + else if (-1 == bytesRead) { + // if we had already read some data in the loop, return that. + if (totalBytesRead > 0) return totalBytesRead + return -1 + } // nothing available to read, retry later. return + else if (0 == bytesRead) { + return totalBytesRead + } + + // toNonEmptyBuffer() + } + + totalBytesRead + } + + // This is essentially a workaround to exposing underlying buffers + // Note: tries to do it efficiently without needing to load everything into memory + // (particularly for diskbacked buffers, etc). + def writeTo(channel: WritableByteChannel, cleanup: Boolean): Long = { + + assert (readable) + releasePendingContainers() + + // this also allows us to avoid nasty corner cases in the loop. + if (! hasRemaining()) throw new BufferUnderflowException + + var totalBytesWritten = 0L + + while (hasRemaining()) { + // Write what we can ... + val buffer = fetchCurrentBuffer() + val bufferRemaining = buffer.remaining() + assert (bufferRemaining > 0) + val bytesWritten = channel.write(buffer) + + if (bytesWritten > 0) { + totalBytesWritten += bytesWritten + // bump position too .. + globalPosition += bytesWritten + if (bytesWritten >= bufferRemaining) toNonEmptyBuffer() + assert (! hasRemaining() || currentRemaining() > 0) + } + else if (0 == bytesWritten) { + return totalBytesWritten + } + + // toNonEmptyBuffer() + } + + assert (! hasRemaining()) + if (cleanup) { + free() + } + totalBytesWritten + } + + // This is essentially a workaround to exposing underlying buffers + def writeTo(outStrm: OutputStream, cleanup: Boolean): Long = { + + assert (readable) + releasePendingContainers() + + // this also allows us to avoid nasty corner cases in the loop. + if (! hasRemaining()) throw new BufferUnderflowException + + var totalBytesWritten = 0L + val buff = new Array[Byte](LargeByteBuffer.TEMP_ARRAY_SIZE) + + while (hasRemaining()) { + // write what we can ... note, since there is no gaurantee that underlying buffer might + // expose array() method, we do double copy - from bytearray to buff and from + // buff to outputstream. see if we can optimize this later ... + val buffer = fetchCurrentBuffer() + val bufferRemaining = buffer.remaining() + val size = math.min(bufferRemaining, buff.length) + buffer.get(buff, 0, size) + outStrm.write(buff, 0, size) + + totalBytesWritten += size + // bump position too .. + globalPosition += size + + if (size >= bufferRemaining) toNonEmptyBuffer() + } + + toNonEmptyBuffer() + if (cleanup) { + free() + } + totalBytesWritten + } + + def asInputStream(): InputStream = { + new InputStream() { + override def read(): Int = { + if (! hasRemaining()) return -1 + get() + } + + override def read(arr: Array[Byte], off: Int, len: Int): Int = { + if (! hasRemaining()) return -1 + + get(arr, off, len) + } + + override def available(): Int = { + // current remaining is what can be read without blocking + // anything higher might need disk access/buffer swapping. + /* + val left = remaining() + math.min(left, Int.MaxValue).asInstanceOf[Int] + */ + currentRemaining() + } + } + } + + def getCleaner() = cleaner + + /** + * @param cleaner The previous cleaner, so that the caller can chain them if required. + * @return + */ + private[spark] def overrideCleaner(cleaner: BufferCleaner): BufferCleaner = { + overrideCleaner(cleaner, allowOverride = true) + } + + private def overrideCleaner(cleaner: BufferCleaner, allowOverride: Boolean): BufferCleaner = { + if (! this.allowCleanerOverride) { + // allowCleanerOverride = false is used for EMPTY_BUFFER - where we do not allow free + return this.cleaner + } + + this.allowCleanerOverride = allowOverride + assert (null != cleaner) + val prev = this.cleaner + this.cleaner = cleaner + // logInfo("Overriding " + prev + " with " + this.cleaner) + prev + } + + private def doReleaseAll() { + for (container <- containers) { + container.release() + } + } + + def free(invokeCleaner: Boolean = true) { + // logInfo("Free on " + this + ", cleaner = " + cleaner) + // always invoking release + doReleaseAll() + + if (invokeCleaner) cleaner.clean(this) + } + + private def doDispose(needRelease: Boolean) { + + if (disposeLocationThrowable ne null) { + logError("Already free'ed earlier at : ", disposeLocationThrowable) + logError("Current at ", new Throwable) + throw new IllegalStateException("Already freed.") + } + disposeLocationThrowable = new Throwable() + + // Forcefully cleanup all + if (needRelease) doReleaseAll() + + // Free in a different loop, in case different containers refer to same resource + // to release (like file) + for (container <- containers) { + container.free() + } + + needReleaseIndices.clear() + + // We should not use this buffer anymore : set the values such that f + // we dont ... + globalPosition = 0 + globalLimit = 0 + globalCapacity = 0 + } + + // copy data over ... MUST be used only for cases where array is known to be + // small to begin with. slightly risky method due to that assumption + def toByteArray(): Array[Byte] = { + val positionBackup = position() + val size = remaining() + if (size > Int.MaxValue) { + throw new IllegalStateException( + "Attempt to convert LargeByteBuffer to byte array when data held is more than 2G") + } + + val retval = new Array[Byte](size.asInstanceOf[Int]) + val readSize = get(retval, 0, retval.length) + assert (readSize == retval.length, + "readSize = " + readSize + ", retval.length = " + retval.length) + + position(positionBackup) + + retval + } + + // copy data over ... MUST be used only for cases where array is known to be + // small to begin with. slightly risky method due to that assumption + def toByteBuffer(): ByteBuffer = { + ByteBuffer.wrap(toByteArray()) + } + + def toInMemoryBuffer(ioConf: IOConfig): LargeByteBuffer = { + val retval = LargeByteBuffer.allocateMemoryBuffer(remaining(), ioConf) + val currentPosition = position() + retval.put(this) + position(currentPosition) + retval.clear() + retval + } + + + + // This is ONLY used for testing : that too as part of development of this and associated classes + // remove before contributing to spark. + def hexDump(): String = { + if (remaining() * 64 > Int.MaxValue) { + throw new UnsupportedOperationException("buffer too large " + remaining()) + } + + val sb = new StringBuilder((remaining() * 2).asInstanceOf[Int]) + + var perLine = 0 + var first = true + for (b <- toByteArray()) { + perLine += 1 + if (perLine % 8 == 0) { + sb.append('\n') + first = true + } + if (! first) sb.append(' ') + first = false + sb.append(java.lang.Integer.toHexString(b & 0xff)) + } + sb.append('\n') + sb.toString() + } + + override def toString: String = { + val sb: StringBuffer = new StringBuffer + sb.append(getClass.getName) + sb.append(' ') + sb.append(System.identityHashCode(this)) + sb.append("@[pos=") + sb.append(position()) + sb.append(" lim=") + sb.append(limit()) + sb.append(" cap=") + sb.append(capacity()) + sb.append("]") + sb.toString + } + + + + override def finalize(): Unit = { + var marked = false + if (containers ne null) { + if (containers.exists(container => container.isAcquired && container.requireRelease())) { + marked = true + logError("BUG: buffer was not released - and now going out of scope. " + + "Potential resource leak. Allocated at ", allocateLocationThrowable) + containers.foreach(_.release()) + } + if (containers.exists(container => !container.isFreed && container.requireFree())) { + if (!marked) { + logError("BUG: buffer was not freed - and now going out of scope. Potential resource leak", + allocateLocationThrowable) + } + else { + logError("BUG: buffer was not freed - and now going out of scope. Potential resource leak") + } + containers.foreach(_.free()) + } + } + super.finalize() + } +} + + +object LargeByteBuffer extends Logging { + + private val noopDisposeFunction = new BufferCleaner() { + protected def doClean(buffer: LargeByteBuffer) { + buffer.free(invokeCleaner = false) + } + } + + val enableExpensiveAssert = false + private val EMPTY_BYTEBUFFER = ByteBuffer.allocate(0) + val EMPTY_BUFFER = new LargeByteBuffer(ArrayBuffer( + new HeapByteBufferContainer(EMPTY_BYTEBUFFER, false)), false, false) + // Do not allow anyone else to override cleaner + EMPTY_BUFFER.overrideCleaner(noopDisposeFunction, allowOverride = false) + + // 8K sufficient ? + private val TEMP_ARRAY_SIZE = 8192 + + /** + * Create a LargeByteBuffer of specified size which is split across + * ByteBuffer's of size DiskStore.MAX_BLOCK_SIZE and backed by in memory + * ByteBuffer + * + */ + def allocateMemoryBuffer(totalSize: Long, ioConf: IOConfig): LargeByteBuffer = { + if (0 == totalSize) { + return EMPTY_BUFFER + } + + assert (totalSize > 0) + + val blockSize = ioConf.getMaxBlockSize(BufferType.MEMORY) + val numBlocks = ioConf.numBlocks(BufferType.MEMORY, totalSize) + val lastBlockSize = ioConf.lastBlockSize(BufferType.MEMORY, totalSize) + + assert (lastBlockSize > 0) + + val bufferArray = { + val arr = new ArrayBuffer[ByteBufferContainer](numBlocks) + for (index <- 0 until numBlocks - 1) { + val buff = ByteBuffer.allocate(blockSize) + // buff.clear() + arr += new HeapByteBufferContainer(buff, true) + } + arr += new HeapByteBufferContainer(ByteBuffer.allocate(lastBlockSize), true) + assert (arr.length == numBlocks) + arr + } + + new LargeByteBuffer(bufferArray, false, false) + } + + /** + * Create a LargeByteBuffer of specified size which is split across + * ByteBuffer's of size DiskStore.MAX_BLOCK_SIZE and backed by on disk + * + */ + private def allocateDiskBuffer(totalSize: Long, + blockManager: BlockManager): LargeByteBuffer = { + if (0 == totalSize) { + return EMPTY_BUFFER + } + + assert (totalSize > 0) + + // Create a file of the specified size. + val file = blockManager.diskBlockManager.createTempBlock()._2 + val raf = new RandomAccessFile(file, "rw") + try { + raf.setLength(totalSize) + } finally { + raf.close() + } + + readWriteDiskSegment(new FileSegment(file, 0, totalSize), + ephemeralDiskBacked = true, blockManager.ioConf) + } + + // The returned buffer takes up ownership of the underlying buffers + // (including dispos'ing that when done) + def fromBuffers(buffers: ByteBuffer*): LargeByteBuffer = { + val nonEmpty = buffers.filter(_.hasRemaining) + + // cleanup the empty buffers + buffers.filter(! _.hasRemaining).foreach(b => BlockManager.dispose(b)) + + + if (nonEmpty.isEmpty) { + return EMPTY_BUFFER + } + + // slice so that offsets match our requirement + new LargeByteBuffer(new ArrayBuffer() ++ nonEmpty.map(b => + new HeapByteBufferContainer(b.slice(), true)), false, false) + } + + def fromByteArrays(byteArrays: Array[Byte]*): LargeByteBuffer = { + // only non empty arrays + val arrays = byteArrays.filter(_.length > 0) + if (0 == arrays.length) return EMPTY_BUFFER + + new LargeByteBuffer(new ArrayBuffer() ++ arrays.map(arr => + new HeapByteBufferContainer(ByteBuffer.wrap(arr), true)), false, false) + } + + def fromLargeByteBuffers(canDispose: Boolean, inputBuffers: LargeByteBuffer*): LargeByteBuffer = { + + if (inputBuffers.isEmpty) return EMPTY_BUFFER + + if (! inputBuffers.exists(_.hasRemaining())) { + if (canDispose) inputBuffers.map(_.free()) + return EMPTY_BUFFER + } + + // release all temp resources acquired + inputBuffers.foreach(buff => buff.releasePendingContainers()) + // free current container if acquired. + inputBuffers.foreach(buff => if (buff.currentContainerIndex < buff.containers.length) { + buff.containers(buff.currentContainerIndex).release() + }) + // inputBuffers.foreach(b => b.doReleaseAll()) + + + // Dispose of any empty buffers + if (canDispose) inputBuffers.filter(! _.hasRemaining()).foreach(_.free()) + + // Find all containers we need. + val buffers = inputBuffers.filter(_.hasRemaining()).map(b => b.createSlice(b.remaining())) + + val containers = buffers.flatMap(_.containers) + assert (! containers.isEmpty) + // The in order containers of "buffers" seq constitute the required return value + val retval = new LargeByteBuffer(new ArrayBuffer() ++ containers, + // if you cant dispose, then we dont own the buffers : in which case, need duplicate + ! canDispose, inputBuffers.exists(_.ephemeralDiskBacked)) + + if (canDispose) { + // override dispose of all other buffers. + val disposeFunctions = inputBuffers.map { + buffer => { + (buffer, buffer.overrideCleaner(noopDisposeFunction)) + } + } + + val cleaner = retval.getCleaner() + val newCleaner = new BufferCleaner { + protected def doClean(buffer: LargeByteBuffer) { + + assert (retval == buffer) + // default cleaner. + cleaner.clean(retval) + // not required, since we are within clean anyway. + // retval.free(invokeCleaner = false) + + // retval.doDispose(needRelease = true) + + // This might actually call dispose twice on some (initially) empty buffers, + // which is fine since we now guard against that. + disposeFunctions.foreach(v => v._2.clean(v._1)) + // Call the free method too : so that buffers are marked free ... + disposeFunctions.foreach(v => v._1.free(invokeCleaner = false)) + } + } + + val prev = retval.overrideCleaner(newCleaner) + assert (prev == cleaner) + } + + retval + } + + private def checkOffsets(arr: Array[Byte], offset: Int, size: Int) { + if (arr == null) { + throw new NullPointerException + } else if (offset < 0 || size < 0 || offset + size > arr.length) { + throw new IndexOutOfBoundsException + } + } + + def allocateTransientBuffer(size: Long, blockManager: BlockManager) = { + if (size <= blockManager.ioConf.maxInMemSize) { + LargeByteBuffer.allocateMemoryBuffer(size, blockManager.ioConf) + } else { + LargeByteBuffer.allocateDiskBuffer(size, blockManager) + } + } + + def readFromDiskSegment(segment: FileSegment, ioConf: IOConfig, + ephemeralDiskBacked: Boolean): LargeByteBuffer = { + // Split the block into multiple of BlockStore.maxBlockSize + val segmentSize = segment.length + val blockSize = ioConf.getMaxBlockSize(BufferType.DISK).asInstanceOf[Long] + val numBlocks = ioConf.numBlocks(BufferType.DISK, segmentSize) + val lastBlockSize = ioConf.lastBlockSize(BufferType.DISK, segmentSize) + + val buffers = new ArrayBuffer[ByteBufferContainer](numBlocks) + + for (index <- 0 until numBlocks - 1) { + buffers += new ReadOnlyFileContainer(new FileSegment(segment.file, + segment.offset + index * blockSize, blockSize), ioConf) + } + + // Last block + buffers += new ReadOnlyFileContainer(new FileSegment(segment.file, + segment.offset + (numBlocks - 1) * blockSize, lastBlockSize), ioConf) + + new LargeByteBuffer(buffers, false, ephemeralDiskBacked) + } + + def readWriteDiskSegment(segment: FileSegment, ephemeralDiskBacked: Boolean, + ioConf: IOConfig): LargeByteBuffer = { + + // Split the block into multiple of BlockStore.maxBlockSize + val segmentSize = segment.length + val blockSize = ioConf.getMaxBlockSize(BufferType.DISK).asInstanceOf[Long] + val numBlocks = ioConf.numBlocks(BufferType.DISK, segmentSize) + val lastBlockSize = ioConf.lastBlockSize(BufferType.DISK, segmentSize) + + logInfo("readWriteDiskSegment = " + segment + ", numBlocks = " + numBlocks + + ", lastBlockSize = " + lastBlockSize) + val buffers = new ArrayBuffer[ByteBufferContainer](numBlocks) + + for (index <- 0 until numBlocks - 1) { + buffers += new ReadWriteFileContainer(new FileSegment(segment.file, + segment.offset + index * blockSize, blockSize), ephemeralDiskBacked, null) + } + + // Last block + buffers += new ReadWriteFileContainer(new FileSegment(segment.file, + segment.offset + (numBlocks - 1) * blockSize, lastBlockSize), ephemeralDiskBacked, null) + + new LargeByteBuffer(buffers, false, ephemeralDiskBacked) + } +} diff --git a/core/src/main/scala/org/apache/spark/io/WrappedByteArrayOutputStream.scala b/core/src/main/scala/org/apache/spark/io/WrappedByteArrayOutputStream.scala new file mode 100644 index 0000000000000..0dd7e8e736ad6 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/io/WrappedByteArrayOutputStream.scala @@ -0,0 +1,121 @@ +/* + * 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.io + +import java.io.OutputStream +import scala.collection.mutable.ArrayBuffer + +import org.apache.spark.Logging +import org.apache.spark.io.IOConfig.BufferType + +/** + * byte array backed streams (FastByteArrayOutputStream, ByteArrayOutputStream, etc) are limited to + * array length of 2 gig - since that is the array size limit. + * + * So we move from one to the next as soon as we hit the limit per stream. + * And once done, asBuffers or toByteArrays can be used to pull data as a sequence of bytebuffers + * or byte arrays. + * @param initialSize initial size for the byte array stream ... + */ +class WrappedByteArrayOutputStream(private val initialSize: Int, + ioConf: IOConfig) extends OutputStream with Logging { + + private val maxStreamSize = ioConf.getMaxBlockSize(BufferType.MEMORY) + + private val allStreams = new ArrayBuffer[SparkByteArrayOutputStream](4) + + private var current: SparkByteArrayOutputStream = null + private var currentWritten = 0 + + nextWriter() + + override def flush(): Unit = { + current.flush() + } + + override def write(b: Int): Unit = { + if (currentWritten >= maxStreamSize) { + nextWriter() + } + current.write(b) + currentWritten += 1 + } + + + override def write(b: Array[Byte], off: Int, len: Int): Unit = { + // invariant checks - from OutputStream.java + if (b == null) { + throw new NullPointerException + } else if ((off < 0) || (off > b.length) || (len < 0) || + ((off + len) > b.length) || ((off + len) < 0)) { + throw new IndexOutOfBoundsException + } else if (len == 0) { + return + } + + // Else, write to stream. + + // common case first + if (currentWritten + len < maxStreamSize) { + current.write(b, off, len) + currentWritten += len + return + } + + // We might need to split the write into two streams. + var startOff = off + var remaining = len + + while (remaining > 0) { + var toCurrent = math.min(remaining, maxStreamSize - currentWritten) + if (toCurrent > 0) { + current.write(b, startOff, toCurrent) + currentWritten += toCurrent + remaining -= toCurrent + startOff += toCurrent + } + + if (currentWritten >= maxStreamSize) { + // to next + nextWriter() + } + } + } + + def toLargeByteBuffer(): LargeByteBuffer = { + current.compact() + val seq = allStreams.filter(_.size > 0).map(_.toByteBuffer) + val retval = LargeByteBuffer.fromBuffers(seq:_*) + + retval + } + + private def nextWriter() { + if (null != current) { + current.flush() + current.compact() + current = null + } + + current = new SparkByteArrayOutputStream(initialSize, ioConf) + currentWritten = 0 + allStreams += current + } +} + + diff --git a/core/src/main/scala/org/apache/spark/storage/BlockStore.scala b/core/src/main/scala/org/apache/spark/storage/BlockStore.scala index 69985c9759e2d..b14b5e91d1794 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockStore.scala @@ -17,18 +17,24 @@ package org.apache.spark.storage -import java.nio.ByteBuffer - import scala.collection.mutable.ArrayBuffer import org.apache.spark.Logging +import org.apache.spark.io.LargeByteBuffer /** * Abstract class to store blocks. */ private[spark] abstract class BlockStore(val blockManager: BlockManager) extends Logging { - - def putBytes(blockId: BlockId, bytes: ByteBuffer, level: StorageLevel): PutResult + // TODO: We have inconsistent usage of the bytes in spark. + // In DiskStore, we simply emit bytes to the file without a rewind + // While in memory and tachyon store, we do a rewind. + // Not sure which is correct - since both seem to be working fine in the tests ! + // There is some underlying assumption which is probably unspecified and incorrect + // in a general case. + // Change: consistently modified to do a rewind before calling this method. + // Now, it validates that position == 0 (and so remaining == limit obviously) + def putBytes(blockId: BlockId, bytes: LargeByteBuffer, level: StorageLevel) : PutResult /** * Put in a block and, possibly, also return its content as either bytes or another Iterator. @@ -37,15 +43,15 @@ private[spark] abstract class BlockStore(val blockManager: BlockManager) extends * @return a PutResult that contains the size of the data, as well as the values put if * returnValues is true (if not, the result's data field can be null) */ - def putIterator( + def putValues( blockId: BlockId, values: Iterator[Any], level: StorageLevel, returnValues: Boolean): PutResult - def putArray( + def putValues( blockId: BlockId, - values: Array[Any], + values: ArrayBuffer[Any], level: StorageLevel, returnValues: Boolean): PutResult @@ -54,7 +60,7 @@ private[spark] abstract class BlockStore(val blockManager: BlockManager) extends */ def getSize(blockId: BlockId): Long - def getBytes(blockId: BlockId): Option[ByteBuffer] + def getBytes(blockId: BlockId): Option[LargeByteBuffer] def getValues(blockId: BlockId): Option[Iterator[Any]] From d6337f03a4ac2971a004ef821281723e857f9008 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Tue, 24 Feb 2015 12:18:38 -0600 Subject: [PATCH 03/97] wip -- changed a bunch of types to LargeByteBuffer; discovered problem on replicate() --- .../org/apache/spark/io/LargeByteBuffer.scala | 2961 +++++++++-------- .../io/WrappedByteArrayOutputStream.scala | 242 +- .../apache/spark/storage/BlockManager.scala | 46 +- .../org/apache/spark/storage/BlockStore.scala | 8 +- .../org/apache/spark/storage/DiskStore.scala | 7 +- .../apache/spark/storage/MemoryStore.scala | 4 +- .../org/apache/spark/storage/PutResult.scala | 4 +- .../apache/spark/storage/TachyonStore.scala | 5 +- .../util/LargeByteBufferInputStream.scala | 82 + .../util/LargeByteBufferOutputStream.scala | 45 + .../spark/util/collection/ChainedBuffer.scala | 126 + .../apache/spark/io/LargeByteBufferTest.scala | 57 + .../util/collection/ChainedBufferTest.scala | 135 + 13 files changed, 2173 insertions(+), 1549 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/util/LargeByteBufferInputStream.scala create mode 100644 core/src/main/scala/org/apache/spark/util/LargeByteBufferOutputStream.scala create mode 100644 core/src/main/scala/org/apache/spark/util/collection/ChainedBuffer.scala create mode 100644 core/src/test/scala/org/apache/spark/io/LargeByteBufferTest.scala create mode 100644 core/src/test/scala/org/apache/spark/util/collection/ChainedBufferTest.scala diff --git a/core/src/main/scala/org/apache/spark/io/LargeByteBuffer.scala b/core/src/main/scala/org/apache/spark/io/LargeByteBuffer.scala index 509fe186793b6..01bd433f55c78 100644 --- a/core/src/main/scala/org/apache/spark/io/LargeByteBuffer.scala +++ b/core/src/main/scala/org/apache/spark/io/LargeByteBuffer.scala @@ -21,275 +21,80 @@ import java.io.{RandomAccessFile, DataInput, InputStream, OutputStream} import java.nio.{ByteBuffer, BufferUnderflowException, BufferOverflowException} import java.nio.channels.{WritableByteChannel, ReadableByteChannel} +import org.apache.spark.util.collection.ChainedBuffer + import scala.collection.mutable.{ArrayBuffer, HashSet} import org.apache.spark.Logging -import org.apache.spark.io.IOConfig.BufferType import org.apache.spark.storage.{FileSegment, BlockManager} -/** - * This is used to control chaining of cleaners. - * For some usecases, invocation of clean on LargeByteBuffer must not immediately clean - * but be tied to clean of other buffers. - * This allows a way to override how/when clean is invoked - * - * TODO: We should revisit this design and see if we can use ref tracking : might be more general - * at higher cost ? Punting on it for now - */ -trait BufferCleaner extends Logging { - @volatile private var invoked = false - - protected def doClean(buffer: LargeByteBuffer) - - final def clean(buffer: LargeByteBuffer) { - if (invoked) return - invoked = true - doClean(buffer) - } -} - -/** - * This is a variant of ByteBuffer to be used internally in spark, which is not limited to 2G - * which ByteBuffers are limited to. - * Externally, it exposes all the api which java.nio.ByteBuffer exposes. - * Internally, it maintains a sequence of Containers which manage the ByteBuffer data. - * Not all the data might be loaded into memory (like disk or tachyon data) - so actual - * memory footprint - heap and vm could be much lower than capacity. - * - * TODO: Currently we are slightly fast and loose in terms of concurrent modifications to this - * buffer, maybe revisit this later ? Note: this is not much different from earlier though ! - * - * TODO: Explore if (at all) we can leverage zero copy transfers. The issue (currently) is that this - * will require the file to be kept open (repeatedly opening/closing file is not good - * for each transfer) and this has an impact on ulimit. Not to mention writing of mmap'ed buffer is - * pretty quick (it is the first failover in case direct transfer is not possible in file zero copy) - * - * TODO: After redesign to containers, we got rid of parent containers to free - the side effect is - * that if there are direct ByteBuffers, we are not handling explicit cleanup of those in some - * cases (when we duplicate/slice them). Currently spark does not need this, but might in future - * so relook at it later. - */ -// We should make this constructor private: but for now, -// leaving it public since TachyonStore needs it -class LargeByteBuffer private[spark](private val inputContainers: ArrayBuffer[ByteBufferContainer], - private val needDuplicate: Boolean, val ephemeralDiskBacked: Boolean) extends Logging { - - // TODO: TEMP code: to flush out potential resource leaks. REMOVE ME - private val allocateLocationThrowable: Throwable = { - if (inputContainers.exists(c => c.requireRelease() || c.requireFree())) { - new Throwable("blockId = " + BlockManager.getLookupBlockId) - } else { - null - } - } - private var disposeLocationThrowable: Throwable = null - - @volatile private var allowCleanerOverride = true - @volatile private var cleaner: BufferCleaner = new BufferCleaner { - override def doClean(buffer: LargeByteBuffer) = { - assert (LargeByteBuffer.this == buffer) - doDispose(needRelease = false) - } - } - - // should not be empty - assert (null != inputContainers && ! inputContainers.isEmpty) - // should not have any null's - assert (inputContainers.find(_ == null).isEmpty) - - // println("Num containers = " + inputContainers.size) - - // Position, limit and capacity relevant over the engire LargeByteBuffer - @volatile private var globalPosition = 0L - @volatile private var globalLimit = 0L - @volatile private var currentContainerIndex = 0 - - // The buffers in which the actual data is held. - private var containers: Array[ByteBufferContainer] = null - - // aggregate capacities of the individual buffers. - // bufferPositionStart(0) will be capacity of 1st buffer, bufferPositionStart(1) will be - // sum of capacity of 0th and 1st block buffer - private var bufferPositionStart: Array[Long] = null - // Contains the indices of a containers which requires release before subsequent invocation of - // read/write should be serviced. This is required since current read/write might have moved the - // position but since we are returning bytebuffers which depend on the validity of the existing - // bytebuffer, we cant release them yet. - private var needReleaseIndices = new HashSet[Int]() - private val readable = ! inputContainers.exists(! _.isReadable) - private val writable = ! inputContainers.exists(! _.isWritable) +trait LargeByteBuffer { +// def position(): Long +// +// def limit(): Long + def capacity(): Long - // initialize - @volatile private var globalCapacity = { + def get(): Byte //needed for ByteBufferInputStream - // Ensure that there are no empty buffers : messes up with our code : unless it - // is a single buffer (for empty buffer for marker case) - assert (inputContainers.find(0 == _.capacity()).isEmpty || 1 == inputContainers.length) + def get(dst: Array[Byte], offset: Int, length: Int): Unit // for ByteBufferInputStream - containers = { - if (needDuplicate) inputContainers.map(_.duplicate()).toArray else inputContainers.toArray - } - containers.foreach(_.validate()) + def position(position: Long): Unit //for ByteBufferInputStream - def initializeBufferPositionStart(arr: Array[ByteBufferContainer]) { - val buff = new ArrayBuffer[Long](arr.length + 1) - buff += 0L + def position(): Long //for ByteBufferInputStream - buff ++= arr.map(_.capacity().asInstanceOf[Long]).scanLeft(0L)(_ + _).slice(1, arr.length + 1) - assert (buff.length == arr.length + 1) - bufferPositionStart = buff.toArray - } + /** doesn't copy data, just copies references & offsets */ + def duplicate(): LargeByteBuffer - initializeBufferPositionStart(containers) + def put(bytes: LargeByteBuffer): Unit - // remove references from inputBuffers - inputContainers.clear() + //also need whatever is necessary for ByteArrayOutputStream for BlockManager#dataSerialize - globalLimit = bufferPositionStart(containers.length) - globalPosition = 0L - currentContainerIndex = 0 - - assert (globalLimit == containers.map(_.capacity().asInstanceOf[Long]).sum) - - globalLimit - } - - final def position(): Long = globalPosition - - final def limit(): Long = globalLimit - - final def capacity(): Long = globalCapacity - - final def limit(newLimit: Long) { - if ((newLimit > capacity()) || (newLimit < 0)) { - throw new IllegalArgumentException("newLimit = " + newLimit + ", capacity = " + capacity()) - } - - globalLimit = newLimit - if (position() > newLimit) position(newLimit) - } - - def skip(skipBy: Long) = position(position() + skipBy) - - private def releasePendingContainers() { - if (! needReleaseIndices.isEmpty) { - val iter = needReleaseIndices.iterator - while (iter.hasNext) { - val index = iter.next() - assert (index >= 0 && index < containers.length) - // It is possible to move from one container to next before the previous - // container was acquired. For example, get forcing move to next container - // since current was exhausted immediatelly followed by a position() - // so the container we moved to was never acquired. - - // assert (containers(index).isAcquired) - // will this always be satisfied ? - // assert (index != currentContainerIndex) - if (containers(index).isAcquired) containers(index).release() - } - needReleaseIndices.clear() - } - } - - private def toNewContainer(newIndex: Int) { - if (newIndex != currentContainerIndex && currentContainerIndex < containers.length) { - - assert (currentContainerIndex >= 0) - needReleaseIndices += currentContainerIndex - } - currentContainerIndex = newIndex - } - - // expensive method, sigh ... optimize it later ? - final def position(newPosition: Long) { - - if ((newPosition > globalLimit) || (newPosition < 0)) throw new IllegalArgumentException() - - if (currentContainerIndex < bufferPositionStart.length - 1 && - newPosition >= bufferPositionStart(currentContainerIndex) && - newPosition < bufferPositionStart(currentContainerIndex + 1)) { - // Same buffer - easy method ... - globalPosition = newPosition - // Changed position - free previously returned buffers. - releasePendingContainers() - return - } - - // Find appropriate currentContainerIndex - // Since bufferPositionStart is sorted, can be replaced with binary search if required. - // For now, not in the perf critical path since buffers size is very low typically. - var index = 0 - val cLen = containers.length - while (index < cLen) { - if (newPosition >= bufferPositionStart(index) && - newPosition < bufferPositionStart(index + 1)) { - globalPosition = newPosition - toNewContainer(index) - // Changed position - free earlier and previously returned buffers. - releasePendingContainers() - return - } - index += 1 - } - - if (newPosition == globalLimit && newPosition == bufferPositionStart(cLen)) { - // boundary. - globalPosition = newPosition - toNewContainer(cLen) - // Changed position - free earlier and previously returned buffers. - releasePendingContainers() - return - } - - assert (assertion = false, "Unexpected to come here .... newPosition = " + newPosition + - ", bufferPositionStart = " + bufferPositionStart.mkString("[", ", ", "]")) - } + //TODO checks on limit semantics /** - * Clears this buffer. The position is set to zero, the limit is set to - * the capacity, and the mark is discarded. - * - *

Invoke this method before using a sequence of channel-read or - * put operations to fill this buffer. - * - *

This method does not actually erase the data in the buffer, but it - * is named as if it did because it will most often be used in situations - * in which that might as well be the case.

+ * Sets this buffer's limit. If the position is larger than the new limit then it is set to the + * new limit. If the mark is defined and larger than the new limit then it is discarded. */ - final def clear() { - // if (0 == globalCapacity) return - - needReleaseIndices += 0 - globalPosition = 0L - toNewContainer(0) - globalLimit = globalCapacity - - // Now free all pending containers - releasePendingContainers() - } + def limit(newLimit: Long): Unit /** - * Flips this buffer. The limit is set to the current position and then - * the position is set to zero. If the mark is defined then it is - * discarded. - * - *

After a sequence of channel-read or put operations, invoke - * this method to prepare for a sequence of channel-write or relative - * get operations. + * return this buffer's limit + * @return */ - final def flip() { - needReleaseIndices += 0 - globalLimit = globalPosition - globalPosition = 0L - toNewContainer(0) - - // Now free all pending containers - releasePendingContainers() - } + def limit(): Long + +// +// def skip(skipBy: Long): Unit +// +// def position(newPosition: Long): Unit +// +// /** +// * Clears this buffer. The position is set to zero, the limit is set to +// * the capacity, and the mark is discarded. +// * +// *

Invoke this method before using a sequence of channel-read or +// * put operations to fill this buffer. +// * +// *

This method does not actually erase the data in the buffer, but it +// * is named as if it did because it will most often be used in situations +// * in which that might as well be the case.

+// */ +// def clear(): Unit +// +// /** +// * Flips this buffer. The limit is set to the current position and then +// * the position is set to zero. If the mark is defined then it is +// * discarded. +// * +// *

After a sequence of channel-read or put operations, invoke +// * this method to prepare for a sequence of channel-write or relative +// * get operations. +// */ +// def flip(): Unit /** * Rewinds this buffer. The position is set to zero and the mark is @@ -299,14 +104,7 @@ class LargeByteBuffer private[spark](private val inputContainers: ArrayBuffer[By * operations, assuming that the limit has already been set * appropriately. */ - final def rewind() { - needReleaseIndices += 0 - globalPosition = 0L - toNewContainer(0) - - // Now free all pending containers - releasePendingContainers() - } + def rewind(): Unit /** * Returns the number of elements between the current position and the @@ -314,1192 +112,1555 @@ class LargeByteBuffer private[spark](private val inputContainers: ArrayBuffer[By * * @return The number of elements remaining in this buffer */ - final def remaining(): Long = { - globalLimit - globalPosition - } - - /** - * Tells whether there are any elements between the current position and - * the limit.

- * - * @return true if, and only if, there is at least one element - * remaining in this buffer - */ - final def hasRemaining() = { - globalPosition < globalLimit - } - - // private def currentBuffer(): ByteBuffer = buffers(currentContainerIndex) - - // number of bytes remaining in currently active underlying buffer - private def currentRemaining(): Int = { - if (hasRemaining()) { - // validate currentContainerIndex is valid - assert (globalPosition >= bufferPositionStart(currentContainerIndex) && - globalPosition < bufferPositionStart(currentContainerIndex + 1), - "globalPosition = " + globalPosition + - ", currentContainerIndex = " + currentContainerIndex + - ", bufferPositionStart = " + bufferPositionStart.mkString("[", ", ", " ]")) - - currentRemaining0(currentContainerIndex) - } else 0 - } - - // Without any validation : required when we are bumping the index (when validation will fail) ... - private def currentRemaining0(which: Int): Int = { - // currentBuffer().remaining() - math.max(0, math.min(bufferPositionStart(which + 1), - globalLimit) - globalPosition).asInstanceOf[Int] - } - - // Set the approppriate position/limit for the current underlying buffer to mirror our - // the LargeByteBuffer's state. - private def fetchCurrentBuffer(): ByteBuffer = { - releasePendingContainers() - - assert (currentContainerIndex < containers.length) - - val container = containers(currentContainerIndex) - if (! container.isAcquired) { - container.acquire() - } - - assert (container.isAcquired) - if (LargeByteBuffer.enableExpensiveAssert) { - assert (! containers.exists( b => (b ne container) && b.isAcquired)) - } - - assert (currentContainerIndex < bufferPositionStart.length && - globalPosition < bufferPositionStart(currentContainerIndex + 1), - "currentContainerIndex = " + currentContainerIndex + ", bufferPositionStart = " + - bufferPositionStart.mkString("[", ", ", "]") + ", this = " + this) - - val buffPosition = (globalPosition - bufferPositionStart(currentContainerIndex)). - asInstanceOf[Int] - - val buffer = container.getByteBuffer - buffer.position(buffPosition) - val diff = buffer.capacity - buffPosition - val left = remaining() - if (diff <= left) { - buffer.limit(buffer.capacity()) - } else { - // Can happen if limit() was called. - buffer.limit(buffPosition + left.asInstanceOf[Int]) - } - - buffer - } - - // To be used ONLY to test in suites. - private[spark] def fetchCurrentBufferForTesting(): ByteBuffer = { - if ("1" != System.getProperty("SPARK_TESTING")) { - throw new IllegalStateException("This method is to be used ONLY within spark test suites") - } - - fetchCurrentBuffer() - } - - // Expects that the invoker has ensured that this can be safely invoked. - // That is, it wont be invoked when the loop wont terminate. - private def toNonEmptyBuffer() { - - if (! hasRemaining()) { - var newIndex = currentContainerIndex - // Ensure we are in the right block or not. - while (newIndex < containers.length && globalPosition >= bufferPositionStart(newIndex + 1)) { - newIndex += 1 - } - toNewContainer(newIndex) - // Do not do this - since we might not yet have consumed the buffer which caused EOF right now - /* - // Add last one also, and release it too - since we are at the end of the buffer with nothing - // more pending. - if (newIndex >= 0 && currentContainerIndex < containers.length) { - needReleaseIndices += newIndex - } - */ - assert (currentContainerIndex >= 0) - // releasePendingContainers() - return - } - - var index = currentContainerIndex - while (0 == currentRemaining0(index) && index < containers.length) { - index += 1 - } - assert (currentContainerIndex < containers.length) - toNewContainer(index) - assert (0 != currentRemaining()) - } - - private def assertPreconditions(containerIndex: Int) { - assert (globalPosition >= bufferPositionStart(containerIndex), - "globalPosition = " + globalPosition + ", containerIndex = " + containerIndex + - ", bufferPositionStart = " + bufferPositionStart.mkString("[", ", ", " ]")) - assert (globalPosition < bufferPositionStart(containerIndex + 1), - "globalPosition = " + globalPosition + ", containerIndex = " + containerIndex + - ", bufferPositionStart = " + bufferPositionStart.mkString("[", ", ", " ]")) - - assert (globalLimit <= globalCapacity) - assert (containerIndex < containers.length) - } - - - /** - * Attempts to return a ByteBuffer of the requested size. - * It is possible to return a buffer of size smaller than requested - * even though hasRemaining == true - * - * On return, position would have been moved 'ahead' by the size of the buffer returned : - * that is, we treat that the returned buffer has been already 'read' from this LargeByteBuffer - * - * - * This is used to primarily retrieve content of this buffer to expose via ByteBuffer - * to some other api which is deemed too cumbersome to move to LargeByteBuffer (like the - * chunked sending of contents via ConnectionManager) Note that the lifecycle of the ByteBuffer - * returned is inherently tied to the state of this LargeByteBuffer. For example,if the underlying - * container is a disk backed container, and we make subsequent calls to get(), the returned - * ByteBuffer can be dispose'ed off - * - * @param maxChunkSize Max size of the ByteBuffer to retrieve. - * @return - */ - - private def fetchBufferOfSize(maxChunkSize: Int): ByteBuffer = { - fetchBufferOfSizeImpl(maxChunkSize, canReleaseContainers = true) - } - - private def fetchBufferOfSizeImpl(maxChunkSize: Int, - canReleaseContainers: Boolean): ByteBuffer = { - if (canReleaseContainers) releasePendingContainers() - assert (maxChunkSize > 0) - - // not checking for degenerate case of maxChunkSize == 0 - if (globalPosition >= globalLimit) { - // throw exception - throw new BufferUnderflowException() - } - - // Check preconditions : disable these later, since they might be expensive to - // evaluate for every IO op - assertPreconditions(currentContainerIndex) - - val currentBufferRemaining = currentRemaining() - - assert (currentBufferRemaining > 0) - - val size = math.min(currentBufferRemaining, maxChunkSize) - - val newBuffer = if (currentBufferRemaining > maxChunkSize) { - val currentBuffer = fetchCurrentBuffer() - val buff = ByteBufferContainer.createSlice(currentBuffer, - currentBuffer.position(), maxChunkSize) - assert (buff.remaining() == maxChunkSize) - buff - } else { - val currentBuffer = fetchCurrentBuffer() - val buff = currentBuffer.slice() - assert (buff.remaining() == currentBufferRemaining) - buff - } - - assert (size == newBuffer.remaining()) - assert (0 == newBuffer.position()) - assert (size == newBuffer.limit()) - assert (newBuffer.capacity() == newBuffer.limit()) - - globalPosition += newBuffer.remaining - toNonEmptyBuffer() - - newBuffer - } - - // Can we service the read/write from the currently active (underlying) bytebuffer or not. - // For almost all cases, this will return true allowing us to optimize away the more expensive - // computations. - private def localReadWritePossible(size: Int) = - size >= 0 && globalPosition + size <= bufferPositionStart(currentContainerIndex + 1) - - - def getLong(): Long = { - assert (readable) - releasePendingContainers() - - if (remaining() < 8) throw new BufferUnderflowException - - if (localReadWritePossible(8)) { - val buff = fetchCurrentBuffer() - assert (buff.remaining() >= 8) - val retval = buff.getLong - globalPosition += 8 - toNonEmptyBuffer() - return retval - } - - val buff = readFully(8) - buff.getLong - } - - def getInt(): Int = { - assert (readable) - releasePendingContainers() - - if (remaining() < 4) throw new BufferUnderflowException - - if (localReadWritePossible(4)) { - val buff = fetchCurrentBuffer() - assert (buff.remaining() >= 4) - val retval = buff.getInt - globalPosition += 4 - toNonEmptyBuffer() - return retval - } - - val buff = readFully(4) - buff.getInt - } + def remaining(): Long +} - def getChar(): Char = { - assert (readable) - releasePendingContainers() +class ChainedLargeByteBuffer(private[io] val underlying: ChainedBuffer) extends LargeByteBuffer { - if (remaining() < 2) throw new BufferUnderflowException + def capacity = underlying.capacity - if (localReadWritePossible(2)) { - val buff = fetchCurrentBuffer() - assert (buff.remaining() >= 2) - val retval = buff.getChar - globalPosition += 2 - toNonEmptyBuffer() - return retval - } + var _pos = 0l - // if slice is becoming too expensive, revisit this ... - val buff = readFully(2) - buff.getChar + def get(dst: Array[Byte],offset: Int,length: Int): Unit = { + underlying.read(_pos, dst, offset, length) + _pos += length } def get(): Byte = { - assert (readable) - releasePendingContainers() - - if (! hasRemaining()) throw new BufferUnderflowException - - // If we have remaining bytes, previous invocations MUST have ensured that we are at - // a buffer which has data to be read. - assert (localReadWritePossible(1)) - - val buff = fetchCurrentBuffer() - assert (buff.remaining() >= 1, "buff.remaining = " + buff.remaining()) - val retval = buff.get() - globalPosition += 1 - toNonEmptyBuffer() - - retval - } - - def get(arr: Array[Byte], offset: Int, size: Int): Int = { - assert (readable) - releasePendingContainers() - - LargeByteBuffer.checkOffsets(arr, offset, size) - - // kyro depends on this it seems ? - // assert (size > 0) - if (0 == size) return 0 - - if (! hasRemaining()) return -1 - - if (localReadWritePossible(size)) { - val buff = fetchCurrentBuffer() - assert (buff.remaining() >= size) - buff.get(arr, offset, size) - globalPosition += size - toNonEmptyBuffer() - return size - } - - var remainingSize = math.min(size, remaining()).asInstanceOf[Int] - var currentOffset = offset - - while (remainingSize > 0) { - val buff = fetchBufferOfSize(remainingSize) - val toCopy = math.min(buff.remaining(), remainingSize) - - buff.get(arr, currentOffset, toCopy) - currentOffset += toCopy - remainingSize -= toCopy - } - - currentOffset - offset - } - - - private def createSlice(size: Long): LargeByteBuffer = { - - releasePendingContainers() - - if (remaining() < size) { - // logInfo("createSlice. remaining = " + remaining() + ", size " + size + ", this = " + this) - throw new BufferOverflowException - } - - // kyro depends on this it seems ? - // assert (size > 0) - if (0 == size) return LargeByteBuffer.EMPTY_BUFFER - - val arr = new ArrayBuffer[ByteBufferContainer](2) - var totalLeft = size - - // assert (currentRemaining() < totalLeft || totalLeft != size || currentAsByteBuffer) - - var containerIndex = currentContainerIndex - while (totalLeft > 0 && hasRemaining()) { - assertPreconditions(containerIndex) - val container = containers(containerIndex) - val currentLeft = currentRemaining0(containerIndex) - - assert (globalPosition + currentLeft <= globalLimit) - assert (globalPosition >= bufferPositionStart(containerIndex) && - (globalPosition < bufferPositionStart(containerIndex + 1))) - - val from = (globalPosition - bufferPositionStart(containerIndex)).asInstanceOf[Int] - val sliceSize = math.min(totalLeft, currentLeft) - assert (from >= 0) - assert (sliceSize > 0 && sliceSize <= Int.MaxValue) - - val slice = container.createSlice(from, sliceSize.asInstanceOf[Int]) - arr += slice - - globalPosition += sliceSize - totalLeft -= sliceSize - if (currentLeft == sliceSize) containerIndex += 1 - } - - // Using toNonEmptyBuffer instead of directly moving to next here so that - // other checks can be performed there. - toNonEmptyBuffer() - // force cleanup - this is fine since we are not using the buffers directly - // which are actively needed (the returned value is on containers which can - // recreate) - releasePendingContainers() - // free current container if acquired. - if (currentContainerIndex < containers.length) { - containers(currentContainerIndex).release() - } - assert (currentContainerIndex == containerIndex) - - val retval = new LargeByteBuffer(arr, false, ephemeralDiskBacked) - retval.overrideCleaner(LargeByteBuffer.noopDisposeFunction) - retval - } - - // Get a composite sequence of ByteBuffer which might straddle one or more underlying buffers - // This is to be used only for writes : and ensures that writes are done into the appropriate - // underlying bytebuffers. - def getCompositeWriteBuffer(size: Long): LargeByteBuffer = { - assert(writable) - assert(size >= 0) - - createSlice(size) - } - - // get a buffer which is of the specified size and contains data from the underlying buffers - // Note, the actual data might be spread across the underlying buffers. - // This MUST BE used only for specific usecases like getInt, etc. Not for bulk copy ! - private def readFully(size: Int): ByteBuffer = { - assert (readable) - - if (remaining() < size) { - // throw exception - throw new BufferUnderflowException() - } - - // kyro depends on this it seems ? - // assert (size > 0) - if (0 == size) return LargeByteBuffer.EMPTY_BYTEBUFFER - - // Expected to be handled elsewhere. - assert (! localReadWritePossible(size)) - - val localBuff = { - val buff = fetchBufferOfSize(size) - // assert(buff.remaining() <= size) - // if (buff.remaining() == size) return buff - assert(buff.remaining() < size) - ByteBuffer.allocate(size).put(buff) - } - - // assert (localBuff.hasRemaining) - - while (localBuff.hasRemaining) { - val buff = fetchBufferOfSize(localBuff.remaining()) - localBuff.put(buff) - } - - localBuff.flip() - localBuff - } - - - - def put(b: Byte) { - assert (writable) - if (remaining() < 1) { - // logInfo("put byte. remaining = " + remaining() + ", this = " + this) - throw new BufferOverflowException - } - - assert (currentRemaining() > 0) - - fetchCurrentBuffer().put(b) - globalPosition += 1 - // Check to need to bump the index ? - toNonEmptyBuffer() - } - - - def put(buffer: ByteBuffer) { - assert (writable) - if (remaining() < buffer.remaining()) { - throw new BufferOverflowException - } - - val bufferRemaining = buffer.remaining() - if (localReadWritePossible(bufferRemaining)) { - - assert (currentRemaining() >= bufferRemaining) - - fetchCurrentBuffer().put(buffer) - - globalPosition += bufferRemaining - toNonEmptyBuffer() - return - } - - while (buffer.hasRemaining) { - val currentBufferRemaining = currentRemaining() - val bufferRemaining = buffer.remaining() - - if (currentBufferRemaining >= bufferRemaining) { - fetchCurrentBuffer().put(buffer) - globalPosition += bufferRemaining - } else { - // Split across buffers. - val currentBuffer = fetchCurrentBuffer() - assert (currentBuffer.remaining() >= currentBufferRemaining) - val sliced = ByteBufferContainer.createSlice(buffer, buffer.position(), - currentBufferRemaining) - assert (sliced.remaining() == currentBufferRemaining) - currentBuffer.put(sliced) - // move buffer pos - buffer.position(buffer.position() + currentBufferRemaining) - - globalPosition += currentBufferRemaining - } - toNonEmptyBuffer() - } - - assert (! hasRemaining() || currentRemaining() > 0) - } - - def put(other: LargeByteBuffer) { - assert (writable) - if (this.remaining() < other.remaining()) { - throw new BufferOverflowException - } - - while (other.hasRemaining()) { - val buffer = other.fetchBufferOfSize(other.currentRemaining()) - this.put(buffer) - } - } - - - def duplicate(): LargeByteBuffer = { - val containersCopy = new ArrayBuffer[ByteBufferContainer](containers.size) - // We do a duplicate as part of construction - so avoid double duplicate. - // containersCopy ++= containers.map(_.duplicate()) - containersCopy ++= containers - val retval = new LargeByteBuffer(containersCopy, true, ephemeralDiskBacked) - - // set limit and position (in that order) ... - retval.limit(this.limit()) - retval.position(this.position()) - - // Now release our containers - if any had been acquired - releasePendingContainers() - - retval - } - - - /** - * 'read' a LargeByteBuffer of size specified and return that. - * Position will be incremented by size - * - * The name might be slightly confusing : rename ? - * - * @param size Amount of data to be read from this buffer and returned - * @return - */ - def readLargeBuffer(size: Long, partialReadAllowed: Boolean): LargeByteBuffer = { - if (! hasRemaining() && ! partialReadAllowed) throw new BufferUnderflowException - if (remaining() < size && ! partialReadAllowed) throw new BufferUnderflowException - - - assert (readable) - assert (size >= 0) - - releasePendingContainers() - - if (0 == size) return LargeByteBuffer.EMPTY_BUFFER - - createSlice(size) - } - - - // This is essentially a workaround to exposing underlying buffers - def readFrom(channel: ReadableByteChannel): Long = { - - assert (writable) - releasePendingContainers() - - // this also allows us to avoid nasty corner cases in the loop. - if (! hasRemaining()) { - // logInfo("readFrom channel. remaining = " + remaining() + ", this = " + this) - throw new BufferOverflowException - } - - var totalBytesRead = 0L - - while (hasRemaining()) { - // read what we can ... - val buffer = fetchCurrentBuffer() - val bufferRemaining = currentRemaining() - val bytesRead = channel.read(buffer) - - if (bytesRead > 0) { - totalBytesRead += bytesRead - // bump position too .. - globalPosition += bytesRead - if (bytesRead >= bufferRemaining) toNonEmptyBuffer() - } - else if (-1 == bytesRead) { - // if we had already read some data in the loop, return that. - if (totalBytesRead > 0) return totalBytesRead - return -1 - } // nothing available to read, retry later. return - else if (0 == bytesRead) { - return totalBytesRead - } - - // toNonEmptyBuffer() - } - - // Cleanup last buffer ? - toNonEmptyBuffer() - totalBytesRead - } - - // This is essentially a workaround to exposing underlying buffers - def readFrom(inStrm: InputStream): Long = { - - assert (writable) - releasePendingContainers() - - // this also allows us to avoid nasty corner cases in the loop. - // if (! hasRemaining()) throw new BufferOverflowException - if (! hasRemaining()) return 0 - - var totalBytesRead = 0L - - val buff = new Array[Byte](LargeByteBuffer.TEMP_ARRAY_SIZE) - - while (hasRemaining()) { - // read what we can ... note, since there is no gaurantee that underlying buffer might - // expose array() method, we do double copy - from stream to buff and from buff to bytearray. - // see if we can optimize this later ... - val buffer = fetchCurrentBuffer() - val bufferRemaining = buffer.remaining() - val max = math.min(buff.length, bufferRemaining) - val bytesRead = inStrm.read(buff, 0, max) - - if (bytesRead > 0) { - buffer.put(buff, 0, bytesRead) - totalBytesRead += bytesRead - // bump position too .. - globalPosition += bytesRead - // buffer.position(buffer.position + bytesRead) - if (bytesRead >= bufferRemaining) toNonEmptyBuffer() - } - else if (-1 == bytesRead) { - // if we had already read some data in the loop, return that. - if (totalBytesRead > 0) return totalBytesRead - return -1 - } // nothing available to read, retry later. return - else if (0 == bytesRead) { - return totalBytesRead - } - - // toNonEmptyBuffer() - } - - totalBytesRead - } - - // This is essentially a workaround to exposing underlying buffers - // Note: very similar to readFrom(InputStream) : not trying anything fancy to reduce - // code for performance reasons. - def readFrom(inStrm: DataInput): Long = { - - assert (writable) - releasePendingContainers() - - // this also allows us to avoid nasty corner cases in the loop. - // if (! hasRemaining()) throw new BufferOverflowException - if (! hasRemaining()) return 0 - - var totalBytesRead = 0L - - val buff = new Array[Byte](LargeByteBuffer.TEMP_ARRAY_SIZE) - - while (hasRemaining()) { - // read what we can ... note, since there is no gaurantee that underlying buffer might - // expose array() method, we do double copy - from stream to buff and from buff to bytearray. - // see if we can optimize this later ... - val buffer = fetchCurrentBuffer() - val bufferRemaining = buffer.remaining() - val max = math.min(buff.length, bufferRemaining) - inStrm.readFully(buff, 0, max) - val bytesRead = max - - if (bytesRead > 0) { - buffer.put(buff, 0, bytesRead) - totalBytesRead += bytesRead - // bump position too .. - globalPosition += bytesRead - // buffer.position(buffer.position() + bytesRead) - if (bytesRead >= bufferRemaining) toNonEmptyBuffer() - } - else if (-1 == bytesRead) { - // if we had already read some data in the loop, return that. - if (totalBytesRead > 0) return totalBytesRead - return -1 - } // nothing available to read, retry later. return - else if (0 == bytesRead) { - return totalBytesRead - } - - // toNonEmptyBuffer() - } - - totalBytesRead - } - - // This is essentially a workaround to exposing underlying buffers - // Note: tries to do it efficiently without needing to load everything into memory - // (particularly for diskbacked buffers, etc). - def writeTo(channel: WritableByteChannel, cleanup: Boolean): Long = { - - assert (readable) - releasePendingContainers() - - // this also allows us to avoid nasty corner cases in the loop. - if (! hasRemaining()) throw new BufferUnderflowException - - var totalBytesWritten = 0L - - while (hasRemaining()) { - // Write what we can ... - val buffer = fetchCurrentBuffer() - val bufferRemaining = buffer.remaining() - assert (bufferRemaining > 0) - val bytesWritten = channel.write(buffer) - - if (bytesWritten > 0) { - totalBytesWritten += bytesWritten - // bump position too .. - globalPosition += bytesWritten - if (bytesWritten >= bufferRemaining) toNonEmptyBuffer() - assert (! hasRemaining() || currentRemaining() > 0) - } - else if (0 == bytesWritten) { - return totalBytesWritten - } - - // toNonEmptyBuffer() - } - - assert (! hasRemaining()) - if (cleanup) { - free() - } - totalBytesWritten - } - - // This is essentially a workaround to exposing underlying buffers - def writeTo(outStrm: OutputStream, cleanup: Boolean): Long = { - - assert (readable) - releasePendingContainers() - - // this also allows us to avoid nasty corner cases in the loop. - if (! hasRemaining()) throw new BufferUnderflowException - - var totalBytesWritten = 0L - val buff = new Array[Byte](LargeByteBuffer.TEMP_ARRAY_SIZE) - - while (hasRemaining()) { - // write what we can ... note, since there is no gaurantee that underlying buffer might - // expose array() method, we do double copy - from bytearray to buff and from - // buff to outputstream. see if we can optimize this later ... - val buffer = fetchCurrentBuffer() - val bufferRemaining = buffer.remaining() - val size = math.min(bufferRemaining, buff.length) - buffer.get(buff, 0, size) - outStrm.write(buff, 0, size) - - totalBytesWritten += size - // bump position too .. - globalPosition += size - - if (size >= bufferRemaining) toNonEmptyBuffer() - } - - toNonEmptyBuffer() - if (cleanup) { - free() - } - totalBytesWritten + val b = underlying.read(_pos) + _pos += 1 + b } - def asInputStream(): InputStream = { - new InputStream() { - override def read(): Int = { - if (! hasRemaining()) return -1 - get() - } - - override def read(arr: Array[Byte], off: Int, len: Int): Int = { - if (! hasRemaining()) return -1 - - get(arr, off, len) - } - - override def available(): Int = { - // current remaining is what can be read without blocking - // anything higher might need disk access/buffer swapping. - /* - val left = remaining() - math.min(left, Int.MaxValue).asInstanceOf[Int] - */ - currentRemaining() - } - } - } - - def getCleaner() = cleaner - - /** - * @param cleaner The previous cleaner, so that the caller can chain them if required. - * @return - */ - private[spark] def overrideCleaner(cleaner: BufferCleaner): BufferCleaner = { - overrideCleaner(cleaner, allowOverride = true) - } - - private def overrideCleaner(cleaner: BufferCleaner, allowOverride: Boolean): BufferCleaner = { - if (! this.allowCleanerOverride) { - // allowCleanerOverride = false is used for EMPTY_BUFFER - where we do not allow free - return this.cleaner - } - - this.allowCleanerOverride = allowOverride - assert (null != cleaner) - val prev = this.cleaner - this.cleaner = cleaner - // logInfo("Overriding " + prev + " with " + this.cleaner) - prev - } - - private def doReleaseAll() { - for (container <- containers) { - container.release() - } - } - - def free(invokeCleaner: Boolean = true) { - // logInfo("Free on " + this + ", cleaner = " + cleaner) - // always invoking release - doReleaseAll() - - if (invokeCleaner) cleaner.clean(this) - } - - private def doDispose(needRelease: Boolean) { - - if (disposeLocationThrowable ne null) { - logError("Already free'ed earlier at : ", disposeLocationThrowable) - logError("Current at ", new Throwable) - throw new IllegalStateException("Already freed.") - } - disposeLocationThrowable = new Throwable() - - // Forcefully cleanup all - if (needRelease) doReleaseAll() - - // Free in a different loop, in case different containers refer to same resource - // to release (like file) - for (container <- containers) { - container.free() - } - - needReleaseIndices.clear() - - // We should not use this buffer anymore : set the values such that f - // we dont ... - globalPosition = 0 - globalLimit = 0 - globalCapacity = 0 + def put(bytes: LargeByteBuffer): Unit = { + ??? } - // copy data over ... MUST be used only for cases where array is known to be - // small to begin with. slightly risky method due to that assumption - def toByteArray(): Array[Byte] = { - val positionBackup = position() - val size = remaining() - if (size > Int.MaxValue) { - throw new IllegalStateException( - "Attempt to convert LargeByteBuffer to byte array when data held is more than 2G") - } - - val retval = new Array[Byte](size.asInstanceOf[Int]) - val readSize = get(retval, 0, retval.length) - assert (readSize == retval.length, - "readSize = " + readSize + ", retval.length = " + retval.length) - - position(positionBackup) - - retval + def position: Long = _pos + def position(position: Long): Unit = { + _pos = position } - - // copy data over ... MUST be used only for cases where array is known to be - // small to begin with. slightly risky method due to that assumption - def toByteBuffer(): ByteBuffer = { - ByteBuffer.wrap(toByteArray()) + def remaining(): Long = { + underlying.size - position } - def toInMemoryBuffer(ioConf: IOConfig): LargeByteBuffer = { - val retval = LargeByteBuffer.allocateMemoryBuffer(remaining(), ioConf) - val currentPosition = position() - retval.put(this) - position(currentPosition) - retval.clear() - retval + def duplicate(): ChainedLargeByteBuffer = { + new ChainedLargeByteBuffer(underlying) } - - - // This is ONLY used for testing : that too as part of development of this and associated classes - // remove before contributing to spark. - def hexDump(): String = { - if (remaining() * 64 > Int.MaxValue) { - throw new UnsupportedOperationException("buffer too large " + remaining()) - } - - val sb = new StringBuilder((remaining() * 2).asInstanceOf[Int]) - - var perLine = 0 - var first = true - for (b <- toByteArray()) { - perLine += 1 - if (perLine % 8 == 0) { - sb.append('\n') - first = true - } - if (! first) sb.append(' ') - first = false - sb.append(java.lang.Integer.toHexString(b & 0xff)) - } - sb.append('\n') - sb.toString() + def rewind(): Unit = { + _pos = 0 } - override def toString: String = { - val sb: StringBuffer = new StringBuffer - sb.append(getClass.getName) - sb.append(' ') - sb.append(System.identityHashCode(this)) - sb.append("@[pos=") - sb.append(position()) - sb.append(" lim=") - sb.append(limit()) - sb.append(" cap=") - sb.append(capacity()) - sb.append("]") - sb.toString + def limit(): Long = { + capacity } - - - override def finalize(): Unit = { - var marked = false - if (containers ne null) { - if (containers.exists(container => container.isAcquired && container.requireRelease())) { - marked = true - logError("BUG: buffer was not released - and now going out of scope. " + - "Potential resource leak. Allocated at ", allocateLocationThrowable) - containers.foreach(_.release()) - } - if (containers.exists(container => !container.isFreed && container.requireFree())) { - if (!marked) { - logError("BUG: buffer was not freed - and now going out of scope. Potential resource leak", - allocateLocationThrowable) - } - else { - logError("BUG: buffer was not freed - and now going out of scope. Potential resource leak") - } - containers.foreach(_.free()) - } - } - super.finalize() + def limit(newLimit: Long): Unit = { + ??? } } +class WrappedLargeByteBuffer(private val underlying: ByteBuffer) extends LargeByteBuffer { + def capacity = underlying.capacity -object LargeByteBuffer extends Logging { - - private val noopDisposeFunction = new BufferCleaner() { - protected def doClean(buffer: LargeByteBuffer) { - buffer.free(invokeCleaner = false) - } + def get(dst: Array[Byte], offset: Int, length: Int): Unit = { + underlying.get(dst, offset, length) } - val enableExpensiveAssert = false - private val EMPTY_BYTEBUFFER = ByteBuffer.allocate(0) - val EMPTY_BUFFER = new LargeByteBuffer(ArrayBuffer( - new HeapByteBufferContainer(EMPTY_BYTEBUFFER, false)), false, false) - // Do not allow anyone else to override cleaner - EMPTY_BUFFER.overrideCleaner(noopDisposeFunction, allowOverride = false) - - // 8K sufficient ? - private val TEMP_ARRAY_SIZE = 8192 - - /** - * Create a LargeByteBuffer of specified size which is split across - * ByteBuffer's of size DiskStore.MAX_BLOCK_SIZE and backed by in memory - * ByteBuffer - * - */ - def allocateMemoryBuffer(totalSize: Long, ioConf: IOConfig): LargeByteBuffer = { - if (0 == totalSize) { - return EMPTY_BUFFER - } - - assert (totalSize > 0) - - val blockSize = ioConf.getMaxBlockSize(BufferType.MEMORY) - val numBlocks = ioConf.numBlocks(BufferType.MEMORY, totalSize) - val lastBlockSize = ioConf.lastBlockSize(BufferType.MEMORY, totalSize) - - assert (lastBlockSize > 0) - - val bufferArray = { - val arr = new ArrayBuffer[ByteBufferContainer](numBlocks) - for (index <- 0 until numBlocks - 1) { - val buff = ByteBuffer.allocate(blockSize) - // buff.clear() - arr += new HeapByteBufferContainer(buff, true) - } - arr += new HeapByteBufferContainer(ByteBuffer.allocate(lastBlockSize), true) - assert (arr.length == numBlocks) - arr - } - - new LargeByteBuffer(bufferArray, false, false) + def get(): Byte = { + underlying.get() } - /** - * Create a LargeByteBuffer of specified size which is split across - * ByteBuffer's of size DiskStore.MAX_BLOCK_SIZE and backed by on disk - * - */ - private def allocateDiskBuffer(totalSize: Long, - blockManager: BlockManager): LargeByteBuffer = { - if (0 == totalSize) { - return EMPTY_BUFFER - } - - assert (totalSize > 0) - - // Create a file of the specified size. - val file = blockManager.diskBlockManager.createTempBlock()._2 - val raf = new RandomAccessFile(file, "rw") - try { - raf.setLength(totalSize) - } finally { - raf.close() - } - - readWriteDiskSegment(new FileSegment(file, 0, totalSize), - ephemeralDiskBacked = true, blockManager.ioConf) + def position: Long = underlying.position + def position(position: Long): Unit = { + //XXX check range? + underlying.position(position.toInt) } - - // The returned buffer takes up ownership of the underlying buffers - // (including dispos'ing that when done) - def fromBuffers(buffers: ByteBuffer*): LargeByteBuffer = { - val nonEmpty = buffers.filter(_.hasRemaining) - - // cleanup the empty buffers - buffers.filter(! _.hasRemaining).foreach(b => BlockManager.dispose(b)) - - - if (nonEmpty.isEmpty) { - return EMPTY_BUFFER - } - - // slice so that offsets match our requirement - new LargeByteBuffer(new ArrayBuffer() ++ nonEmpty.map(b => - new HeapByteBufferContainer(b.slice(), true)), false, false) + def remaining(): Long = { + underlying.remaining() } - def fromByteArrays(byteArrays: Array[Byte]*): LargeByteBuffer = { - // only non empty arrays - val arrays = byteArrays.filter(_.length > 0) - if (0 == arrays.length) return EMPTY_BUFFER - - new LargeByteBuffer(new ArrayBuffer() ++ arrays.map(arr => - new HeapByteBufferContainer(ByteBuffer.wrap(arr), true)), false, false) + def duplicate(): WrappedLargeByteBuffer = { + new WrappedLargeByteBuffer(underlying.duplicate()) } - def fromLargeByteBuffers(canDispose: Boolean, inputBuffers: LargeByteBuffer*): LargeByteBuffer = { - - if (inputBuffers.isEmpty) return EMPTY_BUFFER - - if (! inputBuffers.exists(_.hasRemaining())) { - if (canDispose) inputBuffers.map(_.free()) - return EMPTY_BUFFER - } - - // release all temp resources acquired - inputBuffers.foreach(buff => buff.releasePendingContainers()) - // free current container if acquired. - inputBuffers.foreach(buff => if (buff.currentContainerIndex < buff.containers.length) { - buff.containers(buff.currentContainerIndex).release() - }) - // inputBuffers.foreach(b => b.doReleaseAll()) - - - // Dispose of any empty buffers - if (canDispose) inputBuffers.filter(! _.hasRemaining()).foreach(_.free()) - - // Find all containers we need. - val buffers = inputBuffers.filter(_.hasRemaining()).map(b => b.createSlice(b.remaining())) - - val containers = buffers.flatMap(_.containers) - assert (! containers.isEmpty) - // The in order containers of "buffers" seq constitute the required return value - val retval = new LargeByteBuffer(new ArrayBuffer() ++ containers, - // if you cant dispose, then we dont own the buffers : in which case, need duplicate - ! canDispose, inputBuffers.exists(_.ephemeralDiskBacked)) - - if (canDispose) { - // override dispose of all other buffers. - val disposeFunctions = inputBuffers.map { - buffer => { - (buffer, buffer.overrideCleaner(noopDisposeFunction)) - } - } - - val cleaner = retval.getCleaner() - val newCleaner = new BufferCleaner { - protected def doClean(buffer: LargeByteBuffer) { - - assert (retval == buffer) - // default cleaner. - cleaner.clean(retval) - // not required, since we are within clean anyway. - // retval.free(invokeCleaner = false) - - // retval.doDispose(needRelease = true) - - // This might actually call dispose twice on some (initially) empty buffers, - // which is fine since we now guard against that. - disposeFunctions.foreach(v => v._2.clean(v._1)) - // Call the free method too : so that buffers are marked free ... - disposeFunctions.foreach(v => v._1.free(invokeCleaner = false)) - } - } - - val prev = retval.overrideCleaner(newCleaner) - assert (prev == cleaner) - } - - retval + def rewind(): Unit = { + underlying.duplicate() } - private def checkOffsets(arr: Array[Byte], offset: Int, size: Int) { - if (arr == null) { - throw new NullPointerException - } else if (offset < 0 || size < 0 || offset + size > arr.length) { - throw new IndexOutOfBoundsException - } + def limit(): Long = { + underlying.limit() } - def allocateTransientBuffer(size: Long, blockManager: BlockManager) = { - if (size <= blockManager.ioConf.maxInMemSize) { - LargeByteBuffer.allocateMemoryBuffer(size, blockManager.ioConf) - } else { - LargeByteBuffer.allocateDiskBuffer(size, blockManager) - } + def limit(newLimit: Long) = { + //XXX check range? + underlying.limit(newLimit.toInt) } - def readFromDiskSegment(segment: FileSegment, ioConf: IOConfig, - ephemeralDiskBacked: Boolean): LargeByteBuffer = { - // Split the block into multiple of BlockStore.maxBlockSize - val segmentSize = segment.length - val blockSize = ioConf.getMaxBlockSize(BufferType.DISK).asInstanceOf[Long] - val numBlocks = ioConf.numBlocks(BufferType.DISK, segmentSize) - val lastBlockSize = ioConf.lastBlockSize(BufferType.DISK, segmentSize) - - val buffers = new ArrayBuffer[ByteBufferContainer](numBlocks) - - for (index <- 0 until numBlocks - 1) { - buffers += new ReadOnlyFileContainer(new FileSegment(segment.file, - segment.offset + index * blockSize, blockSize), ioConf) - } - - // Last block - buffers += new ReadOnlyFileContainer(new FileSegment(segment.file, - segment.offset + (numBlocks - 1) * blockSize, lastBlockSize), ioConf) +} - new LargeByteBuffer(buffers, false, ephemeralDiskBacked) +object LargeByteBuffer { + def allocateOnHeap(size: Long, maxChunk: Int): LargeByteBuffer = { + val buffer = ChainedBuffer.withInitialSize(maxChunk, size) + new ChainedLargeByteBuffer(buffer) } +} - def readWriteDiskSegment(segment: FileSegment, ephemeralDiskBacked: Boolean, - ioConf: IOConfig): LargeByteBuffer = { - - // Split the block into multiple of BlockStore.maxBlockSize - val segmentSize = segment.length - val blockSize = ioConf.getMaxBlockSize(BufferType.DISK).asInstanceOf[Long] - val numBlocks = ioConf.numBlocks(BufferType.DISK, segmentSize) - val lastBlockSize = ioConf.lastBlockSize(BufferType.DISK, segmentSize) - - logInfo("readWriteDiskSegment = " + segment + ", numBlocks = " + numBlocks + - ", lastBlockSize = " + lastBlockSize) - val buffers = new ArrayBuffer[ByteBufferContainer](numBlocks) - - for (index <- 0 until numBlocks - 1) { - buffers += new ReadWriteFileContainer(new FileSegment(segment.file, - segment.offset + index * blockSize, blockSize), ephemeralDiskBacked, null) - } - - // Last block - buffers += new ReadWriteFileContainer(new FileSegment(segment.file, - segment.offset + (numBlocks - 1) * blockSize, lastBlockSize), ephemeralDiskBacked, null) - new LargeByteBuffer(buffers, false, ephemeralDiskBacked) - } -} +// +///** +// * This is a variant of ByteBuffer to be used internally in spark, which is not limited to 2G +// * which ByteBuffers are limited to. +// * Externally, it exposes all the api which java.nio.ByteBuffer exposes. +// * Internally, it maintains a sequence of Containers which manage the ByteBuffer data. +// * Not all the data might be loaded into memory (like disk or tachyon data) - so actual +// * memory footprint - heap and vm could be much lower than capacity. +// * +// * TODO: Currently we are slightly fast and loose in terms of concurrent modifications to this +// * buffer, maybe revisit this later ? Note: this is not much different from earlier though ! +// * +// * TODO: Explore if (at all) we can leverage zero copy transfers. The issue (currently) is that this +// * will require the file to be kept open (repeatedly opening/closing file is not good +// * for each transfer) and this has an impact on ulimit. Not to mention writing of mmap'ed buffer is +// * pretty quick (it is the first failover in case direct transfer is not possible in file zero copy) +// * +// * TODO: After redesign to containers, we got rid of parent containers to free - the side effect is +// * that if there are direct ByteBuffers, we are not handling explicit cleanup of those in some +// * cases (when we duplicate/slice them). Currently spark does not need this, but might in future +// * so relook at it later. +// */ +//// We should make this constructor private: but for now, +//// leaving it public since TachyonStore needs it +//class LargeByteBuffer private[spark](private val inputContainers: ArrayBuffer[ByteBufferContainer], +// private val needDuplicate: Boolean, val ephemeralDiskBacked: Boolean) extends Logging { +// +// // TODO: TEMP code: to flush out potential resource leaks. REMOVE ME +// private val allocateLocationThrowable: Throwable = { +// if (inputContainers.exists(c => c.requireRelease() || c.requireFree())) { +// new Throwable("blockId = " + BlockManager.getLookupBlockId) +// } else { +// null +// } +// } +// private var disposeLocationThrowable: Throwable = null +// +// @volatile private var allowCleanerOverride = true +// @volatile private var cleaner: BufferCleaner = new BufferCleaner { +// override def doClean(buffer: LargeByteBuffer) = { +// assert (LargeByteBuffer.this == buffer) +// doDispose(needRelease = false) +// } +// } +// +// // should not be empty +// assert (null != inputContainers && ! inputContainers.isEmpty) +// // should not have any null's +// assert (inputContainers.find(_ == null).isEmpty) +// +// // println("Num containers = " + inputContainers.size) +// +// // Position, limit and capacity relevant over the engire LargeByteBuffer +// @volatile private var globalPosition = 0L +// @volatile private var globalLimit = 0L +// @volatile private var currentContainerIndex = 0 +// +// // The buffers in which the actual data is held. +// private var containers: Array[ByteBufferContainer] = null +// +// // aggregate capacities of the individual buffers. +// // bufferPositionStart(0) will be capacity of 1st buffer, bufferPositionStart(1) will be +// // sum of capacity of 0th and 1st block buffer +// private var bufferPositionStart: Array[Long] = null +// +// // Contains the indices of a containers which requires release before subsequent invocation of +// // read/write should be serviced. This is required since current read/write might have moved the +// // position but since we are returning bytebuffers which depend on the validity of the existing +// // bytebuffer, we cant release them yet. +// private var needReleaseIndices = new HashSet[Int]() +// +// private val readable = ! inputContainers.exists(! _.isReadable) +// private val writable = ! inputContainers.exists(! _.isWritable) +// +// +// // initialize +// @volatile private var globalCapacity = { +// +// // Ensure that there are no empty buffers : messes up with our code : unless it +// // is a single buffer (for empty buffer for marker case) +// assert (inputContainers.find(0 == _.capacity()).isEmpty || 1 == inputContainers.length) +// +// containers = { +// if (needDuplicate) inputContainers.map(_.duplicate()).toArray else inputContainers.toArray +// } +// containers.foreach(_.validate()) +// +// def initializeBufferPositionStart(arr: Array[ByteBufferContainer]) { +// val buff = new ArrayBuffer[Long](arr.length + 1) +// buff += 0L +// +// buff ++= arr.map(_.capacity().asInstanceOf[Long]).scanLeft(0L)(_ + _).slice(1, arr.length + 1) +// assert (buff.length == arr.length + 1) +// bufferPositionStart = buff.toArray +// } +// +// initializeBufferPositionStart(containers) +// +// // remove references from inputBuffers +// inputContainers.clear() +// +// globalLimit = bufferPositionStart(containers.length) +// globalPosition = 0L +// currentContainerIndex = 0 +// +// assert (globalLimit == containers.map(_.capacity().asInstanceOf[Long]).sum) +// +// globalLimit +// } +// +// final def position(): Long = globalPosition +// +// final def limit(): Long = globalLimit +// +// final def capacity(): Long = globalCapacity +// +// final def limit(newLimit: Long) { +// if ((newLimit > capacity()) || (newLimit < 0)) { +// throw new IllegalArgumentException("newLimit = " + newLimit + ", capacity = " + capacity()) +// } +// +// globalLimit = newLimit +// if (position() > newLimit) position(newLimit) +// } +// +// def skip(skipBy: Long) = position(position() + skipBy) +// +// private def releasePendingContainers() { +// if (! needReleaseIndices.isEmpty) { +// val iter = needReleaseIndices.iterator +// while (iter.hasNext) { +// val index = iter.next() +// assert (index >= 0 && index < containers.length) +// // It is possible to move from one container to next before the previous +// // container was acquired. For example, get forcing move to next container +// // since current was exhausted immediatelly followed by a position() +// // so the container we moved to was never acquired. +// +// // assert (containers(index).isAcquired) +// // will this always be satisfied ? +// // assert (index != currentContainerIndex) +// if (containers(index).isAcquired) containers(index).release() +// } +// needReleaseIndices.clear() +// } +// } +// +// private def toNewContainer(newIndex: Int) { +// if (newIndex != currentContainerIndex && currentContainerIndex < containers.length) { +// +// assert (currentContainerIndex >= 0) +// needReleaseIndices += currentContainerIndex +// } +// currentContainerIndex = newIndex +// } +// +// // expensive method, sigh ... optimize it later ? +// final def position(newPosition: Long) { +// +// if ((newPosition > globalLimit) || (newPosition < 0)) throw new IllegalArgumentException() +// +// if (currentContainerIndex < bufferPositionStart.length - 1 && +// newPosition >= bufferPositionStart(currentContainerIndex) && +// newPosition < bufferPositionStart(currentContainerIndex + 1)) { +// // Same buffer - easy method ... +// globalPosition = newPosition +// // Changed position - free previously returned buffers. +// releasePendingContainers() +// return +// } +// +// // Find appropriate currentContainerIndex +// // Since bufferPositionStart is sorted, can be replaced with binary search if required. +// // For now, not in the perf critical path since buffers size is very low typically. +// var index = 0 +// val cLen = containers.length +// while (index < cLen) { +// if (newPosition >= bufferPositionStart(index) && +// newPosition < bufferPositionStart(index + 1)) { +// globalPosition = newPosition +// toNewContainer(index) +// // Changed position - free earlier and previously returned buffers. +// releasePendingContainers() +// return +// } +// index += 1 +// } +// +// if (newPosition == globalLimit && newPosition == bufferPositionStart(cLen)) { +// // boundary. +// globalPosition = newPosition +// toNewContainer(cLen) +// // Changed position - free earlier and previously returned buffers. +// releasePendingContainers() +// return +// } +// +// assert (assertion = false, "Unexpected to come here .... newPosition = " + newPosition + +// ", bufferPositionStart = " + bufferPositionStart.mkString("[", ", ", "]")) +// } +// +// +// /** +// * Clears this buffer. The position is set to zero, the limit is set to +// * the capacity, and the mark is discarded. +// * +// *

Invoke this method before using a sequence of channel-read or +// * put operations to fill this buffer. +// * +// *

This method does not actually erase the data in the buffer, but it +// * is named as if it did because it will most often be used in situations +// * in which that might as well be the case.

+// */ +// final def clear() { +// // if (0 == globalCapacity) return +// +// needReleaseIndices += 0 +// globalPosition = 0L +// toNewContainer(0) +// globalLimit = globalCapacity +// +// // Now free all pending containers +// releasePendingContainers() +// } +// +// /** +// * Flips this buffer. The limit is set to the current position and then +// * the position is set to zero. If the mark is defined then it is +// * discarded. +// * +// *

After a sequence of channel-read or put operations, invoke +// * this method to prepare for a sequence of channel-write or relative +// * get operations. +// */ +// final def flip() { +// needReleaseIndices += 0 +// globalLimit = globalPosition +// globalPosition = 0L +// toNewContainer(0) +// +// // Now free all pending containers +// releasePendingContainers() +// } +// +// /** +// * Rewinds this buffer. The position is set to zero and the mark is +// * discarded. +// * +// *

Invoke this method before a sequence of channel-write or get +// * operations, assuming that the limit has already been set +// * appropriately. +// */ +// final def rewind() { +// needReleaseIndices += 0 +// globalPosition = 0L +// toNewContainer(0) +// +// // Now free all pending containers +// releasePendingContainers() +// } +// +// /** +// * Returns the number of elements between the current position and the +// * limit.

+// * +// * @return The number of elements remaining in this buffer +// */ +// final def remaining(): Long = { +// globalLimit - globalPosition +// } +// +// /** +// * Tells whether there are any elements between the current position and +// * the limit.

+// * +// * @return true if, and only if, there is at least one element +// * remaining in this buffer +// */ +// final def hasRemaining() = { +// globalPosition < globalLimit +// } +// +// // private def currentBuffer(): ByteBuffer = buffers(currentContainerIndex) +// +// // number of bytes remaining in currently active underlying buffer +// private def currentRemaining(): Int = { +// if (hasRemaining()) { +// // validate currentContainerIndex is valid +// assert (globalPosition >= bufferPositionStart(currentContainerIndex) && +// globalPosition < bufferPositionStart(currentContainerIndex + 1), +// "globalPosition = " + globalPosition + +// ", currentContainerIndex = " + currentContainerIndex + +// ", bufferPositionStart = " + bufferPositionStart.mkString("[", ", ", " ]")) +// +// currentRemaining0(currentContainerIndex) +// } else 0 +// } +// +// // Without any validation : required when we are bumping the index (when validation will fail) ... +// private def currentRemaining0(which: Int): Int = { +// // currentBuffer().remaining() +// math.max(0, math.min(bufferPositionStart(which + 1), +// globalLimit) - globalPosition).asInstanceOf[Int] +// } +// +// // Set the approppriate position/limit for the current underlying buffer to mirror our +// // the LargeByteBuffer's state. +// private def fetchCurrentBuffer(): ByteBuffer = { +// releasePendingContainers() +// +// assert (currentContainerIndex < containers.length) +// +// val container = containers(currentContainerIndex) +// if (! container.isAcquired) { +// container.acquire() +// } +// +// assert (container.isAcquired) +// if (LargeByteBuffer.enableExpensiveAssert) { +// assert (! containers.exists( b => (b ne container) && b.isAcquired)) +// } +// +// assert (currentContainerIndex < bufferPositionStart.length && +// globalPosition < bufferPositionStart(currentContainerIndex + 1), +// "currentContainerIndex = " + currentContainerIndex + ", bufferPositionStart = " + +// bufferPositionStart.mkString("[", ", ", "]") + ", this = " + this) +// +// val buffPosition = (globalPosition - bufferPositionStart(currentContainerIndex)). +// asInstanceOf[Int] +// +// val buffer = container.getByteBuffer +// buffer.position(buffPosition) +// val diff = buffer.capacity - buffPosition +// val left = remaining() +// if (diff <= left) { +// buffer.limit(buffer.capacity()) +// } else { +// // Can happen if limit() was called. +// buffer.limit(buffPosition + left.asInstanceOf[Int]) +// } +// +// buffer +// } +// +// // To be used ONLY to test in suites. +// private[spark] def fetchCurrentBufferForTesting(): ByteBuffer = { +// if ("1" != System.getProperty("SPARK_TESTING")) { +// throw new IllegalStateException("This method is to be used ONLY within spark test suites") +// } +// +// fetchCurrentBuffer() +// } +// +// // Expects that the invoker has ensured that this can be safely invoked. +// // That is, it wont be invoked when the loop wont terminate. +// private def toNonEmptyBuffer() { +// +// if (! hasRemaining()) { +// var newIndex = currentContainerIndex +// // Ensure we are in the right block or not. +// while (newIndex < containers.length && globalPosition >= bufferPositionStart(newIndex + 1)) { +// newIndex += 1 +// } +// toNewContainer(newIndex) +// // Do not do this - since we might not yet have consumed the buffer which caused EOF right now +// /* +// // Add last one also, and release it too - since we are at the end of the buffer with nothing +// // more pending. +// if (newIndex >= 0 && currentContainerIndex < containers.length) { +// needReleaseIndices += newIndex +// } +// */ +// assert (currentContainerIndex >= 0) +// // releasePendingContainers() +// return +// } +// +// var index = currentContainerIndex +// while (0 == currentRemaining0(index) && index < containers.length) { +// index += 1 +// } +// assert (currentContainerIndex < containers.length) +// toNewContainer(index) +// assert (0 != currentRemaining()) +// } +// +// private def assertPreconditions(containerIndex: Int) { +// assert (globalPosition >= bufferPositionStart(containerIndex), +// "globalPosition = " + globalPosition + ", containerIndex = " + containerIndex + +// ", bufferPositionStart = " + bufferPositionStart.mkString("[", ", ", " ]")) +// assert (globalPosition < bufferPositionStart(containerIndex + 1), +// "globalPosition = " + globalPosition + ", containerIndex = " + containerIndex + +// ", bufferPositionStart = " + bufferPositionStart.mkString("[", ", ", " ]")) +// +// assert (globalLimit <= globalCapacity) +// assert (containerIndex < containers.length) +// } +// +// +// /** +// * Attempts to return a ByteBuffer of the requested size. +// * It is possible to return a buffer of size smaller than requested +// * even though hasRemaining == true +// * +// * On return, position would have been moved 'ahead' by the size of the buffer returned : +// * that is, we treat that the returned buffer has been already 'read' from this LargeByteBuffer +// * +// * +// * This is used to primarily retrieve content of this buffer to expose via ByteBuffer +// * to some other api which is deemed too cumbersome to move to LargeByteBuffer (like the +// * chunked sending of contents via ConnectionManager) Note that the lifecycle of the ByteBuffer +// * returned is inherently tied to the state of this LargeByteBuffer. For example,if the underlying +// * container is a disk backed container, and we make subsequent calls to get(), the returned +// * ByteBuffer can be dispose'ed off +// * +// * @param maxChunkSize Max size of the ByteBuffer to retrieve. +// * @return +// */ +// +// private def fetchBufferOfSize(maxChunkSize: Int): ByteBuffer = { +// fetchBufferOfSizeImpl(maxChunkSize, canReleaseContainers = true) +// } +// +// private def fetchBufferOfSizeImpl(maxChunkSize: Int, +// canReleaseContainers: Boolean): ByteBuffer = { +// if (canReleaseContainers) releasePendingContainers() +// assert (maxChunkSize > 0) +// +// // not checking for degenerate case of maxChunkSize == 0 +// if (globalPosition >= globalLimit) { +// // throw exception +// throw new BufferUnderflowException() +// } +// +// // Check preconditions : disable these later, since they might be expensive to +// // evaluate for every IO op +// assertPreconditions(currentContainerIndex) +// +// val currentBufferRemaining = currentRemaining() +// +// assert (currentBufferRemaining > 0) +// +// val size = math.min(currentBufferRemaining, maxChunkSize) +// +// val newBuffer = if (currentBufferRemaining > maxChunkSize) { +// val currentBuffer = fetchCurrentBuffer() +// val buff = ByteBufferContainer.createSlice(currentBuffer, +// currentBuffer.position(), maxChunkSize) +// assert (buff.remaining() == maxChunkSize) +// buff +// } else { +// val currentBuffer = fetchCurrentBuffer() +// val buff = currentBuffer.slice() +// assert (buff.remaining() == currentBufferRemaining) +// buff +// } +// +// assert (size == newBuffer.remaining()) +// assert (0 == newBuffer.position()) +// assert (size == newBuffer.limit()) +// assert (newBuffer.capacity() == newBuffer.limit()) +// +// globalPosition += newBuffer.remaining +// toNonEmptyBuffer() +// +// newBuffer +// } +// +// // Can we service the read/write from the currently active (underlying) bytebuffer or not. +// // For almost all cases, this will return true allowing us to optimize away the more expensive +// // computations. +// private def localReadWritePossible(size: Int) = +// size >= 0 && globalPosition + size <= bufferPositionStart(currentContainerIndex + 1) +// +// +// def getLong(): Long = { +// assert (readable) +// releasePendingContainers() +// +// if (remaining() < 8) throw new BufferUnderflowException +// +// if (localReadWritePossible(8)) { +// val buff = fetchCurrentBuffer() +// assert (buff.remaining() >= 8) +// val retval = buff.getLong +// globalPosition += 8 +// toNonEmptyBuffer() +// return retval +// } +// +// val buff = readFully(8) +// buff.getLong +// } +// +// def getInt(): Int = { +// assert (readable) +// releasePendingContainers() +// +// if (remaining() < 4) throw new BufferUnderflowException +// +// if (localReadWritePossible(4)) { +// val buff = fetchCurrentBuffer() +// assert (buff.remaining() >= 4) +// val retval = buff.getInt +// globalPosition += 4 +// toNonEmptyBuffer() +// return retval +// } +// +// val buff = readFully(4) +// buff.getInt +// } +// +// def getChar(): Char = { +// assert (readable) +// releasePendingContainers() +// +// if (remaining() < 2) throw new BufferUnderflowException +// +// if (localReadWritePossible(2)) { +// val buff = fetchCurrentBuffer() +// assert (buff.remaining() >= 2) +// val retval = buff.getChar +// globalPosition += 2 +// toNonEmptyBuffer() +// return retval +// } +// +// // if slice is becoming too expensive, revisit this ... +// val buff = readFully(2) +// buff.getChar +// } +// +// def get(): Byte = { +// assert (readable) +// releasePendingContainers() +// +// if (! hasRemaining()) throw new BufferUnderflowException +// +// // If we have remaining bytes, previous invocations MUST have ensured that we are at +// // a buffer which has data to be read. +// assert (localReadWritePossible(1)) +// +// val buff = fetchCurrentBuffer() +// assert (buff.remaining() >= 1, "buff.remaining = " + buff.remaining()) +// val retval = buff.get() +// globalPosition += 1 +// toNonEmptyBuffer() +// +// retval +// } +// +// def get(arr: Array[Byte], offset: Int, size: Int): Int = { +// assert (readable) +// releasePendingContainers() +// +// LargeByteBuffer.checkOffsets(arr, offset, size) +// +// // kyro depends on this it seems ? +// // assert (size > 0) +// if (0 == size) return 0 +// +// if (! hasRemaining()) return -1 +// +// if (localReadWritePossible(size)) { +// val buff = fetchCurrentBuffer() +// assert (buff.remaining() >= size) +// buff.get(arr, offset, size) +// globalPosition += size +// toNonEmptyBuffer() +// return size +// } +// +// var remainingSize = math.min(size, remaining()).asInstanceOf[Int] +// var currentOffset = offset +// +// while (remainingSize > 0) { +// val buff = fetchBufferOfSize(remainingSize) +// val toCopy = math.min(buff.remaining(), remainingSize) +// +// buff.get(arr, currentOffset, toCopy) +// currentOffset += toCopy +// remainingSize -= toCopy +// } +// +// currentOffset - offset +// } +// +// +// private def createSlice(size: Long): LargeByteBuffer = { +// +// releasePendingContainers() +// +// if (remaining() < size) { +// // logInfo("createSlice. remaining = " + remaining() + ", size " + size + ", this = " + this) +// throw new BufferOverflowException +// } +// +// // kyro depends on this it seems ? +// // assert (size > 0) +// if (0 == size) return LargeByteBuffer.EMPTY_BUFFER +// +// val arr = new ArrayBuffer[ByteBufferContainer](2) +// var totalLeft = size +// +// // assert (currentRemaining() < totalLeft || totalLeft != size || currentAsByteBuffer) +// +// var containerIndex = currentContainerIndex +// while (totalLeft > 0 && hasRemaining()) { +// assertPreconditions(containerIndex) +// val container = containers(containerIndex) +// val currentLeft = currentRemaining0(containerIndex) +// +// assert (globalPosition + currentLeft <= globalLimit) +// assert (globalPosition >= bufferPositionStart(containerIndex) && +// (globalPosition < bufferPositionStart(containerIndex + 1))) +// +// val from = (globalPosition - bufferPositionStart(containerIndex)).asInstanceOf[Int] +// val sliceSize = math.min(totalLeft, currentLeft) +// assert (from >= 0) +// assert (sliceSize > 0 && sliceSize <= Int.MaxValue) +// +// val slice = container.createSlice(from, sliceSize.asInstanceOf[Int]) +// arr += slice +// +// globalPosition += sliceSize +// totalLeft -= sliceSize +// if (currentLeft == sliceSize) containerIndex += 1 +// } +// +// // Using toNonEmptyBuffer instead of directly moving to next here so that +// // other checks can be performed there. +// toNonEmptyBuffer() +// // force cleanup - this is fine since we are not using the buffers directly +// // which are actively needed (the returned value is on containers which can +// // recreate) +// releasePendingContainers() +// // free current container if acquired. +// if (currentContainerIndex < containers.length) { +// containers(currentContainerIndex).release() +// } +// assert (currentContainerIndex == containerIndex) +// +// val retval = new LargeByteBuffer(arr, false, ephemeralDiskBacked) +// retval.overrideCleaner(LargeByteBuffer.noopDisposeFunction) +// retval +// } +// +// // Get a composite sequence of ByteBuffer which might straddle one or more underlying buffers +// // This is to be used only for writes : and ensures that writes are done into the appropriate +// // underlying bytebuffers. +// def getCompositeWriteBuffer(size: Long): LargeByteBuffer = { +// assert(writable) +// assert(size >= 0) +// +// createSlice(size) +// } +// +// // get a buffer which is of the specified size and contains data from the underlying buffers +// // Note, the actual data might be spread across the underlying buffers. +// // This MUST BE used only for specific usecases like getInt, etc. Not for bulk copy ! +// private def readFully(size: Int): ByteBuffer = { +// assert (readable) +// +// if (remaining() < size) { +// // throw exception +// throw new BufferUnderflowException() +// } +// +// // kyro depends on this it seems ? +// // assert (size > 0) +// if (0 == size) return LargeByteBuffer.EMPTY_BYTEBUFFER +// +// // Expected to be handled elsewhere. +// assert (! localReadWritePossible(size)) +// +// val localBuff = { +// val buff = fetchBufferOfSize(size) +// // assert(buff.remaining() <= size) +// // if (buff.remaining() == size) return buff +// assert(buff.remaining() < size) +// ByteBuffer.allocate(size).put(buff) +// } +// +// // assert (localBuff.hasRemaining) +// +// while (localBuff.hasRemaining) { +// val buff = fetchBufferOfSize(localBuff.remaining()) +// localBuff.put(buff) +// } +// +// localBuff.flip() +// localBuff +// } +// +// +// +// def put(b: Byte) { +// assert (writable) +// if (remaining() < 1) { +// // logInfo("put byte. remaining = " + remaining() + ", this = " + this) +// throw new BufferOverflowException +// } +// +// assert (currentRemaining() > 0) +// +// fetchCurrentBuffer().put(b) +// globalPosition += 1 +// // Check to need to bump the index ? +// toNonEmptyBuffer() +// } +// +// +// def put(buffer: ByteBuffer) { +// assert (writable) +// if (remaining() < buffer.remaining()) { +// throw new BufferOverflowException +// } +// +// val bufferRemaining = buffer.remaining() +// if (localReadWritePossible(bufferRemaining)) { +// +// assert (currentRemaining() >= bufferRemaining) +// +// fetchCurrentBuffer().put(buffer) +// +// globalPosition += bufferRemaining +// toNonEmptyBuffer() +// return +// } +// +// while (buffer.hasRemaining) { +// val currentBufferRemaining = currentRemaining() +// val bufferRemaining = buffer.remaining() +// +// if (currentBufferRemaining >= bufferRemaining) { +// fetchCurrentBuffer().put(buffer) +// globalPosition += bufferRemaining +// } else { +// // Split across buffers. +// val currentBuffer = fetchCurrentBuffer() +// assert (currentBuffer.remaining() >= currentBufferRemaining) +// val sliced = ByteBufferContainer.createSlice(buffer, buffer.position(), +// currentBufferRemaining) +// assert (sliced.remaining() == currentBufferRemaining) +// currentBuffer.put(sliced) +// // move buffer pos +// buffer.position(buffer.position() + currentBufferRemaining) +// +// globalPosition += currentBufferRemaining +// } +// toNonEmptyBuffer() +// } +// +// assert (! hasRemaining() || currentRemaining() > 0) +// } +// +// def put(other: LargeByteBuffer) { +// assert (writable) +// if (this.remaining() < other.remaining()) { +// throw new BufferOverflowException +// } +// +// while (other.hasRemaining()) { +// val buffer = other.fetchBufferOfSize(other.currentRemaining()) +// this.put(buffer) +// } +// } +// +// +// def duplicate(): LargeByteBuffer = { +// val containersCopy = new ArrayBuffer[ByteBufferContainer](containers.size) +// // We do a duplicate as part of construction - so avoid double duplicate. +// // containersCopy ++= containers.map(_.duplicate()) +// containersCopy ++= containers +// val retval = new LargeByteBuffer(containersCopy, true, ephemeralDiskBacked) +// +// // set limit and position (in that order) ... +// retval.limit(this.limit()) +// retval.position(this.position()) +// +// // Now release our containers - if any had been acquired +// releasePendingContainers() +// +// retval +// } +// +// +// /** +// * 'read' a LargeByteBuffer of size specified and return that. +// * Position will be incremented by size +// * +// * The name might be slightly confusing : rename ? +// * +// * @param size Amount of data to be read from this buffer and returned +// * @return +// */ +// def readLargeBuffer(size: Long, partialReadAllowed: Boolean): LargeByteBuffer = { +// if (! hasRemaining() && ! partialReadAllowed) throw new BufferUnderflowException +// if (remaining() < size && ! partialReadAllowed) throw new BufferUnderflowException +// +// +// assert (readable) +// assert (size >= 0) +// +// releasePendingContainers() +// +// if (0 == size) return LargeByteBuffer.EMPTY_BUFFER +// +// createSlice(size) +// } +// +// +// // This is essentially a workaround to exposing underlying buffers +// def readFrom(channel: ReadableByteChannel): Long = { +// +// assert (writable) +// releasePendingContainers() +// +// // this also allows us to avoid nasty corner cases in the loop. +// if (! hasRemaining()) { +// // logInfo("readFrom channel. remaining = " + remaining() + ", this = " + this) +// throw new BufferOverflowException +// } +// +// var totalBytesRead = 0L +// +// while (hasRemaining()) { +// // read what we can ... +// val buffer = fetchCurrentBuffer() +// val bufferRemaining = currentRemaining() +// val bytesRead = channel.read(buffer) +// +// if (bytesRead > 0) { +// totalBytesRead += bytesRead +// // bump position too .. +// globalPosition += bytesRead +// if (bytesRead >= bufferRemaining) toNonEmptyBuffer() +// } +// else if (-1 == bytesRead) { +// // if we had already read some data in the loop, return that. +// if (totalBytesRead > 0) return totalBytesRead +// return -1 +// } // nothing available to read, retry later. return +// else if (0 == bytesRead) { +// return totalBytesRead +// } +// +// // toNonEmptyBuffer() +// } +// +// // Cleanup last buffer ? +// toNonEmptyBuffer() +// totalBytesRead +// } +// +// // This is essentially a workaround to exposing underlying buffers +// def readFrom(inStrm: InputStream): Long = { +// +// assert (writable) +// releasePendingContainers() +// +// // this also allows us to avoid nasty corner cases in the loop. +// // if (! hasRemaining()) throw new BufferOverflowException +// if (! hasRemaining()) return 0 +// +// var totalBytesRead = 0L +// +// val buff = new Array[Byte](LargeByteBuffer.TEMP_ARRAY_SIZE) +// +// while (hasRemaining()) { +// // read what we can ... note, since there is no gaurantee that underlying buffer might +// // expose array() method, we do double copy - from stream to buff and from buff to bytearray. +// // see if we can optimize this later ... +// val buffer = fetchCurrentBuffer() +// val bufferRemaining = buffer.remaining() +// val max = math.min(buff.length, bufferRemaining) +// val bytesRead = inStrm.read(buff, 0, max) +// +// if (bytesRead > 0) { +// buffer.put(buff, 0, bytesRead) +// totalBytesRead += bytesRead +// // bump position too .. +// globalPosition += bytesRead +// // buffer.position(buffer.position + bytesRead) +// if (bytesRead >= bufferRemaining) toNonEmptyBuffer() +// } +// else if (-1 == bytesRead) { +// // if we had already read some data in the loop, return that. +// if (totalBytesRead > 0) return totalBytesRead +// return -1 +// } // nothing available to read, retry later. return +// else if (0 == bytesRead) { +// return totalBytesRead +// } +// +// // toNonEmptyBuffer() +// } +// +// totalBytesRead +// } +// +// // This is essentially a workaround to exposing underlying buffers +// // Note: very similar to readFrom(InputStream) : not trying anything fancy to reduce +// // code for performance reasons. +// def readFrom(inStrm: DataInput): Long = { +// +// assert (writable) +// releasePendingContainers() +// +// // this also allows us to avoid nasty corner cases in the loop. +// // if (! hasRemaining()) throw new BufferOverflowException +// if (! hasRemaining()) return 0 +// +// var totalBytesRead = 0L +// +// val buff = new Array[Byte](LargeByteBuffer.TEMP_ARRAY_SIZE) +// +// while (hasRemaining()) { +// // read what we can ... note, since there is no gaurantee that underlying buffer might +// // expose array() method, we do double copy - from stream to buff and from buff to bytearray. +// // see if we can optimize this later ... +// val buffer = fetchCurrentBuffer() +// val bufferRemaining = buffer.remaining() +// val max = math.min(buff.length, bufferRemaining) +// inStrm.readFully(buff, 0, max) +// val bytesRead = max +// +// if (bytesRead > 0) { +// buffer.put(buff, 0, bytesRead) +// totalBytesRead += bytesRead +// // bump position too .. +// globalPosition += bytesRead +// // buffer.position(buffer.position() + bytesRead) +// if (bytesRead >= bufferRemaining) toNonEmptyBuffer() +// } +// else if (-1 == bytesRead) { +// // if we had already read some data in the loop, return that. +// if (totalBytesRead > 0) return totalBytesRead +// return -1 +// } // nothing available to read, retry later. return +// else if (0 == bytesRead) { +// return totalBytesRead +// } +// +// // toNonEmptyBuffer() +// } +// +// totalBytesRead +// } +// +// // This is essentially a workaround to exposing underlying buffers +// // Note: tries to do it efficiently without needing to load everything into memory +// // (particularly for diskbacked buffers, etc). +// def writeTo(channel: WritableByteChannel, cleanup: Boolean): Long = { +// +// assert (readable) +// releasePendingContainers() +// +// // this also allows us to avoid nasty corner cases in the loop. +// if (! hasRemaining()) throw new BufferUnderflowException +// +// var totalBytesWritten = 0L +// +// while (hasRemaining()) { +// // Write what we can ... +// val buffer = fetchCurrentBuffer() +// val bufferRemaining = buffer.remaining() +// assert (bufferRemaining > 0) +// val bytesWritten = channel.write(buffer) +// +// if (bytesWritten > 0) { +// totalBytesWritten += bytesWritten +// // bump position too .. +// globalPosition += bytesWritten +// if (bytesWritten >= bufferRemaining) toNonEmptyBuffer() +// assert (! hasRemaining() || currentRemaining() > 0) +// } +// else if (0 == bytesWritten) { +// return totalBytesWritten +// } +// +// // toNonEmptyBuffer() +// } +// +// assert (! hasRemaining()) +// if (cleanup) { +// free() +// } +// totalBytesWritten +// } +// +// // This is essentially a workaround to exposing underlying buffers +// def writeTo(outStrm: OutputStream, cleanup: Boolean): Long = { +// +// assert (readable) +// releasePendingContainers() +// +// // this also allows us to avoid nasty corner cases in the loop. +// if (! hasRemaining()) throw new BufferUnderflowException +// +// var totalBytesWritten = 0L +// val buff = new Array[Byte](LargeByteBuffer.TEMP_ARRAY_SIZE) +// +// while (hasRemaining()) { +// // write what we can ... note, since there is no gaurantee that underlying buffer might +// // expose array() method, we do double copy - from bytearray to buff and from +// // buff to outputstream. see if we can optimize this later ... +// val buffer = fetchCurrentBuffer() +// val bufferRemaining = buffer.remaining() +// val size = math.min(bufferRemaining, buff.length) +// buffer.get(buff, 0, size) +// outStrm.write(buff, 0, size) +// +// totalBytesWritten += size +// // bump position too .. +// globalPosition += size +// +// if (size >= bufferRemaining) toNonEmptyBuffer() +// } +// +// toNonEmptyBuffer() +// if (cleanup) { +// free() +// } +// totalBytesWritten +// } +// +// def asInputStream(): InputStream = { +// new InputStream() { +// override def read(): Int = { +// if (! hasRemaining()) return -1 +// get() +// } +// +// override def read(arr: Array[Byte], off: Int, len: Int): Int = { +// if (! hasRemaining()) return -1 +// +// get(arr, off, len) +// } +// +// override def available(): Int = { +// // current remaining is what can be read without blocking +// // anything higher might need disk access/buffer swapping. +// /* +// val left = remaining() +// math.min(left, Int.MaxValue).asInstanceOf[Int] +// */ +// currentRemaining() +// } +// } +// } +// +// def getCleaner() = cleaner +// +// /** +// * @param cleaner The previous cleaner, so that the caller can chain them if required. +// * @return +// */ +// private[spark] def overrideCleaner(cleaner: BufferCleaner): BufferCleaner = { +// overrideCleaner(cleaner, allowOverride = true) +// } +// +// private def overrideCleaner(cleaner: BufferCleaner, allowOverride: Boolean): BufferCleaner = { +// if (! this.allowCleanerOverride) { +// // allowCleanerOverride = false is used for EMPTY_BUFFER - where we do not allow free +// return this.cleaner +// } +// +// this.allowCleanerOverride = allowOverride +// assert (null != cleaner) +// val prev = this.cleaner +// this.cleaner = cleaner +// // logInfo("Overriding " + prev + " with " + this.cleaner) +// prev +// } +// +// private def doReleaseAll() { +// for (container <- containers) { +// container.release() +// } +// } +// +// def free(invokeCleaner: Boolean = true) { +// // logInfo("Free on " + this + ", cleaner = " + cleaner) +// // always invoking release +// doReleaseAll() +// +// if (invokeCleaner) cleaner.clean(this) +// } +// +// private def doDispose(needRelease: Boolean) { +// +// if (disposeLocationThrowable ne null) { +// logError("Already free'ed earlier at : ", disposeLocationThrowable) +// logError("Current at ", new Throwable) +// throw new IllegalStateException("Already freed.") +// } +// disposeLocationThrowable = new Throwable() +// +// // Forcefully cleanup all +// if (needRelease) doReleaseAll() +// +// // Free in a different loop, in case different containers refer to same resource +// // to release (like file) +// for (container <- containers) { +// container.free() +// } +// +// needReleaseIndices.clear() +// +// // We should not use this buffer anymore : set the values such that f +// // we dont ... +// globalPosition = 0 +// globalLimit = 0 +// globalCapacity = 0 +// } +// +// // copy data over ... MUST be used only for cases where array is known to be +// // small to begin with. slightly risky method due to that assumption +// def toByteArray(): Array[Byte] = { +// val positionBackup = position() +// val size = remaining() +// if (size > Int.MaxValue) { +// throw new IllegalStateException( +// "Attempt to convert LargeByteBuffer to byte array when data held is more than 2G") +// } +// +// val retval = new Array[Byte](size.asInstanceOf[Int]) +// val readSize = get(retval, 0, retval.length) +// assert (readSize == retval.length, +// "readSize = " + readSize + ", retval.length = " + retval.length) +// +// position(positionBackup) +// +// retval +// } +// +// // copy data over ... MUST be used only for cases where array is known to be +// // small to begin with. slightly risky method due to that assumption +// def toByteBuffer(): ByteBuffer = { +// ByteBuffer.wrap(toByteArray()) +// } +// +// def toInMemoryBuffer(ioConf: IOConfig): LargeByteBuffer = { +// val retval = LargeByteBuffer.allocateMemoryBuffer(remaining(), ioConf) +// val currentPosition = position() +// retval.put(this) +// position(currentPosition) +// retval.clear() +// retval +// } +// +// +// +// // This is ONLY used for testing : that too as part of development of this and associated classes +// // remove before contributing to spark. +// def hexDump(): String = { +// if (remaining() * 64 > Int.MaxValue) { +// throw new UnsupportedOperationException("buffer too large " + remaining()) +// } +// +// val sb = new StringBuilder((remaining() * 2).asInstanceOf[Int]) +// +// var perLine = 0 +// var first = true +// for (b <- toByteArray()) { +// perLine += 1 +// if (perLine % 8 == 0) { +// sb.append('\n') +// first = true +// } +// if (! first) sb.append(' ') +// first = false +// sb.append(java.lang.Integer.toHexString(b & 0xff)) +// } +// sb.append('\n') +// sb.toString() +// } +// +// override def toString: String = { +// val sb: StringBuffer = new StringBuffer +// sb.append(getClass.getName) +// sb.append(' ') +// sb.append(System.identityHashCode(this)) +// sb.append("@[pos=") +// sb.append(position()) +// sb.append(" lim=") +// sb.append(limit()) +// sb.append(" cap=") +// sb.append(capacity()) +// sb.append("]") +// sb.toString +// } +// +// +// +// override def finalize(): Unit = { +// var marked = false +// if (containers ne null) { +// if (containers.exists(container => container.isAcquired && container.requireRelease())) { +// marked = true +// logError("BUG: buffer was not released - and now going out of scope. " + +// "Potential resource leak. Allocated at ", allocateLocationThrowable) +// containers.foreach(_.release()) +// } +// if (containers.exists(container => !container.isFreed && container.requireFree())) { +// if (!marked) { +// logError("BUG: buffer was not freed - and now going out of scope. Potential resource leak", +// allocateLocationThrowable) +// } +// else { +// logError("BUG: buffer was not freed - and now going out of scope. Potential resource leak") +// } +// containers.foreach(_.free()) +// } +// } +// super.finalize() +// } +//} +// +// +//object LargeByteBuffer extends Logging { +// +// private val noopDisposeFunction = new BufferCleaner() { +// protected def doClean(buffer: LargeByteBuffer) { +// buffer.free(invokeCleaner = false) +// } +// } +// +// val enableExpensiveAssert = false +// private val EMPTY_BYTEBUFFER = ByteBuffer.allocate(0) +// val EMPTY_BUFFER = new LargeByteBuffer(ArrayBuffer( +// new HeapByteBufferContainer(EMPTY_BYTEBUFFER, false)), false, false) +// // Do not allow anyone else to override cleaner +// EMPTY_BUFFER.overrideCleaner(noopDisposeFunction, allowOverride = false) +// +// // 8K sufficient ? +// private val TEMP_ARRAY_SIZE = 8192 +// +// /** +// * Create a LargeByteBuffer of specified size which is split across +// * ByteBuffer's of size DiskStore.MAX_BLOCK_SIZE and backed by in memory +// * ByteBuffer +// * +// */ +// def allocateMemoryBuffer(totalSize: Long, ioConf: IOConfig): LargeByteBuffer = { +// if (0 == totalSize) { +// return EMPTY_BUFFER +// } +// +// assert (totalSize > 0) +// +// val blockSize = ioConf.getMaxBlockSize(BufferType.MEMORY) +// val numBlocks = ioConf.numBlocks(BufferType.MEMORY, totalSize) +// val lastBlockSize = ioConf.lastBlockSize(BufferType.MEMORY, totalSize) +// +// assert (lastBlockSize > 0) +// +// val bufferArray = { +// val arr = new ArrayBuffer[ByteBufferContainer](numBlocks) +// for (index <- 0 until numBlocks - 1) { +// val buff = ByteBuffer.allocate(blockSize) +// // buff.clear() +// arr += new HeapByteBufferContainer(buff, true) +// } +// arr += new HeapByteBufferContainer(ByteBuffer.allocate(lastBlockSize), true) +// assert (arr.length == numBlocks) +// arr +// } +// +// new LargeByteBuffer(bufferArray, false, false) +// } +// +// /** +// * Create a LargeByteBuffer of specified size which is split across +// * ByteBuffer's of size DiskStore.MAX_BLOCK_SIZE and backed by on disk +// * +// */ +// private def allocateDiskBuffer(totalSize: Long, +// blockManager: BlockManager): LargeByteBuffer = { +// if (0 == totalSize) { +// return EMPTY_BUFFER +// } +// +// assert (totalSize > 0) +// +// // Create a file of the specified size. +// val file = blockManager.diskBlockManager.createTempBlock()._2 +// val raf = new RandomAccessFile(file, "rw") +// try { +// raf.setLength(totalSize) +// } finally { +// raf.close() +// } +// +// readWriteDiskSegment(new FileSegment(file, 0, totalSize), +// ephemeralDiskBacked = true, blockManager.ioConf) +// } +// +// // The returned buffer takes up ownership of the underlying buffers +// // (including dispos'ing that when done) +// def fromBuffers(buffers: ByteBuffer*): LargeByteBuffer = { +// val nonEmpty = buffers.filter(_.hasRemaining) +// +// // cleanup the empty buffers +// buffers.filter(! _.hasRemaining).foreach(b => BlockManager.dispose(b)) +// +// +// if (nonEmpty.isEmpty) { +// return EMPTY_BUFFER +// } +// +// // slice so that offsets match our requirement +// new LargeByteBuffer(new ArrayBuffer() ++ nonEmpty.map(b => +// new HeapByteBufferContainer(b.slice(), true)), false, false) +// } +// +// def fromByteArrays(byteArrays: Array[Byte]*): LargeByteBuffer = { +// // only non empty arrays +// val arrays = byteArrays.filter(_.length > 0) +// if (0 == arrays.length) return EMPTY_BUFFER +// +// new LargeByteBuffer(new ArrayBuffer() ++ arrays.map(arr => +// new HeapByteBufferContainer(ByteBuffer.wrap(arr), true)), false, false) +// } +// +// def fromLargeByteBuffers(canDispose: Boolean, inputBuffers: LargeByteBuffer*): LargeByteBuffer = { +// +// if (inputBuffers.isEmpty) return EMPTY_BUFFER +// +// if (! inputBuffers.exists(_.hasRemaining())) { +// if (canDispose) inputBuffers.map(_.free()) +// return EMPTY_BUFFER +// } +// +// // release all temp resources acquired +// inputBuffers.foreach(buff => buff.releasePendingContainers()) +// // free current container if acquired. +// inputBuffers.foreach(buff => if (buff.currentContainerIndex < buff.containers.length) { +// buff.containers(buff.currentContainerIndex).release() +// }) +// // inputBuffers.foreach(b => b.doReleaseAll()) +// +// +// // Dispose of any empty buffers +// if (canDispose) inputBuffers.filter(! _.hasRemaining()).foreach(_.free()) +// +// // Find all containers we need. +// val buffers = inputBuffers.filter(_.hasRemaining()).map(b => b.createSlice(b.remaining())) +// +// val containers = buffers.flatMap(_.containers) +// assert (! containers.isEmpty) +// // The in order containers of "buffers" seq constitute the required return value +// val retval = new LargeByteBuffer(new ArrayBuffer() ++ containers, +// // if you cant dispose, then we dont own the buffers : in which case, need duplicate +// ! canDispose, inputBuffers.exists(_.ephemeralDiskBacked)) +// +// if (canDispose) { +// // override dispose of all other buffers. +// val disposeFunctions = inputBuffers.map { +// buffer => { +// (buffer, buffer.overrideCleaner(noopDisposeFunction)) +// } +// } +// +// val cleaner = retval.getCleaner() +// val newCleaner = new BufferCleaner { +// protected def doClean(buffer: LargeByteBuffer) { +// +// assert (retval == buffer) +// // default cleaner. +// cleaner.clean(retval) +// // not required, since we are within clean anyway. +// // retval.free(invokeCleaner = false) +// +// // retval.doDispose(needRelease = true) +// +// // This might actually call dispose twice on some (initially) empty buffers, +// // which is fine since we now guard against that. +// disposeFunctions.foreach(v => v._2.clean(v._1)) +// // Call the free method too : so that buffers are marked free ... +// disposeFunctions.foreach(v => v._1.free(invokeCleaner = false)) +// } +// } +// +// val prev = retval.overrideCleaner(newCleaner) +// assert (prev == cleaner) +// } +// +// retval +// } +// +// private def checkOffsets(arr: Array[Byte], offset: Int, size: Int) { +// if (arr == null) { +// throw new NullPointerException +// } else if (offset < 0 || size < 0 || offset + size > arr.length) { +// throw new IndexOutOfBoundsException +// } +// } +// +// def allocateTransientBuffer(size: Long, blockManager: BlockManager) = { +// if (size <= blockManager.ioConf.maxInMemSize) { +// LargeByteBuffer.allocateMemoryBuffer(size, blockManager.ioConf) +// } else { +// LargeByteBuffer.allocateDiskBuffer(size, blockManager) +// } +// } +// +// def readFromDiskSegment(segment: FileSegment, ioConf: IOConfig, +// ephemeralDiskBacked: Boolean): LargeByteBuffer = { +// // Split the block into multiple of BlockStore.maxBlockSize +// val segmentSize = segment.length +// val blockSize = ioConf.getMaxBlockSize(BufferType.DISK).asInstanceOf[Long] +// val numBlocks = ioConf.numBlocks(BufferType.DISK, segmentSize) +// val lastBlockSize = ioConf.lastBlockSize(BufferType.DISK, segmentSize) +// +// val buffers = new ArrayBuffer[ByteBufferContainer](numBlocks) +// +// for (index <- 0 until numBlocks - 1) { +// buffers += new ReadOnlyFileContainer(new FileSegment(segment.file, +// segment.offset + index * blockSize, blockSize), ioConf) +// } +// +// // Last block +// buffers += new ReadOnlyFileContainer(new FileSegment(segment.file, +// segment.offset + (numBlocks - 1) * blockSize, lastBlockSize), ioConf) +// +// new LargeByteBuffer(buffers, false, ephemeralDiskBacked) +// } +// +// def readWriteDiskSegment(segment: FileSegment, ephemeralDiskBacked: Boolean, +// ioConf: IOConfig): LargeByteBuffer = { +// +// // Split the block into multiple of BlockStore.maxBlockSize +// val segmentSize = segment.length +// val blockSize = ioConf.getMaxBlockSize(BufferType.DISK).asInstanceOf[Long] +// val numBlocks = ioConf.numBlocks(BufferType.DISK, segmentSize) +// val lastBlockSize = ioConf.lastBlockSize(BufferType.DISK, segmentSize) +// +// logInfo("readWriteDiskSegment = " + segment + ", numBlocks = " + numBlocks + +// ", lastBlockSize = " + lastBlockSize) +// val buffers = new ArrayBuffer[ByteBufferContainer](numBlocks) +// +// for (index <- 0 until numBlocks - 1) { +// buffers += new ReadWriteFileContainer(new FileSegment(segment.file, +// segment.offset + index * blockSize, blockSize), ephemeralDiskBacked, null) +// } +// +// // Last block +// buffers += new ReadWriteFileContainer(new FileSegment(segment.file, +// segment.offset + (numBlocks - 1) * blockSize, lastBlockSize), ephemeralDiskBacked, null) +// +// new LargeByteBuffer(buffers, false, ephemeralDiskBacked) +// } +//} diff --git a/core/src/main/scala/org/apache/spark/io/WrappedByteArrayOutputStream.scala b/core/src/main/scala/org/apache/spark/io/WrappedByteArrayOutputStream.scala index 0dd7e8e736ad6..6657c4f7efc52 100644 --- a/core/src/main/scala/org/apache/spark/io/WrappedByteArrayOutputStream.scala +++ b/core/src/main/scala/org/apache/spark/io/WrappedByteArrayOutputStream.scala @@ -1,121 +1,121 @@ -/* - * 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.io - -import java.io.OutputStream -import scala.collection.mutable.ArrayBuffer - -import org.apache.spark.Logging -import org.apache.spark.io.IOConfig.BufferType - -/** - * byte array backed streams (FastByteArrayOutputStream, ByteArrayOutputStream, etc) are limited to - * array length of 2 gig - since that is the array size limit. - * - * So we move from one to the next as soon as we hit the limit per stream. - * And once done, asBuffers or toByteArrays can be used to pull data as a sequence of bytebuffers - * or byte arrays. - * @param initialSize initial size for the byte array stream ... - */ -class WrappedByteArrayOutputStream(private val initialSize: Int, - ioConf: IOConfig) extends OutputStream with Logging { - - private val maxStreamSize = ioConf.getMaxBlockSize(BufferType.MEMORY) - - private val allStreams = new ArrayBuffer[SparkByteArrayOutputStream](4) - - private var current: SparkByteArrayOutputStream = null - private var currentWritten = 0 - - nextWriter() - - override def flush(): Unit = { - current.flush() - } - - override def write(b: Int): Unit = { - if (currentWritten >= maxStreamSize) { - nextWriter() - } - current.write(b) - currentWritten += 1 - } - - - override def write(b: Array[Byte], off: Int, len: Int): Unit = { - // invariant checks - from OutputStream.java - if (b == null) { - throw new NullPointerException - } else if ((off < 0) || (off > b.length) || (len < 0) || - ((off + len) > b.length) || ((off + len) < 0)) { - throw new IndexOutOfBoundsException - } else if (len == 0) { - return - } - - // Else, write to stream. - - // common case first - if (currentWritten + len < maxStreamSize) { - current.write(b, off, len) - currentWritten += len - return - } - - // We might need to split the write into two streams. - var startOff = off - var remaining = len - - while (remaining > 0) { - var toCurrent = math.min(remaining, maxStreamSize - currentWritten) - if (toCurrent > 0) { - current.write(b, startOff, toCurrent) - currentWritten += toCurrent - remaining -= toCurrent - startOff += toCurrent - } - - if (currentWritten >= maxStreamSize) { - // to next - nextWriter() - } - } - } - - def toLargeByteBuffer(): LargeByteBuffer = { - current.compact() - val seq = allStreams.filter(_.size > 0).map(_.toByteBuffer) - val retval = LargeByteBuffer.fromBuffers(seq:_*) - - retval - } - - private def nextWriter() { - if (null != current) { - current.flush() - current.compact() - current = null - } - - current = new SparkByteArrayOutputStream(initialSize, ioConf) - currentWritten = 0 - allStreams += current - } -} - - +///* +// * 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.io +// +//import java.io.OutputStream +//import scala.collection.mutable.ArrayBuffer +// +//import org.apache.spark.Logging +//import org.apache.spark.io.IOConfig.BufferType +// +///** +// * byte array backed streams (FastByteArrayOutputStream, ByteArrayOutputStream, etc) are limited to +// * array length of 2 gig - since that is the array size limit. +// * +// * So we move from one to the next as soon as we hit the limit per stream. +// * And once done, asBuffers or toByteArrays can be used to pull data as a sequence of bytebuffers +// * or byte arrays. +// * @param initialSize initial size for the byte array stream ... +// */ +//class WrappedByteArrayOutputStream(private val initialSize: Int, +// ioConf: IOConfig) extends OutputStream with Logging { +// +// private val maxStreamSize = ioConf.getMaxBlockSize(BufferType.MEMORY) +// +// private val allStreams = new ArrayBuffer[SparkByteArrayOutputStream](4) +// +// private var current: SparkByteArrayOutputStream = null +// private var currentWritten = 0 +// +// nextWriter() +// +// override def flush(): Unit = { +// current.flush() +// } +// +// override def write(b: Int): Unit = { +// if (currentWritten >= maxStreamSize) { +// nextWriter() +// } +// current.write(b) +// currentWritten += 1 +// } +// +// +// override def write(b: Array[Byte], off: Int, len: Int): Unit = { +// // invariant checks - from OutputStream.java +// if (b == null) { +// throw new NullPointerException +// } else if ((off < 0) || (off > b.length) || (len < 0) || +// ((off + len) > b.length) || ((off + len) < 0)) { +// throw new IndexOutOfBoundsException +// } else if (len == 0) { +// return +// } +// +// // Else, write to stream. +// +// // common case first +// if (currentWritten + len < maxStreamSize) { +// current.write(b, off, len) +// currentWritten += len +// return +// } +// +// // We might need to split the write into two streams. +// var startOff = off +// var remaining = len +// +// while (remaining > 0) { +// var toCurrent = math.min(remaining, maxStreamSize - currentWritten) +// if (toCurrent > 0) { +// current.write(b, startOff, toCurrent) +// currentWritten += toCurrent +// remaining -= toCurrent +// startOff += toCurrent +// } +// +// if (currentWritten >= maxStreamSize) { +// // to next +// nextWriter() +// } +// } +// } +// +// def toLargeByteBuffer(): LargeByteBuffer = { +// current.compact() +// val seq = allStreams.filter(_.size > 0).map(_.toByteBuffer) +// val retval = LargeByteBuffer.fromBuffers(seq:_*) +// +// retval +// } +// +// private def nextWriter() { +// if (null != current) { +// current.flush() +// current.compact() +// current = null +// } +// +// current = new SparkByteArrayOutputStream(initialSize, ioConf) +// currentWritten = 0 +// allStreams += current +// } +//} +// +// 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 86dbd89f0ffb8..ad895ff338d54 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -31,7 +31,7 @@ import sun.nio.ch.DirectBuffer import org.apache.spark._ import org.apache.spark.executor._ -import org.apache.spark.io.CompressionCodec +import org.apache.spark.io.{WrappedLargeByteBuffer, ChainedLargeByteBuffer, LargeByteBuffer, CompressionCodec} import org.apache.spark.network._ import org.apache.spark.network.buffer.{ManagedBuffer, NioManagedBuffer} import org.apache.spark.network.netty.SparkTransportConf @@ -43,7 +43,7 @@ import org.apache.spark.shuffle.hash.HashShuffleManager import org.apache.spark.util._ private[spark] sealed trait BlockValues -private[spark] case class ByteBufferValues(buffer: ByteBuffer) extends BlockValues +private[spark] case class ByteBufferValues(buffer: LargeByteBuffer) extends BlockValues private[spark] case class IteratorValues(iterator: Iterator[Any]) extends BlockValues private[spark] case class ArrayValues(buffer: Array[Any]) extends BlockValues @@ -78,6 +78,9 @@ private[spark] class BlockManager( val diskBlockManager = new DiskBlockManager(this, conf) + //XXX + val largeByteBufferChunkSize = 65536 + private val blockInfo = new TimeStampedHashMap[BlockId, BlockInfo] // Actual storage of where blocks are kept @@ -318,7 +321,7 @@ private[spark] class BlockManager( * Put the block locally, using the given storage level. */ override def putBlockData(blockId: BlockId, data: ManagedBuffer, level: StorageLevel): Unit = { - putBytes(blockId, data.nioByteBuffer(), level) + putBytes(blockId, new WrappedLargeByteBuffer(data.nioByteBuffer()), level) } /** @@ -513,7 +516,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 { + val bytes: LargeByteBuffer = diskStore.getBytes(blockId) match { case Some(b) => b case None => throw new BlockException( @@ -535,7 +538,7 @@ private[spark] class BlockManager( /* We'll store the bytes in memory if the block's storage level includes * "memory serialized", or if it should be cached as objects in memory * but we only requested its serialized bytes. */ - val copyForMemory = ByteBuffer.allocate(bytes.limit) + val copyForMemory = LargeByteBuffer.allocateOnHeap(bytes.limit, largeByteBufferChunkSize) copyForMemory.put(bytes) memoryStore.putBytes(blockId, copyForMemory, level) bytes.rewind() @@ -591,8 +594,9 @@ private[spark] class BlockManager( val locations = Random.shuffle(master.getLocations(blockId)) for (loc <- locations) { logDebug(s"Getting remote block $blockId from $loc") - val data = blockTransferService.fetchBlockSync( - loc.host, loc.port, loc.executorId, blockId.toString).nioByteBuffer() + //the fetch will always be one byte buffer till we fix SPARK-5928 + val data: LargeByteBuffer = new WrappedLargeByteBuffer(blockTransferService.fetchBlockSync( + loc.host, loc.port, loc.executorId, blockId.toString).nioByteBuffer()) if (data != null) { if (asBlockResult) { @@ -674,7 +678,7 @@ private[spark] class BlockManager( */ def putBytes( blockId: BlockId, - bytes: ByteBuffer, + bytes: LargeByteBuffer, level: StorageLevel, tellMaster: Boolean = true, effectiveStorageLevel: Option[StorageLevel] = None): Seq[(BlockId, BlockStatus)] = { @@ -736,7 +740,7 @@ private[spark] class BlockManager( var valuesAfterPut: Iterator[Any] = null // Ditto for the bytes after the put - var bytesAfterPut: ByteBuffer = null + var bytesAfterPut: LargeByteBuffer = null // Size of the block in bytes var size = 0L @@ -884,7 +888,7 @@ private[spark] class BlockManager( * Replicate block to another node. Not that this is a blocking call that returns after * the block has been replicated. */ - private def replicate(blockId: BlockId, data: ByteBuffer, level: StorageLevel): Unit = { + private def replicate(blockId: BlockId, data: LargeByteBuffer, level: StorageLevel): Unit = { val maxReplicationFailures = conf.getInt("spark.storage.maxReplicationFailures", 1) val numPeersToReplicateTo = level.replication - 1 val peersForReplication = new ArrayBuffer[BlockManagerId] @@ -940,8 +944,11 @@ private[spark] class BlockManager( val onePeerStartTime = System.currentTimeMillis data.rewind() logTrace(s"Trying to replicate $blockId of ${data.limit()} bytes to $peer") - blockTransferService.uploadBlockSync( - peer.host, peer.port, peer.executorId, blockId, new NioManagedBuffer(data), tLevel) + //TODO + //ACK! here we're stuck -- we can't replicate a large block until we figure out + // how to deal w/ shuffling more than 2 gb +// blockTransferService.uploadBlockSync( +// peer.host, peer.port, peer.executorId, blockId, new NioManagedBuffer(data), tLevel) logTrace(s"Replicated $blockId of ${data.limit()} bytes to $peer in %s ms" .format(System.currentTimeMillis - onePeerStartTime)) peersReplicatedTo += peer @@ -1180,10 +1187,10 @@ private[spark] class BlockManager( def dataSerialize( blockId: BlockId, values: Iterator[Any], - serializer: Serializer = defaultSerializer): ByteBuffer = { - val byteStream = new ByteArrayOutputStream(4096) + serializer: Serializer = defaultSerializer): LargeByteBuffer = { + val byteStream = new LargeByteBufferOutputStream() dataSerializeStream(blockId, byteStream, values, serializer) - ByteBuffer.wrap(byteStream.toByteArray) + byteStream.largeBuffer } /** @@ -1192,10 +1199,10 @@ private[spark] class BlockManager( */ def dataDeserialize( blockId: BlockId, - bytes: ByteBuffer, + bytes: LargeByteBuffer, serializer: Serializer = defaultSerializer): Iterator[Any] = { bytes.rewind() - val stream = wrapForCompression(blockId, new ByteBufferInputStream(bytes, true)) + val stream = wrapForCompression(blockId, new LargeByteBufferInputStream(bytes, true)) serializer.newInstance().deserializeStream(stream).asIterator } @@ -1245,6 +1252,11 @@ private[spark] object BlockManager extends Logging { } } + def dispose(buffer: LargeByteBuffer): Unit = { + // TODO + ??? + } + def blockIdsToBlockManagers( blockIds: Array[BlockId], env: SparkEnv, diff --git a/core/src/main/scala/org/apache/spark/storage/BlockStore.scala b/core/src/main/scala/org/apache/spark/storage/BlockStore.scala index b14b5e91d1794..38989f0c07681 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockStore.scala @@ -17,6 +17,8 @@ package org.apache.spark.storage +import java.nio.ByteBuffer + import scala.collection.mutable.ArrayBuffer import org.apache.spark.Logging @@ -43,15 +45,15 @@ private[spark] abstract class BlockStore(val blockManager: BlockManager) extends * @return a PutResult that contains the size of the data, as well as the values put if * returnValues is true (if not, the result's data field can be null) */ - def putValues( + def putIterator( blockId: BlockId, values: Iterator[Any], level: StorageLevel, returnValues: Boolean): PutResult - def putValues( + def putArray( blockId: BlockId, - values: ArrayBuffer[Any], + values: Array[Any], level: StorageLevel, returnValues: Boolean): PutResult 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 61ef5ff168791..18293f3314a5f 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala @@ -22,6 +22,7 @@ import java.nio.ByteBuffer import java.nio.channels.FileChannel.MapMode import org.apache.spark.Logging +import org.apache.spark.io.LargeByteBuffer import org.apache.spark.serializer.Serializer import org.apache.spark.util.Utils @@ -104,7 +105,7 @@ private[spark] class DiskStore(blockManager: BlockManager, diskManager: DiskBloc } } - private def getBytes(file: File, offset: Long, length: Long): Option[ByteBuffer] = { + private def getBytes(file: File, offset: Long, length: Long): Option[LargeByteBuffer] = { val channel = new RandomAccessFile(file, "r").getChannel try { @@ -128,12 +129,12 @@ private[spark] class DiskStore(blockManager: BlockManager, diskManager: DiskBloc } } - override def getBytes(blockId: BlockId): Option[ByteBuffer] = { + override def getBytes(blockId: BlockId): Option[LargeByteBuffer] = { val file = diskManager.getFile(blockId.name) getBytes(file, 0, file.length) } - def getBytes(segment: FileSegment): Option[ByteBuffer] = { + def getBytes(segment: FileSegment): Option[LargeByteBuffer] = { getBytes(segment.file, segment.offset, segment.length) } 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 71305a46bf570..90ced59104432 100644 --- a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala @@ -20,6 +20,8 @@ package org.apache.spark.storage import java.nio.ByteBuffer import java.util.LinkedHashMap +import org.apache.spark.io.LargeByteBuffer + import scala.collection.mutable import scala.collection.mutable.ArrayBuffer @@ -77,7 +79,7 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) } } - override def putBytes(blockId: BlockId, _bytes: ByteBuffer, level: StorageLevel): PutResult = { + override def putBytes(blockId: BlockId, _bytes: LargeByteBuffer, level: StorageLevel): PutResult = { // Work on a duplicate - since the original input might be used elsewhere. val bytes = _bytes.duplicate() bytes.rewind() diff --git a/core/src/main/scala/org/apache/spark/storage/PutResult.scala b/core/src/main/scala/org/apache/spark/storage/PutResult.scala index f0eac7594ecf6..2e00934bde243 100644 --- a/core/src/main/scala/org/apache/spark/storage/PutResult.scala +++ b/core/src/main/scala/org/apache/spark/storage/PutResult.scala @@ -17,7 +17,7 @@ package org.apache.spark.storage -import java.nio.ByteBuffer +import org.apache.spark.io.LargeByteBuffer /** * Result of adding a block into a BlockStore. This case class contains a few things: @@ -28,5 +28,5 @@ import java.nio.ByteBuffer */ private[spark] case class PutResult( size: Long, - data: Either[Iterator[_], ByteBuffer], + data: Either[Iterator[_], LargeByteBuffer], droppedBlocks: Seq[(BlockId, BlockStatus)] = Seq.empty) diff --git a/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala b/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala index 233d1e2b7c616..fd7cb49ef9d50 100644 --- a/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala @@ -21,6 +21,7 @@ import java.io.IOException import java.nio.ByteBuffer import com.google.common.io.ByteStreams +import org.apache.spark.io.LargeByteBuffer import tachyon.client.{ReadType, WriteType} import org.apache.spark.Logging @@ -64,7 +65,7 @@ private[spark] class TachyonStore( private def putIntoTachyonStore( blockId: BlockId, - bytes: ByteBuffer, + bytes: LargeByteBuffer, returnValues: Boolean): PutResult = { // So that we do not modify the input offsets ! // duplicate does not copy buffer, so inexpensive @@ -100,7 +101,7 @@ private[spark] class TachyonStore( getBytes(blockId).map(buffer => blockManager.dataDeserialize(blockId, buffer)) } - override def getBytes(blockId: BlockId): Option[ByteBuffer] = { + override def getBytes(blockId: BlockId): Option[LargeByteBuffer] = { val file = tachyonManager.getFile(blockId) if (file == null || file.getLocationHosts.size == 0) { return None diff --git a/core/src/main/scala/org/apache/spark/util/LargeByteBufferInputStream.scala b/core/src/main/scala/org/apache/spark/util/LargeByteBufferInputStream.scala new file mode 100644 index 0000000000000..26f2d7848bb29 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/LargeByteBufferInputStream.scala @@ -0,0 +1,82 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util + +import java.io.InputStream +import java.nio.ByteBuffer + +import org.apache.spark.io.LargeByteBuffer +import org.apache.spark.storage.BlockManager + +/** + * Reads data from a ByteBuffer, and optionally cleans it up using BlockManager.dispose() + * at the end of the stream (e.g. to close a memory-mapped file). + */ +private[spark] +class LargeByteBufferInputStream(private var buffer: LargeByteBuffer, dispose: Boolean = false) + extends InputStream { + + override def read(): Int = { + if (buffer == null || buffer.remaining() == 0) { + cleanUp() + -1 + } else { + buffer.get() & 0xFF + } + } + + override def read(dest: Array[Byte]): Int = { + read(dest, 0, dest.length) + } + + override def read(dest: Array[Byte], offset: Int, length: Int): Int = { + if (buffer == null || buffer.remaining() == 0) { + cleanUp() + -1 + } else { + val amountToGet = math.min(buffer.remaining(), length).toInt + buffer.get(dest, offset, amountToGet) + amountToGet + } + } + + override def skip(bytes: Long): Long = { + if (buffer != null) { + val amountToSkip = math.min(bytes, buffer.remaining).toInt + buffer.position(buffer.position + amountToSkip) + if (buffer.remaining() == 0) { + cleanUp() + } + amountToSkip + } else { + 0L + } + } + + /** + * Clean up the buffer, and potentially dispose of it using BlockManager.dispose(). + */ + private def cleanUp() { + if (buffer != null) { + if (dispose) { + BlockManager.dispose(buffer) + } + buffer = null + } + } +} diff --git a/core/src/main/scala/org/apache/spark/util/LargeByteBufferOutputStream.scala b/core/src/main/scala/org/apache/spark/util/LargeByteBufferOutputStream.scala new file mode 100644 index 0000000000000..246ebca999437 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/LargeByteBufferOutputStream.scala @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util + +import java.io.OutputStream + +import org.apache.spark.io.{ChainedLargeByteBuffer, LargeByteBuffer} +import org.apache.spark.util.collection.ChainedBuffer + +private[spark] +class LargeByteBufferOutputStream(chunkSize: Int = 65536) + extends OutputStream { + + val buffer = new ChainedBuffer(chunkSize) + + private var _pos = 0 + + override def write(b: Int): Unit = { + throw new UnsupportedOperationException() + } + + override def write(bytes: Array[Byte], offs: Int, len: Int): Unit = { + buffer.write(_pos, bytes, offs, len) + _pos += len + } + + def pos: Int = _pos + + def largeBuffer: LargeByteBuffer = new ChainedLargeByteBuffer(buffer) +} diff --git a/core/src/main/scala/org/apache/spark/util/collection/ChainedBuffer.scala b/core/src/main/scala/org/apache/spark/util/collection/ChainedBuffer.scala new file mode 100644 index 0000000000000..c39a2fd1f8a11 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/collection/ChainedBuffer.scala @@ -0,0 +1,126 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util.collection + +import java.io.OutputStream + +import scala.collection.mutable.ArrayBuffer + +/** + * A logical byte buffer that wraps a list of byte arrays. All the byte arrays have equal size. The + * advantage of this over a standard ArrayBuffer is that it can grow without claiming large amounts + * of memory and needing to copy the full contents. + */ +private[spark] class ChainedBuffer private(val chunks: ArrayBuffer[Array[Byte]], chunkSize: Int) { + private val chunkSizeLog2 = (math.log(chunkSize) / math.log(2)).toInt + assert(math.pow(2, chunkSizeLog2).toInt == chunkSize) + private var _size: Long = _ + + /** + * Read bytes from this buffer into a byte array. + * + * @param pos Offset in the buffer to read from. + * @param bytes Byte array to read into. + * @param offs Offset in the byte array to read to. + * @param len Number of bytes to read. + */ + def read(pos: Long, bytes: Array[Byte], offs: Int, len: Int): Unit = { + var chunkIndex = (pos >> chunkSizeLog2).toInt + var posInChunk = (pos - (chunkIndex << chunkSizeLog2)).toInt + var moved = 0 + while (moved < len) { + val toRead = math.min(len - moved, chunkSize - posInChunk) + System.arraycopy(chunks(chunkIndex), posInChunk, bytes, offs + moved, toRead) + moved += toRead + chunkIndex += 1 + posInChunk = 0 + } + } + + def read(pos:Long): Byte = { + val chunkIndex = (pos >> chunkSizeLog2).toInt + val posInChunk = (pos - (chunkIndex << chunkSizeLog2)).toInt + chunks(chunkIndex)(posInChunk) + } + + /** + * Write bytes from a byte array into this buffer. + * + * @param pos Offset in the buffer to write to. + * @param bytes Byte array to write from. + * @param offs Offset in the byte array to write from. + * @param len Number of bytes to write. + */ + def write(pos: Long, bytes: Array[Byte], offs: Int, len: Int): Unit = { + // Grow if needed + val endChunkIndex = ((pos + len - 1) >> chunkSizeLog2).toInt + while (endChunkIndex >= chunks.length) { + chunks += new Array[Byte](chunkSize) + } + + var chunkIndex = (pos >> chunkSizeLog2).toInt + var posInChunk = (pos - (chunkIndex << chunkSizeLog2)).toInt + var moved = 0 + while (moved < len) { + val toWrite = math.min(len - moved, chunkSize - posInChunk) + System.arraycopy(bytes, offs + moved, chunks(chunkIndex), posInChunk, toWrite) + moved += toWrite + chunkIndex += 1 + posInChunk = 0 + } + + _size = math.max(_size, pos + len) + } + + /** + * Total size of buffer that can be written to without allocating additional memory. + */ + def capacity: Int = chunks.size * chunkSize + + /** + * Size of the logical buffer. + */ + def size: Long = _size +} + +private[spark] object ChainedBuffer { + def withInitialSize(chunkSize: Int, minInitialSize: Long = 0): ChainedBuffer = { + val nChunks = (((minInitialSize - 1) / chunkSize).toInt) + 1 + val chunks = new ArrayBuffer[Array[Byte]](nChunks) + (0 until nChunks).foreach{idx => chunks(idx) = new Array[Byte](chunkSize)} + new ChainedBuffer(chunks, chunkSize) + } +} + +/** + * Output stream that writes to a ChainedBuffer. + */ +private[spark] class ChainedBufferOutputStream(chainedBuffer: ChainedBuffer) extends OutputStream { + private var _pos = 0 + + override def write(b: Int): Unit = { + throw new UnsupportedOperationException() + } + + override def write(bytes: Array[Byte], offs: Int, len: Int): Unit = { + chainedBuffer.write(_pos, bytes, offs, len) + _pos += len + } + + def pos: Int = _pos +} \ No newline at end of file diff --git a/core/src/test/scala/org/apache/spark/io/LargeByteBufferTest.scala b/core/src/test/scala/org/apache/spark/io/LargeByteBufferTest.scala new file mode 100644 index 0000000000000..06d3d223c3858 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/io/LargeByteBufferTest.scala @@ -0,0 +1,57 @@ +/* + * 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.io + +import java.io.{ObjectInputStream, ObjectOutputStream} + +import org.apache.spark.util.{LargeByteBufferInputStream, LargeByteBufferOutputStream} +import org.scalatest.{Matchers, FunSuite} + +class LargeByteBufferTest extends FunSuite with Matchers { + +// test("allocateOnHeap") { +// val bufs = LargeByteBuffer.allocateOnHeap(10, 3).asInstanceOf[ChainedLargeByteBuffer] +// bufs.underlying.foreach{buf => buf.capacity should be <= 3} +// bufs.underlying.map{_.capacity}.sum should be (10) +// } +// +// test("allocate large") { +// val size = Integer.MAX_VALUE.toLong + 10 +// val bufs = LargeByteBuffer.allocateOnHeap(size, 1e9.toInt).asInstanceOf[WrappedLargeByteBuffer] +// bufs.capacity should be (size) +// bufs.underlying.map{_.capacity.toLong}.sum should be (Integer.MAX_VALUE.toLong + 10) +// } + + + test("io stream roundtrip") { + + val rawOut = new LargeByteBufferOutputStream(128) + val objOut = new ObjectOutputStream(rawOut) + val someObject = (1 to 100).map{x => x -> scala.util.Random.nextInt(x)}.toMap + objOut.writeObject(someObject) + objOut.close() + + rawOut.largeBuffer.asInstanceOf[ChainedLargeByteBuffer].underlying.chunks.size should be > 1 + + val rawIn = new LargeByteBufferInputStream(rawOut.largeBuffer) + val objIn = new ObjectInputStream(rawIn) + val deser = objIn.readObject() + deser should be (someObject) + + } + +} diff --git a/core/src/test/scala/org/apache/spark/util/collection/ChainedBufferTest.scala b/core/src/test/scala/org/apache/spark/util/collection/ChainedBufferTest.scala new file mode 100644 index 0000000000000..e99d5ecc639c9 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/util/collection/ChainedBufferTest.scala @@ -0,0 +1,135 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util.collection + +import java.nio.ByteBuffer + +import org.scalatest.FunSuite +import org.scalatest.Matchers._ + +class ChainedBufferSuite extends FunSuite { + test("write and read at start") { + // write from start of source array + val buffer = new ChainedBuffer(8) + buffer.capacity should be (0) + verifyWriteAndRead(buffer, 0, 0, 0, 4) + buffer.capacity should be (8) + + // write from middle of source array + verifyWriteAndRead(buffer, 0, 5, 0, 4) + buffer.capacity should be (8) + + // read to middle of target array + verifyWriteAndRead(buffer, 0, 0, 5, 4) + buffer.capacity should be (8) + + // write up to border + verifyWriteAndRead(buffer, 0, 0, 0, 8) + buffer.capacity should be (8) + + // expand into second buffer + verifyWriteAndRead(buffer, 0, 0, 0, 12) + buffer.capacity should be (16) + + // expand into multiple buffers + verifyWriteAndRead(buffer, 0, 0, 0, 28) + buffer.capacity should be (32) + } + + test("write and read at middle") { + // write from start of source array + val buffer = new ChainedBuffer(8) + verifyWriteAndRead(buffer, 3, 0, 0, 4) + buffer.capacity should be (8) + + // write from middle of source array + verifyWriteAndRead(buffer, 3, 5, 0, 4) + buffer.capacity should be (8) + + // read to middle of target array + verifyWriteAndRead(buffer, 3, 0, 5, 4) + buffer.capacity should be (8) + + // write up to border + verifyWriteAndRead(buffer, 3, 0, 0, 5) + buffer.capacity should be (8) + + // expand into second buffer + verifyWriteAndRead(buffer, 3, 0, 0, 12) + buffer.capacity should be (16) + + // expand into multiple buffers + verifyWriteAndRead(buffer, 3, 0, 0, 28) + buffer.capacity should be (32) + } + + test("write and read at later buffer") { + // write from start of source array + val buffer = new ChainedBuffer(8) + verifyWriteAndRead(buffer, 11, 0, 0, 4) + buffer.capacity should be (16) + + // write from middle of source array + verifyWriteAndRead(buffer, 11, 5, 0, 4) + buffer.capacity should be (16) + + // read to middle of target array + verifyWriteAndRead(buffer, 11, 0, 5, 4) + buffer.capacity should be (16) + + // write up to border + verifyWriteAndRead(buffer, 11, 0, 0, 5) + buffer.capacity should be (16) + + // expand into second buffer + verifyWriteAndRead(buffer, 11, 0, 0, 12) + buffer.capacity should be (24) + + // expand into multiple buffers + verifyWriteAndRead(buffer, 11, 0, 0, 28) + buffer.capacity should be (40) + } + + + // Used to make sure we're writing different bytes each time + var rangeStart = 0 + + /** + * @param buffer The buffer to write to and read from. + * @param offsetInBuffer The offset to write to in the buffer. + * @param offsetInSource The offset in the array that the bytes are written from. + * @param offsetInTarget The offset in the array to read the bytes into. + * @param length The number of bytes to read and write + */ + def verifyWriteAndRead( + buffer: ChainedBuffer, + offsetInBuffer: Int, + offsetInSource: Int, + offsetInTarget: Int, + length: Int): Unit = { + val source = new Array[Byte](offsetInSource + length) + (rangeStart until rangeStart + length).map(_.toByte).copyToArray(source, offsetInSource) + buffer.write(offsetInBuffer, source, offsetInSource, length) + val target = new Array[Byte](offsetInTarget + length) + buffer.read(offsetInBuffer, target, offsetInTarget, length) + ByteBuffer.wrap(source, offsetInSource, length) should be + (ByteBuffer.wrap(target, offsetInTarget, length)) + + rangeStart += 100 + } +} \ No newline at end of file From a139e97fe1aeac279b9c47119745c0f45eb7d8c5 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 25 Feb 2015 13:27:13 -0600 Subject: [PATCH 04/97] compiling but all sorts of bad casting etc. --- .../spark/broadcast/TorrentBroadcast.scala | 22 +-- .../org/apache/spark/executor/Executor.scala | 5 +- .../org/apache/spark/io/LargeByteBuffer.scala | 128 ++++++++++++++++-- .../spark/scheduler/TaskResultGetter.scala | 5 +- .../shuffle/FileShuffleBlockManager.scala | 7 +- .../shuffle/IndexShuffleBlockManager.scala | 5 +- .../spark/shuffle/ShuffleBlockManager.scala | 4 +- .../apache/spark/storage/BlockManager.scala | 16 +-- .../org/apache/spark/storage/DiskStore.scala | 12 +- .../apache/spark/storage/MemoryStore.scala | 11 +- .../apache/spark/storage/TachyonStore.scala | 8 +- .../util/LargeByteBufferOutputStream.scala | 2 +- .../org/apache/spark/DistributedSuite.scala | 3 +- .../spark/storage/BlockManagerSuite.scala | 10 +- .../util/collection/ChainedBufferTest.scala | 6 +- 15 files changed, 177 insertions(+), 67 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 94142d33369c7..fac7c01e558ff 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala @@ -25,10 +25,10 @@ import scala.reflect.ClassTag import scala.util.Random import org.apache.spark.{Logging, SparkConf, SparkEnv, SparkException} -import org.apache.spark.io.CompressionCodec +import org.apache.spark.io.{LargeByteBuffer, CompressionCodec} import org.apache.spark.serializer.Serializer import org.apache.spark.storage.{BroadcastBlockId, StorageLevel} -import org.apache.spark.util.{ByteBufferInputStream, Utils} +import org.apache.spark.util.{LargeByteBufferInputStream, ByteBufferInputStream, Utils} import org.apache.spark.util.io.ByteArrayChunkOutputStream /** @@ -110,10 +110,10 @@ private[spark] class TorrentBroadcast[T: ClassTag](obj: T, id: Long) } /** Fetch torrent blocks from the driver and/or other executors. */ - private def readBlocks(): Array[ByteBuffer] = { + private def readBlocks(): Array[LargeByteBuffer] = { // Fetch chunks of data. Note that all these chunks are stored in the BlockManager and reported // to the driver, so other executors can pull these chunks from this executor as well. - val blocks = new Array[ByteBuffer](numBlocks) + val blocks = new Array[LargeByteBuffer](numBlocks) val bm = SparkEnv.get.blockManager for (pid <- Random.shuffle(Seq.range(0, numBlocks))) { @@ -122,8 +122,8 @@ private[spark] class TorrentBroadcast[T: ClassTag](obj: T, id: Long) // First try getLocalBytes because there is a chance that previous attempts to fetch the // broadcast blocks have already fetched some of the blocks. In that case, some blocks // would be available locally (on this executor). - def getLocal: Option[ByteBuffer] = bm.getLocalBytes(pieceId) - def getRemote: Option[ByteBuffer] = bm.getRemoteBytes(pieceId).map { block => + def getLocal: Option[LargeByteBuffer] = bm.getLocalBytes(pieceId) + def getRemote: Option[LargeByteBuffer] = bm.getRemoteBytes(pieceId).map { block => // If we found the block from remote executors/driver's BlockManager, put the block // in this executor's BlockManager. SparkEnv.get.blockManager.putBytes( @@ -133,7 +133,7 @@ private[spark] class TorrentBroadcast[T: ClassTag](obj: T, id: Long) tellMaster = true) block } - val block: ByteBuffer = getLocal.orElse(getRemote).getOrElse( + val block: LargeByteBuffer = getLocal.orElse(getRemote).getOrElse( throw new SparkException(s"Failed to get $pieceId of $broadcastId")) blocks(pid) = block } @@ -194,22 +194,22 @@ private object TorrentBroadcast extends Logging { obj: T, blockSize: Int, serializer: Serializer, - compressionCodec: Option[CompressionCodec]): Array[ByteBuffer] = { + compressionCodec: Option[CompressionCodec]): Array[LargeByteBuffer] = { val bos = new ByteArrayChunkOutputStream(blockSize) val out: OutputStream = compressionCodec.map(c => c.compressedOutputStream(bos)).getOrElse(bos) val ser = serializer.newInstance() val serOut = ser.serializeStream(out) serOut.writeObject[T](obj).close() - bos.toArrays.map(ByteBuffer.wrap) + bos.toArrays.map(LargeByteBuffer.asLargeByteBuffer) } def unBlockifyObject[T: ClassTag]( - blocks: Array[ByteBuffer], + blocks: Array[LargeByteBuffer], serializer: Serializer, compressionCodec: Option[CompressionCodec]): T = { require(blocks.nonEmpty, "Cannot unblockify an empty array of blocks") val is = new SequenceInputStream( - asJavaEnumeration(blocks.iterator.map(block => new ByteBufferInputStream(block)))) + asJavaEnumeration(blocks.iterator.map(block => new LargeByteBufferInputStream(block)))) val in: InputStream = compressionCodec.map(c => c.compressedInputStream(is)).getOrElse(is) val ser = serializer.newInstance() val serIn = ser.deserializeStream(in) diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index b684fb704956b..65c74db8ebf3a 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -23,6 +23,8 @@ import java.net.URL import java.nio.ByteBuffer import java.util.concurrent._ +import org.apache.spark.io.LargeByteBuffer + import scala.collection.JavaConversions._ import scala.collection.mutable.{ArrayBuffer, HashMap} import scala.util.control.NonFatal @@ -217,6 +219,7 @@ private[spark] class Executor( val accumUpdates = Accumulators.values val directResult = new DirectTaskResult(valueBytes, accumUpdates, task.metrics.orNull) + //TODO should we allow task results over 2gb? val serializedDirectResult = ser.serialize(directResult) val resultSize = serializedDirectResult.limit @@ -230,7 +233,7 @@ private[spark] class Executor( } else if (resultSize >= akkaFrameSize - AkkaUtils.reservedSizeBytes) { val blockId = TaskResultBlockId(taskId) env.blockManager.putBytes( - blockId, serializedDirectResult, StorageLevel.MEMORY_AND_DISK_SER) + blockId, LargeByteBuffer.asLargeByteBuffer(serializedDirectResult), StorageLevel.MEMORY_AND_DISK_SER) logInfo( s"Finished $taskName (TID $taskId). $resultSize bytes result sent via BlockManager)") ser.serialize(new IndirectTaskResult[Any](blockId, resultSize)) diff --git a/core/src/main/scala/org/apache/spark/io/LargeByteBuffer.scala b/core/src/main/scala/org/apache/spark/io/LargeByteBuffer.scala index 01bd433f55c78..4bb5b5a101149 100644 --- a/core/src/main/scala/org/apache/spark/io/LargeByteBuffer.scala +++ b/core/src/main/scala/org/apache/spark/io/LargeByteBuffer.scala @@ -18,15 +18,14 @@ package org.apache.spark.io import java.io.{RandomAccessFile, DataInput, InputStream, OutputStream} +import java.nio.channels.FileChannel.MapMode import java.nio.{ByteBuffer, BufferUnderflowException, BufferOverflowException} -import java.nio.channels.{WritableByteChannel, ReadableByteChannel} +import java.nio.channels.{FileChannel, WritableByteChannel, ReadableByteChannel} import org.apache.spark.util.collection.ChainedBuffer import scala.collection.mutable.{ArrayBuffer, HashSet} -import org.apache.spark.Logging -import org.apache.spark.storage.{FileSegment, BlockManager} @@ -67,6 +66,10 @@ trait LargeByteBuffer { */ def limit(): Long + + //an alternative to having this method would be having a foreachBuffer(f: Buffer => T) + def writeTo(channel: WritableByteChannel): Long + // // def skip(skipBy: Long): Unit // @@ -159,52 +162,147 @@ class ChainedLargeByteBuffer(private[io] val underlying: ChainedBuffer) extends def limit(newLimit: Long): Unit = { ??? } + + def writeTo(channel:WritableByteChannel): Long = { + var written = 0l + underlying.chunks.foreach{bytes => + //TODO test this + val buffer = ByteBuffer.wrap(bytes) + while (buffer.hasRemaining) + channel.write(buffer) + written += bytes.length + } + written + } } -class WrappedLargeByteBuffer(private val underlying: ByteBuffer) extends LargeByteBuffer { - def capacity = underlying.capacity +class WrappedLargeByteBuffer(private[spark] val underlying: Array[ByteBuffer]) extends LargeByteBuffer { + + val (totalCapacity, chunkOffsets) = { + var sum = 0l + val offsets = new Array[Long](underlying.size) + (0 until underlying.size).foreach{idx => + offsets(idx) = sum + sum += underlying(idx).capacity() + } + (sum, offsets) + } + + private var _pos = 0l + private var currentBufferIdx = 0 + private var currentBuffer = underlying(0) + private var _limit = totalCapacity + + def capacity = totalCapacity def get(dst: Array[Byte], offset: Int, length: Int): Unit = { - underlying.get(dst, offset, length) + var moved = 0 + while (moved < length) { + val toRead = math.min(length - moved, currentBuffer.remaining()) + currentBuffer.get(dst, offset, toRead) + moved += toRead + updateCurrentBuffer() + } } def get(): Byte = { - underlying.get() + val r = currentBuffer.get() + _pos += 1 + updateCurrentBuffer() + r + } + + private def updateCurrentBuffer(): Unit = { + //TODO fix end condition + while(!currentBuffer.hasRemaining()) { + currentBufferIdx += 1 + currentBuffer = underlying(currentBufferIdx) + } + } + + def put(bytes: LargeByteBuffer): Unit = { + ??? } - def position: Long = underlying.position + def position: Long = _pos def position(position: Long): Unit = { //XXX check range? - underlying.position(position.toInt) + _pos = position } def remaining(): Long = { - underlying.remaining() + totalCapacity - _pos } def duplicate(): WrappedLargeByteBuffer = { - new WrappedLargeByteBuffer(underlying.duplicate()) + new WrappedLargeByteBuffer(underlying.map{_.duplicate()}) } def rewind(): Unit = { - underlying.duplicate() + _pos = 0 + underlying.foreach{_.rewind()} } def limit(): Long = { - underlying.limit() + totalCapacity } def limit(newLimit: Long) = { - //XXX check range? - underlying.limit(newLimit.toInt) + //XXX check range? set limits in sub buffers? + _limit = newLimit + } + + def writeTo(channel: WritableByteChannel): Long = { + var written = 0l + underlying.foreach{buffer => + //TODO test this + //XXX do we care about respecting the limit here? + written += buffer.remaining() + while (buffer.hasRemaining) + channel.write(buffer) + } + written } } object LargeByteBuffer { + + def asLargeByteBuffer(byteBuffer: ByteBuffer): LargeByteBuffer = { + new WrappedLargeByteBuffer(Array(byteBuffer)) + } + + def asLargeByteBuffer(bytes: Array[Byte]): LargeByteBuffer = { + new WrappedLargeByteBuffer(Array(ByteBuffer.wrap(bytes))) + } + + def allocateOnHeap(size: Long, maxChunk: Int): LargeByteBuffer = { val buffer = ChainedBuffer.withInitialSize(maxChunk, size) new ChainedLargeByteBuffer(buffer) } + + def mapFile( + channel: FileChannel, + mode: MapMode, + offset: Long, + length: Long, + maxChunk: Int = Integer.MAX_VALUE - 1e6.toInt + ): LargeByteBuffer = { + val offsets = new ArrayBuffer[Long]() + var curOffset = offset + val end = offset + length + while (curOffset < end) { + offsets += curOffset + val length = math.min(end - curOffset, maxChunk) + curOffset += length + } + offsets += end + val chunks = new Array[ByteBuffer](offsets.size - 1) + (0 until offsets.size - 1).foreach{idx => + chunks(idx) = channel.map(mode, offsets(idx), offsets(idx + 1) - offsets(idx)) + } + new WrappedLargeByteBuffer(chunks) + } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala index 3938580aeea59..9428273561cd8 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala @@ -20,6 +20,8 @@ package org.apache.spark.scheduler import java.nio.ByteBuffer import java.util.concurrent.RejectedExecutionException +import org.apache.spark.io.WrappedLargeByteBuffer + import scala.language.existentials import scala.util.control.NonFatal @@ -72,8 +74,9 @@ private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedul taskSetManager, tid, TaskState.FINISHED, TaskResultLost) return } + //TODO either change serializer interface, or ... val deserializedResult = serializer.get().deserialize[DirectTaskResult[_]]( - serializedTaskResult.get) + serializedTaskResult.get.asInstanceOf[WrappedLargeByteBuffer].underlying(0)) sparkEnv.blockManager.master.removeBlock(blockId) (deserializedResult, size) } diff --git a/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockManager.scala b/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockManager.scala index 7de2f9cbb2866..016964fc274d8 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockManager.scala @@ -22,6 +22,8 @@ import java.nio.ByteBuffer import java.util.concurrent.ConcurrentLinkedQueue import java.util.concurrent.atomic.AtomicInteger +import org.apache.spark.io.LargeByteBuffer + import scala.collection.JavaConversions._ import org.apache.spark.{Logging, SparkConf, SparkEnv} @@ -171,9 +173,10 @@ class FileShuffleBlockManager(conf: SparkConf) } } - override def getBytes(blockId: ShuffleBlockId): Option[ByteBuffer] = { + override def getBytes(blockId: ShuffleBlockId): Option[LargeByteBuffer] = { + //TODO val segment = getBlockData(blockId) - Some(segment.nioByteBuffer()) + Some(LargeByteBuffer.asLargeByteBuffer(segment.nioByteBuffer())) } override def getBlockData(blockId: ShuffleBlockId): ManagedBuffer = { diff --git a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockManager.scala b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockManager.scala index b292587d37028..76c2e4180c838 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockManager.scala @@ -21,6 +21,7 @@ import java.io._ import java.nio.ByteBuffer import com.google.common.io.ByteStreams +import org.apache.spark.io.LargeByteBuffer import org.apache.spark.{SparkConf, SparkEnv} import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer} @@ -97,8 +98,8 @@ class IndexShuffleBlockManager(conf: SparkConf) extends ShuffleBlockManager { } } - override def getBytes(blockId: ShuffleBlockId): Option[ByteBuffer] = { - Some(getBlockData(blockId).nioByteBuffer()) + override def getBytes(blockId: ShuffleBlockId): Option[LargeByteBuffer] = { + Some(LargeByteBuffer.asLargeByteBuffer(getBlockData(blockId).nioByteBuffer())) } override def getBlockData(blockId: ShuffleBlockId): ManagedBuffer = { diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockManager.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockManager.scala index b521f0c7fc77e..fa737729b8758 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockManager.scala @@ -17,7 +17,7 @@ package org.apache.spark.shuffle -import java.nio.ByteBuffer +import org.apache.spark.io.LargeByteBuffer import org.apache.spark.network.buffer.ManagedBuffer import org.apache.spark.storage.ShuffleBlockId @@ -29,7 +29,7 @@ trait ShuffleBlockManager { * Get shuffle block data managed by the local ShuffleBlockManager. * @return Some(ByteBuffer) if block found, otherwise None. */ - def getBytes(blockId: ShuffleBlockId): Option[ByteBuffer] + def getBytes(blockId: ShuffleBlockId): Option[LargeByteBuffer] def getBlockData(blockId: ShuffleBlockId): ManagedBuffer 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 ad895ff338d54..479fcf35283ad 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -321,7 +321,7 @@ private[spark] class BlockManager( * Put the block locally, using the given storage level. */ override def putBlockData(blockId: BlockId, data: ManagedBuffer, level: StorageLevel): Unit = { - putBytes(blockId, new WrappedLargeByteBuffer(data.nioByteBuffer()), level) + putBytes(blockId, LargeByteBuffer.asLargeByteBuffer(data.nioByteBuffer()), level) } /** @@ -437,7 +437,7 @@ private[spark] class BlockManager( /** * Get block from the local block manager as serialized bytes. */ - def getLocalBytes(blockId: BlockId): Option[ByteBuffer] = { + def getLocalBytes(blockId: BlockId): Option[LargeByteBuffer] = { logDebug(s"Getting local block $blockId as bytes") // As an optimization for map output fetches, if the block is for a shuffle, return it // without acquiring a lock; the disk store never deletes (recent) items so this should work @@ -451,7 +451,7 @@ private[spark] class BlockManager( blockId, s"Block $blockId not found on disk, though it should be") } } else { - doGetLocal(blockId, asBlockResult = false).asInstanceOf[Option[ByteBuffer]] + doGetLocal(blockId, asBlockResult = false).asInstanceOf[Option[LargeByteBuffer]] } } @@ -584,9 +584,9 @@ private[spark] class BlockManager( /** * Get block from remote block managers as serialized bytes. */ - def getRemoteBytes(blockId: BlockId): Option[ByteBuffer] = { + def getRemoteBytes(blockId: BlockId): Option[LargeByteBuffer] = { logDebug(s"Getting remote block $blockId as bytes") - doGetRemote(blockId, asBlockResult = false).asInstanceOf[Option[ByteBuffer]] + doGetRemote(blockId, asBlockResult = false).asInstanceOf[Option[LargeByteBuffer]] } private def doGetRemote(blockId: BlockId, asBlockResult: Boolean): Option[Any] = { @@ -594,8 +594,8 @@ private[spark] class BlockManager( val locations = Random.shuffle(master.getLocations(blockId)) for (loc <- locations) { logDebug(s"Getting remote block $blockId from $loc") - //the fetch will always be one byte buffer till we fix SPARK-5928 - val data: LargeByteBuffer = new WrappedLargeByteBuffer(blockTransferService.fetchBlockSync( + //TODO the fetch will always be one byte buffer till we fix SPARK-5928 + val data: LargeByteBuffer = LargeByteBuffer.asLargeByteBuffer(blockTransferService.fetchBlockSync( loc.host, loc.port, loc.executorId, blockId.toString).nioByteBuffer()) if (data != null) { @@ -1006,7 +1006,7 @@ private[spark] class BlockManager( */ def dropFromMemory( blockId: BlockId, - data: Either[Array[Any], ByteBuffer]): Option[BlockStatus] = { + data: Either[Array[Any], LargeByteBuffer]): Option[BlockStatus] = { logInfo(s"Dropping block $blockId from memory") val info = blockInfo.get(blockId).orNull 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 18293f3314a5f..180b237a1a1f2 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala @@ -22,7 +22,7 @@ import java.nio.ByteBuffer import java.nio.channels.FileChannel.MapMode import org.apache.spark.Logging -import org.apache.spark.io.LargeByteBuffer +import org.apache.spark.io.{WrappedLargeByteBuffer, LargeByteBuffer} import org.apache.spark.serializer.Serializer import org.apache.spark.util.Utils @@ -39,7 +39,7 @@ private[spark] class DiskStore(blockManager: BlockManager, diskManager: DiskBloc diskManager.getFile(blockId.name).length } - override def putBytes(blockId: BlockId, _bytes: ByteBuffer, level: StorageLevel): PutResult = { + override def putBytes(blockId: BlockId, _bytes: LargeByteBuffer, level: StorageLevel): PutResult = { // So that we do not modify the input offsets ! // duplicate does not copy buffer, so inexpensive val bytes = _bytes.duplicate() @@ -47,9 +47,7 @@ private[spark] class DiskStore(blockManager: BlockManager, diskManager: DiskBloc val startTime = System.currentTimeMillis val file = diskManager.getFile(blockId) val channel = new FileOutputStream(file).getChannel - while (bytes.remaining > 0) { - channel.write(bytes) - } + bytes.writeTo(channel) channel.close() val finishTime = System.currentTimeMillis logDebug("Block %s stored as %s file on disk in %d ms".format( @@ -120,9 +118,9 @@ private[spark] class DiskStore(blockManager: BlockManager, diskManager: DiskBloc } } buf.flip() - Some(buf) + Some(LargeByteBuffer.asLargeByteBuffer(buf)) } else { - Some(channel.map(MapMode.READ_ONLY, offset, length)) + Some(LargeByteBuffer.mapFile(channel, MapMode.READ_ONLY, offset, length)) } } finally { channel.close() 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 90ced59104432..8fccc0f3e78d2 100644 --- a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala @@ -17,7 +17,6 @@ package org.apache.spark.storage -import java.nio.ByteBuffer import java.util.LinkedHashMap import org.apache.spark.io.LargeByteBuffer @@ -154,7 +153,7 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) } } - override def getBytes(blockId: BlockId): Option[ByteBuffer] = { + override def getBytes(blockId: BlockId): Option[LargeByteBuffer] = { val entry = entries.synchronized { entries.get(blockId) } @@ -163,7 +162,7 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) } else if (entry.deserialized) { Some(blockManager.dataSerialize(blockId, entry.value.asInstanceOf[Array[Any]].iterator)) } else { - Some(entry.value.asInstanceOf[ByteBuffer].duplicate()) // Doesn't actually copy the data + Some(entry.value.asInstanceOf[LargeByteBuffer].duplicate()) // Doesn't actually copy the data } } @@ -176,7 +175,7 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) } else if (entry.deserialized) { Some(entry.value.asInstanceOf[Array[Any]].iterator) } else { - val buffer = entry.value.asInstanceOf[ByteBuffer].duplicate() // Doesn't actually copy data + val buffer = entry.value.asInstanceOf[LargeByteBuffer].duplicate() // Doesn't actually copy data Some(blockManager.dataDeserialize(blockId, buffer)) } } @@ -350,7 +349,7 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) val data = if (deserialized) { Left(value.asInstanceOf[Array[Any]]) } else { - Right(value.asInstanceOf[ByteBuffer].duplicate()) + Right(value.asInstanceOf[LargeByteBuffer].duplicate()) } val droppedBlockStatus = blockManager.dropFromMemory(blockId, data) droppedBlockStatus.foreach { status => droppedBlocks += ((blockId, status)) } @@ -416,7 +415,7 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) val data = if (entry.deserialized) { Left(entry.value.asInstanceOf[Array[Any]]) } else { - Right(entry.value.asInstanceOf[ByteBuffer].duplicate()) + Right(entry.value.asInstanceOf[LargeByteBuffer].duplicate()) } val droppedBlockStatus = blockManager.dropFromMemory(blockId, data) droppedBlockStatus.foreach { status => droppedBlocks += ((blockId, status)) } diff --git a/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala b/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala index fd7cb49ef9d50..9f964ed456d5e 100644 --- a/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala @@ -41,7 +41,7 @@ private[spark] class TachyonStore( tachyonManager.getFile(blockId.name).length } - override def putBytes(blockId: BlockId, bytes: ByteBuffer, level: StorageLevel): PutResult = { + override def putBytes(blockId: BlockId, bytes: LargeByteBuffer, level: StorageLevel): PutResult = { putIntoTachyonStore(blockId, bytes, returnValues = true) } @@ -75,7 +75,8 @@ private[spark] class TachyonStore( val startTime = System.currentTimeMillis val file = tachyonManager.getFile(blockId) val os = file.getOutStream(WriteType.TRY_CACHE) - os.write(byteBuffer.array()) + // TODO need a better fix here for tachyon +// os.write(byteBuffer.array()) os.close() val finishTime = System.currentTimeMillis logDebug("Block %s stored as %s file in Tachyon in %d ms".format( @@ -110,9 +111,10 @@ private[spark] class TachyonStore( assert (is != null) try { val size = file.length + //TODO val bs = new Array[Byte](size.asInstanceOf[Int]) ByteStreams.readFully(is, bs) - Some(ByteBuffer.wrap(bs)) + Some(LargeByteBuffer.asLargeByteBuffer(ByteBuffer.wrap(bs))) } catch { case ioe: IOException => logWarning(s"Failed to fetch the block $blockId from Tachyon", ioe) diff --git a/core/src/main/scala/org/apache/spark/util/LargeByteBufferOutputStream.scala b/core/src/main/scala/org/apache/spark/util/LargeByteBufferOutputStream.scala index 246ebca999437..2fe904b10c53c 100644 --- a/core/src/main/scala/org/apache/spark/util/LargeByteBufferOutputStream.scala +++ b/core/src/main/scala/org/apache/spark/util/LargeByteBufferOutputStream.scala @@ -26,7 +26,7 @@ private[spark] class LargeByteBufferOutputStream(chunkSize: Int = 65536) extends OutputStream { - val buffer = new ChainedBuffer(chunkSize) + val buffer = ChainedBuffer.withInitialSize(chunkSize) private var _pos = 0 diff --git a/core/src/test/scala/org/apache/spark/DistributedSuite.scala b/core/src/test/scala/org/apache/spark/DistributedSuite.scala index 97ea3578aa8ba..b183373de5019 100644 --- a/core/src/test/scala/org/apache/spark/DistributedSuite.scala +++ b/core/src/test/scala/org/apache/spark/DistributedSuite.scala @@ -17,6 +17,7 @@ package org.apache.spark +import org.apache.spark.io.LargeByteBuffer import org.scalatest.FunSuite import org.scalatest.concurrent.Timeouts._ import org.scalatest.Matchers @@ -195,7 +196,7 @@ class DistributedSuite extends FunSuite with Matchers with LocalSparkContext { blockManager.master.getLocations(blockId).foreach { cmId => val bytes = blockTransfer.fetchBlockSync(cmId.host, cmId.port, cmId.executorId, blockId.toString) - val deserialized = blockManager.dataDeserialize(blockId, bytes.nioByteBuffer()) + val deserialized = blockManager.dataDeserialize(blockId, LargeByteBuffer.asLargeByteBuffer(bytes.nioByteBuffer())) .asInstanceOf[Iterator[Int]].toList assert(deserialized === (1 to 100).toList) } 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 ffe6f039145ea..f692c0ffe967f 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -21,6 +21,8 @@ import java.nio.{ByteBuffer, MappedByteBuffer} import java.util.Arrays import java.util.concurrent.TimeUnit +import org.apache.spark.io.LargeByteBuffer + import scala.collection.mutable.ArrayBuffer import scala.concurrent.Await import scala.concurrent.duration._ @@ -809,7 +811,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfterEach var counter = 0.toByte def incr = {counter = (counter + 1).toByte; counter;} val bytes = Array.fill[Byte](1000)(incr) - val byteBuffer = ByteBuffer.wrap(bytes) + val byteBuffer = LargeByteBuffer.asLargeByteBuffer(bytes) val blockId = BlockId("rdd_1_2") @@ -834,9 +836,9 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfterEach "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) + def arrayFromByteBuffer(in: LargeByteBuffer): Array[Byte] = { + val array = new Array[Byte](in.remaining().toInt) + in.get(array, 0, in.remaining().toInt) array } diff --git a/core/src/test/scala/org/apache/spark/util/collection/ChainedBufferTest.scala b/core/src/test/scala/org/apache/spark/util/collection/ChainedBufferTest.scala index e99d5ecc639c9..cc96e24e3dc03 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/ChainedBufferTest.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/ChainedBufferTest.scala @@ -25,7 +25,7 @@ import org.scalatest.Matchers._ class ChainedBufferSuite extends FunSuite { test("write and read at start") { // write from start of source array - val buffer = new ChainedBuffer(8) + val buffer = ChainedBuffer.withInitialSize(8) buffer.capacity should be (0) verifyWriteAndRead(buffer, 0, 0, 0, 4) buffer.capacity should be (8) @@ -53,7 +53,7 @@ class ChainedBufferSuite extends FunSuite { test("write and read at middle") { // write from start of source array - val buffer = new ChainedBuffer(8) + val buffer = ChainedBuffer.withInitialSize(8) verifyWriteAndRead(buffer, 3, 0, 0, 4) buffer.capacity should be (8) @@ -80,7 +80,7 @@ class ChainedBufferSuite extends FunSuite { test("write and read at later buffer") { // write from start of source array - val buffer = new ChainedBuffer(8) + val buffer = ChainedBuffer.withInitialSize(8) verifyWriteAndRead(buffer, 11, 0, 0, 4) buffer.capacity should be (16) From 4965bad00574a05e133e7caeba56cd6115fe35b6 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 25 Feb 2015 14:28:16 -0600 Subject: [PATCH 05/97] move LargeByteBuffer to network-common, since we need it there for the shuffles --- .../buffer/FileSegmentManagedBuffer.java | 6 +- .../spark/network/buffer/LargeByteBuffer.java | 56 ++++++++ .../network/buffer/LargeByteBufferHelper.java | 65 +++++++++ .../spark/network/buffer/ManagedBuffer.java | 3 +- .../network/buffer/NettyManagedBuffer.java | 4 +- .../network/buffer/NioManagedBuffer.java | 4 +- .../buffer/WrappedLargeByteBuffer.java | 135 ++++++++++++++++++ 7 files changed, 264 insertions(+), 9 deletions(-) create mode 100644 network/common/src/main/java/org/apache/spark/network/buffer/LargeByteBuffer.java create mode 100644 network/common/src/main/java/org/apache/spark/network/buffer/LargeByteBufferHelper.java create mode 100644 network/common/src/main/java/org/apache/spark/network/buffer/WrappedLargeByteBuffer.java diff --git a/network/common/src/main/java/org/apache/spark/network/buffer/FileSegmentManagedBuffer.java b/network/common/src/main/java/org/apache/spark/network/buffer/FileSegmentManagedBuffer.java index 844eff4f4c701..75c426f379238 100644 --- a/network/common/src/main/java/org/apache/spark/network/buffer/FileSegmentManagedBuffer.java +++ b/network/common/src/main/java/org/apache/spark/network/buffer/FileSegmentManagedBuffer.java @@ -55,7 +55,7 @@ public long size() { } @Override - public ByteBuffer nioByteBuffer() throws IOException { + public LargeByteBuffer nioByteBuffer() throws IOException { FileChannel channel = null; try { channel = new RandomAccessFile(file, "r").getChannel(); @@ -71,9 +71,9 @@ public ByteBuffer nioByteBuffer() throws IOException { } } buf.flip(); - return buf; + return LargeByteBufferHelper.asLargeByteBuffer(buf); } else { - return channel.map(FileChannel.MapMode.READ_ONLY, offset, length); + return LargeByteBufferHelper.mapFile(channel, FileChannel.MapMode.READ_ONLY, offset, length); } } catch (IOException e) { try { diff --git a/network/common/src/main/java/org/apache/spark/network/buffer/LargeByteBuffer.java b/network/common/src/main/java/org/apache/spark/network/buffer/LargeByteBuffer.java new file mode 100644 index 0000000000000..c56fdebb887a1 --- /dev/null +++ b/network/common/src/main/java/org/apache/spark/network/buffer/LargeByteBuffer.java @@ -0,0 +1,56 @@ +/* +* 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.network.buffer; + +import java.io.IOException; +import java.nio.channels.WritableByteChannel; + +public interface LargeByteBuffer { + public long capacity(); + + public byte get(); + + public void get(byte[] dst,int offset, int length); + + public void position(long position); + + public long position(); + + /** doesn't copy data, just copies references & offsets */ + public LargeByteBuffer duplicate(); + + public void put(LargeByteBuffer bytes); + + //TODO checks on limit semantics + + /** + * Sets this buffer's limit. If the position is larger than the new limit then it is set to the + * new limit. If the mark is defined and larger than the new limit then it is discarded. + */ + public void limit(long newLimit); + + /** + * return this buffer's limit + * @return + */ + public long limit(); + + //an alternative to having this method would be having a foreachBuffer(f: Buffer => T) + public long writeTo(WritableByteChannel channel) throws IOException; + + +} diff --git a/network/common/src/main/java/org/apache/spark/network/buffer/LargeByteBufferHelper.java b/network/common/src/main/java/org/apache/spark/network/buffer/LargeByteBufferHelper.java new file mode 100644 index 0000000000000..2585b65b1f969 --- /dev/null +++ b/network/common/src/main/java/org/apache/spark/network/buffer/LargeByteBufferHelper.java @@ -0,0 +1,65 @@ +/* + * 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.network.buffer; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.channels.FileChannel; +import java.util.ArrayList; + +public class LargeByteBufferHelper { + + static final int DEFAULT_MAX_CHUNK = Integer.MAX_VALUE - 1000000; + + public static LargeByteBuffer asLargeByteBuffer(ByteBuffer buffer) { + return new WrappedLargeByteBuffer(new ByteBuffer[]{buffer}); + } + + public static LargeByteBuffer mapFile( + FileChannel channel, + FileChannel.MapMode mode, + long offset, + long length + ) throws IOException { + return mapFile(channel, mode, offset, length, DEFAULT_MAX_CHUNK); + } + + public static LargeByteBuffer mapFile( + FileChannel channel, + FileChannel.MapMode mode, + long offset, + long length, + int maxChunk + ) throws IOException { + ArrayList offsets = new ArrayList(); + long curOffset = offset; + long end = offset + length; + while (curOffset < end) { + offsets.add(curOffset); + int chunkLength = Math.min((int) (end - curOffset), maxChunk); + curOffset += chunkLength; + } + offsets.add(end); + ByteBuffer[] chunks = new ByteBuffer[offsets.size() - 1]; + for (int i = 0; i< offsets.size() - 1; i++) { + chunks[i] = channel.map(mode, offsets.get(i), offsets.get(i+ 1) - offsets.get(i)); + } + return new WrappedLargeByteBuffer(chunks); + } + + +} diff --git a/network/common/src/main/java/org/apache/spark/network/buffer/ManagedBuffer.java b/network/common/src/main/java/org/apache/spark/network/buffer/ManagedBuffer.java index a415db593a788..8ebf72feef3ed 100644 --- a/network/common/src/main/java/org/apache/spark/network/buffer/ManagedBuffer.java +++ b/network/common/src/main/java/org/apache/spark/network/buffer/ManagedBuffer.java @@ -19,7 +19,6 @@ import java.io.IOException; import java.io.InputStream; -import java.nio.ByteBuffer; /** * This interface provides an immutable view for data in the form of bytes. The implementation @@ -44,7 +43,7 @@ public abstract class ManagedBuffer { * returned ByteBuffer should not affect the content of this buffer. */ // TODO: Deprecate this, usage may require expensive memory mapping or allocation. - public abstract ByteBuffer nioByteBuffer() throws IOException; + public abstract LargeByteBuffer nioByteBuffer() throws IOException; /** * Exposes this buffer's data as an InputStream. The underlying implementation does not diff --git a/network/common/src/main/java/org/apache/spark/network/buffer/NettyManagedBuffer.java b/network/common/src/main/java/org/apache/spark/network/buffer/NettyManagedBuffer.java index c806bfa45bef3..aa0fc74f00fd7 100644 --- a/network/common/src/main/java/org/apache/spark/network/buffer/NettyManagedBuffer.java +++ b/network/common/src/main/java/org/apache/spark/network/buffer/NettyManagedBuffer.java @@ -41,8 +41,8 @@ public long size() { } @Override - public ByteBuffer nioByteBuffer() throws IOException { - return buf.nioBuffer(); + public LargeByteBuffer nioByteBuffer() throws IOException { + return LargeByteBufferHelper.asLargeByteBuffer(buf.nioBuffer()); } @Override diff --git a/network/common/src/main/java/org/apache/spark/network/buffer/NioManagedBuffer.java b/network/common/src/main/java/org/apache/spark/network/buffer/NioManagedBuffer.java index f55b884bc45ce..d4746a76e7d13 100644 --- a/network/common/src/main/java/org/apache/spark/network/buffer/NioManagedBuffer.java +++ b/network/common/src/main/java/org/apache/spark/network/buffer/NioManagedBuffer.java @@ -41,8 +41,8 @@ public long size() { } @Override - public ByteBuffer nioByteBuffer() throws IOException { - return buf.duplicate(); + public LargeByteBuffer nioByteBuffer() throws IOException { + return LargeByteBufferHelper.asLargeByteBuffer(buf.duplicate()); } @Override diff --git a/network/common/src/main/java/org/apache/spark/network/buffer/WrappedLargeByteBuffer.java b/network/common/src/main/java/org/apache/spark/network/buffer/WrappedLargeByteBuffer.java new file mode 100644 index 0000000000000..bbe73a87e9105 --- /dev/null +++ b/network/common/src/main/java/org/apache/spark/network/buffer/WrappedLargeByteBuffer.java @@ -0,0 +1,135 @@ +/* +* 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.network.buffer; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.channels.WritableByteChannel; + +public class WrappedLargeByteBuffer implements LargeByteBuffer { + + private final ByteBuffer[] underlying; + private final Long totalCapacity; + private final long[] chunkOffsets; + + private long _pos; + private int currentBufferIdx; + private ByteBuffer currentBuffer; + private long limit; + + + public WrappedLargeByteBuffer(ByteBuffer[] underlying) { + this.underlying = underlying; + long sum = 0l; + chunkOffsets = new long[underlying.length]; + for (int i = 0; i < underlying.length; i++) { + chunkOffsets[i] = sum; + sum += underlying[i].capacity(); + } + totalCapacity = sum; + _pos = 0l; + currentBufferIdx = 0; + currentBuffer = underlying[0]; + limit = totalCapacity; + } + + @Override + public long capacity() {return totalCapacity;} + + @Override + public void get(byte[] dest, int offset, int length){ + int moved = 0; + while (moved < length) { + int toRead = Math.min(length - moved, currentBuffer.remaining()); + currentBuffer.get(dest, offset, toRead); + moved += toRead; + updateCurrentBuffer(); + } + } + + @Override + public byte get() { + byte r = currentBuffer.get(); + _pos += 1; + updateCurrentBuffer(); + return r; + } + + private void updateCurrentBuffer() { + //TODO fix end condition + while(!currentBuffer.hasRemaining()) { + currentBufferIdx += 1; + currentBuffer = underlying[currentBufferIdx]; + } + } + + @Override + public void put(LargeByteBuffer bytes) { + throw new RuntimeException("not yet implemented"); + } + + @Override + public long position() { return _pos;} + + @Override + public void position(long newPosition) { + //XXX check range? + _pos = newPosition; + } + + @Override + public WrappedLargeByteBuffer duplicate() { + ByteBuffer[] duplicates = new ByteBuffer[underlying.length]; + for (int i = 0; i < underlying.length; i++) { + duplicates[i] = underlying[i].duplicate(); + } + //we could also avoid initializing offsets here, if we cared ... + return new WrappedLargeByteBuffer(duplicates); + } + +// @Override +// public void rewind() { +// _pos = 0; +// for (ByteBuffer buf: underlying) { +// buf.rewind(); +// } +// } + + @Override + public long limit() { + return limit; + } + + @Override + public void limit(long newLimit) { + //XXX check range? set limits in sub buffers? + limit = newLimit; + } + + @Override + public long writeTo(WritableByteChannel channel) throws IOException { + long written = 0l; + for(ByteBuffer buffer: underlying) { + //TODO test this + //XXX do we care about respecting the limit here? + written += buffer.remaining(); + while (buffer.hasRemaining()) + channel.write(buffer); + } + return written; + } +} From 149d4fa3fa55403df90109b440a3523d3f4ab92b Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 25 Feb 2015 15:50:33 -0600 Subject: [PATCH 06/97] move large byte buffer to network/common ... still lots of crud --- .../spark/broadcast/TorrentBroadcast.scala | 9 +- .../org/apache/spark/executor/Executor.scala | 4 +- .../spark/io/ChainedLargeByteBuffer.scala | 85 + .../org/apache/spark/io/LargeByteBuffer.scala | 3304 ++++++++--------- .../spark/network/BlockTransferService.scala | 7 +- .../network/netty/NettyBlockRpcServer.scala | 4 +- .../netty/NettyBlockTransferService.scala | 11 +- .../network/nio/NioBlockTransferService.scala | 12 +- .../spark/scheduler/TaskResultGetter.scala | 4 +- .../shuffle/FileShuffleBlockManager.scala | 7 +- .../shuffle/IndexShuffleBlockManager.scala | 6 +- .../spark/shuffle/ShuffleBlockManager.scala | 3 +- .../apache/spark/storage/BlockManager.scala | 24 +- .../org/apache/spark/storage/BlockStore.scala | 6 +- .../org/apache/spark/storage/DiskStore.scala | 6 +- .../apache/spark/storage/MemoryStore.scala | 4 +- .../org/apache/spark/storage/PutResult.scala | 2 +- .../apache/spark/storage/TachyonStore.scala | 6 +- .../util/LargeByteBufferInputStream.scala | 3 +- .../util/LargeByteBufferOutputStream.scala | 3 +- .../NettyBlockTransferSecuritySuite.scala | 4 +- .../netty/NettyBlockTransferSuite.scala | 80 + .../buffer/FileSegmentManagedBuffer.java | 34 +- .../spark/network/buffer/LargeByteBuffer.java | 6 + .../network/buffer/LargeByteBufferHelper.java | 17 + .../network/buffer/NioManagedBuffer.java | 14 +- .../buffer/WrappedLargeByteBuffer.java | 12 +- .../network/buffer/LargeByteBuffer.scala | 1760 +++++++++ 28 files changed, 3694 insertions(+), 1743 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/io/ChainedLargeByteBuffer.scala create mode 100644 core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSuite.scala create mode 100644 network/common/src/main/scala/org/apache/spark/network/buffer/LargeByteBuffer.scala 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 fac7c01e558ff..2b166439485b0 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala @@ -18,17 +18,18 @@ package org.apache.spark.broadcast import java.io._ -import java.nio.ByteBuffer + +import org.apache.spark.network.buffer.{LargeByteBufferHelper, LargeByteBuffer} import scala.collection.JavaConversions.asJavaEnumeration import scala.reflect.ClassTag import scala.util.Random import org.apache.spark.{Logging, SparkConf, SparkEnv, SparkException} -import org.apache.spark.io.{LargeByteBuffer, CompressionCodec} +import org.apache.spark.io.CompressionCodec import org.apache.spark.serializer.Serializer import org.apache.spark.storage.{BroadcastBlockId, StorageLevel} -import org.apache.spark.util.{LargeByteBufferInputStream, ByteBufferInputStream, Utils} +import org.apache.spark.util.{LargeByteBufferInputStream, Utils} import org.apache.spark.util.io.ByteArrayChunkOutputStream /** @@ -200,7 +201,7 @@ private object TorrentBroadcast extends Logging { val ser = serializer.newInstance() val serOut = ser.serializeStream(out) serOut.writeObject[T](obj).close() - bos.toArrays.map(LargeByteBuffer.asLargeByteBuffer) + bos.toArrays.map(LargeByteBufferHelper.asLargeByteBuffer) } def unBlockifyObject[T: ClassTag]( diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 65c74db8ebf3a..faec7463d9f5f 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -23,7 +23,7 @@ import java.net.URL import java.nio.ByteBuffer import java.util.concurrent._ -import org.apache.spark.io.LargeByteBuffer +import org.apache.spark.network.buffer.LargeByteBufferHelper import scala.collection.JavaConversions._ import scala.collection.mutable.{ArrayBuffer, HashMap} @@ -233,7 +233,7 @@ private[spark] class Executor( } else if (resultSize >= akkaFrameSize - AkkaUtils.reservedSizeBytes) { val blockId = TaskResultBlockId(taskId) env.blockManager.putBytes( - blockId, LargeByteBuffer.asLargeByteBuffer(serializedDirectResult), StorageLevel.MEMORY_AND_DISK_SER) + blockId, LargeByteBufferHelper.asLargeByteBuffer(serializedDirectResult), StorageLevel.MEMORY_AND_DISK_SER) logInfo( s"Finished $taskName (TID $taskId). $resultSize bytes result sent via BlockManager)") ser.serialize(new IndirectTaskResult[Any](blockId, resultSize)) diff --git a/core/src/main/scala/org/apache/spark/io/ChainedLargeByteBuffer.scala b/core/src/main/scala/org/apache/spark/io/ChainedLargeByteBuffer.scala new file mode 100644 index 0000000000000..eea1114ec35ca --- /dev/null +++ b/core/src/main/scala/org/apache/spark/io/ChainedLargeByteBuffer.scala @@ -0,0 +1,85 @@ +/* + * 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.io + +import java.nio.ByteBuffer +import java.nio.channels.WritableByteChannel + +import org.apache.spark.network.buffer.LargeByteBuffer +import org.apache.spark.util.collection.ChainedBuffer + +class ChainedLargeByteBuffer(private[io] val underlying: ChainedBuffer) extends LargeByteBuffer { + + def capacity = underlying.capacity + + var _pos = 0l + + def get(dst: Array[Byte],offset: Int,length: Int): Unit = { + underlying.read(_pos, dst, offset, length) + _pos += length + } + + def get(): Byte = { + val b = underlying.read(_pos) + _pos += 1 + b + } + + def put(bytes: LargeByteBuffer): Unit = { + ??? + } + + def position: Long = _pos + def position(position: Long): Unit = { + _pos = position + } + def remaining(): Long = { + underlying.size - position + } + + def duplicate(): ChainedLargeByteBuffer = { + new ChainedLargeByteBuffer(underlying) + } + + def rewind(): Unit = { + _pos = 0 + } + + def limit(): Long = { + capacity + } + + def limit(newLimit: Long): Unit = { + ??? + } + + def writeTo(channel:WritableByteChannel): Long = { + var written = 0l + underlying.chunks.foreach{bytes => + //TODO test this + val buffer = ByteBuffer.wrap(bytes) + while (buffer.hasRemaining) + channel.write(buffer) + written += bytes.length + } + written + } + + override def firstByteBuffer(): ByteBuffer = { + ByteBuffer.wrap(underlying.chunks(0)) + } +} diff --git a/core/src/main/scala/org/apache/spark/io/LargeByteBuffer.scala b/core/src/main/scala/org/apache/spark/io/LargeByteBuffer.scala index 4bb5b5a101149..56cdd9e6938b3 100644 --- a/core/src/main/scala/org/apache/spark/io/LargeByteBuffer.scala +++ b/core/src/main/scala/org/apache/spark/io/LargeByteBuffer.scala @@ -1,554 +1,103 @@ -/* - * 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.io - -import java.io.{RandomAccessFile, DataInput, InputStream, OutputStream} -import java.nio.channels.FileChannel.MapMode -import java.nio.{ByteBuffer, BufferUnderflowException, BufferOverflowException} -import java.nio.channels.{FileChannel, WritableByteChannel, ReadableByteChannel} - -import org.apache.spark.util.collection.ChainedBuffer - -import scala.collection.mutable.{ArrayBuffer, HashSet} - - - - -trait LargeByteBuffer { -// def position(): Long -// -// def limit(): Long - - def capacity(): Long - - def get(): Byte //needed for ByteBufferInputStream - - def get(dst: Array[Byte], offset: Int, length: Int): Unit // for ByteBufferInputStream - - def position(position: Long): Unit //for ByteBufferInputStream - - def position(): Long //for ByteBufferInputStream - - /** doesn't copy data, just copies references & offsets */ - def duplicate(): LargeByteBuffer - - def put(bytes: LargeByteBuffer): Unit - - //also need whatever is necessary for ByteArrayOutputStream for BlockManager#dataSerialize - - - //TODO checks on limit semantics - - /** - * Sets this buffer's limit. If the position is larger than the new limit then it is set to the - * new limit. If the mark is defined and larger than the new limit then it is discarded. - */ - def limit(newLimit: Long): Unit - - /** - * return this buffer's limit - * @return - */ - def limit(): Long - - - //an alternative to having this method would be having a foreachBuffer(f: Buffer => T) - def writeTo(channel: WritableByteChannel): Long - -// -// def skip(skipBy: Long): Unit -// -// def position(newPosition: Long): Unit -// -// /** -// * Clears this buffer. The position is set to zero, the limit is set to -// * the capacity, and the mark is discarded. -// * -// *

Invoke this method before using a sequence of channel-read or -// * put operations to fill this buffer. -// * -// *

This method does not actually erase the data in the buffer, but it -// * is named as if it did because it will most often be used in situations -// * in which that might as well be the case.

-// */ -// def clear(): Unit -// -// /** -// * Flips this buffer. The limit is set to the current position and then -// * the position is set to zero. If the mark is defined then it is -// * discarded. -// * -// *

After a sequence of channel-read or put operations, invoke -// * this method to prepare for a sequence of channel-write or relative -// * get operations. -// */ -// def flip(): Unit - - /** - * Rewinds this buffer. The position is set to zero and the mark is - * discarded. - * - *

Invoke this method before a sequence of channel-write or get - * operations, assuming that the limit has already been set - * appropriately. - */ - def rewind(): Unit - - /** - * Returns the number of elements between the current position and the - * limit.

- * - * @return The number of elements remaining in this buffer - */ - def remaining(): Long -} - -class ChainedLargeByteBuffer(private[io] val underlying: ChainedBuffer) extends LargeByteBuffer { - - def capacity = underlying.capacity - - var _pos = 0l - - def get(dst: Array[Byte],offset: Int,length: Int): Unit = { - underlying.read(_pos, dst, offset, length) - _pos += length - } - - def get(): Byte = { - val b = underlying.read(_pos) - _pos += 1 - b - } - - def put(bytes: LargeByteBuffer): Unit = { - ??? - } - - def position: Long = _pos - def position(position: Long): Unit = { - _pos = position - } - def remaining(): Long = { - underlying.size - position - } - - def duplicate(): ChainedLargeByteBuffer = { - new ChainedLargeByteBuffer(underlying) - } - - def rewind(): Unit = { - _pos = 0 - } - - def limit(): Long = { - capacity - } - - def limit(newLimit: Long): Unit = { - ??? - } - - def writeTo(channel:WritableByteChannel): Long = { - var written = 0l - underlying.chunks.foreach{bytes => - //TODO test this - val buffer = ByteBuffer.wrap(bytes) - while (buffer.hasRemaining) - channel.write(buffer) - written += bytes.length - } - written - } -} - -class WrappedLargeByteBuffer(private[spark] val underlying: Array[ByteBuffer]) extends LargeByteBuffer { - - val (totalCapacity, chunkOffsets) = { - var sum = 0l - val offsets = new Array[Long](underlying.size) - (0 until underlying.size).foreach{idx => - offsets(idx) = sum - sum += underlying(idx).capacity() - } - (sum, offsets) - } - - private var _pos = 0l - private var currentBufferIdx = 0 - private var currentBuffer = underlying(0) - private var _limit = totalCapacity - - def capacity = totalCapacity - - def get(dst: Array[Byte], offset: Int, length: Int): Unit = { - var moved = 0 - while (moved < length) { - val toRead = math.min(length - moved, currentBuffer.remaining()) - currentBuffer.get(dst, offset, toRead) - moved += toRead - updateCurrentBuffer() - } - } - - def get(): Byte = { - val r = currentBuffer.get() - _pos += 1 - updateCurrentBuffer() - r - } - - private def updateCurrentBuffer(): Unit = { - //TODO fix end condition - while(!currentBuffer.hasRemaining()) { - currentBufferIdx += 1 - currentBuffer = underlying(currentBufferIdx) - } - } - - def put(bytes: LargeByteBuffer): Unit = { - ??? - } - - def position: Long = _pos - def position(position: Long): Unit = { - //XXX check range? - _pos = position - } - def remaining(): Long = { - totalCapacity - _pos - } - - def duplicate(): WrappedLargeByteBuffer = { - new WrappedLargeByteBuffer(underlying.map{_.duplicate()}) - } - - def rewind(): Unit = { - _pos = 0 - underlying.foreach{_.rewind()} - } - - def limit(): Long = { - totalCapacity - } - - def limit(newLimit: Long) = { - //XXX check range? set limits in sub buffers? - _limit = newLimit - } - - def writeTo(channel: WritableByteChannel): Long = { - var written = 0l - underlying.foreach{buffer => - //TODO test this - //XXX do we care about respecting the limit here? - written += buffer.remaining() - while (buffer.hasRemaining) - channel.write(buffer) - } - written - } - -} - -object LargeByteBuffer { - - def asLargeByteBuffer(byteBuffer: ByteBuffer): LargeByteBuffer = { - new WrappedLargeByteBuffer(Array(byteBuffer)) - } - - def asLargeByteBuffer(bytes: Array[Byte]): LargeByteBuffer = { - new WrappedLargeByteBuffer(Array(ByteBuffer.wrap(bytes))) - } - - - def allocateOnHeap(size: Long, maxChunk: Int): LargeByteBuffer = { - val buffer = ChainedBuffer.withInitialSize(maxChunk, size) - new ChainedLargeByteBuffer(buffer) - } - - def mapFile( - channel: FileChannel, - mode: MapMode, - offset: Long, - length: Long, - maxChunk: Int = Integer.MAX_VALUE - 1e6.toInt - ): LargeByteBuffer = { - val offsets = new ArrayBuffer[Long]() - var curOffset = offset - val end = offset + length - while (curOffset < end) { - offsets += curOffset - val length = math.min(end - curOffset, maxChunk) - curOffset += length - } - offsets += end - val chunks = new Array[ByteBuffer](offsets.size - 1) - (0 until offsets.size - 1).foreach{idx => - chunks(idx) = channel.map(mode, offsets(idx), offsets(idx + 1) - offsets(idx)) - } - new WrappedLargeByteBuffer(chunks) - } -} - - -// -///** -// * This is a variant of ByteBuffer to be used internally in spark, which is not limited to 2G -// * which ByteBuffers are limited to. -// * Externally, it exposes all the api which java.nio.ByteBuffer exposes. -// * Internally, it maintains a sequence of Containers which manage the ByteBuffer data. -// * Not all the data might be loaded into memory (like disk or tachyon data) - so actual -// * memory footprint - heap and vm could be much lower than capacity. +///* +// * 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 // * -// * TODO: Currently we are slightly fast and loose in terms of concurrent modifications to this -// * buffer, maybe revisit this later ? Note: this is not much different from earlier though ! +// * http://www.apache.org/licenses/LICENSE-2.0 // * -// * TODO: Explore if (at all) we can leverage zero copy transfers. The issue (currently) is that this -// * will require the file to be kept open (repeatedly opening/closing file is not good -// * for each transfer) and this has an impact on ulimit. Not to mention writing of mmap'ed buffer is -// * pretty quick (it is the first failover in case direct transfer is not possible in file zero copy) -// * -// * TODO: After redesign to containers, we got rid of parent containers to free - the side effect is -// * that if there are direct ByteBuffers, we are not handling explicit cleanup of those in some -// * cases (when we duplicate/slice them). Currently spark does not need this, but might in future -// * so relook at it later. +// * 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. // */ -//// We should make this constructor private: but for now, -//// leaving it public since TachyonStore needs it -//class LargeByteBuffer private[spark](private val inputContainers: ArrayBuffer[ByteBufferContainer], -// private val needDuplicate: Boolean, val ephemeralDiskBacked: Boolean) extends Logging { -// -// // TODO: TEMP code: to flush out potential resource leaks. REMOVE ME -// private val allocateLocationThrowable: Throwable = { -// if (inputContainers.exists(c => c.requireRelease() || c.requireFree())) { -// new Throwable("blockId = " + BlockManager.getLookupBlockId) -// } else { -// null -// } -// } -// private var disposeLocationThrowable: Throwable = null -// -// @volatile private var allowCleanerOverride = true -// @volatile private var cleaner: BufferCleaner = new BufferCleaner { -// override def doClean(buffer: LargeByteBuffer) = { -// assert (LargeByteBuffer.this == buffer) -// doDispose(needRelease = false) -// } -// } // -// // should not be empty -// assert (null != inputContainers && ! inputContainers.isEmpty) -// // should not have any null's -// assert (inputContainers.find(_ == null).isEmpty) +//package org.apache.spark.io // -// // println("Num containers = " + inputContainers.size) +//import java.io.{RandomAccessFile, DataInput, InputStream, OutputStream} +//import java.nio.channels.FileChannel.MapMode +//import java.nio.{ByteBuffer, BufferUnderflowException, BufferOverflowException} +//import java.nio.channels.{FileChannel, WritableByteChannel, ReadableByteChannel} // -// // Position, limit and capacity relevant over the engire LargeByteBuffer -// @volatile private var globalPosition = 0L -// @volatile private var globalLimit = 0L -// @volatile private var currentContainerIndex = 0 +//import org.apache.spark.util.collection.ChainedBuffer // -// // The buffers in which the actual data is held. -// private var containers: Array[ByteBufferContainer] = null +//import scala.collection.mutable.{ArrayBuffer, HashSet} // -// // aggregate capacities of the individual buffers. -// // bufferPositionStart(0) will be capacity of 1st buffer, bufferPositionStart(1) will be -// // sum of capacity of 0th and 1st block buffer -// private var bufferPositionStart: Array[Long] = null // -// // Contains the indices of a containers which requires release before subsequent invocation of -// // read/write should be serviced. This is required since current read/write might have moved the -// // position but since we are returning bytebuffers which depend on the validity of the existing -// // bytebuffer, we cant release them yet. -// private var needReleaseIndices = new HashSet[Int]() // -// private val readable = ! inputContainers.exists(! _.isReadable) -// private val writable = ! inputContainers.exists(! _.isWritable) // +//trait LargeByteBuffer { +//// def position(): Long +//// +//// def limit(): Long // -// // initialize -// @volatile private var globalCapacity = { +// def capacity(): Long // -// // Ensure that there are no empty buffers : messes up with our code : unless it -// // is a single buffer (for empty buffer for marker case) -// assert (inputContainers.find(0 == _.capacity()).isEmpty || 1 == inputContainers.length) +// def get(): Byte //needed for ByteBufferInputStream // -// containers = { -// if (needDuplicate) inputContainers.map(_.duplicate()).toArray else inputContainers.toArray -// } -// containers.foreach(_.validate()) +// def get(dst: Array[Byte], offset: Int, length: Int): Unit // for ByteBufferInputStream // -// def initializeBufferPositionStart(arr: Array[ByteBufferContainer]) { -// val buff = new ArrayBuffer[Long](arr.length + 1) -// buff += 0L +// def position(position: Long): Unit //for ByteBufferInputStream // -// buff ++= arr.map(_.capacity().asInstanceOf[Long]).scanLeft(0L)(_ + _).slice(1, arr.length + 1) -// assert (buff.length == arr.length + 1) -// bufferPositionStart = buff.toArray -// } +// def position(): Long //for ByteBufferInputStream // -// initializeBufferPositionStart(containers) +// /** doesn't copy data, just copies references & offsets */ +// def duplicate(): LargeByteBuffer // -// // remove references from inputBuffers -// inputContainers.clear() +// def put(bytes: LargeByteBuffer): Unit // -// globalLimit = bufferPositionStart(containers.length) -// globalPosition = 0L -// currentContainerIndex = 0 +// //also need whatever is necessary for ByteArrayOutputStream for BlockManager#dataSerialize // -// assert (globalLimit == containers.map(_.capacity().asInstanceOf[Long]).sum) -// -// globalLimit -// } -// -// final def position(): Long = globalPosition -// -// final def limit(): Long = globalLimit -// -// final def capacity(): Long = globalCapacity -// -// final def limit(newLimit: Long) { -// if ((newLimit > capacity()) || (newLimit < 0)) { -// throw new IllegalArgumentException("newLimit = " + newLimit + ", capacity = " + capacity()) -// } -// -// globalLimit = newLimit -// if (position() > newLimit) position(newLimit) -// } -// -// def skip(skipBy: Long) = position(position() + skipBy) -// -// private def releasePendingContainers() { -// if (! needReleaseIndices.isEmpty) { -// val iter = needReleaseIndices.iterator -// while (iter.hasNext) { -// val index = iter.next() -// assert (index >= 0 && index < containers.length) -// // It is possible to move from one container to next before the previous -// // container was acquired. For example, get forcing move to next container -// // since current was exhausted immediatelly followed by a position() -// // so the container we moved to was never acquired. -// -// // assert (containers(index).isAcquired) -// // will this always be satisfied ? -// // assert (index != currentContainerIndex) -// if (containers(index).isAcquired) containers(index).release() -// } -// needReleaseIndices.clear() -// } -// } -// -// private def toNewContainer(newIndex: Int) { -// if (newIndex != currentContainerIndex && currentContainerIndex < containers.length) { -// -// assert (currentContainerIndex >= 0) -// needReleaseIndices += currentContainerIndex -// } -// currentContainerIndex = newIndex -// } -// -// // expensive method, sigh ... optimize it later ? -// final def position(newPosition: Long) { -// -// if ((newPosition > globalLimit) || (newPosition < 0)) throw new IllegalArgumentException() -// -// if (currentContainerIndex < bufferPositionStart.length - 1 && -// newPosition >= bufferPositionStart(currentContainerIndex) && -// newPosition < bufferPositionStart(currentContainerIndex + 1)) { -// // Same buffer - easy method ... -// globalPosition = newPosition -// // Changed position - free previously returned buffers. -// releasePendingContainers() -// return -// } -// -// // Find appropriate currentContainerIndex -// // Since bufferPositionStart is sorted, can be replaced with binary search if required. -// // For now, not in the perf critical path since buffers size is very low typically. -// var index = 0 -// val cLen = containers.length -// while (index < cLen) { -// if (newPosition >= bufferPositionStart(index) && -// newPosition < bufferPositionStart(index + 1)) { -// globalPosition = newPosition -// toNewContainer(index) -// // Changed position - free earlier and previously returned buffers. -// releasePendingContainers() -// return -// } -// index += 1 -// } -// -// if (newPosition == globalLimit && newPosition == bufferPositionStart(cLen)) { -// // boundary. -// globalPosition = newPosition -// toNewContainer(cLen) -// // Changed position - free earlier and previously returned buffers. -// releasePendingContainers() -// return -// } -// -// assert (assertion = false, "Unexpected to come here .... newPosition = " + newPosition + -// ", bufferPositionStart = " + bufferPositionStart.mkString("[", ", ", "]")) -// } // +// //TODO checks on limit semantics // // /** -// * Clears this buffer. The position is set to zero, the limit is set to -// * the capacity, and the mark is discarded. -// * -// *

Invoke this method before using a sequence of channel-read or -// * put operations to fill this buffer. -// * -// *

This method does not actually erase the data in the buffer, but it -// * is named as if it did because it will most often be used in situations -// * in which that might as well be the case.

+// * Sets this buffer's limit. If the position is larger than the new limit then it is set to the +// * new limit. If the mark is defined and larger than the new limit then it is discarded. // */ -// final def clear() { -// // if (0 == globalCapacity) return -// -// needReleaseIndices += 0 -// globalPosition = 0L -// toNewContainer(0) -// globalLimit = globalCapacity -// -// // Now free all pending containers -// releasePendingContainers() -// } +// def limit(newLimit: Long): Unit // // /** -// * Flips this buffer. The limit is set to the current position and then -// * the position is set to zero. If the mark is defined then it is -// * discarded. -// * -// *

After a sequence of channel-read or put operations, invoke -// * this method to prepare for a sequence of channel-write or relative -// * get operations. +// * return this buffer's limit +// * @return // */ -// final def flip() { -// needReleaseIndices += 0 -// globalLimit = globalPosition -// globalPosition = 0L -// toNewContainer(0) +// def limit(): Long // -// // Now free all pending containers -// releasePendingContainers() -// } +// +// //an alternative to having this method would be having a foreachBuffer(f: Buffer => T) +// def writeTo(channel: WritableByteChannel): Long +// +//// +//// def skip(skipBy: Long): Unit +//// +//// def position(newPosition: Long): Unit +//// +//// /** +//// * Clears this buffer. The position is set to zero, the limit is set to +//// * the capacity, and the mark is discarded. +//// * +//// *

Invoke this method before using a sequence of channel-read or +//// * put operations to fill this buffer. +//// * +//// *

This method does not actually erase the data in the buffer, but it +//// * is named as if it did because it will most often be used in situations +//// * in which that might as well be the case.

+//// */ +//// def clear(): Unit +//// +//// /** +//// * Flips this buffer. The limit is set to the current position and then +//// * the position is set to zero. If the mark is defined then it is +//// * discarded. +//// * +//// *

After a sequence of channel-read or put operations, invoke +//// * this method to prepare for a sequence of channel-write or relative +//// * get operations. +//// */ +//// def flip(): Unit // // /** // * Rewinds this buffer. The position is set to zero and the mark is @@ -558,14 +107,7 @@ object LargeByteBuffer { // * operations, assuming that the limit has already been set // * appropriately. // */ -// final def rewind() { -// needReleaseIndices += 0 -// globalPosition = 0L -// toNewContainer(0) -// -// // Now free all pending containers -// releasePendingContainers() -// } +// def rewind(): Unit // // /** // * Returns the number of elements between the current position and the @@ -573,1192 +115,1650 @@ object LargeByteBuffer { // * // * @return The number of elements remaining in this buffer // */ -// final def remaining(): Long = { -// globalLimit - globalPosition -// } -// -// /** -// * Tells whether there are any elements between the current position and -// * the limit.

-// * -// * @return true if, and only if, there is at least one element -// * remaining in this buffer -// */ -// final def hasRemaining() = { -// globalPosition < globalLimit -// } -// -// // private def currentBuffer(): ByteBuffer = buffers(currentContainerIndex) -// -// // number of bytes remaining in currently active underlying buffer -// private def currentRemaining(): Int = { -// if (hasRemaining()) { -// // validate currentContainerIndex is valid -// assert (globalPosition >= bufferPositionStart(currentContainerIndex) && -// globalPosition < bufferPositionStart(currentContainerIndex + 1), -// "globalPosition = " + globalPosition + -// ", currentContainerIndex = " + currentContainerIndex + -// ", bufferPositionStart = " + bufferPositionStart.mkString("[", ", ", " ]")) -// -// currentRemaining0(currentContainerIndex) -// } else 0 -// } -// -// // Without any validation : required when we are bumping the index (when validation will fail) ... -// private def currentRemaining0(which: Int): Int = { -// // currentBuffer().remaining() -// math.max(0, math.min(bufferPositionStart(which + 1), -// globalLimit) - globalPosition).asInstanceOf[Int] -// } -// -// // Set the approppriate position/limit for the current underlying buffer to mirror our -// // the LargeByteBuffer's state. -// private def fetchCurrentBuffer(): ByteBuffer = { -// releasePendingContainers() -// -// assert (currentContainerIndex < containers.length) -// -// val container = containers(currentContainerIndex) -// if (! container.isAcquired) { -// container.acquire() -// } -// -// assert (container.isAcquired) -// if (LargeByteBuffer.enableExpensiveAssert) { -// assert (! containers.exists( b => (b ne container) && b.isAcquired)) -// } -// -// assert (currentContainerIndex < bufferPositionStart.length && -// globalPosition < bufferPositionStart(currentContainerIndex + 1), -// "currentContainerIndex = " + currentContainerIndex + ", bufferPositionStart = " + -// bufferPositionStart.mkString("[", ", ", "]") + ", this = " + this) -// -// val buffPosition = (globalPosition - bufferPositionStart(currentContainerIndex)). -// asInstanceOf[Int] -// -// val buffer = container.getByteBuffer -// buffer.position(buffPosition) -// val diff = buffer.capacity - buffPosition -// val left = remaining() -// if (diff <= left) { -// buffer.limit(buffer.capacity()) -// } else { -// // Can happen if limit() was called. -// buffer.limit(buffPosition + left.asInstanceOf[Int]) -// } -// -// buffer -// } -// -// // To be used ONLY to test in suites. -// private[spark] def fetchCurrentBufferForTesting(): ByteBuffer = { -// if ("1" != System.getProperty("SPARK_TESTING")) { -// throw new IllegalStateException("This method is to be used ONLY within spark test suites") -// } -// -// fetchCurrentBuffer() -// } -// -// // Expects that the invoker has ensured that this can be safely invoked. -// // That is, it wont be invoked when the loop wont terminate. -// private def toNonEmptyBuffer() { -// -// if (! hasRemaining()) { -// var newIndex = currentContainerIndex -// // Ensure we are in the right block or not. -// while (newIndex < containers.length && globalPosition >= bufferPositionStart(newIndex + 1)) { -// newIndex += 1 -// } -// toNewContainer(newIndex) -// // Do not do this - since we might not yet have consumed the buffer which caused EOF right now -// /* -// // Add last one also, and release it too - since we are at the end of the buffer with nothing -// // more pending. -// if (newIndex >= 0 && currentContainerIndex < containers.length) { -// needReleaseIndices += newIndex -// } -// */ -// assert (currentContainerIndex >= 0) -// // releasePendingContainers() -// return -// } -// -// var index = currentContainerIndex -// while (0 == currentRemaining0(index) && index < containers.length) { -// index += 1 -// } -// assert (currentContainerIndex < containers.length) -// toNewContainer(index) -// assert (0 != currentRemaining()) -// } -// -// private def assertPreconditions(containerIndex: Int) { -// assert (globalPosition >= bufferPositionStart(containerIndex), -// "globalPosition = " + globalPosition + ", containerIndex = " + containerIndex + -// ", bufferPositionStart = " + bufferPositionStart.mkString("[", ", ", " ]")) -// assert (globalPosition < bufferPositionStart(containerIndex + 1), -// "globalPosition = " + globalPosition + ", containerIndex = " + containerIndex + -// ", bufferPositionStart = " + bufferPositionStart.mkString("[", ", ", " ]")) -// -// assert (globalLimit <= globalCapacity) -// assert (containerIndex < containers.length) -// } -// -// -// /** -// * Attempts to return a ByteBuffer of the requested size. -// * It is possible to return a buffer of size smaller than requested -// * even though hasRemaining == true -// * -// * On return, position would have been moved 'ahead' by the size of the buffer returned : -// * that is, we treat that the returned buffer has been already 'read' from this LargeByteBuffer -// * -// * -// * This is used to primarily retrieve content of this buffer to expose via ByteBuffer -// * to some other api which is deemed too cumbersome to move to LargeByteBuffer (like the -// * chunked sending of contents via ConnectionManager) Note that the lifecycle of the ByteBuffer -// * returned is inherently tied to the state of this LargeByteBuffer. For example,if the underlying -// * container is a disk backed container, and we make subsequent calls to get(), the returned -// * ByteBuffer can be dispose'ed off -// * -// * @param maxChunkSize Max size of the ByteBuffer to retrieve. -// * @return -// */ -// -// private def fetchBufferOfSize(maxChunkSize: Int): ByteBuffer = { -// fetchBufferOfSizeImpl(maxChunkSize, canReleaseContainers = true) -// } -// -// private def fetchBufferOfSizeImpl(maxChunkSize: Int, -// canReleaseContainers: Boolean): ByteBuffer = { -// if (canReleaseContainers) releasePendingContainers() -// assert (maxChunkSize > 0) -// -// // not checking for degenerate case of maxChunkSize == 0 -// if (globalPosition >= globalLimit) { -// // throw exception -// throw new BufferUnderflowException() -// } -// -// // Check preconditions : disable these later, since they might be expensive to -// // evaluate for every IO op -// assertPreconditions(currentContainerIndex) -// -// val currentBufferRemaining = currentRemaining() -// -// assert (currentBufferRemaining > 0) -// -// val size = math.min(currentBufferRemaining, maxChunkSize) -// -// val newBuffer = if (currentBufferRemaining > maxChunkSize) { -// val currentBuffer = fetchCurrentBuffer() -// val buff = ByteBufferContainer.createSlice(currentBuffer, -// currentBuffer.position(), maxChunkSize) -// assert (buff.remaining() == maxChunkSize) -// buff -// } else { -// val currentBuffer = fetchCurrentBuffer() -// val buff = currentBuffer.slice() -// assert (buff.remaining() == currentBufferRemaining) -// buff -// } -// -// assert (size == newBuffer.remaining()) -// assert (0 == newBuffer.position()) -// assert (size == newBuffer.limit()) -// assert (newBuffer.capacity() == newBuffer.limit()) -// -// globalPosition += newBuffer.remaining -// toNonEmptyBuffer() -// -// newBuffer -// } -// -// // Can we service the read/write from the currently active (underlying) bytebuffer or not. -// // For almost all cases, this will return true allowing us to optimize away the more expensive -// // computations. -// private def localReadWritePossible(size: Int) = -// size >= 0 && globalPosition + size <= bufferPositionStart(currentContainerIndex + 1) -// -// -// def getLong(): Long = { -// assert (readable) -// releasePendingContainers() -// -// if (remaining() < 8) throw new BufferUnderflowException -// -// if (localReadWritePossible(8)) { -// val buff = fetchCurrentBuffer() -// assert (buff.remaining() >= 8) -// val retval = buff.getLong -// globalPosition += 8 -// toNonEmptyBuffer() -// return retval -// } -// -// val buff = readFully(8) -// buff.getLong -// } -// -// def getInt(): Int = { -// assert (readable) -// releasePendingContainers() -// -// if (remaining() < 4) throw new BufferUnderflowException -// -// if (localReadWritePossible(4)) { -// val buff = fetchCurrentBuffer() -// assert (buff.remaining() >= 4) -// val retval = buff.getInt -// globalPosition += 4 -// toNonEmptyBuffer() -// return retval -// } -// -// val buff = readFully(4) -// buff.getInt -// } +// def remaining(): Long +//} // -// def getChar(): Char = { -// assert (readable) -// releasePendingContainers() +//class ChainedLargeByteBuffer(private[io] val underlying: ChainedBuffer) extends LargeByteBuffer { // -// if (remaining() < 2) throw new BufferUnderflowException +// def capacity = underlying.capacity // -// if (localReadWritePossible(2)) { -// val buff = fetchCurrentBuffer() -// assert (buff.remaining() >= 2) -// val retval = buff.getChar -// globalPosition += 2 -// toNonEmptyBuffer() -// return retval -// } +// var _pos = 0l // -// // if slice is becoming too expensive, revisit this ... -// val buff = readFully(2) -// buff.getChar +// def get(dst: Array[Byte],offset: Int,length: Int): Unit = { +// underlying.read(_pos, dst, offset, length) +// _pos += length // } // // def get(): Byte = { -// assert (readable) -// releasePendingContainers() -// -// if (! hasRemaining()) throw new BufferUnderflowException -// -// // If we have remaining bytes, previous invocations MUST have ensured that we are at -// // a buffer which has data to be read. -// assert (localReadWritePossible(1)) -// -// val buff = fetchCurrentBuffer() -// assert (buff.remaining() >= 1, "buff.remaining = " + buff.remaining()) -// val retval = buff.get() -// globalPosition += 1 -// toNonEmptyBuffer() -// -// retval -// } -// -// def get(arr: Array[Byte], offset: Int, size: Int): Int = { -// assert (readable) -// releasePendingContainers() -// -// LargeByteBuffer.checkOffsets(arr, offset, size) -// -// // kyro depends on this it seems ? -// // assert (size > 0) -// if (0 == size) return 0 -// -// if (! hasRemaining()) return -1 -// -// if (localReadWritePossible(size)) { -// val buff = fetchCurrentBuffer() -// assert (buff.remaining() >= size) -// buff.get(arr, offset, size) -// globalPosition += size -// toNonEmptyBuffer() -// return size -// } -// -// var remainingSize = math.min(size, remaining()).asInstanceOf[Int] -// var currentOffset = offset -// -// while (remainingSize > 0) { -// val buff = fetchBufferOfSize(remainingSize) -// val toCopy = math.min(buff.remaining(), remainingSize) -// -// buff.get(arr, currentOffset, toCopy) -// currentOffset += toCopy -// remainingSize -= toCopy -// } -// -// currentOffset - offset -// } -// -// -// private def createSlice(size: Long): LargeByteBuffer = { -// -// releasePendingContainers() -// -// if (remaining() < size) { -// // logInfo("createSlice. remaining = " + remaining() + ", size " + size + ", this = " + this) -// throw new BufferOverflowException -// } -// -// // kyro depends on this it seems ? -// // assert (size > 0) -// if (0 == size) return LargeByteBuffer.EMPTY_BUFFER -// -// val arr = new ArrayBuffer[ByteBufferContainer](2) -// var totalLeft = size -// -// // assert (currentRemaining() < totalLeft || totalLeft != size || currentAsByteBuffer) -// -// var containerIndex = currentContainerIndex -// while (totalLeft > 0 && hasRemaining()) { -// assertPreconditions(containerIndex) -// val container = containers(containerIndex) -// val currentLeft = currentRemaining0(containerIndex) -// -// assert (globalPosition + currentLeft <= globalLimit) -// assert (globalPosition >= bufferPositionStart(containerIndex) && -// (globalPosition < bufferPositionStart(containerIndex + 1))) -// -// val from = (globalPosition - bufferPositionStart(containerIndex)).asInstanceOf[Int] -// val sliceSize = math.min(totalLeft, currentLeft) -// assert (from >= 0) -// assert (sliceSize > 0 && sliceSize <= Int.MaxValue) -// -// val slice = container.createSlice(from, sliceSize.asInstanceOf[Int]) -// arr += slice -// -// globalPosition += sliceSize -// totalLeft -= sliceSize -// if (currentLeft == sliceSize) containerIndex += 1 -// } -// -// // Using toNonEmptyBuffer instead of directly moving to next here so that -// // other checks can be performed there. -// toNonEmptyBuffer() -// // force cleanup - this is fine since we are not using the buffers directly -// // which are actively needed (the returned value is on containers which can -// // recreate) -// releasePendingContainers() -// // free current container if acquired. -// if (currentContainerIndex < containers.length) { -// containers(currentContainerIndex).release() -// } -// assert (currentContainerIndex == containerIndex) -// -// val retval = new LargeByteBuffer(arr, false, ephemeralDiskBacked) -// retval.overrideCleaner(LargeByteBuffer.noopDisposeFunction) -// retval -// } -// -// // Get a composite sequence of ByteBuffer which might straddle one or more underlying buffers -// // This is to be used only for writes : and ensures that writes are done into the appropriate -// // underlying bytebuffers. -// def getCompositeWriteBuffer(size: Long): LargeByteBuffer = { -// assert(writable) -// assert(size >= 0) -// -// createSlice(size) +// val b = underlying.read(_pos) +// _pos += 1 +// b // } // -// // get a buffer which is of the specified size and contains data from the underlying buffers -// // Note, the actual data might be spread across the underlying buffers. -// // This MUST BE used only for specific usecases like getInt, etc. Not for bulk copy ! -// private def readFully(size: Int): ByteBuffer = { -// assert (readable) -// -// if (remaining() < size) { -// // throw exception -// throw new BufferUnderflowException() -// } -// -// // kyro depends on this it seems ? -// // assert (size > 0) -// if (0 == size) return LargeByteBuffer.EMPTY_BYTEBUFFER -// -// // Expected to be handled elsewhere. -// assert (! localReadWritePossible(size)) -// -// val localBuff = { -// val buff = fetchBufferOfSize(size) -// // assert(buff.remaining() <= size) -// // if (buff.remaining() == size) return buff -// assert(buff.remaining() < size) -// ByteBuffer.allocate(size).put(buff) -// } -// -// // assert (localBuff.hasRemaining) -// -// while (localBuff.hasRemaining) { -// val buff = fetchBufferOfSize(localBuff.remaining()) -// localBuff.put(buff) -// } -// -// localBuff.flip() -// localBuff +// def put(bytes: LargeByteBuffer): Unit = { +// ??? // } // -// -// -// def put(b: Byte) { -// assert (writable) -// if (remaining() < 1) { -// // logInfo("put byte. remaining = " + remaining() + ", this = " + this) -// throw new BufferOverflowException -// } -// -// assert (currentRemaining() > 0) -// -// fetchCurrentBuffer().put(b) -// globalPosition += 1 -// // Check to need to bump the index ? -// toNonEmptyBuffer() +// def position: Long = _pos +// def position(position: Long): Unit = { +// _pos = position // } -// -// -// def put(buffer: ByteBuffer) { -// assert (writable) -// if (remaining() < buffer.remaining()) { -// throw new BufferOverflowException -// } -// -// val bufferRemaining = buffer.remaining() -// if (localReadWritePossible(bufferRemaining)) { -// -// assert (currentRemaining() >= bufferRemaining) -// -// fetchCurrentBuffer().put(buffer) -// -// globalPosition += bufferRemaining -// toNonEmptyBuffer() -// return -// } -// -// while (buffer.hasRemaining) { -// val currentBufferRemaining = currentRemaining() -// val bufferRemaining = buffer.remaining() -// -// if (currentBufferRemaining >= bufferRemaining) { -// fetchCurrentBuffer().put(buffer) -// globalPosition += bufferRemaining -// } else { -// // Split across buffers. -// val currentBuffer = fetchCurrentBuffer() -// assert (currentBuffer.remaining() >= currentBufferRemaining) -// val sliced = ByteBufferContainer.createSlice(buffer, buffer.position(), -// currentBufferRemaining) -// assert (sliced.remaining() == currentBufferRemaining) -// currentBuffer.put(sliced) -// // move buffer pos -// buffer.position(buffer.position() + currentBufferRemaining) -// -// globalPosition += currentBufferRemaining -// } -// toNonEmptyBuffer() -// } -// -// assert (! hasRemaining() || currentRemaining() > 0) +// def remaining(): Long = { +// underlying.size - position // } // -// def put(other: LargeByteBuffer) { -// assert (writable) -// if (this.remaining() < other.remaining()) { -// throw new BufferOverflowException -// } -// -// while (other.hasRemaining()) { -// val buffer = other.fetchBufferOfSize(other.currentRemaining()) -// this.put(buffer) -// } +// def duplicate(): ChainedLargeByteBuffer = { +// new ChainedLargeByteBuffer(underlying) // } // -// -// def duplicate(): LargeByteBuffer = { -// val containersCopy = new ArrayBuffer[ByteBufferContainer](containers.size) -// // We do a duplicate as part of construction - so avoid double duplicate. -// // containersCopy ++= containers.map(_.duplicate()) -// containersCopy ++= containers -// val retval = new LargeByteBuffer(containersCopy, true, ephemeralDiskBacked) -// -// // set limit and position (in that order) ... -// retval.limit(this.limit()) -// retval.position(this.position()) -// -// // Now release our containers - if any had been acquired -// releasePendingContainers() -// -// retval +// def rewind(): Unit = { +// _pos = 0 // } // -// -// /** -// * 'read' a LargeByteBuffer of size specified and return that. -// * Position will be incremented by size -// * -// * The name might be slightly confusing : rename ? -// * -// * @param size Amount of data to be read from this buffer and returned -// * @return -// */ -// def readLargeBuffer(size: Long, partialReadAllowed: Boolean): LargeByteBuffer = { -// if (! hasRemaining() && ! partialReadAllowed) throw new BufferUnderflowException -// if (remaining() < size && ! partialReadAllowed) throw new BufferUnderflowException -// -// -// assert (readable) -// assert (size >= 0) -// -// releasePendingContainers() -// -// if (0 == size) return LargeByteBuffer.EMPTY_BUFFER -// -// createSlice(size) +// def limit(): Long = { +// capacity // } // -// -// // This is essentially a workaround to exposing underlying buffers -// def readFrom(channel: ReadableByteChannel): Long = { -// -// assert (writable) -// releasePendingContainers() -// -// // this also allows us to avoid nasty corner cases in the loop. -// if (! hasRemaining()) { -// // logInfo("readFrom channel. remaining = " + remaining() + ", this = " + this) -// throw new BufferOverflowException -// } -// -// var totalBytesRead = 0L -// -// while (hasRemaining()) { -// // read what we can ... -// val buffer = fetchCurrentBuffer() -// val bufferRemaining = currentRemaining() -// val bytesRead = channel.read(buffer) -// -// if (bytesRead > 0) { -// totalBytesRead += bytesRead -// // bump position too .. -// globalPosition += bytesRead -// if (bytesRead >= bufferRemaining) toNonEmptyBuffer() -// } -// else if (-1 == bytesRead) { -// // if we had already read some data in the loop, return that. -// if (totalBytesRead > 0) return totalBytesRead -// return -1 -// } // nothing available to read, retry later. return -// else if (0 == bytesRead) { -// return totalBytesRead -// } -// -// // toNonEmptyBuffer() -// } -// -// // Cleanup last buffer ? -// toNonEmptyBuffer() -// totalBytesRead +// def limit(newLimit: Long): Unit = { +// ??? // } // -// // This is essentially a workaround to exposing underlying buffers -// def readFrom(inStrm: InputStream): Long = { -// -// assert (writable) -// releasePendingContainers() -// -// // this also allows us to avoid nasty corner cases in the loop. -// // if (! hasRemaining()) throw new BufferOverflowException -// if (! hasRemaining()) return 0 -// -// var totalBytesRead = 0L -// -// val buff = new Array[Byte](LargeByteBuffer.TEMP_ARRAY_SIZE) -// -// while (hasRemaining()) { -// // read what we can ... note, since there is no gaurantee that underlying buffer might -// // expose array() method, we do double copy - from stream to buff and from buff to bytearray. -// // see if we can optimize this later ... -// val buffer = fetchCurrentBuffer() -// val bufferRemaining = buffer.remaining() -// val max = math.min(buff.length, bufferRemaining) -// val bytesRead = inStrm.read(buff, 0, max) -// -// if (bytesRead > 0) { -// buffer.put(buff, 0, bytesRead) -// totalBytesRead += bytesRead -// // bump position too .. -// globalPosition += bytesRead -// // buffer.position(buffer.position + bytesRead) -// if (bytesRead >= bufferRemaining) toNonEmptyBuffer() -// } -// else if (-1 == bytesRead) { -// // if we had already read some data in the loop, return that. -// if (totalBytesRead > 0) return totalBytesRead -// return -1 -// } // nothing available to read, retry later. return -// else if (0 == bytesRead) { -// return totalBytesRead -// } -// -// // toNonEmptyBuffer() +// def writeTo(channel:WritableByteChannel): Long = { +// var written = 0l +// underlying.chunks.foreach{bytes => +// //TODO test this +// val buffer = ByteBuffer.wrap(bytes) +// while (buffer.hasRemaining) +// channel.write(buffer) +// written += bytes.length // } -// -// totalBytesRead +// written // } +//} // -// // This is essentially a workaround to exposing underlying buffers -// // Note: very similar to readFrom(InputStream) : not trying anything fancy to reduce -// // code for performance reasons. -// def readFrom(inStrm: DataInput): Long = { -// -// assert (writable) -// releasePendingContainers() -// -// // this also allows us to avoid nasty corner cases in the loop. -// // if (! hasRemaining()) throw new BufferOverflowException -// if (! hasRemaining()) return 0 -// -// var totalBytesRead = 0L -// -// val buff = new Array[Byte](LargeByteBuffer.TEMP_ARRAY_SIZE) -// -// while (hasRemaining()) { -// // read what we can ... note, since there is no gaurantee that underlying buffer might -// // expose array() method, we do double copy - from stream to buff and from buff to bytearray. -// // see if we can optimize this later ... -// val buffer = fetchCurrentBuffer() -// val bufferRemaining = buffer.remaining() -// val max = math.min(buff.length, bufferRemaining) -// inStrm.readFully(buff, 0, max) -// val bytesRead = max -// -// if (bytesRead > 0) { -// buffer.put(buff, 0, bytesRead) -// totalBytesRead += bytesRead -// // bump position too .. -// globalPosition += bytesRead -// // buffer.position(buffer.position() + bytesRead) -// if (bytesRead >= bufferRemaining) toNonEmptyBuffer() -// } -// else if (-1 == bytesRead) { -// // if we had already read some data in the loop, return that. -// if (totalBytesRead > 0) return totalBytesRead -// return -1 -// } // nothing available to read, retry later. return -// else if (0 == bytesRead) { -// return totalBytesRead -// } +//class WrappedLargeByteBuffer(private[spark] val underlying: Array[ByteBuffer]) extends LargeByteBuffer { // -// // toNonEmptyBuffer() +// val (totalCapacity, chunkOffsets) = { +// var sum = 0l +// val offsets = new Array[Long](underlying.size) +// (0 until underlying.size).foreach{idx => +// offsets(idx) = sum +// sum += underlying(idx).capacity() // } -// -// totalBytesRead +// (sum, offsets) // } // -// // This is essentially a workaround to exposing underlying buffers -// // Note: tries to do it efficiently without needing to load everything into memory -// // (particularly for diskbacked buffers, etc). -// def writeTo(channel: WritableByteChannel, cleanup: Boolean): Long = { -// -// assert (readable) -// releasePendingContainers() -// -// // this also allows us to avoid nasty corner cases in the loop. -// if (! hasRemaining()) throw new BufferUnderflowException -// -// var totalBytesWritten = 0L -// -// while (hasRemaining()) { -// // Write what we can ... -// val buffer = fetchCurrentBuffer() -// val bufferRemaining = buffer.remaining() -// assert (bufferRemaining > 0) -// val bytesWritten = channel.write(buffer) -// -// if (bytesWritten > 0) { -// totalBytesWritten += bytesWritten -// // bump position too .. -// globalPosition += bytesWritten -// if (bytesWritten >= bufferRemaining) toNonEmptyBuffer() -// assert (! hasRemaining() || currentRemaining() > 0) -// } -// else if (0 == bytesWritten) { -// return totalBytesWritten -// } +// private var _pos = 0l +// private var currentBufferIdx = 0 +// private var currentBuffer = underlying(0) +// private var _limit = totalCapacity // -// // toNonEmptyBuffer() -// } +// def capacity = totalCapacity // -// assert (! hasRemaining()) -// if (cleanup) { -// free() +// def get(dst: Array[Byte], offset: Int, length: Int): Unit = { +// var moved = 0 +// while (moved < length) { +// val toRead = math.min(length - moved, currentBuffer.remaining()) +// currentBuffer.get(dst, offset, toRead) +// moved += toRead +// updateCurrentBuffer() // } -// totalBytesWritten // } // -// // This is essentially a workaround to exposing underlying buffers -// def writeTo(outStrm: OutputStream, cleanup: Boolean): Long = { -// -// assert (readable) -// releasePendingContainers() -// -// // this also allows us to avoid nasty corner cases in the loop. -// if (! hasRemaining()) throw new BufferUnderflowException -// -// var totalBytesWritten = 0L -// val buff = new Array[Byte](LargeByteBuffer.TEMP_ARRAY_SIZE) -// -// while (hasRemaining()) { -// // write what we can ... note, since there is no gaurantee that underlying buffer might -// // expose array() method, we do double copy - from bytearray to buff and from -// // buff to outputstream. see if we can optimize this later ... -// val buffer = fetchCurrentBuffer() -// val bufferRemaining = buffer.remaining() -// val size = math.min(bufferRemaining, buff.length) -// buffer.get(buff, 0, size) -// outStrm.write(buff, 0, size) -// -// totalBytesWritten += size -// // bump position too .. -// globalPosition += size -// -// if (size >= bufferRemaining) toNonEmptyBuffer() -// } -// -// toNonEmptyBuffer() -// if (cleanup) { -// free() -// } -// totalBytesWritten +// def get(): Byte = { +// val r = currentBuffer.get() +// _pos += 1 +// updateCurrentBuffer() +// r // } // -// def asInputStream(): InputStream = { -// new InputStream() { -// override def read(): Int = { -// if (! hasRemaining()) return -1 -// get() -// } -// -// override def read(arr: Array[Byte], off: Int, len: Int): Int = { -// if (! hasRemaining()) return -1 -// -// get(arr, off, len) -// } -// -// override def available(): Int = { -// // current remaining is what can be read without blocking -// // anything higher might need disk access/buffer swapping. -// /* -// val left = remaining() -// math.min(left, Int.MaxValue).asInstanceOf[Int] -// */ -// currentRemaining() -// } +// private def updateCurrentBuffer(): Unit = { +// //TODO fix end condition +// while(!currentBuffer.hasRemaining()) { +// currentBufferIdx += 1 +// currentBuffer = underlying(currentBufferIdx) // } // } // -// def getCleaner() = cleaner -// -// /** -// * @param cleaner The previous cleaner, so that the caller can chain them if required. -// * @return -// */ -// private[spark] def overrideCleaner(cleaner: BufferCleaner): BufferCleaner = { -// overrideCleaner(cleaner, allowOverride = true) +// def put(bytes: LargeByteBuffer): Unit = { +// ??? // } // -// private def overrideCleaner(cleaner: BufferCleaner, allowOverride: Boolean): BufferCleaner = { -// if (! this.allowCleanerOverride) { -// // allowCleanerOverride = false is used for EMPTY_BUFFER - where we do not allow free -// return this.cleaner -// } -// -// this.allowCleanerOverride = allowOverride -// assert (null != cleaner) -// val prev = this.cleaner -// this.cleaner = cleaner -// // logInfo("Overriding " + prev + " with " + this.cleaner) -// prev +// def position: Long = _pos +// def position(position: Long): Unit = { +// //XXX check range? +// _pos = position // } -// -// private def doReleaseAll() { -// for (container <- containers) { -// container.release() -// } +// def remaining(): Long = { +// totalCapacity - _pos // } // -// def free(invokeCleaner: Boolean = true) { -// // logInfo("Free on " + this + ", cleaner = " + cleaner) -// // always invoking release -// doReleaseAll() -// -// if (invokeCleaner) cleaner.clean(this) +// def duplicate(): WrappedLargeByteBuffer = { +// new WrappedLargeByteBuffer(underlying.map{_.duplicate()}) // } // -// private def doDispose(needRelease: Boolean) { -// -// if (disposeLocationThrowable ne null) { -// logError("Already free'ed earlier at : ", disposeLocationThrowable) -// logError("Current at ", new Throwable) -// throw new IllegalStateException("Already freed.") -// } -// disposeLocationThrowable = new Throwable() -// -// // Forcefully cleanup all -// if (needRelease) doReleaseAll() -// -// // Free in a different loop, in case different containers refer to same resource -// // to release (like file) -// for (container <- containers) { -// container.free() -// } -// -// needReleaseIndices.clear() -// -// // We should not use this buffer anymore : set the values such that f -// // we dont ... -// globalPosition = 0 -// globalLimit = 0 -// globalCapacity = 0 +// def rewind(): Unit = { +// _pos = 0 +// underlying.foreach{_.rewind()} // } // -// // copy data over ... MUST be used only for cases where array is known to be -// // small to begin with. slightly risky method due to that assumption -// def toByteArray(): Array[Byte] = { -// val positionBackup = position() -// val size = remaining() -// if (size > Int.MaxValue) { -// throw new IllegalStateException( -// "Attempt to convert LargeByteBuffer to byte array when data held is more than 2G") -// } -// -// val retval = new Array[Byte](size.asInstanceOf[Int]) -// val readSize = get(retval, 0, retval.length) -// assert (readSize == retval.length, -// "readSize = " + readSize + ", retval.length = " + retval.length) -// -// position(positionBackup) -// -// retval +// def limit(): Long = { +// totalCapacity // } // -// // copy data over ... MUST be used only for cases where array is known to be -// // small to begin with. slightly risky method due to that assumption -// def toByteBuffer(): ByteBuffer = { -// ByteBuffer.wrap(toByteArray()) +// def limit(newLimit: Long) = { +// //XXX check range? set limits in sub buffers? +// _limit = newLimit // } // -// def toInMemoryBuffer(ioConf: IOConfig): LargeByteBuffer = { -// val retval = LargeByteBuffer.allocateMemoryBuffer(remaining(), ioConf) -// val currentPosition = position() -// retval.put(this) -// position(currentPosition) -// retval.clear() -// retval -// } -// -// -// -// // This is ONLY used for testing : that too as part of development of this and associated classes -// // remove before contributing to spark. -// def hexDump(): String = { -// if (remaining() * 64 > Int.MaxValue) { -// throw new UnsupportedOperationException("buffer too large " + remaining()) -// } -// -// val sb = new StringBuilder((remaining() * 2).asInstanceOf[Int]) -// -// var perLine = 0 -// var first = true -// for (b <- toByteArray()) { -// perLine += 1 -// if (perLine % 8 == 0) { -// sb.append('\n') -// first = true -// } -// if (! first) sb.append(' ') -// first = false -// sb.append(java.lang.Integer.toHexString(b & 0xff)) +// def writeTo(channel: WritableByteChannel): Long = { +// var written = 0l +// underlying.foreach{buffer => +// //TODO test this +// //XXX do we care about respecting the limit here? +// written += buffer.remaining() +// while (buffer.hasRemaining) +// channel.write(buffer) // } -// sb.append('\n') -// sb.toString() +// written // } // -// override def toString: String = { -// val sb: StringBuffer = new StringBuffer -// sb.append(getClass.getName) -// sb.append(' ') -// sb.append(System.identityHashCode(this)) -// sb.append("@[pos=") -// sb.append(position()) -// sb.append(" lim=") -// sb.append(limit()) -// sb.append(" cap=") -// sb.append(capacity()) -// sb.append("]") -// sb.toString -// } -// -// -// -// override def finalize(): Unit = { -// var marked = false -// if (containers ne null) { -// if (containers.exists(container => container.isAcquired && container.requireRelease())) { -// marked = true -// logError("BUG: buffer was not released - and now going out of scope. " + -// "Potential resource leak. Allocated at ", allocateLocationThrowable) -// containers.foreach(_.release()) -// } -// if (containers.exists(container => !container.isFreed && container.requireFree())) { -// if (!marked) { -// logError("BUG: buffer was not freed - and now going out of scope. Potential resource leak", -// allocateLocationThrowable) -// } -// else { -// logError("BUG: buffer was not freed - and now going out of scope. Potential resource leak") -// } -// containers.foreach(_.free()) -// } -// } -// super.finalize() -// } //} // +//object LargeByteBuffer { // -//object LargeByteBuffer extends Logging { -// -// private val noopDisposeFunction = new BufferCleaner() { -// protected def doClean(buffer: LargeByteBuffer) { -// buffer.free(invokeCleaner = false) -// } -// } -// -// val enableExpensiveAssert = false -// private val EMPTY_BYTEBUFFER = ByteBuffer.allocate(0) -// val EMPTY_BUFFER = new LargeByteBuffer(ArrayBuffer( -// new HeapByteBufferContainer(EMPTY_BYTEBUFFER, false)), false, false) -// // Do not allow anyone else to override cleaner -// EMPTY_BUFFER.overrideCleaner(noopDisposeFunction, allowOverride = false) -// -// // 8K sufficient ? -// private val TEMP_ARRAY_SIZE = 8192 -// -// /** -// * Create a LargeByteBuffer of specified size which is split across -// * ByteBuffer's of size DiskStore.MAX_BLOCK_SIZE and backed by in memory -// * ByteBuffer -// * -// */ -// def allocateMemoryBuffer(totalSize: Long, ioConf: IOConfig): LargeByteBuffer = { -// if (0 == totalSize) { -// return EMPTY_BUFFER -// } -// -// assert (totalSize > 0) -// -// val blockSize = ioConf.getMaxBlockSize(BufferType.MEMORY) -// val numBlocks = ioConf.numBlocks(BufferType.MEMORY, totalSize) -// val lastBlockSize = ioConf.lastBlockSize(BufferType.MEMORY, totalSize) -// -// assert (lastBlockSize > 0) -// -// val bufferArray = { -// val arr = new ArrayBuffer[ByteBufferContainer](numBlocks) -// for (index <- 0 until numBlocks - 1) { -// val buff = ByteBuffer.allocate(blockSize) -// // buff.clear() -// arr += new HeapByteBufferContainer(buff, true) -// } -// arr += new HeapByteBufferContainer(ByteBuffer.allocate(lastBlockSize), true) -// assert (arr.length == numBlocks) -// arr -// } -// -// new LargeByteBuffer(bufferArray, false, false) -// } -// -// /** -// * Create a LargeByteBuffer of specified size which is split across -// * ByteBuffer's of size DiskStore.MAX_BLOCK_SIZE and backed by on disk -// * -// */ -// private def allocateDiskBuffer(totalSize: Long, -// blockManager: BlockManager): LargeByteBuffer = { -// if (0 == totalSize) { -// return EMPTY_BUFFER -// } -// -// assert (totalSize > 0) -// -// // Create a file of the specified size. -// val file = blockManager.diskBlockManager.createTempBlock()._2 -// val raf = new RandomAccessFile(file, "rw") -// try { -// raf.setLength(totalSize) -// } finally { -// raf.close() -// } -// -// readWriteDiskSegment(new FileSegment(file, 0, totalSize), -// ephemeralDiskBacked = true, blockManager.ioConf) -// } -// -// // The returned buffer takes up ownership of the underlying buffers -// // (including dispos'ing that when done) -// def fromBuffers(buffers: ByteBuffer*): LargeByteBuffer = { -// val nonEmpty = buffers.filter(_.hasRemaining) -// -// // cleanup the empty buffers -// buffers.filter(! _.hasRemaining).foreach(b => BlockManager.dispose(b)) -// -// -// if (nonEmpty.isEmpty) { -// return EMPTY_BUFFER -// } -// -// // slice so that offsets match our requirement -// new LargeByteBuffer(new ArrayBuffer() ++ nonEmpty.map(b => -// new HeapByteBufferContainer(b.slice(), true)), false, false) +// def asLargeByteBuffer(byteBuffer: ByteBuffer): LargeByteBuffer = { +// new WrappedLargeByteBuffer(Array(byteBuffer)) // } // -// def fromByteArrays(byteArrays: Array[Byte]*): LargeByteBuffer = { -// // only non empty arrays -// val arrays = byteArrays.filter(_.length > 0) -// if (0 == arrays.length) return EMPTY_BUFFER -// -// new LargeByteBuffer(new ArrayBuffer() ++ arrays.map(arr => -// new HeapByteBufferContainer(ByteBuffer.wrap(arr), true)), false, false) -// } -// -// def fromLargeByteBuffers(canDispose: Boolean, inputBuffers: LargeByteBuffer*): LargeByteBuffer = { -// -// if (inputBuffers.isEmpty) return EMPTY_BUFFER -// -// if (! inputBuffers.exists(_.hasRemaining())) { -// if (canDispose) inputBuffers.map(_.free()) -// return EMPTY_BUFFER -// } -// -// // release all temp resources acquired -// inputBuffers.foreach(buff => buff.releasePendingContainers()) -// // free current container if acquired. -// inputBuffers.foreach(buff => if (buff.currentContainerIndex < buff.containers.length) { -// buff.containers(buff.currentContainerIndex).release() -// }) -// // inputBuffers.foreach(b => b.doReleaseAll()) -// -// -// // Dispose of any empty buffers -// if (canDispose) inputBuffers.filter(! _.hasRemaining()).foreach(_.free()) -// -// // Find all containers we need. -// val buffers = inputBuffers.filter(_.hasRemaining()).map(b => b.createSlice(b.remaining())) -// -// val containers = buffers.flatMap(_.containers) -// assert (! containers.isEmpty) -// // The in order containers of "buffers" seq constitute the required return value -// val retval = new LargeByteBuffer(new ArrayBuffer() ++ containers, -// // if you cant dispose, then we dont own the buffers : in which case, need duplicate -// ! canDispose, inputBuffers.exists(_.ephemeralDiskBacked)) -// -// if (canDispose) { -// // override dispose of all other buffers. -// val disposeFunctions = inputBuffers.map { -// buffer => { -// (buffer, buffer.overrideCleaner(noopDisposeFunction)) -// } -// } -// -// val cleaner = retval.getCleaner() -// val newCleaner = new BufferCleaner { -// protected def doClean(buffer: LargeByteBuffer) { -// -// assert (retval == buffer) -// // default cleaner. -// cleaner.clean(retval) -// // not required, since we are within clean anyway. -// // retval.free(invokeCleaner = false) -// -// // retval.doDispose(needRelease = true) -// -// // This might actually call dispose twice on some (initially) empty buffers, -// // which is fine since we now guard against that. -// disposeFunctions.foreach(v => v._2.clean(v._1)) -// // Call the free method too : so that buffers are marked free ... -// disposeFunctions.foreach(v => v._1.free(invokeCleaner = false)) -// } -// } -// -// val prev = retval.overrideCleaner(newCleaner) -// assert (prev == cleaner) -// } -// -// retval +// def asLargeByteBuffer(bytes: Array[Byte]): LargeByteBuffer = { +// new WrappedLargeByteBuffer(Array(ByteBuffer.wrap(bytes))) // } // -// private def checkOffsets(arr: Array[Byte], offset: Int, size: Int) { -// if (arr == null) { -// throw new NullPointerException -// } else if (offset < 0 || size < 0 || offset + size > arr.length) { -// throw new IndexOutOfBoundsException -// } -// } // -// def allocateTransientBuffer(size: Long, blockManager: BlockManager) = { -// if (size <= blockManager.ioConf.maxInMemSize) { -// LargeByteBuffer.allocateMemoryBuffer(size, blockManager.ioConf) -// } else { -// LargeByteBuffer.allocateDiskBuffer(size, blockManager) -// } +// def allocateOnHeap(size: Long, maxChunk: Int): LargeByteBuffer = { +// val buffer = ChainedBuffer.withInitialSize(maxChunk, size) +// new ChainedLargeByteBuffer(buffer) // } // -// def readFromDiskSegment(segment: FileSegment, ioConf: IOConfig, -// ephemeralDiskBacked: Boolean): LargeByteBuffer = { -// // Split the block into multiple of BlockStore.maxBlockSize -// val segmentSize = segment.length -// val blockSize = ioConf.getMaxBlockSize(BufferType.DISK).asInstanceOf[Long] -// val numBlocks = ioConf.numBlocks(BufferType.DISK, segmentSize) -// val lastBlockSize = ioConf.lastBlockSize(BufferType.DISK, segmentSize) -// -// val buffers = new ArrayBuffer[ByteBufferContainer](numBlocks) -// -// for (index <- 0 until numBlocks - 1) { -// buffers += new ReadOnlyFileContainer(new FileSegment(segment.file, -// segment.offset + index * blockSize, blockSize), ioConf) -// } -// -// // Last block -// buffers += new ReadOnlyFileContainer(new FileSegment(segment.file, -// segment.offset + (numBlocks - 1) * blockSize, lastBlockSize), ioConf) -// -// new LargeByteBuffer(buffers, false, ephemeralDiskBacked) +// def mapFile( +// channel: FileChannel, +// mode: MapMode, +// offset: Long, +// length: Long, +// maxChunk: Int = Integer.MAX_VALUE - 1e6.toInt +// ): LargeByteBuffer = { +// val offsets = new ArrayBuffer[Long]() +// var curOffset = offset +// val end = offset + length +// while (curOffset < end) { +// offsets += curOffset +// val length = math.min(end - curOffset, maxChunk) +// curOffset += length +// } +// offsets += end +// val chunks = new Array[ByteBuffer](offsets.size - 1) +// (0 until offsets.size - 1).foreach{idx => +// chunks(idx) = channel.map(mode, offsets(idx), offsets(idx + 1) - offsets(idx)) +// } +// new WrappedLargeByteBuffer(chunks) // } +//} // -// def readWriteDiskSegment(segment: FileSegment, ephemeralDiskBacked: Boolean, -// ioConf: IOConfig): LargeByteBuffer = { -// -// // Split the block into multiple of BlockStore.maxBlockSize -// val segmentSize = segment.length -// val blockSize = ioConf.getMaxBlockSize(BufferType.DISK).asInstanceOf[Long] -// val numBlocks = ioConf.numBlocks(BufferType.DISK, segmentSize) -// val lastBlockSize = ioConf.lastBlockSize(BufferType.DISK, segmentSize) -// -// logInfo("readWriteDiskSegment = " + segment + ", numBlocks = " + numBlocks + -// ", lastBlockSize = " + lastBlockSize) -// val buffers = new ArrayBuffer[ByteBufferContainer](numBlocks) -// -// for (index <- 0 until numBlocks - 1) { -// buffers += new ReadWriteFileContainer(new FileSegment(segment.file, -// segment.offset + index * blockSize, blockSize), ephemeralDiskBacked, null) -// } -// -// // Last block -// buffers += new ReadWriteFileContainer(new FileSegment(segment.file, -// segment.offset + (numBlocks - 1) * blockSize, lastBlockSize), ephemeralDiskBacked, null) // -// new LargeByteBuffer(buffers, false, ephemeralDiskBacked) -// } -//} +//// +/////** +//// * This is a variant of ByteBuffer to be used internally in spark, which is not limited to 2G +//// * which ByteBuffers are limited to. +//// * Externally, it exposes all the api which java.nio.ByteBuffer exposes. +//// * Internally, it maintains a sequence of Containers which manage the ByteBuffer data. +//// * Not all the data might be loaded into memory (like disk or tachyon data) - so actual +//// * memory footprint - heap and vm could be much lower than capacity. +//// * +//// * TODO: Currently we are slightly fast and loose in terms of concurrent modifications to this +//// * buffer, maybe revisit this later ? Note: this is not much different from earlier though ! +//// * +//// * TODO: Explore if (at all) we can leverage zero copy transfers. The issue (currently) is that this +//// * will require the file to be kept open (repeatedly opening/closing file is not good +//// * for each transfer) and this has an impact on ulimit. Not to mention writing of mmap'ed buffer is +//// * pretty quick (it is the first failover in case direct transfer is not possible in file zero copy) +//// * +//// * TODO: After redesign to containers, we got rid of parent containers to free - the side effect is +//// * that if there are direct ByteBuffers, we are not handling explicit cleanup of those in some +//// * cases (when we duplicate/slice them). Currently spark does not need this, but might in future +//// * so relook at it later. +//// */ +////// We should make this constructor private: but for now, +////// leaving it public since TachyonStore needs it +////class LargeByteBuffer private[spark](private val inputContainers: ArrayBuffer[ByteBufferContainer], +//// private val needDuplicate: Boolean, val ephemeralDiskBacked: Boolean) extends Logging { +//// +//// // TODO: TEMP code: to flush out potential resource leaks. REMOVE ME +//// private val allocateLocationThrowable: Throwable = { +//// if (inputContainers.exists(c => c.requireRelease() || c.requireFree())) { +//// new Throwable("blockId = " + BlockManager.getLookupBlockId) +//// } else { +//// null +//// } +//// } +//// private var disposeLocationThrowable: Throwable = null +//// +//// @volatile private var allowCleanerOverride = true +//// @volatile private var cleaner: BufferCleaner = new BufferCleaner { +//// override def doClean(buffer: LargeByteBuffer) = { +//// assert (LargeByteBuffer.this == buffer) +//// doDispose(needRelease = false) +//// } +//// } +//// +//// // should not be empty +//// assert (null != inputContainers && ! inputContainers.isEmpty) +//// // should not have any null's +//// assert (inputContainers.find(_ == null).isEmpty) +//// +//// // println("Num containers = " + inputContainers.size) +//// +//// // Position, limit and capacity relevant over the engire LargeByteBuffer +//// @volatile private var globalPosition = 0L +//// @volatile private var globalLimit = 0L +//// @volatile private var currentContainerIndex = 0 +//// +//// // The buffers in which the actual data is held. +//// private var containers: Array[ByteBufferContainer] = null +//// +//// // aggregate capacities of the individual buffers. +//// // bufferPositionStart(0) will be capacity of 1st buffer, bufferPositionStart(1) will be +//// // sum of capacity of 0th and 1st block buffer +//// private var bufferPositionStart: Array[Long] = null +//// +//// // Contains the indices of a containers which requires release before subsequent invocation of +//// // read/write should be serviced. This is required since current read/write might have moved the +//// // position but since we are returning bytebuffers which depend on the validity of the existing +//// // bytebuffer, we cant release them yet. +//// private var needReleaseIndices = new HashSet[Int]() +//// +//// private val readable = ! inputContainers.exists(! _.isReadable) +//// private val writable = ! inputContainers.exists(! _.isWritable) +//// +//// +//// // initialize +//// @volatile private var globalCapacity = { +//// +//// // Ensure that there are no empty buffers : messes up with our code : unless it +//// // is a single buffer (for empty buffer for marker case) +//// assert (inputContainers.find(0 == _.capacity()).isEmpty || 1 == inputContainers.length) +//// +//// containers = { +//// if (needDuplicate) inputContainers.map(_.duplicate()).toArray else inputContainers.toArray +//// } +//// containers.foreach(_.validate()) +//// +//// def initializeBufferPositionStart(arr: Array[ByteBufferContainer]) { +//// val buff = new ArrayBuffer[Long](arr.length + 1) +//// buff += 0L +//// +//// buff ++= arr.map(_.capacity().asInstanceOf[Long]).scanLeft(0L)(_ + _).slice(1, arr.length + 1) +//// assert (buff.length == arr.length + 1) +//// bufferPositionStart = buff.toArray +//// } +//// +//// initializeBufferPositionStart(containers) +//// +//// // remove references from inputBuffers +//// inputContainers.clear() +//// +//// globalLimit = bufferPositionStart(containers.length) +//// globalPosition = 0L +//// currentContainerIndex = 0 +//// +//// assert (globalLimit == containers.map(_.capacity().asInstanceOf[Long]).sum) +//// +//// globalLimit +//// } +//// +//// final def position(): Long = globalPosition +//// +//// final def limit(): Long = globalLimit +//// +//// final def capacity(): Long = globalCapacity +//// +//// final def limit(newLimit: Long) { +//// if ((newLimit > capacity()) || (newLimit < 0)) { +//// throw new IllegalArgumentException("newLimit = " + newLimit + ", capacity = " + capacity()) +//// } +//// +//// globalLimit = newLimit +//// if (position() > newLimit) position(newLimit) +//// } +//// +//// def skip(skipBy: Long) = position(position() + skipBy) +//// +//// private def releasePendingContainers() { +//// if (! needReleaseIndices.isEmpty) { +//// val iter = needReleaseIndices.iterator +//// while (iter.hasNext) { +//// val index = iter.next() +//// assert (index >= 0 && index < containers.length) +//// // It is possible to move from one container to next before the previous +//// // container was acquired. For example, get forcing move to next container +//// // since current was exhausted immediatelly followed by a position() +//// // so the container we moved to was never acquired. +//// +//// // assert (containers(index).isAcquired) +//// // will this always be satisfied ? +//// // assert (index != currentContainerIndex) +//// if (containers(index).isAcquired) containers(index).release() +//// } +//// needReleaseIndices.clear() +//// } +//// } +//// +//// private def toNewContainer(newIndex: Int) { +//// if (newIndex != currentContainerIndex && currentContainerIndex < containers.length) { +//// +//// assert (currentContainerIndex >= 0) +//// needReleaseIndices += currentContainerIndex +//// } +//// currentContainerIndex = newIndex +//// } +//// +//// // expensive method, sigh ... optimize it later ? +//// final def position(newPosition: Long) { +//// +//// if ((newPosition > globalLimit) || (newPosition < 0)) throw new IllegalArgumentException() +//// +//// if (currentContainerIndex < bufferPositionStart.length - 1 && +//// newPosition >= bufferPositionStart(currentContainerIndex) && +//// newPosition < bufferPositionStart(currentContainerIndex + 1)) { +//// // Same buffer - easy method ... +//// globalPosition = newPosition +//// // Changed position - free previously returned buffers. +//// releasePendingContainers() +//// return +//// } +//// +//// // Find appropriate currentContainerIndex +//// // Since bufferPositionStart is sorted, can be replaced with binary search if required. +//// // For now, not in the perf critical path since buffers size is very low typically. +//// var index = 0 +//// val cLen = containers.length +//// while (index < cLen) { +//// if (newPosition >= bufferPositionStart(index) && +//// newPosition < bufferPositionStart(index + 1)) { +//// globalPosition = newPosition +//// toNewContainer(index) +//// // Changed position - free earlier and previously returned buffers. +//// releasePendingContainers() +//// return +//// } +//// index += 1 +//// } +//// +//// if (newPosition == globalLimit && newPosition == bufferPositionStart(cLen)) { +//// // boundary. +//// globalPosition = newPosition +//// toNewContainer(cLen) +//// // Changed position - free earlier and previously returned buffers. +//// releasePendingContainers() +//// return +//// } +//// +//// assert (assertion = false, "Unexpected to come here .... newPosition = " + newPosition + +//// ", bufferPositionStart = " + bufferPositionStart.mkString("[", ", ", "]")) +//// } +//// +//// +//// /** +//// * Clears this buffer. The position is set to zero, the limit is set to +//// * the capacity, and the mark is discarded. +//// * +//// *

Invoke this method before using a sequence of channel-read or +//// * put operations to fill this buffer. +//// * +//// *

This method does not actually erase the data in the buffer, but it +//// * is named as if it did because it will most often be used in situations +//// * in which that might as well be the case.

+//// */ +//// final def clear() { +//// // if (0 == globalCapacity) return +//// +//// needReleaseIndices += 0 +//// globalPosition = 0L +//// toNewContainer(0) +//// globalLimit = globalCapacity +//// +//// // Now free all pending containers +//// releasePendingContainers() +//// } +//// +//// /** +//// * Flips this buffer. The limit is set to the current position and then +//// * the position is set to zero. If the mark is defined then it is +//// * discarded. +//// * +//// *

After a sequence of channel-read or put operations, invoke +//// * this method to prepare for a sequence of channel-write or relative +//// * get operations. +//// */ +//// final def flip() { +//// needReleaseIndices += 0 +//// globalLimit = globalPosition +//// globalPosition = 0L +//// toNewContainer(0) +//// +//// // Now free all pending containers +//// releasePendingContainers() +//// } +//// +//// /** +//// * Rewinds this buffer. The position is set to zero and the mark is +//// * discarded. +//// * +//// *

Invoke this method before a sequence of channel-write or get +//// * operations, assuming that the limit has already been set +//// * appropriately. +//// */ +//// final def rewind() { +//// needReleaseIndices += 0 +//// globalPosition = 0L +//// toNewContainer(0) +//// +//// // Now free all pending containers +//// releasePendingContainers() +//// } +//// +//// /** +//// * Returns the number of elements between the current position and the +//// * limit.

+//// * +//// * @return The number of elements remaining in this buffer +//// */ +//// final def remaining(): Long = { +//// globalLimit - globalPosition +//// } +//// +//// /** +//// * Tells whether there are any elements between the current position and +//// * the limit.

+//// * +//// * @return true if, and only if, there is at least one element +//// * remaining in this buffer +//// */ +//// final def hasRemaining() = { +//// globalPosition < globalLimit +//// } +//// +//// // private def currentBuffer(): ByteBuffer = buffers(currentContainerIndex) +//// +//// // number of bytes remaining in currently active underlying buffer +//// private def currentRemaining(): Int = { +//// if (hasRemaining()) { +//// // validate currentContainerIndex is valid +//// assert (globalPosition >= bufferPositionStart(currentContainerIndex) && +//// globalPosition < bufferPositionStart(currentContainerIndex + 1), +//// "globalPosition = " + globalPosition + +//// ", currentContainerIndex = " + currentContainerIndex + +//// ", bufferPositionStart = " + bufferPositionStart.mkString("[", ", ", " ]")) +//// +//// currentRemaining0(currentContainerIndex) +//// } else 0 +//// } +//// +//// // Without any validation : required when we are bumping the index (when validation will fail) ... +//// private def currentRemaining0(which: Int): Int = { +//// // currentBuffer().remaining() +//// math.max(0, math.min(bufferPositionStart(which + 1), +//// globalLimit) - globalPosition).asInstanceOf[Int] +//// } +//// +//// // Set the approppriate position/limit for the current underlying buffer to mirror our +//// // the LargeByteBuffer's state. +//// private def fetchCurrentBuffer(): ByteBuffer = { +//// releasePendingContainers() +//// +//// assert (currentContainerIndex < containers.length) +//// +//// val container = containers(currentContainerIndex) +//// if (! container.isAcquired) { +//// container.acquire() +//// } +//// +//// assert (container.isAcquired) +//// if (LargeByteBuffer.enableExpensiveAssert) { +//// assert (! containers.exists( b => (b ne container) && b.isAcquired)) +//// } +//// +//// assert (currentContainerIndex < bufferPositionStart.length && +//// globalPosition < bufferPositionStart(currentContainerIndex + 1), +//// "currentContainerIndex = " + currentContainerIndex + ", bufferPositionStart = " + +//// bufferPositionStart.mkString("[", ", ", "]") + ", this = " + this) +//// +//// val buffPosition = (globalPosition - bufferPositionStart(currentContainerIndex)). +//// asInstanceOf[Int] +//// +//// val buffer = container.getByteBuffer +//// buffer.position(buffPosition) +//// val diff = buffer.capacity - buffPosition +//// val left = remaining() +//// if (diff <= left) { +//// buffer.limit(buffer.capacity()) +//// } else { +//// // Can happen if limit() was called. +//// buffer.limit(buffPosition + left.asInstanceOf[Int]) +//// } +//// +//// buffer +//// } +//// +//// // To be used ONLY to test in suites. +//// private[spark] def fetchCurrentBufferForTesting(): ByteBuffer = { +//// if ("1" != System.getProperty("SPARK_TESTING")) { +//// throw new IllegalStateException("This method is to be used ONLY within spark test suites") +//// } +//// +//// fetchCurrentBuffer() +//// } +//// +//// // Expects that the invoker has ensured that this can be safely invoked. +//// // That is, it wont be invoked when the loop wont terminate. +//// private def toNonEmptyBuffer() { +//// +//// if (! hasRemaining()) { +//// var newIndex = currentContainerIndex +//// // Ensure we are in the right block or not. +//// while (newIndex < containers.length && globalPosition >= bufferPositionStart(newIndex + 1)) { +//// newIndex += 1 +//// } +//// toNewContainer(newIndex) +//// // Do not do this - since we might not yet have consumed the buffer which caused EOF right now +//// /* +//// // Add last one also, and release it too - since we are at the end of the buffer with nothing +//// // more pending. +//// if (newIndex >= 0 && currentContainerIndex < containers.length) { +//// needReleaseIndices += newIndex +//// } +//// */ +//// assert (currentContainerIndex >= 0) +//// // releasePendingContainers() +//// return +//// } +//// +//// var index = currentContainerIndex +//// while (0 == currentRemaining0(index) && index < containers.length) { +//// index += 1 +//// } +//// assert (currentContainerIndex < containers.length) +//// toNewContainer(index) +//// assert (0 != currentRemaining()) +//// } +//// +//// private def assertPreconditions(containerIndex: Int) { +//// assert (globalPosition >= bufferPositionStart(containerIndex), +//// "globalPosition = " + globalPosition + ", containerIndex = " + containerIndex + +//// ", bufferPositionStart = " + bufferPositionStart.mkString("[", ", ", " ]")) +//// assert (globalPosition < bufferPositionStart(containerIndex + 1), +//// "globalPosition = " + globalPosition + ", containerIndex = " + containerIndex + +//// ", bufferPositionStart = " + bufferPositionStart.mkString("[", ", ", " ]")) +//// +//// assert (globalLimit <= globalCapacity) +//// assert (containerIndex < containers.length) +//// } +//// +//// +//// /** +//// * Attempts to return a ByteBuffer of the requested size. +//// * It is possible to return a buffer of size smaller than requested +//// * even though hasRemaining == true +//// * +//// * On return, position would have been moved 'ahead' by the size of the buffer returned : +//// * that is, we treat that the returned buffer has been already 'read' from this LargeByteBuffer +//// * +//// * +//// * This is used to primarily retrieve content of this buffer to expose via ByteBuffer +//// * to some other api which is deemed too cumbersome to move to LargeByteBuffer (like the +//// * chunked sending of contents via ConnectionManager) Note that the lifecycle of the ByteBuffer +//// * returned is inherently tied to the state of this LargeByteBuffer. For example,if the underlying +//// * container is a disk backed container, and we make subsequent calls to get(), the returned +//// * ByteBuffer can be dispose'ed off +//// * +//// * @param maxChunkSize Max size of the ByteBuffer to retrieve. +//// * @return +//// */ +//// +//// private def fetchBufferOfSize(maxChunkSize: Int): ByteBuffer = { +//// fetchBufferOfSizeImpl(maxChunkSize, canReleaseContainers = true) +//// } +//// +//// private def fetchBufferOfSizeImpl(maxChunkSize: Int, +//// canReleaseContainers: Boolean): ByteBuffer = { +//// if (canReleaseContainers) releasePendingContainers() +//// assert (maxChunkSize > 0) +//// +//// // not checking for degenerate case of maxChunkSize == 0 +//// if (globalPosition >= globalLimit) { +//// // throw exception +//// throw new BufferUnderflowException() +//// } +//// +//// // Check preconditions : disable these later, since they might be expensive to +//// // evaluate for every IO op +//// assertPreconditions(currentContainerIndex) +//// +//// val currentBufferRemaining = currentRemaining() +//// +//// assert (currentBufferRemaining > 0) +//// +//// val size = math.min(currentBufferRemaining, maxChunkSize) +//// +//// val newBuffer = if (currentBufferRemaining > maxChunkSize) { +//// val currentBuffer = fetchCurrentBuffer() +//// val buff = ByteBufferContainer.createSlice(currentBuffer, +//// currentBuffer.position(), maxChunkSize) +//// assert (buff.remaining() == maxChunkSize) +//// buff +//// } else { +//// val currentBuffer = fetchCurrentBuffer() +//// val buff = currentBuffer.slice() +//// assert (buff.remaining() == currentBufferRemaining) +//// buff +//// } +//// +//// assert (size == newBuffer.remaining()) +//// assert (0 == newBuffer.position()) +//// assert (size == newBuffer.limit()) +//// assert (newBuffer.capacity() == newBuffer.limit()) +//// +//// globalPosition += newBuffer.remaining +//// toNonEmptyBuffer() +//// +//// newBuffer +//// } +//// +//// // Can we service the read/write from the currently active (underlying) bytebuffer or not. +//// // For almost all cases, this will return true allowing us to optimize away the more expensive +//// // computations. +//// private def localReadWritePossible(size: Int) = +//// size >= 0 && globalPosition + size <= bufferPositionStart(currentContainerIndex + 1) +//// +//// +//// def getLong(): Long = { +//// assert (readable) +//// releasePendingContainers() +//// +//// if (remaining() < 8) throw new BufferUnderflowException +//// +//// if (localReadWritePossible(8)) { +//// val buff = fetchCurrentBuffer() +//// assert (buff.remaining() >= 8) +//// val retval = buff.getLong +//// globalPosition += 8 +//// toNonEmptyBuffer() +//// return retval +//// } +//// +//// val buff = readFully(8) +//// buff.getLong +//// } +//// +//// def getInt(): Int = { +//// assert (readable) +//// releasePendingContainers() +//// +//// if (remaining() < 4) throw new BufferUnderflowException +//// +//// if (localReadWritePossible(4)) { +//// val buff = fetchCurrentBuffer() +//// assert (buff.remaining() >= 4) +//// val retval = buff.getInt +//// globalPosition += 4 +//// toNonEmptyBuffer() +//// return retval +//// } +//// +//// val buff = readFully(4) +//// buff.getInt +//// } +//// +//// def getChar(): Char = { +//// assert (readable) +//// releasePendingContainers() +//// +//// if (remaining() < 2) throw new BufferUnderflowException +//// +//// if (localReadWritePossible(2)) { +//// val buff = fetchCurrentBuffer() +//// assert (buff.remaining() >= 2) +//// val retval = buff.getChar +//// globalPosition += 2 +//// toNonEmptyBuffer() +//// return retval +//// } +//// +//// // if slice is becoming too expensive, revisit this ... +//// val buff = readFully(2) +//// buff.getChar +//// } +//// +//// def get(): Byte = { +//// assert (readable) +//// releasePendingContainers() +//// +//// if (! hasRemaining()) throw new BufferUnderflowException +//// +//// // If we have remaining bytes, previous invocations MUST have ensured that we are at +//// // a buffer which has data to be read. +//// assert (localReadWritePossible(1)) +//// +//// val buff = fetchCurrentBuffer() +//// assert (buff.remaining() >= 1, "buff.remaining = " + buff.remaining()) +//// val retval = buff.get() +//// globalPosition += 1 +//// toNonEmptyBuffer() +//// +//// retval +//// } +//// +//// def get(arr: Array[Byte], offset: Int, size: Int): Int = { +//// assert (readable) +//// releasePendingContainers() +//// +//// LargeByteBuffer.checkOffsets(arr, offset, size) +//// +//// // kyro depends on this it seems ? +//// // assert (size > 0) +//// if (0 == size) return 0 +//// +//// if (! hasRemaining()) return -1 +//// +//// if (localReadWritePossible(size)) { +//// val buff = fetchCurrentBuffer() +//// assert (buff.remaining() >= size) +//// buff.get(arr, offset, size) +//// globalPosition += size +//// toNonEmptyBuffer() +//// return size +//// } +//// +//// var remainingSize = math.min(size, remaining()).asInstanceOf[Int] +//// var currentOffset = offset +//// +//// while (remainingSize > 0) { +//// val buff = fetchBufferOfSize(remainingSize) +//// val toCopy = math.min(buff.remaining(), remainingSize) +//// +//// buff.get(arr, currentOffset, toCopy) +//// currentOffset += toCopy +//// remainingSize -= toCopy +//// } +//// +//// currentOffset - offset +//// } +//// +//// +//// private def createSlice(size: Long): LargeByteBuffer = { +//// +//// releasePendingContainers() +//// +//// if (remaining() < size) { +//// // logInfo("createSlice. remaining = " + remaining() + ", size " + size + ", this = " + this) +//// throw new BufferOverflowException +//// } +//// +//// // kyro depends on this it seems ? +//// // assert (size > 0) +//// if (0 == size) return LargeByteBuffer.EMPTY_BUFFER +//// +//// val arr = new ArrayBuffer[ByteBufferContainer](2) +//// var totalLeft = size +//// +//// // assert (currentRemaining() < totalLeft || totalLeft != size || currentAsByteBuffer) +//// +//// var containerIndex = currentContainerIndex +//// while (totalLeft > 0 && hasRemaining()) { +//// assertPreconditions(containerIndex) +//// val container = containers(containerIndex) +//// val currentLeft = currentRemaining0(containerIndex) +//// +//// assert (globalPosition + currentLeft <= globalLimit) +//// assert (globalPosition >= bufferPositionStart(containerIndex) && +//// (globalPosition < bufferPositionStart(containerIndex + 1))) +//// +//// val from = (globalPosition - bufferPositionStart(containerIndex)).asInstanceOf[Int] +//// val sliceSize = math.min(totalLeft, currentLeft) +//// assert (from >= 0) +//// assert (sliceSize > 0 && sliceSize <= Int.MaxValue) +//// +//// val slice = container.createSlice(from, sliceSize.asInstanceOf[Int]) +//// arr += slice +//// +//// globalPosition += sliceSize +//// totalLeft -= sliceSize +//// if (currentLeft == sliceSize) containerIndex += 1 +//// } +//// +//// // Using toNonEmptyBuffer instead of directly moving to next here so that +//// // other checks can be performed there. +//// toNonEmptyBuffer() +//// // force cleanup - this is fine since we are not using the buffers directly +//// // which are actively needed (the returned value is on containers which can +//// // recreate) +//// releasePendingContainers() +//// // free current container if acquired. +//// if (currentContainerIndex < containers.length) { +//// containers(currentContainerIndex).release() +//// } +//// assert (currentContainerIndex == containerIndex) +//// +//// val retval = new LargeByteBuffer(arr, false, ephemeralDiskBacked) +//// retval.overrideCleaner(LargeByteBuffer.noopDisposeFunction) +//// retval +//// } +//// +//// // Get a composite sequence of ByteBuffer which might straddle one or more underlying buffers +//// // This is to be used only for writes : and ensures that writes are done into the appropriate +//// // underlying bytebuffers. +//// def getCompositeWriteBuffer(size: Long): LargeByteBuffer = { +//// assert(writable) +//// assert(size >= 0) +//// +//// createSlice(size) +//// } +//// +//// // get a buffer which is of the specified size and contains data from the underlying buffers +//// // Note, the actual data might be spread across the underlying buffers. +//// // This MUST BE used only for specific usecases like getInt, etc. Not for bulk copy ! +//// private def readFully(size: Int): ByteBuffer = { +//// assert (readable) +//// +//// if (remaining() < size) { +//// // throw exception +//// throw new BufferUnderflowException() +//// } +//// +//// // kyro depends on this it seems ? +//// // assert (size > 0) +//// if (0 == size) return LargeByteBuffer.EMPTY_BYTEBUFFER +//// +//// // Expected to be handled elsewhere. +//// assert (! localReadWritePossible(size)) +//// +//// val localBuff = { +//// val buff = fetchBufferOfSize(size) +//// // assert(buff.remaining() <= size) +//// // if (buff.remaining() == size) return buff +//// assert(buff.remaining() < size) +//// ByteBuffer.allocate(size).put(buff) +//// } +//// +//// // assert (localBuff.hasRemaining) +//// +//// while (localBuff.hasRemaining) { +//// val buff = fetchBufferOfSize(localBuff.remaining()) +//// localBuff.put(buff) +//// } +//// +//// localBuff.flip() +//// localBuff +//// } +//// +//// +//// +//// def put(b: Byte) { +//// assert (writable) +//// if (remaining() < 1) { +//// // logInfo("put byte. remaining = " + remaining() + ", this = " + this) +//// throw new BufferOverflowException +//// } +//// +//// assert (currentRemaining() > 0) +//// +//// fetchCurrentBuffer().put(b) +//// globalPosition += 1 +//// // Check to need to bump the index ? +//// toNonEmptyBuffer() +//// } +//// +//// +//// def put(buffer: ByteBuffer) { +//// assert (writable) +//// if (remaining() < buffer.remaining()) { +//// throw new BufferOverflowException +//// } +//// +//// val bufferRemaining = buffer.remaining() +//// if (localReadWritePossible(bufferRemaining)) { +//// +//// assert (currentRemaining() >= bufferRemaining) +//// +//// fetchCurrentBuffer().put(buffer) +//// +//// globalPosition += bufferRemaining +//// toNonEmptyBuffer() +//// return +//// } +//// +//// while (buffer.hasRemaining) { +//// val currentBufferRemaining = currentRemaining() +//// val bufferRemaining = buffer.remaining() +//// +//// if (currentBufferRemaining >= bufferRemaining) { +//// fetchCurrentBuffer().put(buffer) +//// globalPosition += bufferRemaining +//// } else { +//// // Split across buffers. +//// val currentBuffer = fetchCurrentBuffer() +//// assert (currentBuffer.remaining() >= currentBufferRemaining) +//// val sliced = ByteBufferContainer.createSlice(buffer, buffer.position(), +//// currentBufferRemaining) +//// assert (sliced.remaining() == currentBufferRemaining) +//// currentBuffer.put(sliced) +//// // move buffer pos +//// buffer.position(buffer.position() + currentBufferRemaining) +//// +//// globalPosition += currentBufferRemaining +//// } +//// toNonEmptyBuffer() +//// } +//// +//// assert (! hasRemaining() || currentRemaining() > 0) +//// } +//// +//// def put(other: LargeByteBuffer) { +//// assert (writable) +//// if (this.remaining() < other.remaining()) { +//// throw new BufferOverflowException +//// } +//// +//// while (other.hasRemaining()) { +//// val buffer = other.fetchBufferOfSize(other.currentRemaining()) +//// this.put(buffer) +//// } +//// } +//// +//// +//// def duplicate(): LargeByteBuffer = { +//// val containersCopy = new ArrayBuffer[ByteBufferContainer](containers.size) +//// // We do a duplicate as part of construction - so avoid double duplicate. +//// // containersCopy ++= containers.map(_.duplicate()) +//// containersCopy ++= containers +//// val retval = new LargeByteBuffer(containersCopy, true, ephemeralDiskBacked) +//// +//// // set limit and position (in that order) ... +//// retval.limit(this.limit()) +//// retval.position(this.position()) +//// +//// // Now release our containers - if any had been acquired +//// releasePendingContainers() +//// +//// retval +//// } +//// +//// +//// /** +//// * 'read' a LargeByteBuffer of size specified and return that. +//// * Position will be incremented by size +//// * +//// * The name might be slightly confusing : rename ? +//// * +//// * @param size Amount of data to be read from this buffer and returned +//// * @return +//// */ +//// def readLargeBuffer(size: Long, partialReadAllowed: Boolean): LargeByteBuffer = { +//// if (! hasRemaining() && ! partialReadAllowed) throw new BufferUnderflowException +//// if (remaining() < size && ! partialReadAllowed) throw new BufferUnderflowException +//// +//// +//// assert (readable) +//// assert (size >= 0) +//// +//// releasePendingContainers() +//// +//// if (0 == size) return LargeByteBuffer.EMPTY_BUFFER +//// +//// createSlice(size) +//// } +//// +//// +//// // This is essentially a workaround to exposing underlying buffers +//// def readFrom(channel: ReadableByteChannel): Long = { +//// +//// assert (writable) +//// releasePendingContainers() +//// +//// // this also allows us to avoid nasty corner cases in the loop. +//// if (! hasRemaining()) { +//// // logInfo("readFrom channel. remaining = " + remaining() + ", this = " + this) +//// throw new BufferOverflowException +//// } +//// +//// var totalBytesRead = 0L +//// +//// while (hasRemaining()) { +//// // read what we can ... +//// val buffer = fetchCurrentBuffer() +//// val bufferRemaining = currentRemaining() +//// val bytesRead = channel.read(buffer) +//// +//// if (bytesRead > 0) { +//// totalBytesRead += bytesRead +//// // bump position too .. +//// globalPosition += bytesRead +//// if (bytesRead >= bufferRemaining) toNonEmptyBuffer() +//// } +//// else if (-1 == bytesRead) { +//// // if we had already read some data in the loop, return that. +//// if (totalBytesRead > 0) return totalBytesRead +//// return -1 +//// } // nothing available to read, retry later. return +//// else if (0 == bytesRead) { +//// return totalBytesRead +//// } +//// +//// // toNonEmptyBuffer() +//// } +//// +//// // Cleanup last buffer ? +//// toNonEmptyBuffer() +//// totalBytesRead +//// } +//// +//// // This is essentially a workaround to exposing underlying buffers +//// def readFrom(inStrm: InputStream): Long = { +//// +//// assert (writable) +//// releasePendingContainers() +//// +//// // this also allows us to avoid nasty corner cases in the loop. +//// // if (! hasRemaining()) throw new BufferOverflowException +//// if (! hasRemaining()) return 0 +//// +//// var totalBytesRead = 0L +//// +//// val buff = new Array[Byte](LargeByteBuffer.TEMP_ARRAY_SIZE) +//// +//// while (hasRemaining()) { +//// // read what we can ... note, since there is no gaurantee that underlying buffer might +//// // expose array() method, we do double copy - from stream to buff and from buff to bytearray. +//// // see if we can optimize this later ... +//// val buffer = fetchCurrentBuffer() +//// val bufferRemaining = buffer.remaining() +//// val max = math.min(buff.length, bufferRemaining) +//// val bytesRead = inStrm.read(buff, 0, max) +//// +//// if (bytesRead > 0) { +//// buffer.put(buff, 0, bytesRead) +//// totalBytesRead += bytesRead +//// // bump position too .. +//// globalPosition += bytesRead +//// // buffer.position(buffer.position + bytesRead) +//// if (bytesRead >= bufferRemaining) toNonEmptyBuffer() +//// } +//// else if (-1 == bytesRead) { +//// // if we had already read some data in the loop, return that. +//// if (totalBytesRead > 0) return totalBytesRead +//// return -1 +//// } // nothing available to read, retry later. return +//// else if (0 == bytesRead) { +//// return totalBytesRead +//// } +//// +//// // toNonEmptyBuffer() +//// } +//// +//// totalBytesRead +//// } +//// +//// // This is essentially a workaround to exposing underlying buffers +//// // Note: very similar to readFrom(InputStream) : not trying anything fancy to reduce +//// // code for performance reasons. +//// def readFrom(inStrm: DataInput): Long = { +//// +//// assert (writable) +//// releasePendingContainers() +//// +//// // this also allows us to avoid nasty corner cases in the loop. +//// // if (! hasRemaining()) throw new BufferOverflowException +//// if (! hasRemaining()) return 0 +//// +//// var totalBytesRead = 0L +//// +//// val buff = new Array[Byte](LargeByteBuffer.TEMP_ARRAY_SIZE) +//// +//// while (hasRemaining()) { +//// // read what we can ... note, since there is no gaurantee that underlying buffer might +//// // expose array() method, we do double copy - from stream to buff and from buff to bytearray. +//// // see if we can optimize this later ... +//// val buffer = fetchCurrentBuffer() +//// val bufferRemaining = buffer.remaining() +//// val max = math.min(buff.length, bufferRemaining) +//// inStrm.readFully(buff, 0, max) +//// val bytesRead = max +//// +//// if (bytesRead > 0) { +//// buffer.put(buff, 0, bytesRead) +//// totalBytesRead += bytesRead +//// // bump position too .. +//// globalPosition += bytesRead +//// // buffer.position(buffer.position() + bytesRead) +//// if (bytesRead >= bufferRemaining) toNonEmptyBuffer() +//// } +//// else if (-1 == bytesRead) { +//// // if we had already read some data in the loop, return that. +//// if (totalBytesRead > 0) return totalBytesRead +//// return -1 +//// } // nothing available to read, retry later. return +//// else if (0 == bytesRead) { +//// return totalBytesRead +//// } +//// +//// // toNonEmptyBuffer() +//// } +//// +//// totalBytesRead +//// } +//// +//// // This is essentially a workaround to exposing underlying buffers +//// // Note: tries to do it efficiently without needing to load everything into memory +//// // (particularly for diskbacked buffers, etc). +//// def writeTo(channel: WritableByteChannel, cleanup: Boolean): Long = { +//// +//// assert (readable) +//// releasePendingContainers() +//// +//// // this also allows us to avoid nasty corner cases in the loop. +//// if (! hasRemaining()) throw new BufferUnderflowException +//// +//// var totalBytesWritten = 0L +//// +//// while (hasRemaining()) { +//// // Write what we can ... +//// val buffer = fetchCurrentBuffer() +//// val bufferRemaining = buffer.remaining() +//// assert (bufferRemaining > 0) +//// val bytesWritten = channel.write(buffer) +//// +//// if (bytesWritten > 0) { +//// totalBytesWritten += bytesWritten +//// // bump position too .. +//// globalPosition += bytesWritten +//// if (bytesWritten >= bufferRemaining) toNonEmptyBuffer() +//// assert (! hasRemaining() || currentRemaining() > 0) +//// } +//// else if (0 == bytesWritten) { +//// return totalBytesWritten +//// } +//// +//// // toNonEmptyBuffer() +//// } +//// +//// assert (! hasRemaining()) +//// if (cleanup) { +//// free() +//// } +//// totalBytesWritten +//// } +//// +//// // This is essentially a workaround to exposing underlying buffers +//// def writeTo(outStrm: OutputStream, cleanup: Boolean): Long = { +//// +//// assert (readable) +//// releasePendingContainers() +//// +//// // this also allows us to avoid nasty corner cases in the loop. +//// if (! hasRemaining()) throw new BufferUnderflowException +//// +//// var totalBytesWritten = 0L +//// val buff = new Array[Byte](LargeByteBuffer.TEMP_ARRAY_SIZE) +//// +//// while (hasRemaining()) { +//// // write what we can ... note, since there is no gaurantee that underlying buffer might +//// // expose array() method, we do double copy - from bytearray to buff and from +//// // buff to outputstream. see if we can optimize this later ... +//// val buffer = fetchCurrentBuffer() +//// val bufferRemaining = buffer.remaining() +//// val size = math.min(bufferRemaining, buff.length) +//// buffer.get(buff, 0, size) +//// outStrm.write(buff, 0, size) +//// +//// totalBytesWritten += size +//// // bump position too .. +//// globalPosition += size +//// +//// if (size >= bufferRemaining) toNonEmptyBuffer() +//// } +//// +//// toNonEmptyBuffer() +//// if (cleanup) { +//// free() +//// } +//// totalBytesWritten +//// } +//// +//// def asInputStream(): InputStream = { +//// new InputStream() { +//// override def read(): Int = { +//// if (! hasRemaining()) return -1 +//// get() +//// } +//// +//// override def read(arr: Array[Byte], off: Int, len: Int): Int = { +//// if (! hasRemaining()) return -1 +//// +//// get(arr, off, len) +//// } +//// +//// override def available(): Int = { +//// // current remaining is what can be read without blocking +//// // anything higher might need disk access/buffer swapping. +//// /* +//// val left = remaining() +//// math.min(left, Int.MaxValue).asInstanceOf[Int] +//// */ +//// currentRemaining() +//// } +//// } +//// } +//// +//// def getCleaner() = cleaner +//// +//// /** +//// * @param cleaner The previous cleaner, so that the caller can chain them if required. +//// * @return +//// */ +//// private[spark] def overrideCleaner(cleaner: BufferCleaner): BufferCleaner = { +//// overrideCleaner(cleaner, allowOverride = true) +//// } +//// +//// private def overrideCleaner(cleaner: BufferCleaner, allowOverride: Boolean): BufferCleaner = { +//// if (! this.allowCleanerOverride) { +//// // allowCleanerOverride = false is used for EMPTY_BUFFER - where we do not allow free +//// return this.cleaner +//// } +//// +//// this.allowCleanerOverride = allowOverride +//// assert (null != cleaner) +//// val prev = this.cleaner +//// this.cleaner = cleaner +//// // logInfo("Overriding " + prev + " with " + this.cleaner) +//// prev +//// } +//// +//// private def doReleaseAll() { +//// for (container <- containers) { +//// container.release() +//// } +//// } +//// +//// def free(invokeCleaner: Boolean = true) { +//// // logInfo("Free on " + this + ", cleaner = " + cleaner) +//// // always invoking release +//// doReleaseAll() +//// +//// if (invokeCleaner) cleaner.clean(this) +//// } +//// +//// private def doDispose(needRelease: Boolean) { +//// +//// if (disposeLocationThrowable ne null) { +//// logError("Already free'ed earlier at : ", disposeLocationThrowable) +//// logError("Current at ", new Throwable) +//// throw new IllegalStateException("Already freed.") +//// } +//// disposeLocationThrowable = new Throwable() +//// +//// // Forcefully cleanup all +//// if (needRelease) doReleaseAll() +//// +//// // Free in a different loop, in case different containers refer to same resource +//// // to release (like file) +//// for (container <- containers) { +//// container.free() +//// } +//// +//// needReleaseIndices.clear() +//// +//// // We should not use this buffer anymore : set the values such that f +//// // we dont ... +//// globalPosition = 0 +//// globalLimit = 0 +//// globalCapacity = 0 +//// } +//// +//// // copy data over ... MUST be used only for cases where array is known to be +//// // small to begin with. slightly risky method due to that assumption +//// def toByteArray(): Array[Byte] = { +//// val positionBackup = position() +//// val size = remaining() +//// if (size > Int.MaxValue) { +//// throw new IllegalStateException( +//// "Attempt to convert LargeByteBuffer to byte array when data held is more than 2G") +//// } +//// +//// val retval = new Array[Byte](size.asInstanceOf[Int]) +//// val readSize = get(retval, 0, retval.length) +//// assert (readSize == retval.length, +//// "readSize = " + readSize + ", retval.length = " + retval.length) +//// +//// position(positionBackup) +//// +//// retval +//// } +//// +//// // copy data over ... MUST be used only for cases where array is known to be +//// // small to begin with. slightly risky method due to that assumption +//// def toByteBuffer(): ByteBuffer = { +//// ByteBuffer.wrap(toByteArray()) +//// } +//// +//// def toInMemoryBuffer(ioConf: IOConfig): LargeByteBuffer = { +//// val retval = LargeByteBuffer.allocateMemoryBuffer(remaining(), ioConf) +//// val currentPosition = position() +//// retval.put(this) +//// position(currentPosition) +//// retval.clear() +//// retval +//// } +//// +//// +//// +//// // This is ONLY used for testing : that too as part of development of this and associated classes +//// // remove before contributing to spark. +//// def hexDump(): String = { +//// if (remaining() * 64 > Int.MaxValue) { +//// throw new UnsupportedOperationException("buffer too large " + remaining()) +//// } +//// +//// val sb = new StringBuilder((remaining() * 2).asInstanceOf[Int]) +//// +//// var perLine = 0 +//// var first = true +//// for (b <- toByteArray()) { +//// perLine += 1 +//// if (perLine % 8 == 0) { +//// sb.append('\n') +//// first = true +//// } +//// if (! first) sb.append(' ') +//// first = false +//// sb.append(java.lang.Integer.toHexString(b & 0xff)) +//// } +//// sb.append('\n') +//// sb.toString() +//// } +//// +//// override def toString: String = { +//// val sb: StringBuffer = new StringBuffer +//// sb.append(getClass.getName) +//// sb.append(' ') +//// sb.append(System.identityHashCode(this)) +//// sb.append("@[pos=") +//// sb.append(position()) +//// sb.append(" lim=") +//// sb.append(limit()) +//// sb.append(" cap=") +//// sb.append(capacity()) +//// sb.append("]") +//// sb.toString +//// } +//// +//// +//// +//// override def finalize(): Unit = { +//// var marked = false +//// if (containers ne null) { +//// if (containers.exists(container => container.isAcquired && container.requireRelease())) { +//// marked = true +//// logError("BUG: buffer was not released - and now going out of scope. " + +//// "Potential resource leak. Allocated at ", allocateLocationThrowable) +//// containers.foreach(_.release()) +//// } +//// if (containers.exists(container => !container.isFreed && container.requireFree())) { +//// if (!marked) { +//// logError("BUG: buffer was not freed - and now going out of scope. Potential resource leak", +//// allocateLocationThrowable) +//// } +//// else { +//// logError("BUG: buffer was not freed - and now going out of scope. Potential resource leak") +//// } +//// containers.foreach(_.free()) +//// } +//// } +//// super.finalize() +//// } +////} +//// +//// +////object LargeByteBuffer extends Logging { +//// +//// private val noopDisposeFunction = new BufferCleaner() { +//// protected def doClean(buffer: LargeByteBuffer) { +//// buffer.free(invokeCleaner = false) +//// } +//// } +//// +//// val enableExpensiveAssert = false +//// private val EMPTY_BYTEBUFFER = ByteBuffer.allocate(0) +//// val EMPTY_BUFFER = new LargeByteBuffer(ArrayBuffer( +//// new HeapByteBufferContainer(EMPTY_BYTEBUFFER, false)), false, false) +//// // Do not allow anyone else to override cleaner +//// EMPTY_BUFFER.overrideCleaner(noopDisposeFunction, allowOverride = false) +//// +//// // 8K sufficient ? +//// private val TEMP_ARRAY_SIZE = 8192 +//// +//// /** +//// * Create a LargeByteBuffer of specified size which is split across +//// * ByteBuffer's of size DiskStore.MAX_BLOCK_SIZE and backed by in memory +//// * ByteBuffer +//// * +//// */ +//// def allocateMemoryBuffer(totalSize: Long, ioConf: IOConfig): LargeByteBuffer = { +//// if (0 == totalSize) { +//// return EMPTY_BUFFER +//// } +//// +//// assert (totalSize > 0) +//// +//// val blockSize = ioConf.getMaxBlockSize(BufferType.MEMORY) +//// val numBlocks = ioConf.numBlocks(BufferType.MEMORY, totalSize) +//// val lastBlockSize = ioConf.lastBlockSize(BufferType.MEMORY, totalSize) +//// +//// assert (lastBlockSize > 0) +//// +//// val bufferArray = { +//// val arr = new ArrayBuffer[ByteBufferContainer](numBlocks) +//// for (index <- 0 until numBlocks - 1) { +//// val buff = ByteBuffer.allocate(blockSize) +//// // buff.clear() +//// arr += new HeapByteBufferContainer(buff, true) +//// } +//// arr += new HeapByteBufferContainer(ByteBuffer.allocate(lastBlockSize), true) +//// assert (arr.length == numBlocks) +//// arr +//// } +//// +//// new LargeByteBuffer(bufferArray, false, false) +//// } +//// +//// /** +//// * Create a LargeByteBuffer of specified size which is split across +//// * ByteBuffer's of size DiskStore.MAX_BLOCK_SIZE and backed by on disk +//// * +//// */ +//// private def allocateDiskBuffer(totalSize: Long, +//// blockManager: BlockManager): LargeByteBuffer = { +//// if (0 == totalSize) { +//// return EMPTY_BUFFER +//// } +//// +//// assert (totalSize > 0) +//// +//// // Create a file of the specified size. +//// val file = blockManager.diskBlockManager.createTempBlock()._2 +//// val raf = new RandomAccessFile(file, "rw") +//// try { +//// raf.setLength(totalSize) +//// } finally { +//// raf.close() +//// } +//// +//// readWriteDiskSegment(new FileSegment(file, 0, totalSize), +//// ephemeralDiskBacked = true, blockManager.ioConf) +//// } +//// +//// // The returned buffer takes up ownership of the underlying buffers +//// // (including dispos'ing that when done) +//// def fromBuffers(buffers: ByteBuffer*): LargeByteBuffer = { +//// val nonEmpty = buffers.filter(_.hasRemaining) +//// +//// // cleanup the empty buffers +//// buffers.filter(! _.hasRemaining).foreach(b => BlockManager.dispose(b)) +//// +//// +//// if (nonEmpty.isEmpty) { +//// return EMPTY_BUFFER +//// } +//// +//// // slice so that offsets match our requirement +//// new LargeByteBuffer(new ArrayBuffer() ++ nonEmpty.map(b => +//// new HeapByteBufferContainer(b.slice(), true)), false, false) +//// } +//// +//// def fromByteArrays(byteArrays: Array[Byte]*): LargeByteBuffer = { +//// // only non empty arrays +//// val arrays = byteArrays.filter(_.length > 0) +//// if (0 == arrays.length) return EMPTY_BUFFER +//// +//// new LargeByteBuffer(new ArrayBuffer() ++ arrays.map(arr => +//// new HeapByteBufferContainer(ByteBuffer.wrap(arr), true)), false, false) +//// } +//// +//// def fromLargeByteBuffers(canDispose: Boolean, inputBuffers: LargeByteBuffer*): LargeByteBuffer = { +//// +//// if (inputBuffers.isEmpty) return EMPTY_BUFFER +//// +//// if (! inputBuffers.exists(_.hasRemaining())) { +//// if (canDispose) inputBuffers.map(_.free()) +//// return EMPTY_BUFFER +//// } +//// +//// // release all temp resources acquired +//// inputBuffers.foreach(buff => buff.releasePendingContainers()) +//// // free current container if acquired. +//// inputBuffers.foreach(buff => if (buff.currentContainerIndex < buff.containers.length) { +//// buff.containers(buff.currentContainerIndex).release() +//// }) +//// // inputBuffers.foreach(b => b.doReleaseAll()) +//// +//// +//// // Dispose of any empty buffers +//// if (canDispose) inputBuffers.filter(! _.hasRemaining()).foreach(_.free()) +//// +//// // Find all containers we need. +//// val buffers = inputBuffers.filter(_.hasRemaining()).map(b => b.createSlice(b.remaining())) +//// +//// val containers = buffers.flatMap(_.containers) +//// assert (! containers.isEmpty) +//// // The in order containers of "buffers" seq constitute the required return value +//// val retval = new LargeByteBuffer(new ArrayBuffer() ++ containers, +//// // if you cant dispose, then we dont own the buffers : in which case, need duplicate +//// ! canDispose, inputBuffers.exists(_.ephemeralDiskBacked)) +//// +//// if (canDispose) { +//// // override dispose of all other buffers. +//// val disposeFunctions = inputBuffers.map { +//// buffer => { +//// (buffer, buffer.overrideCleaner(noopDisposeFunction)) +//// } +//// } +//// +//// val cleaner = retval.getCleaner() +//// val newCleaner = new BufferCleaner { +//// protected def doClean(buffer: LargeByteBuffer) { +//// +//// assert (retval == buffer) +//// // default cleaner. +//// cleaner.clean(retval) +//// // not required, since we are within clean anyway. +//// // retval.free(invokeCleaner = false) +//// +//// // retval.doDispose(needRelease = true) +//// +//// // This might actually call dispose twice on some (initially) empty buffers, +//// // which is fine since we now guard against that. +//// disposeFunctions.foreach(v => v._2.clean(v._1)) +//// // Call the free method too : so that buffers are marked free ... +//// disposeFunctions.foreach(v => v._1.free(invokeCleaner = false)) +//// } +//// } +//// +//// val prev = retval.overrideCleaner(newCleaner) +//// assert (prev == cleaner) +//// } +//// +//// retval +//// } +//// +//// private def checkOffsets(arr: Array[Byte], offset: Int, size: Int) { +//// if (arr == null) { +//// throw new NullPointerException +//// } else if (offset < 0 || size < 0 || offset + size > arr.length) { +//// throw new IndexOutOfBoundsException +//// } +//// } +//// +//// def allocateTransientBuffer(size: Long, blockManager: BlockManager) = { +//// if (size <= blockManager.ioConf.maxInMemSize) { +//// LargeByteBuffer.allocateMemoryBuffer(size, blockManager.ioConf) +//// } else { +//// LargeByteBuffer.allocateDiskBuffer(size, blockManager) +//// } +//// } +//// +//// def readFromDiskSegment(segment: FileSegment, ioConf: IOConfig, +//// ephemeralDiskBacked: Boolean): LargeByteBuffer = { +//// // Split the block into multiple of BlockStore.maxBlockSize +//// val segmentSize = segment.length +//// val blockSize = ioConf.getMaxBlockSize(BufferType.DISK).asInstanceOf[Long] +//// val numBlocks = ioConf.numBlocks(BufferType.DISK, segmentSize) +//// val lastBlockSize = ioConf.lastBlockSize(BufferType.DISK, segmentSize) +//// +//// val buffers = new ArrayBuffer[ByteBufferContainer](numBlocks) +//// +//// for (index <- 0 until numBlocks - 1) { +//// buffers += new ReadOnlyFileContainer(new FileSegment(segment.file, +//// segment.offset + index * blockSize, blockSize), ioConf) +//// } +//// +//// // Last block +//// buffers += new ReadOnlyFileContainer(new FileSegment(segment.file, +//// segment.offset + (numBlocks - 1) * blockSize, lastBlockSize), ioConf) +//// +//// new LargeByteBuffer(buffers, false, ephemeralDiskBacked) +//// } +//// +//// def readWriteDiskSegment(segment: FileSegment, ephemeralDiskBacked: Boolean, +//// ioConf: IOConfig): LargeByteBuffer = { +//// +//// // Split the block into multiple of BlockStore.maxBlockSize +//// val segmentSize = segment.length +//// val blockSize = ioConf.getMaxBlockSize(BufferType.DISK).asInstanceOf[Long] +//// val numBlocks = ioConf.numBlocks(BufferType.DISK, segmentSize) +//// val lastBlockSize = ioConf.lastBlockSize(BufferType.DISK, segmentSize) +//// +//// logInfo("readWriteDiskSegment = " + segment + ", numBlocks = " + numBlocks + +//// ", lastBlockSize = " + lastBlockSize) +//// val buffers = new ArrayBuffer[ByteBufferContainer](numBlocks) +//// +//// for (index <- 0 until numBlocks - 1) { +//// buffers += new ReadWriteFileContainer(new FileSegment(segment.file, +//// segment.offset + index * blockSize, blockSize), ephemeralDiskBacked, null) +//// } +//// +//// // Last block +//// buffers += new ReadWriteFileContainer(new FileSegment(segment.file, +//// segment.offset + (numBlocks - 1) * blockSize, lastBlockSize), ephemeralDiskBacked, null) +//// +//// new LargeByteBuffer(buffers, false, ephemeralDiskBacked) +//// } +////} diff --git a/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala b/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala index dcbda5a8515dd..644544cf869df 100644 --- a/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala +++ b/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala @@ -24,7 +24,7 @@ import scala.concurrent.{Promise, Await, Future} import scala.concurrent.duration.Duration import org.apache.spark.Logging -import org.apache.spark.network.buffer.{NioManagedBuffer, ManagedBuffer} +import org.apache.spark.network.buffer.{LargeByteBufferHelper, NioManagedBuffer, ManagedBuffer} import org.apache.spark.network.shuffle.{ShuffleClient, BlockFetchingListener} import org.apache.spark.storage.{BlockManagerId, BlockId, StorageLevel} @@ -92,9 +92,10 @@ abstract class BlockTransferService extends ShuffleClient with Closeable with Lo result.failure(exception) } override def onBlockFetchSuccess(blockId: String, data: ManagedBuffer): Unit = { - val ret = ByteBuffer.allocate(data.size.toInt) + val ret = LargeByteBufferHelper.allocate(data.size) ret.put(data.nioByteBuffer()) - ret.flip() + //XXX do we need ret.flip()?? + ret.position(0l) result.success(new NioManagedBuffer(ret)) } }) diff --git a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala index b089da8596e2b..86df34920a666 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala @@ -23,7 +23,7 @@ import scala.collection.JavaConversions._ import org.apache.spark.Logging import org.apache.spark.network.BlockDataManager -import org.apache.spark.network.buffer.{ManagedBuffer, NioManagedBuffer} +import org.apache.spark.network.buffer.{LargeByteBufferHelper, ManagedBuffer, NioManagedBuffer} import org.apache.spark.network.client.{RpcResponseCallback, TransportClient} import org.apache.spark.network.server.{OneForOneStreamManager, RpcHandler, StreamManager} import org.apache.spark.network.shuffle.protocol.{BlockTransferMessage, OpenBlocks, StreamHandle, UploadBlock} @@ -63,7 +63,7 @@ class NettyBlockRpcServer( // StorageLevel is serialized as bytes using our JavaSerializer. val level: StorageLevel = serializer.newInstance().deserialize(ByteBuffer.wrap(uploadBlock.metadata)) - val data = new NioManagedBuffer(ByteBuffer.wrap(uploadBlock.blockData)) + val data = new NioManagedBuffer(LargeByteBufferHelper.asLargeByteBuffer(uploadBlock.blockData)) blockManager.putBlockData(BlockId(uploadBlock.blockId), data, level) responseContext.onSuccess(new Array[Byte](0)) } diff --git a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala index 3f0950dae1f24..9824c7c38c188 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala @@ -115,13 +115,10 @@ class NettyBlockTransferService(conf: SparkConf, securityManager: SecurityManage // Convert or copy nio buffer into array in order to serialize it. val nioBuffer = blockData.nioByteBuffer() - val array = if (nioBuffer.hasArray) { - nioBuffer.array() - } else { - val data = new Array[Byte](nioBuffer.remaining()) - nioBuffer.get(data) - data - } + //TODO key change -- multiple uploads here + // this stub is not even efficient when the buffer actually is small + val array = new Array[Byte](nioBuffer.remaining().toInt) + nioBuffer.get(array, 0, nioBuffer.remaining().toInt) client.sendRpc(new UploadBlock(appId, execId, blockId.toString, levelBytes, array).toByteArray, new RpcResponseCallback { diff --git a/core/src/main/scala/org/apache/spark/network/nio/NioBlockTransferService.scala b/core/src/main/scala/org/apache/spark/network/nio/NioBlockTransferService.scala index b2aec160635c7..d0ba9d8948594 100644 --- a/core/src/main/scala/org/apache/spark/network/nio/NioBlockTransferService.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/NioBlockTransferService.scala @@ -20,7 +20,7 @@ package org.apache.spark.network.nio import java.nio.ByteBuffer import org.apache.spark.network._ -import org.apache.spark.network.buffer.{ManagedBuffer, NioManagedBuffer} +import org.apache.spark.network.buffer.{LargeByteBuffer, LargeByteBufferHelper, ManagedBuffer, NioManagedBuffer} import org.apache.spark.network.shuffle.BlockFetchingListener import org.apache.spark.storage.{BlockId, StorageLevel} import org.apache.spark.util.Utils @@ -116,7 +116,7 @@ final class NioBlockTransferService(conf: SparkConf, securityManager: SecurityMa val blockId = blockMessage.getId val networkSize = blockMessage.getData.limit() listener.onBlockFetchSuccess( - blockId.toString, new NioManagedBuffer(blockMessage.getData)) + blockId.toString, new NioManagedBuffer(LargeByteBufferHelper.asLargeByteBuffer(blockMessage.getData))) } } } @@ -143,7 +143,7 @@ final class NioBlockTransferService(conf: SparkConf, securityManager: SecurityMa level: StorageLevel) : Future[Unit] = { checkInit() - val msg = PutBlock(blockId, blockData.nioByteBuffer(), level) + val msg = PutBlock(blockId, blockData.nioByteBuffer().firstByteBuffer(), level) val blockMessageArray = new BlockMessageArray(BlockMessage.fromPutBlock(msg)) val remoteCmId = new ConnectionManagerId(hostName, port) val reply = cm.sendMessageReliably(remoteCmId, blockMessageArray.toBufferMessage) @@ -192,7 +192,7 @@ final class NioBlockTransferService(conf: SparkConf, securityManager: SecurityMa if (buffer == null) { return None } - Some(BlockMessage.fromGotBlock(GotBlock(msg.id, buffer))) + Some(BlockMessage.fromGotBlock(GotBlock(msg.id, buffer.firstByteBuffer()))) case _ => None } @@ -201,12 +201,12 @@ final class NioBlockTransferService(conf: SparkConf, securityManager: SecurityMa private def putBlock(blockId: BlockId, bytes: ByteBuffer, level: StorageLevel) { val startTimeMs = System.currentTimeMillis() logDebug("PutBlock " + blockId + " started from " + startTimeMs + " with data: " + bytes) - blockDataManager.putBlockData(blockId, new NioManagedBuffer(bytes), level) + blockDataManager.putBlockData(blockId, new NioManagedBuffer(LargeByteBufferHelper.asLargeByteBuffer(bytes)), level) logDebug("PutBlock " + blockId + " used " + Utils.getUsedTimeMs(startTimeMs) + " with data size: " + bytes.limit) } - private def getBlock(blockId: BlockId): ByteBuffer = { + private def getBlock(blockId: BlockId): LargeByteBuffer = { val startTimeMs = System.currentTimeMillis() logDebug("GetBlock " + blockId + " started from " + startTimeMs) val buffer = blockDataManager.getBlockData(blockId) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala index 9428273561cd8..1ea26ee8f9946 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala @@ -20,7 +20,7 @@ package org.apache.spark.scheduler import java.nio.ByteBuffer import java.util.concurrent.RejectedExecutionException -import org.apache.spark.io.WrappedLargeByteBuffer +import org.apache.spark.network.buffer.WrappedLargeByteBuffer import scala.language.existentials import scala.util.control.NonFatal @@ -76,7 +76,7 @@ private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedul } //TODO either change serializer interface, or ... val deserializedResult = serializer.get().deserialize[DirectTaskResult[_]]( - serializedTaskResult.get.asInstanceOf[WrappedLargeByteBuffer].underlying(0)) + serializedTaskResult.get.firstByteBuffer()) sparkEnv.blockManager.master.removeBlock(blockId) (deserializedResult, size) } diff --git a/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockManager.scala b/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockManager.scala index 016964fc274d8..4c0bc9ccec06e 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockManager.scala @@ -18,17 +18,14 @@ package org.apache.spark.shuffle import java.io.File -import java.nio.ByteBuffer import java.util.concurrent.ConcurrentLinkedQueue import java.util.concurrent.atomic.AtomicInteger -import org.apache.spark.io.LargeByteBuffer - import scala.collection.JavaConversions._ import org.apache.spark.{Logging, SparkConf, SparkEnv} import org.apache.spark.executor.ShuffleWriteMetrics -import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer} +import org.apache.spark.network.buffer.{LargeByteBuffer, FileSegmentManagedBuffer, ManagedBuffer} import org.apache.spark.network.netty.SparkTransportConf import org.apache.spark.serializer.Serializer import org.apache.spark.shuffle.FileShuffleBlockManager.ShuffleFileGroup @@ -176,7 +173,7 @@ class FileShuffleBlockManager(conf: SparkConf) override def getBytes(blockId: ShuffleBlockId): Option[LargeByteBuffer] = { //TODO val segment = getBlockData(blockId) - Some(LargeByteBuffer.asLargeByteBuffer(segment.nioByteBuffer())) + Some(segment.nioByteBuffer()) } override def getBlockData(blockId: ShuffleBlockId): ManagedBuffer = { diff --git a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockManager.scala b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockManager.scala index 76c2e4180c838..4dfdf9987a5fe 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockManager.scala @@ -18,13 +18,11 @@ package org.apache.spark.shuffle import java.io._ -import java.nio.ByteBuffer import com.google.common.io.ByteStreams -import org.apache.spark.io.LargeByteBuffer import org.apache.spark.{SparkConf, SparkEnv} -import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer} +import org.apache.spark.network.buffer.{LargeByteBuffer, FileSegmentManagedBuffer, ManagedBuffer} import org.apache.spark.network.netty.SparkTransportConf import org.apache.spark.storage._ @@ -99,7 +97,7 @@ class IndexShuffleBlockManager(conf: SparkConf) extends ShuffleBlockManager { } override def getBytes(blockId: ShuffleBlockId): Option[LargeByteBuffer] = { - Some(LargeByteBuffer.asLargeByteBuffer(getBlockData(blockId).nioByteBuffer())) + Some(getBlockData(blockId).nioByteBuffer()) } override def getBlockData(blockId: ShuffleBlockId): ManagedBuffer = { diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockManager.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockManager.scala index fa737729b8758..96dde7f53b84b 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockManager.scala @@ -17,8 +17,7 @@ package org.apache.spark.shuffle -import org.apache.spark.io.LargeByteBuffer -import org.apache.spark.network.buffer.ManagedBuffer +import org.apache.spark.network.buffer.{LargeByteBuffer, ManagedBuffer} import org.apache.spark.storage.ShuffleBlockId private[spark] 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 479fcf35283ad..be63f9cb03d29 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -17,7 +17,7 @@ package org.apache.spark.storage -import java.io.{BufferedOutputStream, ByteArrayOutputStream, File, InputStream, OutputStream} +import java.io.{BufferedOutputStream, File, InputStream, OutputStream} import java.nio.{ByteBuffer, MappedByteBuffer} import scala.collection.mutable.{ArrayBuffer, HashMap} @@ -31,9 +31,9 @@ import sun.nio.ch.DirectBuffer import org.apache.spark._ import org.apache.spark.executor._ -import org.apache.spark.io.{WrappedLargeByteBuffer, ChainedLargeByteBuffer, LargeByteBuffer, CompressionCodec} +import org.apache.spark.io.CompressionCodec import org.apache.spark.network._ -import org.apache.spark.network.buffer.{ManagedBuffer, NioManagedBuffer} +import org.apache.spark.network.buffer.{LargeByteBufferHelper, LargeByteBuffer, ManagedBuffer, NioManagedBuffer} import org.apache.spark.network.netty.SparkTransportConf import org.apache.spark.network.shuffle.ExternalShuffleClient import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo @@ -307,7 +307,7 @@ private[spark] class BlockManager( shuffleManager.shuffleBlockManager.getBlockData(blockId.asInstanceOf[ShuffleBlockId]) } else { val blockBytesOpt = doGetLocal(blockId, asBlockResult = false) - .asInstanceOf[Option[ByteBuffer]] + .asInstanceOf[Option[LargeByteBuffer]] if (blockBytesOpt.isDefined) { val buffer = blockBytesOpt.get new NioManagedBuffer(buffer) @@ -321,7 +321,7 @@ private[spark] class BlockManager( * Put the block locally, using the given storage level. */ override def putBlockData(blockId: BlockId, data: ManagedBuffer, level: StorageLevel): Unit = { - putBytes(blockId, LargeByteBuffer.asLargeByteBuffer(data.nioByteBuffer()), level) + putBytes(blockId, data.nioByteBuffer(), level) } /** @@ -538,10 +538,10 @@ private[spark] class BlockManager( /* We'll store the bytes in memory if the block's storage level includes * "memory serialized", or if it should be cached as objects in memory * but we only requested its serialized bytes. */ - val copyForMemory = LargeByteBuffer.allocateOnHeap(bytes.limit, largeByteBufferChunkSize) + val copyForMemory = LargeByteBufferHelper.allocate(bytes.limit) copyForMemory.put(bytes) memoryStore.putBytes(blockId, copyForMemory, level) - bytes.rewind() + bytes.position(0l) } if (!asBlockResult) { return Some(bytes) @@ -595,8 +595,8 @@ private[spark] class BlockManager( for (loc <- locations) { logDebug(s"Getting remote block $blockId from $loc") //TODO the fetch will always be one byte buffer till we fix SPARK-5928 - val data: LargeByteBuffer = LargeByteBuffer.asLargeByteBuffer(blockTransferService.fetchBlockSync( - loc.host, loc.port, loc.executorId, blockId.toString).nioByteBuffer()) + val data: LargeByteBuffer = blockTransferService.fetchBlockSync( + loc.host, loc.port, loc.executorId, blockId.toString).nioByteBuffer() if (data != null) { if (asBlockResult) { @@ -791,7 +791,7 @@ private[spark] class BlockManager( case ArrayValues(array) => blockStore.putArray(blockId, array, putLevel, returnValues) case ByteBufferValues(bytes) => - bytes.rewind() + bytes.position(0l) blockStore.putBytes(blockId, bytes, putLevel) } size = result.size @@ -942,7 +942,7 @@ private[spark] class BlockManager( case Some(peer) => try { val onePeerStartTime = System.currentTimeMillis - data.rewind() + data.position(0l) logTrace(s"Trying to replicate $blockId of ${data.limit()} bytes to $peer") //TODO //ACK! here we're stuck -- we can't replicate a large block until we figure out @@ -1201,7 +1201,7 @@ private[spark] class BlockManager( blockId: BlockId, bytes: LargeByteBuffer, serializer: Serializer = defaultSerializer): Iterator[Any] = { - bytes.rewind() + bytes.position(0); val stream = wrapForCompression(blockId, new LargeByteBufferInputStream(bytes, true)) serializer.newInstance().deserializeStream(stream).asIterator } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockStore.scala b/core/src/main/scala/org/apache/spark/storage/BlockStore.scala index 38989f0c07681..e1b48bf11bcc7 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockStore.scala @@ -17,12 +17,8 @@ package org.apache.spark.storage -import java.nio.ByteBuffer - -import scala.collection.mutable.ArrayBuffer - import org.apache.spark.Logging -import org.apache.spark.io.LargeByteBuffer +import org.apache.spark.network.buffer.LargeByteBuffer /** * Abstract class to store blocks. 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 180b237a1a1f2..5ef9929feaabf 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala @@ -22,7 +22,7 @@ import java.nio.ByteBuffer import java.nio.channels.FileChannel.MapMode import org.apache.spark.Logging -import org.apache.spark.io.{WrappedLargeByteBuffer, LargeByteBuffer} +import org.apache.spark.network.buffer.{LargeByteBufferHelper, LargeByteBuffer} import org.apache.spark.serializer.Serializer import org.apache.spark.util.Utils @@ -118,9 +118,9 @@ private[spark] class DiskStore(blockManager: BlockManager, diskManager: DiskBloc } } buf.flip() - Some(LargeByteBuffer.asLargeByteBuffer(buf)) + Some(LargeByteBufferHelper.asLargeByteBuffer(buf)) } else { - Some(LargeByteBuffer.mapFile(channel, MapMode.READ_ONLY, offset, length)) + Some(LargeByteBufferHelper.mapFile(channel, MapMode.READ_ONLY, offset, length)) } } finally { channel.close() 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 8fccc0f3e78d2..074919fdeb706 100644 --- a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala @@ -19,7 +19,7 @@ package org.apache.spark.storage import java.util.LinkedHashMap -import org.apache.spark.io.LargeByteBuffer +import org.apache.spark.network.buffer.LargeByteBuffer import scala.collection.mutable import scala.collection.mutable.ArrayBuffer @@ -81,7 +81,7 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) override def putBytes(blockId: BlockId, _bytes: LargeByteBuffer, level: StorageLevel): PutResult = { // Work on a duplicate - since the original input might be used elsewhere. val bytes = _bytes.duplicate() - bytes.rewind() + bytes.position(0l); if (level.deserialized) { val values = blockManager.dataDeserialize(blockId, bytes) putIterator(blockId, values, level, returnValues = true) diff --git a/core/src/main/scala/org/apache/spark/storage/PutResult.scala b/core/src/main/scala/org/apache/spark/storage/PutResult.scala index 2e00934bde243..aa9176791b319 100644 --- a/core/src/main/scala/org/apache/spark/storage/PutResult.scala +++ b/core/src/main/scala/org/apache/spark/storage/PutResult.scala @@ -17,7 +17,7 @@ package org.apache.spark.storage -import org.apache.spark.io.LargeByteBuffer +import org.apache.spark.network.buffer.LargeByteBuffer /** * Result of adding a block into a BlockStore. This case class contains a few things: diff --git a/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala b/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala index 9f964ed456d5e..62cfd9e65eaec 100644 --- a/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala @@ -21,7 +21,7 @@ import java.io.IOException import java.nio.ByteBuffer import com.google.common.io.ByteStreams -import org.apache.spark.io.LargeByteBuffer +import org.apache.spark.network.buffer.{LargeByteBufferHelper, LargeByteBuffer} import tachyon.client.{ReadType, WriteType} import org.apache.spark.Logging @@ -70,7 +70,7 @@ private[spark] class TachyonStore( // So that we do not modify the input offsets ! // duplicate does not copy buffer, so inexpensive val byteBuffer = bytes.duplicate() - byteBuffer.rewind() + byteBuffer.position(0l) logDebug(s"Attempting to put block $blockId into Tachyon") val startTime = System.currentTimeMillis val file = tachyonManager.getFile(blockId) @@ -114,7 +114,7 @@ private[spark] class TachyonStore( //TODO val bs = new Array[Byte](size.asInstanceOf[Int]) ByteStreams.readFully(is, bs) - Some(LargeByteBuffer.asLargeByteBuffer(ByteBuffer.wrap(bs))) + Some(LargeByteBufferHelper.asLargeByteBuffer(bs)) } catch { case ioe: IOException => logWarning(s"Failed to fetch the block $blockId from Tachyon", ioe) diff --git a/core/src/main/scala/org/apache/spark/util/LargeByteBufferInputStream.scala b/core/src/main/scala/org/apache/spark/util/LargeByteBufferInputStream.scala index 26f2d7848bb29..98e41906b8251 100644 --- a/core/src/main/scala/org/apache/spark/util/LargeByteBufferInputStream.scala +++ b/core/src/main/scala/org/apache/spark/util/LargeByteBufferInputStream.scala @@ -18,9 +18,8 @@ package org.apache.spark.util import java.io.InputStream -import java.nio.ByteBuffer -import org.apache.spark.io.LargeByteBuffer +import org.apache.spark.network.buffer.LargeByteBuffer import org.apache.spark.storage.BlockManager /** diff --git a/core/src/main/scala/org/apache/spark/util/LargeByteBufferOutputStream.scala b/core/src/main/scala/org/apache/spark/util/LargeByteBufferOutputStream.scala index 2fe904b10c53c..6c61d52ecca75 100644 --- a/core/src/main/scala/org/apache/spark/util/LargeByteBufferOutputStream.scala +++ b/core/src/main/scala/org/apache/spark/util/LargeByteBufferOutputStream.scala @@ -19,7 +19,8 @@ package org.apache.spark.util import java.io.OutputStream -import org.apache.spark.io.{ChainedLargeByteBuffer, LargeByteBuffer} +import org.apache.spark.io.ChainedLargeByteBuffer +import org.apache.spark.network.buffer.LargeByteBuffer import org.apache.spark.util.collection.ChainedBuffer private[spark] diff --git a/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSecuritySuite.scala b/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSecuritySuite.scala index 94bfa67451892..6c01db61168b3 100644 --- a/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSecuritySuite.scala +++ b/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSecuritySuite.scala @@ -25,7 +25,7 @@ import scala.concurrent.{Await, Promise} import scala.util.{Failure, Success, Try} import org.apache.commons.io.IOUtils -import org.apache.spark.network.buffer.{ManagedBuffer, NioManagedBuffer} +import org.apache.spark.network.buffer.{LargeByteBufferHelper, LargeByteBuffer, ManagedBuffer, NioManagedBuffer} import org.apache.spark.network.shuffle.BlockFetchingListener import org.apache.spark.network.{BlockDataManager, BlockTransferService} import org.apache.spark.storage.{BlockId, ShuffleBlockId} @@ -100,7 +100,7 @@ class NettyBlockTransferSecuritySuite extends FunSuite with MockitoSugar with Sh val blockManager = mock[BlockDataManager] val blockId = ShuffleBlockId(0, 1, 2) val blockString = "Hello, world!" - val blockBuffer = new NioManagedBuffer(ByteBuffer.wrap(blockString.getBytes)) + val blockBuffer = new NioManagedBuffer(LargeByteBufferHelper.asLargeByteBuffer(blockString.getBytes)) when(blockManager.getBlockData(blockId)).thenReturn(blockBuffer) val securityManager0 = new SecurityManager(conf0) diff --git a/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSuite.scala b/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSuite.scala new file mode 100644 index 0000000000000..b242ff593f04a --- /dev/null +++ b/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSuite.scala @@ -0,0 +1,80 @@ +/* + * 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.network.netty + +import java.util.concurrent.TimeUnit + +import org.apache.spark.network.BlockDataManager +import org.apache.spark.network.buffer.{ManagedBuffer, LargeByteBufferHelper, NioManagedBuffer} +import org.apache.spark.network.shuffle.BlockFetchingListener +import org.apache.spark.storage.ShuffleBlockId +import org.apache.spark.{SecurityManager, SparkConf} +import org.mockito.Mockito._ +import org.scalatest.mock.MockitoSugar +import org.scalatest.{Matchers, FunSuite} + +import scala.concurrent.duration.FiniteDuration +import scala.concurrent.{Await, Promise} + +class NettyBlockTransferSuite extends FunSuite with Matchers with MockitoSugar { + + val conf = new SparkConf() + .set("spark.app.id", "app-id") + val securityManager = new SecurityManager(conf) + + + + test("simple fetch") { + + val blockManager = mock[BlockDataManager] + val blockId = ShuffleBlockId(0, 1, 2) + val blockString = "Hello, world!" + val blockBuffer = new NioManagedBuffer(LargeByteBufferHelper.asLargeByteBuffer(blockString.getBytes)) + when(blockManager.getBlockData(blockId)).thenReturn(blockBuffer) + + val from = new NettyBlockTransferService(conf, securityManager, numCores = 1) + from.init(blockManager) + val to = new NettyBlockTransferService(conf, securityManager, numCores = 1) + to.init(blockManager) + + try { + val promise = Promise[ManagedBuffer]() + + to.fetchBlocks(from.hostName, from.port, "1", Array(blockId.toString), + new BlockFetchingListener { + override def onBlockFetchFailure(blockId: String, exception: Throwable): Unit = { + promise.failure(exception) + } + + override def onBlockFetchSuccess(blockId: String, data: ManagedBuffer): Unit = { + promise.success(data.retain()) + } + }) + + Await.ready(promise.future, FiniteDuration(1000, TimeUnit.MILLISECONDS)) + } finally { + from.close() + to.close() + } + + + } + + + + +} diff --git a/network/common/src/main/java/org/apache/spark/network/buffer/FileSegmentManagedBuffer.java b/network/common/src/main/java/org/apache/spark/network/buffer/FileSegmentManagedBuffer.java index 75c426f379238..213013e14b7c6 100644 --- a/network/common/src/main/java/org/apache/spark/network/buffer/FileSegmentManagedBuffer.java +++ b/network/common/src/main/java/org/apache/spark/network/buffer/FileSegmentManagedBuffer.java @@ -1,19 +1,19 @@ /* - * 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. - */ +* 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.network.buffer; @@ -34,8 +34,8 @@ import org.apache.spark.network.util.TransportConf; /** - * A {@link ManagedBuffer} backed by a segment in a file. - */ +* A {@link ManagedBuffer} backed by a segment in a file. +*/ public final class FileSegmentManagedBuffer extends ManagedBuffer { private final TransportConf conf; private final File file; diff --git a/network/common/src/main/java/org/apache/spark/network/buffer/LargeByteBuffer.java b/network/common/src/main/java/org/apache/spark/network/buffer/LargeByteBuffer.java index c56fdebb887a1..4997dcecc3370 100644 --- a/network/common/src/main/java/org/apache/spark/network/buffer/LargeByteBuffer.java +++ b/network/common/src/main/java/org/apache/spark/network/buffer/LargeByteBuffer.java @@ -17,6 +17,7 @@ package org.apache.spark.network.buffer; import java.io.IOException; +import java.nio.ByteBuffer; import java.nio.channels.WritableByteChannel; public interface LargeByteBuffer { @@ -35,6 +36,8 @@ public interface LargeByteBuffer { public void put(LargeByteBuffer bytes); + public long remaining(); + //TODO checks on limit semantics /** @@ -53,4 +56,7 @@ public interface LargeByteBuffer { public long writeTo(WritableByteChannel channel) throws IOException; + //TODO this should be deleted -- just to help me get going + public ByteBuffer firstByteBuffer(); + } diff --git a/network/common/src/main/java/org/apache/spark/network/buffer/LargeByteBufferHelper.java b/network/common/src/main/java/org/apache/spark/network/buffer/LargeByteBufferHelper.java index 2585b65b1f969..5fe01d87e36e9 100644 --- a/network/common/src/main/java/org/apache/spark/network/buffer/LargeByteBufferHelper.java +++ b/network/common/src/main/java/org/apache/spark/network/buffer/LargeByteBufferHelper.java @@ -29,6 +29,23 @@ public static LargeByteBuffer asLargeByteBuffer(ByteBuffer buffer) { return new WrappedLargeByteBuffer(new ByteBuffer[]{buffer}); } + public static LargeByteBuffer asLargeByteBuffer(byte[] bytes) { + return new WrappedLargeByteBuffer(new ByteBuffer[]{ByteBuffer.wrap(bytes)}); + } + + public static LargeByteBuffer allocate(long size) { + ArrayList chunks = new ArrayList(); + long remaining = size; + while (remaining > 0) { + int nextSize = (int)Math.min(remaining, DEFAULT_MAX_CHUNK); + ByteBuffer next = ByteBuffer.allocate(nextSize); + remaining -= nextSize; + chunks.add(next); + } + return new WrappedLargeByteBuffer(chunks.toArray(new ByteBuffer[chunks.size()])); + } + + public static LargeByteBuffer mapFile( FileChannel channel, FileChannel.MapMode mode, diff --git a/network/common/src/main/java/org/apache/spark/network/buffer/NioManagedBuffer.java b/network/common/src/main/java/org/apache/spark/network/buffer/NioManagedBuffer.java index d4746a76e7d13..2febfe7f761c5 100644 --- a/network/common/src/main/java/org/apache/spark/network/buffer/NioManagedBuffer.java +++ b/network/common/src/main/java/org/apache/spark/network/buffer/NioManagedBuffer.java @@ -29,9 +29,9 @@ * A {@link ManagedBuffer} backed by {@link ByteBuffer}. */ public final class NioManagedBuffer extends ManagedBuffer { - private final ByteBuffer buf; + private final LargeByteBuffer buf; - public NioManagedBuffer(ByteBuffer buf) { + public NioManagedBuffer(LargeByteBuffer buf) { this.buf = buf; } @@ -42,12 +42,14 @@ public long size() { @Override public LargeByteBuffer nioByteBuffer() throws IOException { - return LargeByteBufferHelper.asLargeByteBuffer(buf.duplicate()); + return buf.duplicate(); } @Override public InputStream createInputStream() throws IOException { - return new ByteBufInputStream(Unpooled.wrappedBuffer(buf)); + //TODO + ByteBuffer aBuf = ((WrappedLargeByteBuffer) buf).underlying[0]; + return new ByteBufInputStream(Unpooled.wrappedBuffer(aBuf)); } @Override @@ -62,7 +64,9 @@ public ManagedBuffer release() { @Override public Object convertToNetty() throws IOException { - return Unpooled.wrappedBuffer(buf); + //TODO + ByteBuffer aBuf = ((WrappedLargeByteBuffer) buf).underlying[0]; + return Unpooled.wrappedBuffer(aBuf); } @Override diff --git a/network/common/src/main/java/org/apache/spark/network/buffer/WrappedLargeByteBuffer.java b/network/common/src/main/java/org/apache/spark/network/buffer/WrappedLargeByteBuffer.java index bbe73a87e9105..2ecb569b3a23c 100644 --- a/network/common/src/main/java/org/apache/spark/network/buffer/WrappedLargeByteBuffer.java +++ b/network/common/src/main/java/org/apache/spark/network/buffer/WrappedLargeByteBuffer.java @@ -22,7 +22,7 @@ public class WrappedLargeByteBuffer implements LargeByteBuffer { - private final ByteBuffer[] underlying; + final ByteBuffer[] underlying; private final Long totalCapacity; private final long[] chunkOffsets; @@ -91,6 +91,11 @@ public void position(long newPosition) { _pos = newPosition; } + @Override + public long remaining() { + return limit - _pos; + } + @Override public WrappedLargeByteBuffer duplicate() { ByteBuffer[] duplicates = new ByteBuffer[underlying.length]; @@ -132,4 +137,9 @@ public long writeTo(WritableByteChannel channel) throws IOException { } return written; } + + @Override + public ByteBuffer firstByteBuffer() { + return underlying[0]; + } } diff --git a/network/common/src/main/scala/org/apache/spark/network/buffer/LargeByteBuffer.scala b/network/common/src/main/scala/org/apache/spark/network/buffer/LargeByteBuffer.scala new file mode 100644 index 0000000000000..eb7520d3c6325 --- /dev/null +++ b/network/common/src/main/scala/org/apache/spark/network/buffer/LargeByteBuffer.scala @@ -0,0 +1,1760 @@ +///* +// * 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.network.buffer +// +//import java.nio.ByteBuffer +//import java.nio.channels.FileChannel.MapMode +//import java.nio.channels.{FileChannel, WritableByteChannel} +// +//import scala.collection.mutable.ArrayBuffer +// +// +// +//trait LargeByteBuffer { +//// def position(): Long +//// +//// def limit(): Long +// +// def capacity(): Long +// +// def get(): Byte //needed for ByteBufferInputStream +// +// def get(dst: Array[Byte], offset: Int, length: Int): Unit // for ByteBufferInputStream +// +// def position(position: Long): Unit //for ByteBufferInputStream +// +// def position(): Long //for ByteBufferInputStream +// +// /** doesn't copy data, just copies references & offsets */ +// def duplicate(): LargeByteBuffer +// +// def put(bytes: LargeByteBuffer): Unit +// +// //also need whatever is necessary for ByteArrayOutputStream for BlockManager#dataSerialize +// +// +// //TODO checks on limit semantics +// +// /** +// * Sets this buffer's limit. If the position is larger than the new limit then it is set to the +// * new limit. If the mark is defined and larger than the new limit then it is discarded. +// */ +// def limit(newLimit: Long): Unit +// +// /** +// * return this buffer's limit +// * @return +// */ +// def limit(): Long +// +// +// //an alternative to having this method would be having a foreachBuffer(f: Buffer => T) +// def writeTo(channel: WritableByteChannel): Long +// +//// +//// def skip(skipBy: Long): Unit +//// +//// def position(newPosition: Long): Unit +//// +//// /** +//// * Clears this buffer. The position is set to zero, the limit is set to +//// * the capacity, and the mark is discarded. +//// * +//// *

Invoke this method before using a sequence of channel-read or +//// * put operations to fill this buffer. +//// * +//// *

This method does not actually erase the data in the buffer, but it +//// * is named as if it did because it will most often be used in situations +//// * in which that might as well be the case.

+//// */ +//// def clear(): Unit +//// +//// /** +//// * Flips this buffer. The limit is set to the current position and then +//// * the position is set to zero. If the mark is defined then it is +//// * discarded. +//// * +//// *

After a sequence of channel-read or put operations, invoke +//// * this method to prepare for a sequence of channel-write or relative +//// * get operations. +//// */ +//// def flip(): Unit +// +// /** +// * Rewinds this buffer. The position is set to zero and the mark is +// * discarded. +// * +// *

Invoke this method before a sequence of channel-write or get +// * operations, assuming that the limit has already been set +// * appropriately. +// */ +// def rewind(): Unit +// +// /** +// * Returns the number of elements between the current position and the +// * limit.

+// * +// * @return The number of elements remaining in this buffer +// */ +// def remaining(): Long +//} +//// +////class ChainedLargeByteBuffer(private[network] val underlying: ChainedBuffer) extends LargeByteBuffer { +//// +//// def capacity = underlying.capacity +//// +//// var _pos = 0l +//// +//// def get(dst: Array[Byte],offset: Int,length: Int): Unit = { +//// underlying.read(_pos, dst, offset, length) +//// _pos += length +//// } +//// +//// def get(): Byte = { +//// val b = underlying.read(_pos) +//// _pos += 1 +//// b +//// } +//// +//// def put(bytes: LargeByteBuffer): Unit = { +//// ??? +//// } +//// +//// def position: Long = _pos +//// def position(position: Long): Unit = { +//// _pos = position +//// } +//// def remaining(): Long = { +//// underlying.size - position +//// } +//// +//// def duplicate(): ChainedLargeByteBuffer = { +//// new ChainedLargeByteBuffer(underlying) +//// } +//// +//// def rewind(): Unit = { +//// _pos = 0 +//// } +//// +//// def limit(): Long = { +//// capacity +//// } +//// +//// def limit(newLimit: Long): Unit = { +//// ??? +//// } +//// +//// def writeTo(channel:WritableByteChannel): Long = { +//// var written = 0l +//// underlying.chunks.foreach{bytes => +//// //TODO test this +//// val buffer = ByteBuffer.wrap(bytes) +//// while (buffer.hasRemaining) +//// channel.write(buffer) +//// written += bytes.length +//// } +//// written +//// } +////} +// +//class WrappedLargeByteBuffer(private[spark] val underlying: Array[ByteBuffer]) extends LargeByteBuffer { +// +// val (totalCapacity, chunkOffsets) = { +// var sum = 0l +// val offsets = new Array[Long](underlying.size) +// (0 until underlying.size).foreach{idx => +// offsets(idx) = sum +// sum += underlying(idx).capacity() +// } +// (sum, offsets) +// } +// +// private var _pos = 0l +// private var currentBufferIdx = 0 +// private var currentBuffer = underlying(0) +// private var _limit = totalCapacity +// +// def capacity = totalCapacity +// +// def get(dst: Array[Byte], offset: Int, length: Int): Unit = { +// var moved = 0 +// while (moved < length) { +// val toRead = math.min(length - moved, currentBuffer.remaining()) +// currentBuffer.get(dst, offset, toRead) +// moved += toRead +// updateCurrentBuffer() +// } +// } +// +// def get(): Byte = { +// val r = currentBuffer.get() +// _pos += 1 +// updateCurrentBuffer() +// r +// } +// +// private def updateCurrentBuffer(): Unit = { +// //TODO fix end condition +// while(!currentBuffer.hasRemaining()) { +// currentBufferIdx += 1 +// currentBuffer = underlying(currentBufferIdx) +// } +// } +// +// def put(bytes: LargeByteBuffer): Unit = { +// ??? +// } +// +// def position: Long = _pos +// def position(position: Long): Unit = { +// //XXX check range? +// _pos = position +// } +// def remaining(): Long = { +// totalCapacity - _pos +// } +// +// def duplicate(): WrappedLargeByteBuffer = { +// new WrappedLargeByteBuffer(underlying.map{_.duplicate()}) +// } +// +// def rewind(): Unit = { +// _pos = 0 +// underlying.foreach{_.rewind()} +// } +// +// def limit(): Long = { +// totalCapacity +// } +// +// def limit(newLimit: Long) = { +// //XXX check range? set limits in sub buffers? +// _limit = newLimit +// } +// +// def writeTo(channel: WritableByteChannel): Long = { +// var written = 0l +// underlying.foreach{buffer => +// //TODO test this +// //XXX do we care about respecting the limit here? +// written += buffer.remaining() +// while (buffer.hasRemaining) +// channel.write(buffer) +// } +// written +// } +// +//} +// +//object LargeByteBuffer { +// +// def asLargeByteBuffer(byteBuffer: ByteBuffer): LargeByteBuffer = { +// new WrappedLargeByteBuffer(Array(byteBuffer)) +// } +// +// def asLargeByteBuffer(bytes: Array[Byte]): LargeByteBuffer = { +// new WrappedLargeByteBuffer(Array(ByteBuffer.wrap(bytes))) +// } +// +//// +//// def allocateOnHeap(size: Long, maxChunk: Int): LargeByteBuffer = { +//// val buffer = ChainedBuffer.withInitialSize(maxChunk, size) +//// new ChainedLargeByteBuffer(buffer) +//// } +// +// def mapFile( +// channel: FileChannel, +// mode: MapMode, +// offset: Long, +// length: Long, +// maxChunk: Int = Integer.MAX_VALUE - 1e6.toInt +// ): LargeByteBuffer = { +// val offsets = new ArrayBuffer[Long]() +// var curOffset = offset +// val end = offset + length +// while (curOffset < end) { +// offsets += curOffset +// val length = math.min(end - curOffset, maxChunk) +// curOffset += length +// } +// offsets += end +// val chunks = new Array[ByteBuffer](offsets.size - 1) +// (0 until offsets.size - 1).foreach{idx => +// chunks(idx) = channel.map(mode, offsets(idx), offsets(idx + 1) - offsets(idx)) +// } +// new WrappedLargeByteBuffer(chunks) +// } +//} +// +// +//// +/////** +//// * This is a variant of ByteBuffer to be used internally in spark, which is not limited to 2G +//// * which ByteBuffers are limited to. +//// * Externally, it exposes all the api which java.nio.ByteBuffer exposes. +//// * Internally, it maintains a sequence of Containers which manage the ByteBuffer data. +//// * Not all the data might be loaded into memory (like disk or tachyon data) - so actual +//// * memory footprint - heap and vm could be much lower than capacity. +//// * +//// * TODO: Currently we are slightly fast and loose in terms of concurrent modifications to this +//// * buffer, maybe revisit this later ? Note: this is not much different from earlier though ! +//// * +//// * TODO: Explore if (at all) we can leverage zero copy transfers. The issue (currently) is that this +//// * will require the file to be kept open (repeatedly opening/closing file is not good +//// * for each transfer) and this has an impact on ulimit. Not to mention writing of mmap'ed buffer is +//// * pretty quick (it is the first failover in case direct transfer is not possible in file zero copy) +//// * +//// * TODO: After redesign to containers, we got rid of parent containers to free - the side effect is +//// * that if there are direct ByteBuffers, we are not handling explicit cleanup of those in some +//// * cases (when we duplicate/slice them). Currently spark does not need this, but might in future +//// * so relook at it later. +//// */ +////// We should make this constructor private: but for now, +////// leaving it public since TachyonStore needs it +////class LargeByteBuffer private[spark](private val inputContainers: ArrayBuffer[ByteBufferContainer], +//// private val needDuplicate: Boolean, val ephemeralDiskBacked: Boolean) extends Logging { +//// +//// // TODO: TEMP code: to flush out potential resource leaks. REMOVE ME +//// private val allocateLocationThrowable: Throwable = { +//// if (inputContainers.exists(c => c.requireRelease() || c.requireFree())) { +//// new Throwable("blockId = " + BlockManager.getLookupBlockId) +//// } else { +//// null +//// } +//// } +//// private var disposeLocationThrowable: Throwable = null +//// +//// @volatile private var allowCleanerOverride = true +//// @volatile private var cleaner: BufferCleaner = new BufferCleaner { +//// override def doClean(buffer: LargeByteBuffer) = { +//// assert (LargeByteBuffer.this == buffer) +//// doDispose(needRelease = false) +//// } +//// } +//// +//// // should not be empty +//// assert (null != inputContainers && ! inputContainers.isEmpty) +//// // should not have any null's +//// assert (inputContainers.find(_ == null).isEmpty) +//// +//// // println("Num containers = " + inputContainers.size) +//// +//// // Position, limit and capacity relevant over the engire LargeByteBuffer +//// @volatile private var globalPosition = 0L +//// @volatile private var globalLimit = 0L +//// @volatile private var currentContainerIndex = 0 +//// +//// // The buffers in which the actual data is held. +//// private var containers: Array[ByteBufferContainer] = null +//// +//// // aggregate capacities of the individual buffers. +//// // bufferPositionStart(0) will be capacity of 1st buffer, bufferPositionStart(1) will be +//// // sum of capacity of 0th and 1st block buffer +//// private var bufferPositionStart: Array[Long] = null +//// +//// // Contains the indices of a containers which requires release before subsequent invocation of +//// // read/write should be serviced. This is required since current read/write might have moved the +//// // position but since we are returning bytebuffers which depend on the validity of the existing +//// // bytebuffer, we cant release them yet. +//// private var needReleaseIndices = new HashSet[Int]() +//// +//// private val readable = ! inputContainers.exists(! _.isReadable) +//// private val writable = ! inputContainers.exists(! _.isWritable) +//// +//// +//// // initialize +//// @volatile private var globalCapacity = { +//// +//// // Ensure that there are no empty buffers : messes up with our code : unless it +//// // is a single buffer (for empty buffer for marker case) +//// assert (inputContainers.find(0 == _.capacity()).isEmpty || 1 == inputContainers.length) +//// +//// containers = { +//// if (needDuplicate) inputContainers.map(_.duplicate()).toArray else inputContainers.toArray +//// } +//// containers.foreach(_.validate()) +//// +//// def initializeBufferPositionStart(arr: Array[ByteBufferContainer]) { +//// val buff = new ArrayBuffer[Long](arr.length + 1) +//// buff += 0L +//// +//// buff ++= arr.map(_.capacity().asInstanceOf[Long]).scanLeft(0L)(_ + _).slice(1, arr.length + 1) +//// assert (buff.length == arr.length + 1) +//// bufferPositionStart = buff.toArray +//// } +//// +//// initializeBufferPositionStart(containers) +//// +//// // remove references from inputBuffers +//// inputContainers.clear() +//// +//// globalLimit = bufferPositionStart(containers.length) +//// globalPosition = 0L +//// currentContainerIndex = 0 +//// +//// assert (globalLimit == containers.map(_.capacity().asInstanceOf[Long]).sum) +//// +//// globalLimit +//// } +//// +//// final def position(): Long = globalPosition +//// +//// final def limit(): Long = globalLimit +//// +//// final def capacity(): Long = globalCapacity +//// +//// final def limit(newLimit: Long) { +//// if ((newLimit > capacity()) || (newLimit < 0)) { +//// throw new IllegalArgumentException("newLimit = " + newLimit + ", capacity = " + capacity()) +//// } +//// +//// globalLimit = newLimit +//// if (position() > newLimit) position(newLimit) +//// } +//// +//// def skip(skipBy: Long) = position(position() + skipBy) +//// +//// private def releasePendingContainers() { +//// if (! needReleaseIndices.isEmpty) { +//// val iter = needReleaseIndices.iterator +//// while (iter.hasNext) { +//// val index = iter.next() +//// assert (index >= 0 && index < containers.length) +//// // It is possible to move from one container to next before the previous +//// // container was acquired. For example, get forcing move to next container +//// // since current was exhausted immediatelly followed by a position() +//// // so the container we moved to was never acquired. +//// +//// // assert (containers(index).isAcquired) +//// // will this always be satisfied ? +//// // assert (index != currentContainerIndex) +//// if (containers(index).isAcquired) containers(index).release() +//// } +//// needReleaseIndices.clear() +//// } +//// } +//// +//// private def toNewContainer(newIndex: Int) { +//// if (newIndex != currentContainerIndex && currentContainerIndex < containers.length) { +//// +//// assert (currentContainerIndex >= 0) +//// needReleaseIndices += currentContainerIndex +//// } +//// currentContainerIndex = newIndex +//// } +//// +//// // expensive method, sigh ... optimize it later ? +//// final def position(newPosition: Long) { +//// +//// if ((newPosition > globalLimit) || (newPosition < 0)) throw new IllegalArgumentException() +//// +//// if (currentContainerIndex < bufferPositionStart.length - 1 && +//// newPosition >= bufferPositionStart(currentContainerIndex) && +//// newPosition < bufferPositionStart(currentContainerIndex + 1)) { +//// // Same buffer - easy method ... +//// globalPosition = newPosition +//// // Changed position - free previously returned buffers. +//// releasePendingContainers() +//// return +//// } +//// +//// // Find appropriate currentContainerIndex +//// // Since bufferPositionStart is sorted, can be replaced with binary search if required. +//// // For now, not in the perf critical path since buffers size is very low typically. +//// var index = 0 +//// val cLen = containers.length +//// while (index < cLen) { +//// if (newPosition >= bufferPositionStart(index) && +//// newPosition < bufferPositionStart(index + 1)) { +//// globalPosition = newPosition +//// toNewContainer(index) +//// // Changed position - free earlier and previously returned buffers. +//// releasePendingContainers() +//// return +//// } +//// index += 1 +//// } +//// +//// if (newPosition == globalLimit && newPosition == bufferPositionStart(cLen)) { +//// // boundary. +//// globalPosition = newPosition +//// toNewContainer(cLen) +//// // Changed position - free earlier and previously returned buffers. +//// releasePendingContainers() +//// return +//// } +//// +//// assert (assertion = false, "Unexpected to come here .... newPosition = " + newPosition + +//// ", bufferPositionStart = " + bufferPositionStart.mkString("[", ", ", "]")) +//// } +//// +//// +//// /** +//// * Clears this buffer. The position is set to zero, the limit is set to +//// * the capacity, and the mark is discarded. +//// * +//// *

Invoke this method before using a sequence of channel-read or +//// * put operations to fill this buffer. +//// * +//// *

This method does not actually erase the data in the buffer, but it +//// * is named as if it did because it will most often be used in situations +//// * in which that might as well be the case.

+//// */ +//// final def clear() { +//// // if (0 == globalCapacity) return +//// +//// needReleaseIndices += 0 +//// globalPosition = 0L +//// toNewContainer(0) +//// globalLimit = globalCapacity +//// +//// // Now free all pending containers +//// releasePendingContainers() +//// } +//// +//// /** +//// * Flips this buffer. The limit is set to the current position and then +//// * the position is set to zero. If the mark is defined then it is +//// * discarded. +//// * +//// *

After a sequence of channel-read or put operations, invoke +//// * this method to prepare for a sequence of channel-write or relative +//// * get operations. +//// */ +//// final def flip() { +//// needReleaseIndices += 0 +//// globalLimit = globalPosition +//// globalPosition = 0L +//// toNewContainer(0) +//// +//// // Now free all pending containers +//// releasePendingContainers() +//// } +//// +//// /** +//// * Rewinds this buffer. The position is set to zero and the mark is +//// * discarded. +//// * +//// *

Invoke this method before a sequence of channel-write or get +//// * operations, assuming that the limit has already been set +//// * appropriately. +//// */ +//// final def rewind() { +//// needReleaseIndices += 0 +//// globalPosition = 0L +//// toNewContainer(0) +//// +//// // Now free all pending containers +//// releasePendingContainers() +//// } +//// +//// /** +//// * Returns the number of elements between the current position and the +//// * limit.

+//// * +//// * @return The number of elements remaining in this buffer +//// */ +//// final def remaining(): Long = { +//// globalLimit - globalPosition +//// } +//// +//// /** +//// * Tells whether there are any elements between the current position and +//// * the limit.

+//// * +//// * @return true if, and only if, there is at least one element +//// * remaining in this buffer +//// */ +//// final def hasRemaining() = { +//// globalPosition < globalLimit +//// } +//// +//// // private def currentBuffer(): ByteBuffer = buffers(currentContainerIndex) +//// +//// // number of bytes remaining in currently active underlying buffer +//// private def currentRemaining(): Int = { +//// if (hasRemaining()) { +//// // validate currentContainerIndex is valid +//// assert (globalPosition >= bufferPositionStart(currentContainerIndex) && +//// globalPosition < bufferPositionStart(currentContainerIndex + 1), +//// "globalPosition = " + globalPosition + +//// ", currentContainerIndex = " + currentContainerIndex + +//// ", bufferPositionStart = " + bufferPositionStart.mkString("[", ", ", " ]")) +//// +//// currentRemaining0(currentContainerIndex) +//// } else 0 +//// } +//// +//// // Without any validation : required when we are bumping the index (when validation will fail) ... +//// private def currentRemaining0(which: Int): Int = { +//// // currentBuffer().remaining() +//// math.max(0, math.min(bufferPositionStart(which + 1), +//// globalLimit) - globalPosition).asInstanceOf[Int] +//// } +//// +//// // Set the approppriate position/limit for the current underlying buffer to mirror our +//// // the LargeByteBuffer's state. +//// private def fetchCurrentBuffer(): ByteBuffer = { +//// releasePendingContainers() +//// +//// assert (currentContainerIndex < containers.length) +//// +//// val container = containers(currentContainerIndex) +//// if (! container.isAcquired) { +//// container.acquire() +//// } +//// +//// assert (container.isAcquired) +//// if (LargeByteBuffer.enableExpensiveAssert) { +//// assert (! containers.exists( b => (b ne container) && b.isAcquired)) +//// } +//// +//// assert (currentContainerIndex < bufferPositionStart.length && +//// globalPosition < bufferPositionStart(currentContainerIndex + 1), +//// "currentContainerIndex = " + currentContainerIndex + ", bufferPositionStart = " + +//// bufferPositionStart.mkString("[", ", ", "]") + ", this = " + this) +//// +//// val buffPosition = (globalPosition - bufferPositionStart(currentContainerIndex)). +//// asInstanceOf[Int] +//// +//// val buffer = container.getByteBuffer +//// buffer.position(buffPosition) +//// val diff = buffer.capacity - buffPosition +//// val left = remaining() +//// if (diff <= left) { +//// buffer.limit(buffer.capacity()) +//// } else { +//// // Can happen if limit() was called. +//// buffer.limit(buffPosition + left.asInstanceOf[Int]) +//// } +//// +//// buffer +//// } +//// +//// // To be used ONLY to test in suites. +//// private[spark] def fetchCurrentBufferForTesting(): ByteBuffer = { +//// if ("1" != System.getProperty("SPARK_TESTING")) { +//// throw new IllegalStateException("This method is to be used ONLY within spark test suites") +//// } +//// +//// fetchCurrentBuffer() +//// } +//// +//// // Expects that the invoker has ensured that this can be safely invoked. +//// // That is, it wont be invoked when the loop wont terminate. +//// private def toNonEmptyBuffer() { +//// +//// if (! hasRemaining()) { +//// var newIndex = currentContainerIndex +//// // Ensure we are in the right block or not. +//// while (newIndex < containers.length && globalPosition >= bufferPositionStart(newIndex + 1)) { +//// newIndex += 1 +//// } +//// toNewContainer(newIndex) +//// // Do not do this - since we might not yet have consumed the buffer which caused EOF right now +//// /* +//// // Add last one also, and release it too - since we are at the end of the buffer with nothing +//// // more pending. +//// if (newIndex >= 0 && currentContainerIndex < containers.length) { +//// needReleaseIndices += newIndex +//// } +//// */ +//// assert (currentContainerIndex >= 0) +//// // releasePendingContainers() +//// return +//// } +//// +//// var index = currentContainerIndex +//// while (0 == currentRemaining0(index) && index < containers.length) { +//// index += 1 +//// } +//// assert (currentContainerIndex < containers.length) +//// toNewContainer(index) +//// assert (0 != currentRemaining()) +//// } +//// +//// private def assertPreconditions(containerIndex: Int) { +//// assert (globalPosition >= bufferPositionStart(containerIndex), +//// "globalPosition = " + globalPosition + ", containerIndex = " + containerIndex + +//// ", bufferPositionStart = " + bufferPositionStart.mkString("[", ", ", " ]")) +//// assert (globalPosition < bufferPositionStart(containerIndex + 1), +//// "globalPosition = " + globalPosition + ", containerIndex = " + containerIndex + +//// ", bufferPositionStart = " + bufferPositionStart.mkString("[", ", ", " ]")) +//// +//// assert (globalLimit <= globalCapacity) +//// assert (containerIndex < containers.length) +//// } +//// +//// +//// /** +//// * Attempts to return a ByteBuffer of the requested size. +//// * It is possible to return a buffer of size smaller than requested +//// * even though hasRemaining == true +//// * +//// * On return, position would have been moved 'ahead' by the size of the buffer returned : +//// * that is, we treat that the returned buffer has been already 'read' from this LargeByteBuffer +//// * +//// * +//// * This is used to primarily retrieve content of this buffer to expose via ByteBuffer +//// * to some other api which is deemed too cumbersome to move to LargeByteBuffer (like the +//// * chunked sending of contents via ConnectionManager) Note that the lifecycle of the ByteBuffer +//// * returned is inherently tied to the state of this LargeByteBuffer. For example,if the underlying +//// * container is a disk backed container, and we make subsequent calls to get(), the returned +//// * ByteBuffer can be dispose'ed off +//// * +//// * @param maxChunkSize Max size of the ByteBuffer to retrieve. +//// * @return +//// */ +//// +//// private def fetchBufferOfSize(maxChunkSize: Int): ByteBuffer = { +//// fetchBufferOfSizeImpl(maxChunkSize, canReleaseContainers = true) +//// } +//// +//// private def fetchBufferOfSizeImpl(maxChunkSize: Int, +//// canReleaseContainers: Boolean): ByteBuffer = { +//// if (canReleaseContainers) releasePendingContainers() +//// assert (maxChunkSize > 0) +//// +//// // not checking for degenerate case of maxChunkSize == 0 +//// if (globalPosition >= globalLimit) { +//// // throw exception +//// throw new BufferUnderflowException() +//// } +//// +//// // Check preconditions : disable these later, since they might be expensive to +//// // evaluate for every IO op +//// assertPreconditions(currentContainerIndex) +//// +//// val currentBufferRemaining = currentRemaining() +//// +//// assert (currentBufferRemaining > 0) +//// +//// val size = math.min(currentBufferRemaining, maxChunkSize) +//// +//// val newBuffer = if (currentBufferRemaining > maxChunkSize) { +//// val currentBuffer = fetchCurrentBuffer() +//// val buff = ByteBufferContainer.createSlice(currentBuffer, +//// currentBuffer.position(), maxChunkSize) +//// assert (buff.remaining() == maxChunkSize) +//// buff +//// } else { +//// val currentBuffer = fetchCurrentBuffer() +//// val buff = currentBuffer.slice() +//// assert (buff.remaining() == currentBufferRemaining) +//// buff +//// } +//// +//// assert (size == newBuffer.remaining()) +//// assert (0 == newBuffer.position()) +//// assert (size == newBuffer.limit()) +//// assert (newBuffer.capacity() == newBuffer.limit()) +//// +//// globalPosition += newBuffer.remaining +//// toNonEmptyBuffer() +//// +//// newBuffer +//// } +//// +//// // Can we service the read/write from the currently active (underlying) bytebuffer or not. +//// // For almost all cases, this will return true allowing us to optimize away the more expensive +//// // computations. +//// private def localReadWritePossible(size: Int) = +//// size >= 0 && globalPosition + size <= bufferPositionStart(currentContainerIndex + 1) +//// +//// +//// def getLong(): Long = { +//// assert (readable) +//// releasePendingContainers() +//// +//// if (remaining() < 8) throw new BufferUnderflowException +//// +//// if (localReadWritePossible(8)) { +//// val buff = fetchCurrentBuffer() +//// assert (buff.remaining() >= 8) +//// val retval = buff.getLong +//// globalPosition += 8 +//// toNonEmptyBuffer() +//// return retval +//// } +//// +//// val buff = readFully(8) +//// buff.getLong +//// } +//// +//// def getInt(): Int = { +//// assert (readable) +//// releasePendingContainers() +//// +//// if (remaining() < 4) throw new BufferUnderflowException +//// +//// if (localReadWritePossible(4)) { +//// val buff = fetchCurrentBuffer() +//// assert (buff.remaining() >= 4) +//// val retval = buff.getInt +//// globalPosition += 4 +//// toNonEmptyBuffer() +//// return retval +//// } +//// +//// val buff = readFully(4) +//// buff.getInt +//// } +//// +//// def getChar(): Char = { +//// assert (readable) +//// releasePendingContainers() +//// +//// if (remaining() < 2) throw new BufferUnderflowException +//// +//// if (localReadWritePossible(2)) { +//// val buff = fetchCurrentBuffer() +//// assert (buff.remaining() >= 2) +//// val retval = buff.getChar +//// globalPosition += 2 +//// toNonEmptyBuffer() +//// return retval +//// } +//// +//// // if slice is becoming too expensive, revisit this ... +//// val buff = readFully(2) +//// buff.getChar +//// } +//// +//// def get(): Byte = { +//// assert (readable) +//// releasePendingContainers() +//// +//// if (! hasRemaining()) throw new BufferUnderflowException +//// +//// // If we have remaining bytes, previous invocations MUST have ensured that we are at +//// // a buffer which has data to be read. +//// assert (localReadWritePossible(1)) +//// +//// val buff = fetchCurrentBuffer() +//// assert (buff.remaining() >= 1, "buff.remaining = " + buff.remaining()) +//// val retval = buff.get() +//// globalPosition += 1 +//// toNonEmptyBuffer() +//// +//// retval +//// } +//// +//// def get(arr: Array[Byte], offset: Int, size: Int): Int = { +//// assert (readable) +//// releasePendingContainers() +//// +//// LargeByteBuffer.checkOffsets(arr, offset, size) +//// +//// // kyro depends on this it seems ? +//// // assert (size > 0) +//// if (0 == size) return 0 +//// +//// if (! hasRemaining()) return -1 +//// +//// if (localReadWritePossible(size)) { +//// val buff = fetchCurrentBuffer() +//// assert (buff.remaining() >= size) +//// buff.get(arr, offset, size) +//// globalPosition += size +//// toNonEmptyBuffer() +//// return size +//// } +//// +//// var remainingSize = math.min(size, remaining()).asInstanceOf[Int] +//// var currentOffset = offset +//// +//// while (remainingSize > 0) { +//// val buff = fetchBufferOfSize(remainingSize) +//// val toCopy = math.min(buff.remaining(), remainingSize) +//// +//// buff.get(arr, currentOffset, toCopy) +//// currentOffset += toCopy +//// remainingSize -= toCopy +//// } +//// +//// currentOffset - offset +//// } +//// +//// +//// private def createSlice(size: Long): LargeByteBuffer = { +//// +//// releasePendingContainers() +//// +//// if (remaining() < size) { +//// // logInfo("createSlice. remaining = " + remaining() + ", size " + size + ", this = " + this) +//// throw new BufferOverflowException +//// } +//// +//// // kyro depends on this it seems ? +//// // assert (size > 0) +//// if (0 == size) return LargeByteBuffer.EMPTY_BUFFER +//// +//// val arr = new ArrayBuffer[ByteBufferContainer](2) +//// var totalLeft = size +//// +//// // assert (currentRemaining() < totalLeft || totalLeft != size || currentAsByteBuffer) +//// +//// var containerIndex = currentContainerIndex +//// while (totalLeft > 0 && hasRemaining()) { +//// assertPreconditions(containerIndex) +//// val container = containers(containerIndex) +//// val currentLeft = currentRemaining0(containerIndex) +//// +//// assert (globalPosition + currentLeft <= globalLimit) +//// assert (globalPosition >= bufferPositionStart(containerIndex) && +//// (globalPosition < bufferPositionStart(containerIndex + 1))) +//// +//// val from = (globalPosition - bufferPositionStart(containerIndex)).asInstanceOf[Int] +//// val sliceSize = math.min(totalLeft, currentLeft) +//// assert (from >= 0) +//// assert (sliceSize > 0 && sliceSize <= Int.MaxValue) +//// +//// val slice = container.createSlice(from, sliceSize.asInstanceOf[Int]) +//// arr += slice +//// +//// globalPosition += sliceSize +//// totalLeft -= sliceSize +//// if (currentLeft == sliceSize) containerIndex += 1 +//// } +//// +//// // Using toNonEmptyBuffer instead of directly moving to next here so that +//// // other checks can be performed there. +//// toNonEmptyBuffer() +//// // force cleanup - this is fine since we are not using the buffers directly +//// // which are actively needed (the returned value is on containers which can +//// // recreate) +//// releasePendingContainers() +//// // free current container if acquired. +//// if (currentContainerIndex < containers.length) { +//// containers(currentContainerIndex).release() +//// } +//// assert (currentContainerIndex == containerIndex) +//// +//// val retval = new LargeByteBuffer(arr, false, ephemeralDiskBacked) +//// retval.overrideCleaner(LargeByteBuffer.noopDisposeFunction) +//// retval +//// } +//// +//// // Get a composite sequence of ByteBuffer which might straddle one or more underlying buffers +//// // This is to be used only for writes : and ensures that writes are done into the appropriate +//// // underlying bytebuffers. +//// def getCompositeWriteBuffer(size: Long): LargeByteBuffer = { +//// assert(writable) +//// assert(size >= 0) +//// +//// createSlice(size) +//// } +//// +//// // get a buffer which is of the specified size and contains data from the underlying buffers +//// // Note, the actual data might be spread across the underlying buffers. +//// // This MUST BE used only for specific usecases like getInt, etc. Not for bulk copy ! +//// private def readFully(size: Int): ByteBuffer = { +//// assert (readable) +//// +//// if (remaining() < size) { +//// // throw exception +//// throw new BufferUnderflowException() +//// } +//// +//// // kyro depends on this it seems ? +//// // assert (size > 0) +//// if (0 == size) return LargeByteBuffer.EMPTY_BYTEBUFFER +//// +//// // Expected to be handled elsewhere. +//// assert (! localReadWritePossible(size)) +//// +//// val localBuff = { +//// val buff = fetchBufferOfSize(size) +//// // assert(buff.remaining() <= size) +//// // if (buff.remaining() == size) return buff +//// assert(buff.remaining() < size) +//// ByteBuffer.allocate(size).put(buff) +//// } +//// +//// // assert (localBuff.hasRemaining) +//// +//// while (localBuff.hasRemaining) { +//// val buff = fetchBufferOfSize(localBuff.remaining()) +//// localBuff.put(buff) +//// } +//// +//// localBuff.flip() +//// localBuff +//// } +//// +//// +//// +//// def put(b: Byte) { +//// assert (writable) +//// if (remaining() < 1) { +//// // logInfo("put byte. remaining = " + remaining() + ", this = " + this) +//// throw new BufferOverflowException +//// } +//// +//// assert (currentRemaining() > 0) +//// +//// fetchCurrentBuffer().put(b) +//// globalPosition += 1 +//// // Check to need to bump the index ? +//// toNonEmptyBuffer() +//// } +//// +//// +//// def put(buffer: ByteBuffer) { +//// assert (writable) +//// if (remaining() < buffer.remaining()) { +//// throw new BufferOverflowException +//// } +//// +//// val bufferRemaining = buffer.remaining() +//// if (localReadWritePossible(bufferRemaining)) { +//// +//// assert (currentRemaining() >= bufferRemaining) +//// +//// fetchCurrentBuffer().put(buffer) +//// +//// globalPosition += bufferRemaining +//// toNonEmptyBuffer() +//// return +//// } +//// +//// while (buffer.hasRemaining) { +//// val currentBufferRemaining = currentRemaining() +//// val bufferRemaining = buffer.remaining() +//// +//// if (currentBufferRemaining >= bufferRemaining) { +//// fetchCurrentBuffer().put(buffer) +//// globalPosition += bufferRemaining +//// } else { +//// // Split across buffers. +//// val currentBuffer = fetchCurrentBuffer() +//// assert (currentBuffer.remaining() >= currentBufferRemaining) +//// val sliced = ByteBufferContainer.createSlice(buffer, buffer.position(), +//// currentBufferRemaining) +//// assert (sliced.remaining() == currentBufferRemaining) +//// currentBuffer.put(sliced) +//// // move buffer pos +//// buffer.position(buffer.position() + currentBufferRemaining) +//// +//// globalPosition += currentBufferRemaining +//// } +//// toNonEmptyBuffer() +//// } +//// +//// assert (! hasRemaining() || currentRemaining() > 0) +//// } +//// +//// def put(other: LargeByteBuffer) { +//// assert (writable) +//// if (this.remaining() < other.remaining()) { +//// throw new BufferOverflowException +//// } +//// +//// while (other.hasRemaining()) { +//// val buffer = other.fetchBufferOfSize(other.currentRemaining()) +//// this.put(buffer) +//// } +//// } +//// +//// +//// def duplicate(): LargeByteBuffer = { +//// val containersCopy = new ArrayBuffer[ByteBufferContainer](containers.size) +//// // We do a duplicate as part of construction - so avoid double duplicate. +//// // containersCopy ++= containers.map(_.duplicate()) +//// containersCopy ++= containers +//// val retval = new LargeByteBuffer(containersCopy, true, ephemeralDiskBacked) +//// +//// // set limit and position (in that order) ... +//// retval.limit(this.limit()) +//// retval.position(this.position()) +//// +//// // Now release our containers - if any had been acquired +//// releasePendingContainers() +//// +//// retval +//// } +//// +//// +//// /** +//// * 'read' a LargeByteBuffer of size specified and return that. +//// * Position will be incremented by size +//// * +//// * The name might be slightly confusing : rename ? +//// * +//// * @param size Amount of data to be read from this buffer and returned +//// * @return +//// */ +//// def readLargeBuffer(size: Long, partialReadAllowed: Boolean): LargeByteBuffer = { +//// if (! hasRemaining() && ! partialReadAllowed) throw new BufferUnderflowException +//// if (remaining() < size && ! partialReadAllowed) throw new BufferUnderflowException +//// +//// +//// assert (readable) +//// assert (size >= 0) +//// +//// releasePendingContainers() +//// +//// if (0 == size) return LargeByteBuffer.EMPTY_BUFFER +//// +//// createSlice(size) +//// } +//// +//// +//// // This is essentially a workaround to exposing underlying buffers +//// def readFrom(channel: ReadableByteChannel): Long = { +//// +//// assert (writable) +//// releasePendingContainers() +//// +//// // this also allows us to avoid nasty corner cases in the loop. +//// if (! hasRemaining()) { +//// // logInfo("readFrom channel. remaining = " + remaining() + ", this = " + this) +//// throw new BufferOverflowException +//// } +//// +//// var totalBytesRead = 0L +//// +//// while (hasRemaining()) { +//// // read what we can ... +//// val buffer = fetchCurrentBuffer() +//// val bufferRemaining = currentRemaining() +//// val bytesRead = channel.read(buffer) +//// +//// if (bytesRead > 0) { +//// totalBytesRead += bytesRead +//// // bump position too .. +//// globalPosition += bytesRead +//// if (bytesRead >= bufferRemaining) toNonEmptyBuffer() +//// } +//// else if (-1 == bytesRead) { +//// // if we had already read some data in the loop, return that. +//// if (totalBytesRead > 0) return totalBytesRead +//// return -1 +//// } // nothing available to read, retry later. return +//// else if (0 == bytesRead) { +//// return totalBytesRead +//// } +//// +//// // toNonEmptyBuffer() +//// } +//// +//// // Cleanup last buffer ? +//// toNonEmptyBuffer() +//// totalBytesRead +//// } +//// +//// // This is essentially a workaround to exposing underlying buffers +//// def readFrom(inStrm: InputStream): Long = { +//// +//// assert (writable) +//// releasePendingContainers() +//// +//// // this also allows us to avoid nasty corner cases in the loop. +//// // if (! hasRemaining()) throw new BufferOverflowException +//// if (! hasRemaining()) return 0 +//// +//// var totalBytesRead = 0L +//// +//// val buff = new Array[Byte](LargeByteBuffer.TEMP_ARRAY_SIZE) +//// +//// while (hasRemaining()) { +//// // read what we can ... note, since there is no gaurantee that underlying buffer might +//// // expose array() method, we do double copy - from stream to buff and from buff to bytearray. +//// // see if we can optimize this later ... +//// val buffer = fetchCurrentBuffer() +//// val bufferRemaining = buffer.remaining() +//// val max = math.min(buff.length, bufferRemaining) +//// val bytesRead = inStrm.read(buff, 0, max) +//// +//// if (bytesRead > 0) { +//// buffer.put(buff, 0, bytesRead) +//// totalBytesRead += bytesRead +//// // bump position too .. +//// globalPosition += bytesRead +//// // buffer.position(buffer.position + bytesRead) +//// if (bytesRead >= bufferRemaining) toNonEmptyBuffer() +//// } +//// else if (-1 == bytesRead) { +//// // if we had already read some data in the loop, return that. +//// if (totalBytesRead > 0) return totalBytesRead +//// return -1 +//// } // nothing available to read, retry later. return +//// else if (0 == bytesRead) { +//// return totalBytesRead +//// } +//// +//// // toNonEmptyBuffer() +//// } +//// +//// totalBytesRead +//// } +//// +//// // This is essentially a workaround to exposing underlying buffers +//// // Note: very similar to readFrom(InputStream) : not trying anything fancy to reduce +//// // code for performance reasons. +//// def readFrom(inStrm: DataInput): Long = { +//// +//// assert (writable) +//// releasePendingContainers() +//// +//// // this also allows us to avoid nasty corner cases in the loop. +//// // if (! hasRemaining()) throw new BufferOverflowException +//// if (! hasRemaining()) return 0 +//// +//// var totalBytesRead = 0L +//// +//// val buff = new Array[Byte](LargeByteBuffer.TEMP_ARRAY_SIZE) +//// +//// while (hasRemaining()) { +//// // read what we can ... note, since there is no gaurantee that underlying buffer might +//// // expose array() method, we do double copy - from stream to buff and from buff to bytearray. +//// // see if we can optimize this later ... +//// val buffer = fetchCurrentBuffer() +//// val bufferRemaining = buffer.remaining() +//// val max = math.min(buff.length, bufferRemaining) +//// inStrm.readFully(buff, 0, max) +//// val bytesRead = max +//// +//// if (bytesRead > 0) { +//// buffer.put(buff, 0, bytesRead) +//// totalBytesRead += bytesRead +//// // bump position too .. +//// globalPosition += bytesRead +//// // buffer.position(buffer.position() + bytesRead) +//// if (bytesRead >= bufferRemaining) toNonEmptyBuffer() +//// } +//// else if (-1 == bytesRead) { +//// // if we had already read some data in the loop, return that. +//// if (totalBytesRead > 0) return totalBytesRead +//// return -1 +//// } // nothing available to read, retry later. return +//// else if (0 == bytesRead) { +//// return totalBytesRead +//// } +//// +//// // toNonEmptyBuffer() +//// } +//// +//// totalBytesRead +//// } +//// +//// // This is essentially a workaround to exposing underlying buffers +//// // Note: tries to do it efficiently without needing to load everything into memory +//// // (particularly for diskbacked buffers, etc). +//// def writeTo(channel: WritableByteChannel, cleanup: Boolean): Long = { +//// +//// assert (readable) +//// releasePendingContainers() +//// +//// // this also allows us to avoid nasty corner cases in the loop. +//// if (! hasRemaining()) throw new BufferUnderflowException +//// +//// var totalBytesWritten = 0L +//// +//// while (hasRemaining()) { +//// // Write what we can ... +//// val buffer = fetchCurrentBuffer() +//// val bufferRemaining = buffer.remaining() +//// assert (bufferRemaining > 0) +//// val bytesWritten = channel.write(buffer) +//// +//// if (bytesWritten > 0) { +//// totalBytesWritten += bytesWritten +//// // bump position too .. +//// globalPosition += bytesWritten +//// if (bytesWritten >= bufferRemaining) toNonEmptyBuffer() +//// assert (! hasRemaining() || currentRemaining() > 0) +//// } +//// else if (0 == bytesWritten) { +//// return totalBytesWritten +//// } +//// +//// // toNonEmptyBuffer() +//// } +//// +//// assert (! hasRemaining()) +//// if (cleanup) { +//// free() +//// } +//// totalBytesWritten +//// } +//// +//// // This is essentially a workaround to exposing underlying buffers +//// def writeTo(outStrm: OutputStream, cleanup: Boolean): Long = { +//// +//// assert (readable) +//// releasePendingContainers() +//// +//// // this also allows us to avoid nasty corner cases in the loop. +//// if (! hasRemaining()) throw new BufferUnderflowException +//// +//// var totalBytesWritten = 0L +//// val buff = new Array[Byte](LargeByteBuffer.TEMP_ARRAY_SIZE) +//// +//// while (hasRemaining()) { +//// // write what we can ... note, since there is no gaurantee that underlying buffer might +//// // expose array() method, we do double copy - from bytearray to buff and from +//// // buff to outputstream. see if we can optimize this later ... +//// val buffer = fetchCurrentBuffer() +//// val bufferRemaining = buffer.remaining() +//// val size = math.min(bufferRemaining, buff.length) +//// buffer.get(buff, 0, size) +//// outStrm.write(buff, 0, size) +//// +//// totalBytesWritten += size +//// // bump position too .. +//// globalPosition += size +//// +//// if (size >= bufferRemaining) toNonEmptyBuffer() +//// } +//// +//// toNonEmptyBuffer() +//// if (cleanup) { +//// free() +//// } +//// totalBytesWritten +//// } +//// +//// def asInputStream(): InputStream = { +//// new InputStream() { +//// override def read(): Int = { +//// if (! hasRemaining()) return -1 +//// get() +//// } +//// +//// override def read(arr: Array[Byte], off: Int, len: Int): Int = { +//// if (! hasRemaining()) return -1 +//// +//// get(arr, off, len) +//// } +//// +//// override def available(): Int = { +//// // current remaining is what can be read without blocking +//// // anything higher might need disk access/buffer swapping. +//// /* +//// val left = remaining() +//// math.min(left, Int.MaxValue).asInstanceOf[Int] +//// */ +//// currentRemaining() +//// } +//// } +//// } +//// +//// def getCleaner() = cleaner +//// +//// /** +//// * @param cleaner The previous cleaner, so that the caller can chain them if required. +//// * @return +//// */ +//// private[spark] def overrideCleaner(cleaner: BufferCleaner): BufferCleaner = { +//// overrideCleaner(cleaner, allowOverride = true) +//// } +//// +//// private def overrideCleaner(cleaner: BufferCleaner, allowOverride: Boolean): BufferCleaner = { +//// if (! this.allowCleanerOverride) { +//// // allowCleanerOverride = false is used for EMPTY_BUFFER - where we do not allow free +//// return this.cleaner +//// } +//// +//// this.allowCleanerOverride = allowOverride +//// assert (null != cleaner) +//// val prev = this.cleaner +//// this.cleaner = cleaner +//// // logInfo("Overriding " + prev + " with " + this.cleaner) +//// prev +//// } +//// +//// private def doReleaseAll() { +//// for (container <- containers) { +//// container.release() +//// } +//// } +//// +//// def free(invokeCleaner: Boolean = true) { +//// // logInfo("Free on " + this + ", cleaner = " + cleaner) +//// // always invoking release +//// doReleaseAll() +//// +//// if (invokeCleaner) cleaner.clean(this) +//// } +//// +//// private def doDispose(needRelease: Boolean) { +//// +//// if (disposeLocationThrowable ne null) { +//// logError("Already free'ed earlier at : ", disposeLocationThrowable) +//// logError("Current at ", new Throwable) +//// throw new IllegalStateException("Already freed.") +//// } +//// disposeLocationThrowable = new Throwable() +//// +//// // Forcefully cleanup all +//// if (needRelease) doReleaseAll() +//// +//// // Free in a different loop, in case different containers refer to same resource +//// // to release (like file) +//// for (container <- containers) { +//// container.free() +//// } +//// +//// needReleaseIndices.clear() +//// +//// // We should not use this buffer anymore : set the values such that f +//// // we dont ... +//// globalPosition = 0 +//// globalLimit = 0 +//// globalCapacity = 0 +//// } +//// +//// // copy data over ... MUST be used only for cases where array is known to be +//// // small to begin with. slightly risky method due to that assumption +//// def toByteArray(): Array[Byte] = { +//// val positionBackup = position() +//// val size = remaining() +//// if (size > Int.MaxValue) { +//// throw new IllegalStateException( +//// "Attempt to convert LargeByteBuffer to byte array when data held is more than 2G") +//// } +//// +//// val retval = new Array[Byte](size.asInstanceOf[Int]) +//// val readSize = get(retval, 0, retval.length) +//// assert (readSize == retval.length, +//// "readSize = " + readSize + ", retval.length = " + retval.length) +//// +//// position(positionBackup) +//// +//// retval +//// } +//// +//// // copy data over ... MUST be used only for cases where array is known to be +//// // small to begin with. slightly risky method due to that assumption +//// def toByteBuffer(): ByteBuffer = { +//// ByteBuffer.wrap(toByteArray()) +//// } +//// +//// def toInMemoryBuffer(ioConf: IOConfig): LargeByteBuffer = { +//// val retval = LargeByteBuffer.allocateMemoryBuffer(remaining(), ioConf) +//// val currentPosition = position() +//// retval.put(this) +//// position(currentPosition) +//// retval.clear() +//// retval +//// } +//// +//// +//// +//// // This is ONLY used for testing : that too as part of development of this and associated classes +//// // remove before contributing to spark. +//// def hexDump(): String = { +//// if (remaining() * 64 > Int.MaxValue) { +//// throw new UnsupportedOperationException("buffer too large " + remaining()) +//// } +//// +//// val sb = new StringBuilder((remaining() * 2).asInstanceOf[Int]) +//// +//// var perLine = 0 +//// var first = true +//// for (b <- toByteArray()) { +//// perLine += 1 +//// if (perLine % 8 == 0) { +//// sb.append('\n') +//// first = true +//// } +//// if (! first) sb.append(' ') +//// first = false +//// sb.append(java.lang.Integer.toHexString(b & 0xff)) +//// } +//// sb.append('\n') +//// sb.toString() +//// } +//// +//// override def toString: String = { +//// val sb: StringBuffer = new StringBuffer +//// sb.append(getClass.getName) +//// sb.append(' ') +//// sb.append(System.identityHashCode(this)) +//// sb.append("@[pos=") +//// sb.append(position()) +//// sb.append(" lim=") +//// sb.append(limit()) +//// sb.append(" cap=") +//// sb.append(capacity()) +//// sb.append("]") +//// sb.toString +//// } +//// +//// +//// +//// override def finalize(): Unit = { +//// var marked = false +//// if (containers ne null) { +//// if (containers.exists(container => container.isAcquired && container.requireRelease())) { +//// marked = true +//// logError("BUG: buffer was not released - and now going out of scope. " + +//// "Potential resource leak. Allocated at ", allocateLocationThrowable) +//// containers.foreach(_.release()) +//// } +//// if (containers.exists(container => !container.isFreed && container.requireFree())) { +//// if (!marked) { +//// logError("BUG: buffer was not freed - and now going out of scope. Potential resource leak", +//// allocateLocationThrowable) +//// } +//// else { +//// logError("BUG: buffer was not freed - and now going out of scope. Potential resource leak") +//// } +//// containers.foreach(_.free()) +//// } +//// } +//// super.finalize() +//// } +////} +//// +//// +////object LargeByteBuffer extends Logging { +//// +//// private val noopDisposeFunction = new BufferCleaner() { +//// protected def doClean(buffer: LargeByteBuffer) { +//// buffer.free(invokeCleaner = false) +//// } +//// } +//// +//// val enableExpensiveAssert = false +//// private val EMPTY_BYTEBUFFER = ByteBuffer.allocate(0) +//// val EMPTY_BUFFER = new LargeByteBuffer(ArrayBuffer( +//// new HeapByteBufferContainer(EMPTY_BYTEBUFFER, false)), false, false) +//// // Do not allow anyone else to override cleaner +//// EMPTY_BUFFER.overrideCleaner(noopDisposeFunction, allowOverride = false) +//// +//// // 8K sufficient ? +//// private val TEMP_ARRAY_SIZE = 8192 +//// +//// /** +//// * Create a LargeByteBuffer of specified size which is split across +//// * ByteBuffer's of size DiskStore.MAX_BLOCK_SIZE and backed by in memory +//// * ByteBuffer +//// * +//// */ +//// def allocateMemoryBuffer(totalSize: Long, ioConf: IOConfig): LargeByteBuffer = { +//// if (0 == totalSize) { +//// return EMPTY_BUFFER +//// } +//// +//// assert (totalSize > 0) +//// +//// val blockSize = ioConf.getMaxBlockSize(BufferType.MEMORY) +//// val numBlocks = ioConf.numBlocks(BufferType.MEMORY, totalSize) +//// val lastBlockSize = ioConf.lastBlockSize(BufferType.MEMORY, totalSize) +//// +//// assert (lastBlockSize > 0) +//// +//// val bufferArray = { +//// val arr = new ArrayBuffer[ByteBufferContainer](numBlocks) +//// for (index <- 0 until numBlocks - 1) { +//// val buff = ByteBuffer.allocate(blockSize) +//// // buff.clear() +//// arr += new HeapByteBufferContainer(buff, true) +//// } +//// arr += new HeapByteBufferContainer(ByteBuffer.allocate(lastBlockSize), true) +//// assert (arr.length == numBlocks) +//// arr +//// } +//// +//// new LargeByteBuffer(bufferArray, false, false) +//// } +//// +//// /** +//// * Create a LargeByteBuffer of specified size which is split across +//// * ByteBuffer's of size DiskStore.MAX_BLOCK_SIZE and backed by on disk +//// * +//// */ +//// private def allocateDiskBuffer(totalSize: Long, +//// blockManager: BlockManager): LargeByteBuffer = { +//// if (0 == totalSize) { +//// return EMPTY_BUFFER +//// } +//// +//// assert (totalSize > 0) +//// +//// // Create a file of the specified size. +//// val file = blockManager.diskBlockManager.createTempBlock()._2 +//// val raf = new RandomAccessFile(file, "rw") +//// try { +//// raf.setLength(totalSize) +//// } finally { +//// raf.close() +//// } +//// +//// readWriteDiskSegment(new FileSegment(file, 0, totalSize), +//// ephemeralDiskBacked = true, blockManager.ioConf) +//// } +//// +//// // The returned buffer takes up ownership of the underlying buffers +//// // (including dispos'ing that when done) +//// def fromBuffers(buffers: ByteBuffer*): LargeByteBuffer = { +//// val nonEmpty = buffers.filter(_.hasRemaining) +//// +//// // cleanup the empty buffers +//// buffers.filter(! _.hasRemaining).foreach(b => BlockManager.dispose(b)) +//// +//// +//// if (nonEmpty.isEmpty) { +//// return EMPTY_BUFFER +//// } +//// +//// // slice so that offsets match our requirement +//// new LargeByteBuffer(new ArrayBuffer() ++ nonEmpty.map(b => +//// new HeapByteBufferContainer(b.slice(), true)), false, false) +//// } +//// +//// def fromByteArrays(byteArrays: Array[Byte]*): LargeByteBuffer = { +//// // only non empty arrays +//// val arrays = byteArrays.filter(_.length > 0) +//// if (0 == arrays.length) return EMPTY_BUFFER +//// +//// new LargeByteBuffer(new ArrayBuffer() ++ arrays.map(arr => +//// new HeapByteBufferContainer(ByteBuffer.wrap(arr), true)), false, false) +//// } +//// +//// def fromLargeByteBuffers(canDispose: Boolean, inputBuffers: LargeByteBuffer*): LargeByteBuffer = { +//// +//// if (inputBuffers.isEmpty) return EMPTY_BUFFER +//// +//// if (! inputBuffers.exists(_.hasRemaining())) { +//// if (canDispose) inputBuffers.map(_.free()) +//// return EMPTY_BUFFER +//// } +//// +//// // release all temp resources acquired +//// inputBuffers.foreach(buff => buff.releasePendingContainers()) +//// // free current container if acquired. +//// inputBuffers.foreach(buff => if (buff.currentContainerIndex < buff.containers.length) { +//// buff.containers(buff.currentContainerIndex).release() +//// }) +//// // inputBuffers.foreach(b => b.doReleaseAll()) +//// +//// +//// // Dispose of any empty buffers +//// if (canDispose) inputBuffers.filter(! _.hasRemaining()).foreach(_.free()) +//// +//// // Find all containers we need. +//// val buffers = inputBuffers.filter(_.hasRemaining()).map(b => b.createSlice(b.remaining())) +//// +//// val containers = buffers.flatMap(_.containers) +//// assert (! containers.isEmpty) +//// // The in order containers of "buffers" seq constitute the required return value +//// val retval = new LargeByteBuffer(new ArrayBuffer() ++ containers, +//// // if you cant dispose, then we dont own the buffers : in which case, need duplicate +//// ! canDispose, inputBuffers.exists(_.ephemeralDiskBacked)) +//// +//// if (canDispose) { +//// // override dispose of all other buffers. +//// val disposeFunctions = inputBuffers.map { +//// buffer => { +//// (buffer, buffer.overrideCleaner(noopDisposeFunction)) +//// } +//// } +//// +//// val cleaner = retval.getCleaner() +//// val newCleaner = new BufferCleaner { +//// protected def doClean(buffer: LargeByteBuffer) { +//// +//// assert (retval == buffer) +//// // default cleaner. +//// cleaner.clean(retval) +//// // not required, since we are within clean anyway. +//// // retval.free(invokeCleaner = false) +//// +//// // retval.doDispose(needRelease = true) +//// +//// // This might actually call dispose twice on some (initially) empty buffers, +//// // which is fine since we now guard against that. +//// disposeFunctions.foreach(v => v._2.clean(v._1)) +//// // Call the free method too : so that buffers are marked free ... +//// disposeFunctions.foreach(v => v._1.free(invokeCleaner = false)) +//// } +//// } +//// +//// val prev = retval.overrideCleaner(newCleaner) +//// assert (prev == cleaner) +//// } +//// +//// retval +//// } +//// +//// private def checkOffsets(arr: Array[Byte], offset: Int, size: Int) { +//// if (arr == null) { +//// throw new NullPointerException +//// } else if (offset < 0 || size < 0 || offset + size > arr.length) { +//// throw new IndexOutOfBoundsException +//// } +//// } +//// +//// def allocateTransientBuffer(size: Long, blockManager: BlockManager) = { +//// if (size <= blockManager.ioConf.maxInMemSize) { +//// LargeByteBuffer.allocateMemoryBuffer(size, blockManager.ioConf) +//// } else { +//// LargeByteBuffer.allocateDiskBuffer(size, blockManager) +//// } +//// } +//// +//// def readFromDiskSegment(segment: FileSegment, ioConf: IOConfig, +//// ephemeralDiskBacked: Boolean): LargeByteBuffer = { +//// // Split the block into multiple of BlockStore.maxBlockSize +//// val segmentSize = segment.length +//// val blockSize = ioConf.getMaxBlockSize(BufferType.DISK).asInstanceOf[Long] +//// val numBlocks = ioConf.numBlocks(BufferType.DISK, segmentSize) +//// val lastBlockSize = ioConf.lastBlockSize(BufferType.DISK, segmentSize) +//// +//// val buffers = new ArrayBuffer[ByteBufferContainer](numBlocks) +//// +//// for (index <- 0 until numBlocks - 1) { +//// buffers += new ReadOnlyFileContainer(new FileSegment(segment.file, +//// segment.offset + index * blockSize, blockSize), ioConf) +//// } +//// +//// // Last block +//// buffers += new ReadOnlyFileContainer(new FileSegment(segment.file, +//// segment.offset + (numBlocks - 1) * blockSize, lastBlockSize), ioConf) +//// +//// new LargeByteBuffer(buffers, false, ephemeralDiskBacked) +//// } +//// +//// def readWriteDiskSegment(segment: FileSegment, ephemeralDiskBacked: Boolean, +//// ioConf: IOConfig): LargeByteBuffer = { +//// +//// // Split the block into multiple of BlockStore.maxBlockSize +//// val segmentSize = segment.length +//// val blockSize = ioConf.getMaxBlockSize(BufferType.DISK).asInstanceOf[Long] +//// val numBlocks = ioConf.numBlocks(BufferType.DISK, segmentSize) +//// val lastBlockSize = ioConf.lastBlockSize(BufferType.DISK, segmentSize) +//// +//// logInfo("readWriteDiskSegment = " + segment + ", numBlocks = " + numBlocks + +//// ", lastBlockSize = " + lastBlockSize) +//// val buffers = new ArrayBuffer[ByteBufferContainer](numBlocks) +//// +//// for (index <- 0 until numBlocks - 1) { +//// buffers += new ReadWriteFileContainer(new FileSegment(segment.file, +//// segment.offset + index * blockSize, blockSize), ephemeralDiskBacked, null) +//// } +//// +//// // Last block +//// buffers += new ReadWriteFileContainer(new FileSegment(segment.file, +//// segment.offset + (numBlocks - 1) * blockSize, lastBlockSize), ephemeralDiskBacked, null) +//// +//// new LargeByteBuffer(buffers, false, ephemeralDiskBacked) +//// } +////} From 01cafbf15026fdcbfd58566335802082493a491c Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 25 Feb 2015 15:53:22 -0600 Subject: [PATCH 07/97] tests compile too --- core/src/test/scala/org/apache/spark/DistributedSuite.scala | 3 +-- .../scala/org/apache/spark/storage/BlockManagerSuite.scala | 4 ++-- 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/DistributedSuite.scala b/core/src/test/scala/org/apache/spark/DistributedSuite.scala index b183373de5019..97ea3578aa8ba 100644 --- a/core/src/test/scala/org/apache/spark/DistributedSuite.scala +++ b/core/src/test/scala/org/apache/spark/DistributedSuite.scala @@ -17,7 +17,6 @@ package org.apache.spark -import org.apache.spark.io.LargeByteBuffer import org.scalatest.FunSuite import org.scalatest.concurrent.Timeouts._ import org.scalatest.Matchers @@ -196,7 +195,7 @@ class DistributedSuite extends FunSuite with Matchers with LocalSparkContext { blockManager.master.getLocations(blockId).foreach { cmId => val bytes = blockTransfer.fetchBlockSync(cmId.host, cmId.port, cmId.executorId, blockId.toString) - val deserialized = blockManager.dataDeserialize(blockId, LargeByteBuffer.asLargeByteBuffer(bytes.nioByteBuffer())) + val deserialized = blockManager.dataDeserialize(blockId, bytes.nioByteBuffer()) .asInstanceOf[Iterator[Int]].toList assert(deserialized === (1 to 100).toList) } 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 f692c0ffe967f..05107fca1a5d1 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -21,7 +21,7 @@ import java.nio.{ByteBuffer, MappedByteBuffer} import java.util.Arrays import java.util.concurrent.TimeUnit -import org.apache.spark.io.LargeByteBuffer +import org.apache.spark.network.buffer.{LargeByteBufferHelper, LargeByteBuffer} import scala.collection.mutable.ArrayBuffer import scala.concurrent.Await @@ -811,7 +811,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfterEach var counter = 0.toByte def incr = {counter = (counter + 1).toByte; counter;} val bytes = Array.fill[Byte](1000)(incr) - val byteBuffer = LargeByteBuffer.asLargeByteBuffer(bytes) + val byteBuffer = LargeByteBufferHelper.asLargeByteBuffer(bytes) val blockId = BlockId("rdd_1_2") From ce391a0dbbba3d169d4013d2e387b7808065b3f8 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 25 Feb 2015 16:00:00 -0600 Subject: [PATCH 08/97] failing test case (though its crappy) --- .../network/netty/NettyBlockTransferSuite.scala | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSuite.scala b/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSuite.scala index b242ff593f04a..2133b3286ff36 100644 --- a/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSuite.scala +++ b/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.network.netty import java.util.concurrent.TimeUnit +import org.apache.commons.io.IOUtils import org.apache.spark.network.BlockDataManager import org.apache.spark.network.buffer.{ManagedBuffer, LargeByteBufferHelper, NioManagedBuffer} import org.apache.spark.network.shuffle.BlockFetchingListener @@ -42,14 +43,16 @@ class NettyBlockTransferSuite extends FunSuite with Matchers with MockitoSugar { val blockManager = mock[BlockDataManager] val blockId = ShuffleBlockId(0, 1, 2) - val blockString = "Hello, world!" - val blockBuffer = new NioManagedBuffer(LargeByteBufferHelper.asLargeByteBuffer(blockString.getBytes)) + val buf = LargeByteBufferHelper.allocate(Integer.MAX_VALUE.toLong + 100l) + val blockBuffer = new NioManagedBuffer(buf) when(blockManager.getBlockData(blockId)).thenReturn(blockBuffer) val from = new NettyBlockTransferService(conf, securityManager, numCores = 1) from.init(blockManager) + println("from: " + from.hostName + ":" + from.port) val to = new NettyBlockTransferService(conf, securityManager, numCores = 1) to.init(blockManager) + println("to: " + to.hostName + ":" + to.port) try { val promise = Promise[ManagedBuffer]() @@ -65,7 +68,10 @@ class NettyBlockTransferSuite extends FunSuite with Matchers with MockitoSugar { } }) - Await.ready(promise.future, FiniteDuration(1000, TimeUnit.MILLISECONDS)) + Await.ready(promise.future, FiniteDuration(100, TimeUnit.SECONDS)) + val v = promise.future.value.get.get +// IOUtils.toString(v.createInputStream()) should equal(blockString) + println(v.nioByteBuffer().limit()) } finally { from.close() to.close() From 29f0a8a10c685ea2742d239a748bc6c5d7798380 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Fri, 27 Feb 2015 13:19:12 -0600 Subject: [PATCH 09/97] fix use of LargeByteBuffer in some tests, create UploadPartialBlock --- .../buffer/WrappedLargeByteBuffer.java | 4 +- .../network/buffer/LargeByteBuffer.scala | 1760 ----------------- .../network/ChunkFetchIntegrationSuite.java | 15 +- .../spark/network/TestManagedBuffer.java | 5 +- .../protocol/BlockTransferMessage.java | 3 +- .../shuffle/protocol/UploadPartialBlock.java | 128 ++ .../shuffle/BlockTransferMessagesSuite.java | 2 + .../ExternalShuffleBlockHandlerSuite.java | 5 +- .../ExternalShuffleIntegrationSuite.java | 13 +- .../shuffle/OneForOneBlockFetcherSuite.java | 13 +- .../shuffle/RetryingBlockFetcherSuite.java | 7 +- 11 files changed, 166 insertions(+), 1789 deletions(-) delete mode 100644 network/common/src/main/scala/org/apache/spark/network/buffer/LargeByteBuffer.scala create mode 100644 network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/UploadPartialBlock.java diff --git a/network/common/src/main/java/org/apache/spark/network/buffer/WrappedLargeByteBuffer.java b/network/common/src/main/java/org/apache/spark/network/buffer/WrappedLargeByteBuffer.java index 2ecb569b3a23c..e948fa67581d0 100644 --- a/network/common/src/main/java/org/apache/spark/network/buffer/WrappedLargeByteBuffer.java +++ b/network/common/src/main/java/org/apache/spark/network/buffer/WrappedLargeByteBuffer.java @@ -71,9 +71,9 @@ public byte get() { private void updateCurrentBuffer() { //TODO fix end condition - while(!currentBuffer.hasRemaining()) { + while(currentBuffer != null && !currentBuffer.hasRemaining()) { currentBufferIdx += 1; - currentBuffer = underlying[currentBufferIdx]; + currentBuffer = currentBufferIdx < underlying.length ? underlying[currentBufferIdx] : null; } } diff --git a/network/common/src/main/scala/org/apache/spark/network/buffer/LargeByteBuffer.scala b/network/common/src/main/scala/org/apache/spark/network/buffer/LargeByteBuffer.scala deleted file mode 100644 index eb7520d3c6325..0000000000000 --- a/network/common/src/main/scala/org/apache/spark/network/buffer/LargeByteBuffer.scala +++ /dev/null @@ -1,1760 +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.network.buffer -// -//import java.nio.ByteBuffer -//import java.nio.channels.FileChannel.MapMode -//import java.nio.channels.{FileChannel, WritableByteChannel} -// -//import scala.collection.mutable.ArrayBuffer -// -// -// -//trait LargeByteBuffer { -//// def position(): Long -//// -//// def limit(): Long -// -// def capacity(): Long -// -// def get(): Byte //needed for ByteBufferInputStream -// -// def get(dst: Array[Byte], offset: Int, length: Int): Unit // for ByteBufferInputStream -// -// def position(position: Long): Unit //for ByteBufferInputStream -// -// def position(): Long //for ByteBufferInputStream -// -// /** doesn't copy data, just copies references & offsets */ -// def duplicate(): LargeByteBuffer -// -// def put(bytes: LargeByteBuffer): Unit -// -// //also need whatever is necessary for ByteArrayOutputStream for BlockManager#dataSerialize -// -// -// //TODO checks on limit semantics -// -// /** -// * Sets this buffer's limit. If the position is larger than the new limit then it is set to the -// * new limit. If the mark is defined and larger than the new limit then it is discarded. -// */ -// def limit(newLimit: Long): Unit -// -// /** -// * return this buffer's limit -// * @return -// */ -// def limit(): Long -// -// -// //an alternative to having this method would be having a foreachBuffer(f: Buffer => T) -// def writeTo(channel: WritableByteChannel): Long -// -//// -//// def skip(skipBy: Long): Unit -//// -//// def position(newPosition: Long): Unit -//// -//// /** -//// * Clears this buffer. The position is set to zero, the limit is set to -//// * the capacity, and the mark is discarded. -//// * -//// *

Invoke this method before using a sequence of channel-read or -//// * put operations to fill this buffer. -//// * -//// *

This method does not actually erase the data in the buffer, but it -//// * is named as if it did because it will most often be used in situations -//// * in which that might as well be the case.

-//// */ -//// def clear(): Unit -//// -//// /** -//// * Flips this buffer. The limit is set to the current position and then -//// * the position is set to zero. If the mark is defined then it is -//// * discarded. -//// * -//// *

After a sequence of channel-read or put operations, invoke -//// * this method to prepare for a sequence of channel-write or relative -//// * get operations. -//// */ -//// def flip(): Unit -// -// /** -// * Rewinds this buffer. The position is set to zero and the mark is -// * discarded. -// * -// *

Invoke this method before a sequence of channel-write or get -// * operations, assuming that the limit has already been set -// * appropriately. -// */ -// def rewind(): Unit -// -// /** -// * Returns the number of elements between the current position and the -// * limit.

-// * -// * @return The number of elements remaining in this buffer -// */ -// def remaining(): Long -//} -//// -////class ChainedLargeByteBuffer(private[network] val underlying: ChainedBuffer) extends LargeByteBuffer { -//// -//// def capacity = underlying.capacity -//// -//// var _pos = 0l -//// -//// def get(dst: Array[Byte],offset: Int,length: Int): Unit = { -//// underlying.read(_pos, dst, offset, length) -//// _pos += length -//// } -//// -//// def get(): Byte = { -//// val b = underlying.read(_pos) -//// _pos += 1 -//// b -//// } -//// -//// def put(bytes: LargeByteBuffer): Unit = { -//// ??? -//// } -//// -//// def position: Long = _pos -//// def position(position: Long): Unit = { -//// _pos = position -//// } -//// def remaining(): Long = { -//// underlying.size - position -//// } -//// -//// def duplicate(): ChainedLargeByteBuffer = { -//// new ChainedLargeByteBuffer(underlying) -//// } -//// -//// def rewind(): Unit = { -//// _pos = 0 -//// } -//// -//// def limit(): Long = { -//// capacity -//// } -//// -//// def limit(newLimit: Long): Unit = { -//// ??? -//// } -//// -//// def writeTo(channel:WritableByteChannel): Long = { -//// var written = 0l -//// underlying.chunks.foreach{bytes => -//// //TODO test this -//// val buffer = ByteBuffer.wrap(bytes) -//// while (buffer.hasRemaining) -//// channel.write(buffer) -//// written += bytes.length -//// } -//// written -//// } -////} -// -//class WrappedLargeByteBuffer(private[spark] val underlying: Array[ByteBuffer]) extends LargeByteBuffer { -// -// val (totalCapacity, chunkOffsets) = { -// var sum = 0l -// val offsets = new Array[Long](underlying.size) -// (0 until underlying.size).foreach{idx => -// offsets(idx) = sum -// sum += underlying(idx).capacity() -// } -// (sum, offsets) -// } -// -// private var _pos = 0l -// private var currentBufferIdx = 0 -// private var currentBuffer = underlying(0) -// private var _limit = totalCapacity -// -// def capacity = totalCapacity -// -// def get(dst: Array[Byte], offset: Int, length: Int): Unit = { -// var moved = 0 -// while (moved < length) { -// val toRead = math.min(length - moved, currentBuffer.remaining()) -// currentBuffer.get(dst, offset, toRead) -// moved += toRead -// updateCurrentBuffer() -// } -// } -// -// def get(): Byte = { -// val r = currentBuffer.get() -// _pos += 1 -// updateCurrentBuffer() -// r -// } -// -// private def updateCurrentBuffer(): Unit = { -// //TODO fix end condition -// while(!currentBuffer.hasRemaining()) { -// currentBufferIdx += 1 -// currentBuffer = underlying(currentBufferIdx) -// } -// } -// -// def put(bytes: LargeByteBuffer): Unit = { -// ??? -// } -// -// def position: Long = _pos -// def position(position: Long): Unit = { -// //XXX check range? -// _pos = position -// } -// def remaining(): Long = { -// totalCapacity - _pos -// } -// -// def duplicate(): WrappedLargeByteBuffer = { -// new WrappedLargeByteBuffer(underlying.map{_.duplicate()}) -// } -// -// def rewind(): Unit = { -// _pos = 0 -// underlying.foreach{_.rewind()} -// } -// -// def limit(): Long = { -// totalCapacity -// } -// -// def limit(newLimit: Long) = { -// //XXX check range? set limits in sub buffers? -// _limit = newLimit -// } -// -// def writeTo(channel: WritableByteChannel): Long = { -// var written = 0l -// underlying.foreach{buffer => -// //TODO test this -// //XXX do we care about respecting the limit here? -// written += buffer.remaining() -// while (buffer.hasRemaining) -// channel.write(buffer) -// } -// written -// } -// -//} -// -//object LargeByteBuffer { -// -// def asLargeByteBuffer(byteBuffer: ByteBuffer): LargeByteBuffer = { -// new WrappedLargeByteBuffer(Array(byteBuffer)) -// } -// -// def asLargeByteBuffer(bytes: Array[Byte]): LargeByteBuffer = { -// new WrappedLargeByteBuffer(Array(ByteBuffer.wrap(bytes))) -// } -// -//// -//// def allocateOnHeap(size: Long, maxChunk: Int): LargeByteBuffer = { -//// val buffer = ChainedBuffer.withInitialSize(maxChunk, size) -//// new ChainedLargeByteBuffer(buffer) -//// } -// -// def mapFile( -// channel: FileChannel, -// mode: MapMode, -// offset: Long, -// length: Long, -// maxChunk: Int = Integer.MAX_VALUE - 1e6.toInt -// ): LargeByteBuffer = { -// val offsets = new ArrayBuffer[Long]() -// var curOffset = offset -// val end = offset + length -// while (curOffset < end) { -// offsets += curOffset -// val length = math.min(end - curOffset, maxChunk) -// curOffset += length -// } -// offsets += end -// val chunks = new Array[ByteBuffer](offsets.size - 1) -// (0 until offsets.size - 1).foreach{idx => -// chunks(idx) = channel.map(mode, offsets(idx), offsets(idx + 1) - offsets(idx)) -// } -// new WrappedLargeByteBuffer(chunks) -// } -//} -// -// -//// -/////** -//// * This is a variant of ByteBuffer to be used internally in spark, which is not limited to 2G -//// * which ByteBuffers are limited to. -//// * Externally, it exposes all the api which java.nio.ByteBuffer exposes. -//// * Internally, it maintains a sequence of Containers which manage the ByteBuffer data. -//// * Not all the data might be loaded into memory (like disk or tachyon data) - so actual -//// * memory footprint - heap and vm could be much lower than capacity. -//// * -//// * TODO: Currently we are slightly fast and loose in terms of concurrent modifications to this -//// * buffer, maybe revisit this later ? Note: this is not much different from earlier though ! -//// * -//// * TODO: Explore if (at all) we can leverage zero copy transfers. The issue (currently) is that this -//// * will require the file to be kept open (repeatedly opening/closing file is not good -//// * for each transfer) and this has an impact on ulimit. Not to mention writing of mmap'ed buffer is -//// * pretty quick (it is the first failover in case direct transfer is not possible in file zero copy) -//// * -//// * TODO: After redesign to containers, we got rid of parent containers to free - the side effect is -//// * that if there are direct ByteBuffers, we are not handling explicit cleanup of those in some -//// * cases (when we duplicate/slice them). Currently spark does not need this, but might in future -//// * so relook at it later. -//// */ -////// We should make this constructor private: but for now, -////// leaving it public since TachyonStore needs it -////class LargeByteBuffer private[spark](private val inputContainers: ArrayBuffer[ByteBufferContainer], -//// private val needDuplicate: Boolean, val ephemeralDiskBacked: Boolean) extends Logging { -//// -//// // TODO: TEMP code: to flush out potential resource leaks. REMOVE ME -//// private val allocateLocationThrowable: Throwable = { -//// if (inputContainers.exists(c => c.requireRelease() || c.requireFree())) { -//// new Throwable("blockId = " + BlockManager.getLookupBlockId) -//// } else { -//// null -//// } -//// } -//// private var disposeLocationThrowable: Throwable = null -//// -//// @volatile private var allowCleanerOverride = true -//// @volatile private var cleaner: BufferCleaner = new BufferCleaner { -//// override def doClean(buffer: LargeByteBuffer) = { -//// assert (LargeByteBuffer.this == buffer) -//// doDispose(needRelease = false) -//// } -//// } -//// -//// // should not be empty -//// assert (null != inputContainers && ! inputContainers.isEmpty) -//// // should not have any null's -//// assert (inputContainers.find(_ == null).isEmpty) -//// -//// // println("Num containers = " + inputContainers.size) -//// -//// // Position, limit and capacity relevant over the engire LargeByteBuffer -//// @volatile private var globalPosition = 0L -//// @volatile private var globalLimit = 0L -//// @volatile private var currentContainerIndex = 0 -//// -//// // The buffers in which the actual data is held. -//// private var containers: Array[ByteBufferContainer] = null -//// -//// // aggregate capacities of the individual buffers. -//// // bufferPositionStart(0) will be capacity of 1st buffer, bufferPositionStart(1) will be -//// // sum of capacity of 0th and 1st block buffer -//// private var bufferPositionStart: Array[Long] = null -//// -//// // Contains the indices of a containers which requires release before subsequent invocation of -//// // read/write should be serviced. This is required since current read/write might have moved the -//// // position but since we are returning bytebuffers which depend on the validity of the existing -//// // bytebuffer, we cant release them yet. -//// private var needReleaseIndices = new HashSet[Int]() -//// -//// private val readable = ! inputContainers.exists(! _.isReadable) -//// private val writable = ! inputContainers.exists(! _.isWritable) -//// -//// -//// // initialize -//// @volatile private var globalCapacity = { -//// -//// // Ensure that there are no empty buffers : messes up with our code : unless it -//// // is a single buffer (for empty buffer for marker case) -//// assert (inputContainers.find(0 == _.capacity()).isEmpty || 1 == inputContainers.length) -//// -//// containers = { -//// if (needDuplicate) inputContainers.map(_.duplicate()).toArray else inputContainers.toArray -//// } -//// containers.foreach(_.validate()) -//// -//// def initializeBufferPositionStart(arr: Array[ByteBufferContainer]) { -//// val buff = new ArrayBuffer[Long](arr.length + 1) -//// buff += 0L -//// -//// buff ++= arr.map(_.capacity().asInstanceOf[Long]).scanLeft(0L)(_ + _).slice(1, arr.length + 1) -//// assert (buff.length == arr.length + 1) -//// bufferPositionStart = buff.toArray -//// } -//// -//// initializeBufferPositionStart(containers) -//// -//// // remove references from inputBuffers -//// inputContainers.clear() -//// -//// globalLimit = bufferPositionStart(containers.length) -//// globalPosition = 0L -//// currentContainerIndex = 0 -//// -//// assert (globalLimit == containers.map(_.capacity().asInstanceOf[Long]).sum) -//// -//// globalLimit -//// } -//// -//// final def position(): Long = globalPosition -//// -//// final def limit(): Long = globalLimit -//// -//// final def capacity(): Long = globalCapacity -//// -//// final def limit(newLimit: Long) { -//// if ((newLimit > capacity()) || (newLimit < 0)) { -//// throw new IllegalArgumentException("newLimit = " + newLimit + ", capacity = " + capacity()) -//// } -//// -//// globalLimit = newLimit -//// if (position() > newLimit) position(newLimit) -//// } -//// -//// def skip(skipBy: Long) = position(position() + skipBy) -//// -//// private def releasePendingContainers() { -//// if (! needReleaseIndices.isEmpty) { -//// val iter = needReleaseIndices.iterator -//// while (iter.hasNext) { -//// val index = iter.next() -//// assert (index >= 0 && index < containers.length) -//// // It is possible to move from one container to next before the previous -//// // container was acquired. For example, get forcing move to next container -//// // since current was exhausted immediatelly followed by a position() -//// // so the container we moved to was never acquired. -//// -//// // assert (containers(index).isAcquired) -//// // will this always be satisfied ? -//// // assert (index != currentContainerIndex) -//// if (containers(index).isAcquired) containers(index).release() -//// } -//// needReleaseIndices.clear() -//// } -//// } -//// -//// private def toNewContainer(newIndex: Int) { -//// if (newIndex != currentContainerIndex && currentContainerIndex < containers.length) { -//// -//// assert (currentContainerIndex >= 0) -//// needReleaseIndices += currentContainerIndex -//// } -//// currentContainerIndex = newIndex -//// } -//// -//// // expensive method, sigh ... optimize it later ? -//// final def position(newPosition: Long) { -//// -//// if ((newPosition > globalLimit) || (newPosition < 0)) throw new IllegalArgumentException() -//// -//// if (currentContainerIndex < bufferPositionStart.length - 1 && -//// newPosition >= bufferPositionStart(currentContainerIndex) && -//// newPosition < bufferPositionStart(currentContainerIndex + 1)) { -//// // Same buffer - easy method ... -//// globalPosition = newPosition -//// // Changed position - free previously returned buffers. -//// releasePendingContainers() -//// return -//// } -//// -//// // Find appropriate currentContainerIndex -//// // Since bufferPositionStart is sorted, can be replaced with binary search if required. -//// // For now, not in the perf critical path since buffers size is very low typically. -//// var index = 0 -//// val cLen = containers.length -//// while (index < cLen) { -//// if (newPosition >= bufferPositionStart(index) && -//// newPosition < bufferPositionStart(index + 1)) { -//// globalPosition = newPosition -//// toNewContainer(index) -//// // Changed position - free earlier and previously returned buffers. -//// releasePendingContainers() -//// return -//// } -//// index += 1 -//// } -//// -//// if (newPosition == globalLimit && newPosition == bufferPositionStart(cLen)) { -//// // boundary. -//// globalPosition = newPosition -//// toNewContainer(cLen) -//// // Changed position - free earlier and previously returned buffers. -//// releasePendingContainers() -//// return -//// } -//// -//// assert (assertion = false, "Unexpected to come here .... newPosition = " + newPosition + -//// ", bufferPositionStart = " + bufferPositionStart.mkString("[", ", ", "]")) -//// } -//// -//// -//// /** -//// * Clears this buffer. The position is set to zero, the limit is set to -//// * the capacity, and the mark is discarded. -//// * -//// *

Invoke this method before using a sequence of channel-read or -//// * put operations to fill this buffer. -//// * -//// *

This method does not actually erase the data in the buffer, but it -//// * is named as if it did because it will most often be used in situations -//// * in which that might as well be the case.

-//// */ -//// final def clear() { -//// // if (0 == globalCapacity) return -//// -//// needReleaseIndices += 0 -//// globalPosition = 0L -//// toNewContainer(0) -//// globalLimit = globalCapacity -//// -//// // Now free all pending containers -//// releasePendingContainers() -//// } -//// -//// /** -//// * Flips this buffer. The limit is set to the current position and then -//// * the position is set to zero. If the mark is defined then it is -//// * discarded. -//// * -//// *

After a sequence of channel-read or put operations, invoke -//// * this method to prepare for a sequence of channel-write or relative -//// * get operations. -//// */ -//// final def flip() { -//// needReleaseIndices += 0 -//// globalLimit = globalPosition -//// globalPosition = 0L -//// toNewContainer(0) -//// -//// // Now free all pending containers -//// releasePendingContainers() -//// } -//// -//// /** -//// * Rewinds this buffer. The position is set to zero and the mark is -//// * discarded. -//// * -//// *

Invoke this method before a sequence of channel-write or get -//// * operations, assuming that the limit has already been set -//// * appropriately. -//// */ -//// final def rewind() { -//// needReleaseIndices += 0 -//// globalPosition = 0L -//// toNewContainer(0) -//// -//// // Now free all pending containers -//// releasePendingContainers() -//// } -//// -//// /** -//// * Returns the number of elements between the current position and the -//// * limit.

-//// * -//// * @return The number of elements remaining in this buffer -//// */ -//// final def remaining(): Long = { -//// globalLimit - globalPosition -//// } -//// -//// /** -//// * Tells whether there are any elements between the current position and -//// * the limit.

-//// * -//// * @return true if, and only if, there is at least one element -//// * remaining in this buffer -//// */ -//// final def hasRemaining() = { -//// globalPosition < globalLimit -//// } -//// -//// // private def currentBuffer(): ByteBuffer = buffers(currentContainerIndex) -//// -//// // number of bytes remaining in currently active underlying buffer -//// private def currentRemaining(): Int = { -//// if (hasRemaining()) { -//// // validate currentContainerIndex is valid -//// assert (globalPosition >= bufferPositionStart(currentContainerIndex) && -//// globalPosition < bufferPositionStart(currentContainerIndex + 1), -//// "globalPosition = " + globalPosition + -//// ", currentContainerIndex = " + currentContainerIndex + -//// ", bufferPositionStart = " + bufferPositionStart.mkString("[", ", ", " ]")) -//// -//// currentRemaining0(currentContainerIndex) -//// } else 0 -//// } -//// -//// // Without any validation : required when we are bumping the index (when validation will fail) ... -//// private def currentRemaining0(which: Int): Int = { -//// // currentBuffer().remaining() -//// math.max(0, math.min(bufferPositionStart(which + 1), -//// globalLimit) - globalPosition).asInstanceOf[Int] -//// } -//// -//// // Set the approppriate position/limit for the current underlying buffer to mirror our -//// // the LargeByteBuffer's state. -//// private def fetchCurrentBuffer(): ByteBuffer = { -//// releasePendingContainers() -//// -//// assert (currentContainerIndex < containers.length) -//// -//// val container = containers(currentContainerIndex) -//// if (! container.isAcquired) { -//// container.acquire() -//// } -//// -//// assert (container.isAcquired) -//// if (LargeByteBuffer.enableExpensiveAssert) { -//// assert (! containers.exists( b => (b ne container) && b.isAcquired)) -//// } -//// -//// assert (currentContainerIndex < bufferPositionStart.length && -//// globalPosition < bufferPositionStart(currentContainerIndex + 1), -//// "currentContainerIndex = " + currentContainerIndex + ", bufferPositionStart = " + -//// bufferPositionStart.mkString("[", ", ", "]") + ", this = " + this) -//// -//// val buffPosition = (globalPosition - bufferPositionStart(currentContainerIndex)). -//// asInstanceOf[Int] -//// -//// val buffer = container.getByteBuffer -//// buffer.position(buffPosition) -//// val diff = buffer.capacity - buffPosition -//// val left = remaining() -//// if (diff <= left) { -//// buffer.limit(buffer.capacity()) -//// } else { -//// // Can happen if limit() was called. -//// buffer.limit(buffPosition + left.asInstanceOf[Int]) -//// } -//// -//// buffer -//// } -//// -//// // To be used ONLY to test in suites. -//// private[spark] def fetchCurrentBufferForTesting(): ByteBuffer = { -//// if ("1" != System.getProperty("SPARK_TESTING")) { -//// throw new IllegalStateException("This method is to be used ONLY within spark test suites") -//// } -//// -//// fetchCurrentBuffer() -//// } -//// -//// // Expects that the invoker has ensured that this can be safely invoked. -//// // That is, it wont be invoked when the loop wont terminate. -//// private def toNonEmptyBuffer() { -//// -//// if (! hasRemaining()) { -//// var newIndex = currentContainerIndex -//// // Ensure we are in the right block or not. -//// while (newIndex < containers.length && globalPosition >= bufferPositionStart(newIndex + 1)) { -//// newIndex += 1 -//// } -//// toNewContainer(newIndex) -//// // Do not do this - since we might not yet have consumed the buffer which caused EOF right now -//// /* -//// // Add last one also, and release it too - since we are at the end of the buffer with nothing -//// // more pending. -//// if (newIndex >= 0 && currentContainerIndex < containers.length) { -//// needReleaseIndices += newIndex -//// } -//// */ -//// assert (currentContainerIndex >= 0) -//// // releasePendingContainers() -//// return -//// } -//// -//// var index = currentContainerIndex -//// while (0 == currentRemaining0(index) && index < containers.length) { -//// index += 1 -//// } -//// assert (currentContainerIndex < containers.length) -//// toNewContainer(index) -//// assert (0 != currentRemaining()) -//// } -//// -//// private def assertPreconditions(containerIndex: Int) { -//// assert (globalPosition >= bufferPositionStart(containerIndex), -//// "globalPosition = " + globalPosition + ", containerIndex = " + containerIndex + -//// ", bufferPositionStart = " + bufferPositionStart.mkString("[", ", ", " ]")) -//// assert (globalPosition < bufferPositionStart(containerIndex + 1), -//// "globalPosition = " + globalPosition + ", containerIndex = " + containerIndex + -//// ", bufferPositionStart = " + bufferPositionStart.mkString("[", ", ", " ]")) -//// -//// assert (globalLimit <= globalCapacity) -//// assert (containerIndex < containers.length) -//// } -//// -//// -//// /** -//// * Attempts to return a ByteBuffer of the requested size. -//// * It is possible to return a buffer of size smaller than requested -//// * even though hasRemaining == true -//// * -//// * On return, position would have been moved 'ahead' by the size of the buffer returned : -//// * that is, we treat that the returned buffer has been already 'read' from this LargeByteBuffer -//// * -//// * -//// * This is used to primarily retrieve content of this buffer to expose via ByteBuffer -//// * to some other api which is deemed too cumbersome to move to LargeByteBuffer (like the -//// * chunked sending of contents via ConnectionManager) Note that the lifecycle of the ByteBuffer -//// * returned is inherently tied to the state of this LargeByteBuffer. For example,if the underlying -//// * container is a disk backed container, and we make subsequent calls to get(), the returned -//// * ByteBuffer can be dispose'ed off -//// * -//// * @param maxChunkSize Max size of the ByteBuffer to retrieve. -//// * @return -//// */ -//// -//// private def fetchBufferOfSize(maxChunkSize: Int): ByteBuffer = { -//// fetchBufferOfSizeImpl(maxChunkSize, canReleaseContainers = true) -//// } -//// -//// private def fetchBufferOfSizeImpl(maxChunkSize: Int, -//// canReleaseContainers: Boolean): ByteBuffer = { -//// if (canReleaseContainers) releasePendingContainers() -//// assert (maxChunkSize > 0) -//// -//// // not checking for degenerate case of maxChunkSize == 0 -//// if (globalPosition >= globalLimit) { -//// // throw exception -//// throw new BufferUnderflowException() -//// } -//// -//// // Check preconditions : disable these later, since they might be expensive to -//// // evaluate for every IO op -//// assertPreconditions(currentContainerIndex) -//// -//// val currentBufferRemaining = currentRemaining() -//// -//// assert (currentBufferRemaining > 0) -//// -//// val size = math.min(currentBufferRemaining, maxChunkSize) -//// -//// val newBuffer = if (currentBufferRemaining > maxChunkSize) { -//// val currentBuffer = fetchCurrentBuffer() -//// val buff = ByteBufferContainer.createSlice(currentBuffer, -//// currentBuffer.position(), maxChunkSize) -//// assert (buff.remaining() == maxChunkSize) -//// buff -//// } else { -//// val currentBuffer = fetchCurrentBuffer() -//// val buff = currentBuffer.slice() -//// assert (buff.remaining() == currentBufferRemaining) -//// buff -//// } -//// -//// assert (size == newBuffer.remaining()) -//// assert (0 == newBuffer.position()) -//// assert (size == newBuffer.limit()) -//// assert (newBuffer.capacity() == newBuffer.limit()) -//// -//// globalPosition += newBuffer.remaining -//// toNonEmptyBuffer() -//// -//// newBuffer -//// } -//// -//// // Can we service the read/write from the currently active (underlying) bytebuffer or not. -//// // For almost all cases, this will return true allowing us to optimize away the more expensive -//// // computations. -//// private def localReadWritePossible(size: Int) = -//// size >= 0 && globalPosition + size <= bufferPositionStart(currentContainerIndex + 1) -//// -//// -//// def getLong(): Long = { -//// assert (readable) -//// releasePendingContainers() -//// -//// if (remaining() < 8) throw new BufferUnderflowException -//// -//// if (localReadWritePossible(8)) { -//// val buff = fetchCurrentBuffer() -//// assert (buff.remaining() >= 8) -//// val retval = buff.getLong -//// globalPosition += 8 -//// toNonEmptyBuffer() -//// return retval -//// } -//// -//// val buff = readFully(8) -//// buff.getLong -//// } -//// -//// def getInt(): Int = { -//// assert (readable) -//// releasePendingContainers() -//// -//// if (remaining() < 4) throw new BufferUnderflowException -//// -//// if (localReadWritePossible(4)) { -//// val buff = fetchCurrentBuffer() -//// assert (buff.remaining() >= 4) -//// val retval = buff.getInt -//// globalPosition += 4 -//// toNonEmptyBuffer() -//// return retval -//// } -//// -//// val buff = readFully(4) -//// buff.getInt -//// } -//// -//// def getChar(): Char = { -//// assert (readable) -//// releasePendingContainers() -//// -//// if (remaining() < 2) throw new BufferUnderflowException -//// -//// if (localReadWritePossible(2)) { -//// val buff = fetchCurrentBuffer() -//// assert (buff.remaining() >= 2) -//// val retval = buff.getChar -//// globalPosition += 2 -//// toNonEmptyBuffer() -//// return retval -//// } -//// -//// // if slice is becoming too expensive, revisit this ... -//// val buff = readFully(2) -//// buff.getChar -//// } -//// -//// def get(): Byte = { -//// assert (readable) -//// releasePendingContainers() -//// -//// if (! hasRemaining()) throw new BufferUnderflowException -//// -//// // If we have remaining bytes, previous invocations MUST have ensured that we are at -//// // a buffer which has data to be read. -//// assert (localReadWritePossible(1)) -//// -//// val buff = fetchCurrentBuffer() -//// assert (buff.remaining() >= 1, "buff.remaining = " + buff.remaining()) -//// val retval = buff.get() -//// globalPosition += 1 -//// toNonEmptyBuffer() -//// -//// retval -//// } -//// -//// def get(arr: Array[Byte], offset: Int, size: Int): Int = { -//// assert (readable) -//// releasePendingContainers() -//// -//// LargeByteBuffer.checkOffsets(arr, offset, size) -//// -//// // kyro depends on this it seems ? -//// // assert (size > 0) -//// if (0 == size) return 0 -//// -//// if (! hasRemaining()) return -1 -//// -//// if (localReadWritePossible(size)) { -//// val buff = fetchCurrentBuffer() -//// assert (buff.remaining() >= size) -//// buff.get(arr, offset, size) -//// globalPosition += size -//// toNonEmptyBuffer() -//// return size -//// } -//// -//// var remainingSize = math.min(size, remaining()).asInstanceOf[Int] -//// var currentOffset = offset -//// -//// while (remainingSize > 0) { -//// val buff = fetchBufferOfSize(remainingSize) -//// val toCopy = math.min(buff.remaining(), remainingSize) -//// -//// buff.get(arr, currentOffset, toCopy) -//// currentOffset += toCopy -//// remainingSize -= toCopy -//// } -//// -//// currentOffset - offset -//// } -//// -//// -//// private def createSlice(size: Long): LargeByteBuffer = { -//// -//// releasePendingContainers() -//// -//// if (remaining() < size) { -//// // logInfo("createSlice. remaining = " + remaining() + ", size " + size + ", this = " + this) -//// throw new BufferOverflowException -//// } -//// -//// // kyro depends on this it seems ? -//// // assert (size > 0) -//// if (0 == size) return LargeByteBuffer.EMPTY_BUFFER -//// -//// val arr = new ArrayBuffer[ByteBufferContainer](2) -//// var totalLeft = size -//// -//// // assert (currentRemaining() < totalLeft || totalLeft != size || currentAsByteBuffer) -//// -//// var containerIndex = currentContainerIndex -//// while (totalLeft > 0 && hasRemaining()) { -//// assertPreconditions(containerIndex) -//// val container = containers(containerIndex) -//// val currentLeft = currentRemaining0(containerIndex) -//// -//// assert (globalPosition + currentLeft <= globalLimit) -//// assert (globalPosition >= bufferPositionStart(containerIndex) && -//// (globalPosition < bufferPositionStart(containerIndex + 1))) -//// -//// val from = (globalPosition - bufferPositionStart(containerIndex)).asInstanceOf[Int] -//// val sliceSize = math.min(totalLeft, currentLeft) -//// assert (from >= 0) -//// assert (sliceSize > 0 && sliceSize <= Int.MaxValue) -//// -//// val slice = container.createSlice(from, sliceSize.asInstanceOf[Int]) -//// arr += slice -//// -//// globalPosition += sliceSize -//// totalLeft -= sliceSize -//// if (currentLeft == sliceSize) containerIndex += 1 -//// } -//// -//// // Using toNonEmptyBuffer instead of directly moving to next here so that -//// // other checks can be performed there. -//// toNonEmptyBuffer() -//// // force cleanup - this is fine since we are not using the buffers directly -//// // which are actively needed (the returned value is on containers which can -//// // recreate) -//// releasePendingContainers() -//// // free current container if acquired. -//// if (currentContainerIndex < containers.length) { -//// containers(currentContainerIndex).release() -//// } -//// assert (currentContainerIndex == containerIndex) -//// -//// val retval = new LargeByteBuffer(arr, false, ephemeralDiskBacked) -//// retval.overrideCleaner(LargeByteBuffer.noopDisposeFunction) -//// retval -//// } -//// -//// // Get a composite sequence of ByteBuffer which might straddle one or more underlying buffers -//// // This is to be used only for writes : and ensures that writes are done into the appropriate -//// // underlying bytebuffers. -//// def getCompositeWriteBuffer(size: Long): LargeByteBuffer = { -//// assert(writable) -//// assert(size >= 0) -//// -//// createSlice(size) -//// } -//// -//// // get a buffer which is of the specified size and contains data from the underlying buffers -//// // Note, the actual data might be spread across the underlying buffers. -//// // This MUST BE used only for specific usecases like getInt, etc. Not for bulk copy ! -//// private def readFully(size: Int): ByteBuffer = { -//// assert (readable) -//// -//// if (remaining() < size) { -//// // throw exception -//// throw new BufferUnderflowException() -//// } -//// -//// // kyro depends on this it seems ? -//// // assert (size > 0) -//// if (0 == size) return LargeByteBuffer.EMPTY_BYTEBUFFER -//// -//// // Expected to be handled elsewhere. -//// assert (! localReadWritePossible(size)) -//// -//// val localBuff = { -//// val buff = fetchBufferOfSize(size) -//// // assert(buff.remaining() <= size) -//// // if (buff.remaining() == size) return buff -//// assert(buff.remaining() < size) -//// ByteBuffer.allocate(size).put(buff) -//// } -//// -//// // assert (localBuff.hasRemaining) -//// -//// while (localBuff.hasRemaining) { -//// val buff = fetchBufferOfSize(localBuff.remaining()) -//// localBuff.put(buff) -//// } -//// -//// localBuff.flip() -//// localBuff -//// } -//// -//// -//// -//// def put(b: Byte) { -//// assert (writable) -//// if (remaining() < 1) { -//// // logInfo("put byte. remaining = " + remaining() + ", this = " + this) -//// throw new BufferOverflowException -//// } -//// -//// assert (currentRemaining() > 0) -//// -//// fetchCurrentBuffer().put(b) -//// globalPosition += 1 -//// // Check to need to bump the index ? -//// toNonEmptyBuffer() -//// } -//// -//// -//// def put(buffer: ByteBuffer) { -//// assert (writable) -//// if (remaining() < buffer.remaining()) { -//// throw new BufferOverflowException -//// } -//// -//// val bufferRemaining = buffer.remaining() -//// if (localReadWritePossible(bufferRemaining)) { -//// -//// assert (currentRemaining() >= bufferRemaining) -//// -//// fetchCurrentBuffer().put(buffer) -//// -//// globalPosition += bufferRemaining -//// toNonEmptyBuffer() -//// return -//// } -//// -//// while (buffer.hasRemaining) { -//// val currentBufferRemaining = currentRemaining() -//// val bufferRemaining = buffer.remaining() -//// -//// if (currentBufferRemaining >= bufferRemaining) { -//// fetchCurrentBuffer().put(buffer) -//// globalPosition += bufferRemaining -//// } else { -//// // Split across buffers. -//// val currentBuffer = fetchCurrentBuffer() -//// assert (currentBuffer.remaining() >= currentBufferRemaining) -//// val sliced = ByteBufferContainer.createSlice(buffer, buffer.position(), -//// currentBufferRemaining) -//// assert (sliced.remaining() == currentBufferRemaining) -//// currentBuffer.put(sliced) -//// // move buffer pos -//// buffer.position(buffer.position() + currentBufferRemaining) -//// -//// globalPosition += currentBufferRemaining -//// } -//// toNonEmptyBuffer() -//// } -//// -//// assert (! hasRemaining() || currentRemaining() > 0) -//// } -//// -//// def put(other: LargeByteBuffer) { -//// assert (writable) -//// if (this.remaining() < other.remaining()) { -//// throw new BufferOverflowException -//// } -//// -//// while (other.hasRemaining()) { -//// val buffer = other.fetchBufferOfSize(other.currentRemaining()) -//// this.put(buffer) -//// } -//// } -//// -//// -//// def duplicate(): LargeByteBuffer = { -//// val containersCopy = new ArrayBuffer[ByteBufferContainer](containers.size) -//// // We do a duplicate as part of construction - so avoid double duplicate. -//// // containersCopy ++= containers.map(_.duplicate()) -//// containersCopy ++= containers -//// val retval = new LargeByteBuffer(containersCopy, true, ephemeralDiskBacked) -//// -//// // set limit and position (in that order) ... -//// retval.limit(this.limit()) -//// retval.position(this.position()) -//// -//// // Now release our containers - if any had been acquired -//// releasePendingContainers() -//// -//// retval -//// } -//// -//// -//// /** -//// * 'read' a LargeByteBuffer of size specified and return that. -//// * Position will be incremented by size -//// * -//// * The name might be slightly confusing : rename ? -//// * -//// * @param size Amount of data to be read from this buffer and returned -//// * @return -//// */ -//// def readLargeBuffer(size: Long, partialReadAllowed: Boolean): LargeByteBuffer = { -//// if (! hasRemaining() && ! partialReadAllowed) throw new BufferUnderflowException -//// if (remaining() < size && ! partialReadAllowed) throw new BufferUnderflowException -//// -//// -//// assert (readable) -//// assert (size >= 0) -//// -//// releasePendingContainers() -//// -//// if (0 == size) return LargeByteBuffer.EMPTY_BUFFER -//// -//// createSlice(size) -//// } -//// -//// -//// // This is essentially a workaround to exposing underlying buffers -//// def readFrom(channel: ReadableByteChannel): Long = { -//// -//// assert (writable) -//// releasePendingContainers() -//// -//// // this also allows us to avoid nasty corner cases in the loop. -//// if (! hasRemaining()) { -//// // logInfo("readFrom channel. remaining = " + remaining() + ", this = " + this) -//// throw new BufferOverflowException -//// } -//// -//// var totalBytesRead = 0L -//// -//// while (hasRemaining()) { -//// // read what we can ... -//// val buffer = fetchCurrentBuffer() -//// val bufferRemaining = currentRemaining() -//// val bytesRead = channel.read(buffer) -//// -//// if (bytesRead > 0) { -//// totalBytesRead += bytesRead -//// // bump position too .. -//// globalPosition += bytesRead -//// if (bytesRead >= bufferRemaining) toNonEmptyBuffer() -//// } -//// else if (-1 == bytesRead) { -//// // if we had already read some data in the loop, return that. -//// if (totalBytesRead > 0) return totalBytesRead -//// return -1 -//// } // nothing available to read, retry later. return -//// else if (0 == bytesRead) { -//// return totalBytesRead -//// } -//// -//// // toNonEmptyBuffer() -//// } -//// -//// // Cleanup last buffer ? -//// toNonEmptyBuffer() -//// totalBytesRead -//// } -//// -//// // This is essentially a workaround to exposing underlying buffers -//// def readFrom(inStrm: InputStream): Long = { -//// -//// assert (writable) -//// releasePendingContainers() -//// -//// // this also allows us to avoid nasty corner cases in the loop. -//// // if (! hasRemaining()) throw new BufferOverflowException -//// if (! hasRemaining()) return 0 -//// -//// var totalBytesRead = 0L -//// -//// val buff = new Array[Byte](LargeByteBuffer.TEMP_ARRAY_SIZE) -//// -//// while (hasRemaining()) { -//// // read what we can ... note, since there is no gaurantee that underlying buffer might -//// // expose array() method, we do double copy - from stream to buff and from buff to bytearray. -//// // see if we can optimize this later ... -//// val buffer = fetchCurrentBuffer() -//// val bufferRemaining = buffer.remaining() -//// val max = math.min(buff.length, bufferRemaining) -//// val bytesRead = inStrm.read(buff, 0, max) -//// -//// if (bytesRead > 0) { -//// buffer.put(buff, 0, bytesRead) -//// totalBytesRead += bytesRead -//// // bump position too .. -//// globalPosition += bytesRead -//// // buffer.position(buffer.position + bytesRead) -//// if (bytesRead >= bufferRemaining) toNonEmptyBuffer() -//// } -//// else if (-1 == bytesRead) { -//// // if we had already read some data in the loop, return that. -//// if (totalBytesRead > 0) return totalBytesRead -//// return -1 -//// } // nothing available to read, retry later. return -//// else if (0 == bytesRead) { -//// return totalBytesRead -//// } -//// -//// // toNonEmptyBuffer() -//// } -//// -//// totalBytesRead -//// } -//// -//// // This is essentially a workaround to exposing underlying buffers -//// // Note: very similar to readFrom(InputStream) : not trying anything fancy to reduce -//// // code for performance reasons. -//// def readFrom(inStrm: DataInput): Long = { -//// -//// assert (writable) -//// releasePendingContainers() -//// -//// // this also allows us to avoid nasty corner cases in the loop. -//// // if (! hasRemaining()) throw new BufferOverflowException -//// if (! hasRemaining()) return 0 -//// -//// var totalBytesRead = 0L -//// -//// val buff = new Array[Byte](LargeByteBuffer.TEMP_ARRAY_SIZE) -//// -//// while (hasRemaining()) { -//// // read what we can ... note, since there is no gaurantee that underlying buffer might -//// // expose array() method, we do double copy - from stream to buff and from buff to bytearray. -//// // see if we can optimize this later ... -//// val buffer = fetchCurrentBuffer() -//// val bufferRemaining = buffer.remaining() -//// val max = math.min(buff.length, bufferRemaining) -//// inStrm.readFully(buff, 0, max) -//// val bytesRead = max -//// -//// if (bytesRead > 0) { -//// buffer.put(buff, 0, bytesRead) -//// totalBytesRead += bytesRead -//// // bump position too .. -//// globalPosition += bytesRead -//// // buffer.position(buffer.position() + bytesRead) -//// if (bytesRead >= bufferRemaining) toNonEmptyBuffer() -//// } -//// else if (-1 == bytesRead) { -//// // if we had already read some data in the loop, return that. -//// if (totalBytesRead > 0) return totalBytesRead -//// return -1 -//// } // nothing available to read, retry later. return -//// else if (0 == bytesRead) { -//// return totalBytesRead -//// } -//// -//// // toNonEmptyBuffer() -//// } -//// -//// totalBytesRead -//// } -//// -//// // This is essentially a workaround to exposing underlying buffers -//// // Note: tries to do it efficiently without needing to load everything into memory -//// // (particularly for diskbacked buffers, etc). -//// def writeTo(channel: WritableByteChannel, cleanup: Boolean): Long = { -//// -//// assert (readable) -//// releasePendingContainers() -//// -//// // this also allows us to avoid nasty corner cases in the loop. -//// if (! hasRemaining()) throw new BufferUnderflowException -//// -//// var totalBytesWritten = 0L -//// -//// while (hasRemaining()) { -//// // Write what we can ... -//// val buffer = fetchCurrentBuffer() -//// val bufferRemaining = buffer.remaining() -//// assert (bufferRemaining > 0) -//// val bytesWritten = channel.write(buffer) -//// -//// if (bytesWritten > 0) { -//// totalBytesWritten += bytesWritten -//// // bump position too .. -//// globalPosition += bytesWritten -//// if (bytesWritten >= bufferRemaining) toNonEmptyBuffer() -//// assert (! hasRemaining() || currentRemaining() > 0) -//// } -//// else if (0 == bytesWritten) { -//// return totalBytesWritten -//// } -//// -//// // toNonEmptyBuffer() -//// } -//// -//// assert (! hasRemaining()) -//// if (cleanup) { -//// free() -//// } -//// totalBytesWritten -//// } -//// -//// // This is essentially a workaround to exposing underlying buffers -//// def writeTo(outStrm: OutputStream, cleanup: Boolean): Long = { -//// -//// assert (readable) -//// releasePendingContainers() -//// -//// // this also allows us to avoid nasty corner cases in the loop. -//// if (! hasRemaining()) throw new BufferUnderflowException -//// -//// var totalBytesWritten = 0L -//// val buff = new Array[Byte](LargeByteBuffer.TEMP_ARRAY_SIZE) -//// -//// while (hasRemaining()) { -//// // write what we can ... note, since there is no gaurantee that underlying buffer might -//// // expose array() method, we do double copy - from bytearray to buff and from -//// // buff to outputstream. see if we can optimize this later ... -//// val buffer = fetchCurrentBuffer() -//// val bufferRemaining = buffer.remaining() -//// val size = math.min(bufferRemaining, buff.length) -//// buffer.get(buff, 0, size) -//// outStrm.write(buff, 0, size) -//// -//// totalBytesWritten += size -//// // bump position too .. -//// globalPosition += size -//// -//// if (size >= bufferRemaining) toNonEmptyBuffer() -//// } -//// -//// toNonEmptyBuffer() -//// if (cleanup) { -//// free() -//// } -//// totalBytesWritten -//// } -//// -//// def asInputStream(): InputStream = { -//// new InputStream() { -//// override def read(): Int = { -//// if (! hasRemaining()) return -1 -//// get() -//// } -//// -//// override def read(arr: Array[Byte], off: Int, len: Int): Int = { -//// if (! hasRemaining()) return -1 -//// -//// get(arr, off, len) -//// } -//// -//// override def available(): Int = { -//// // current remaining is what can be read without blocking -//// // anything higher might need disk access/buffer swapping. -//// /* -//// val left = remaining() -//// math.min(left, Int.MaxValue).asInstanceOf[Int] -//// */ -//// currentRemaining() -//// } -//// } -//// } -//// -//// def getCleaner() = cleaner -//// -//// /** -//// * @param cleaner The previous cleaner, so that the caller can chain them if required. -//// * @return -//// */ -//// private[spark] def overrideCleaner(cleaner: BufferCleaner): BufferCleaner = { -//// overrideCleaner(cleaner, allowOverride = true) -//// } -//// -//// private def overrideCleaner(cleaner: BufferCleaner, allowOverride: Boolean): BufferCleaner = { -//// if (! this.allowCleanerOverride) { -//// // allowCleanerOverride = false is used for EMPTY_BUFFER - where we do not allow free -//// return this.cleaner -//// } -//// -//// this.allowCleanerOverride = allowOverride -//// assert (null != cleaner) -//// val prev = this.cleaner -//// this.cleaner = cleaner -//// // logInfo("Overriding " + prev + " with " + this.cleaner) -//// prev -//// } -//// -//// private def doReleaseAll() { -//// for (container <- containers) { -//// container.release() -//// } -//// } -//// -//// def free(invokeCleaner: Boolean = true) { -//// // logInfo("Free on " + this + ", cleaner = " + cleaner) -//// // always invoking release -//// doReleaseAll() -//// -//// if (invokeCleaner) cleaner.clean(this) -//// } -//// -//// private def doDispose(needRelease: Boolean) { -//// -//// if (disposeLocationThrowable ne null) { -//// logError("Already free'ed earlier at : ", disposeLocationThrowable) -//// logError("Current at ", new Throwable) -//// throw new IllegalStateException("Already freed.") -//// } -//// disposeLocationThrowable = new Throwable() -//// -//// // Forcefully cleanup all -//// if (needRelease) doReleaseAll() -//// -//// // Free in a different loop, in case different containers refer to same resource -//// // to release (like file) -//// for (container <- containers) { -//// container.free() -//// } -//// -//// needReleaseIndices.clear() -//// -//// // We should not use this buffer anymore : set the values such that f -//// // we dont ... -//// globalPosition = 0 -//// globalLimit = 0 -//// globalCapacity = 0 -//// } -//// -//// // copy data over ... MUST be used only for cases where array is known to be -//// // small to begin with. slightly risky method due to that assumption -//// def toByteArray(): Array[Byte] = { -//// val positionBackup = position() -//// val size = remaining() -//// if (size > Int.MaxValue) { -//// throw new IllegalStateException( -//// "Attempt to convert LargeByteBuffer to byte array when data held is more than 2G") -//// } -//// -//// val retval = new Array[Byte](size.asInstanceOf[Int]) -//// val readSize = get(retval, 0, retval.length) -//// assert (readSize == retval.length, -//// "readSize = " + readSize + ", retval.length = " + retval.length) -//// -//// position(positionBackup) -//// -//// retval -//// } -//// -//// // copy data over ... MUST be used only for cases where array is known to be -//// // small to begin with. slightly risky method due to that assumption -//// def toByteBuffer(): ByteBuffer = { -//// ByteBuffer.wrap(toByteArray()) -//// } -//// -//// def toInMemoryBuffer(ioConf: IOConfig): LargeByteBuffer = { -//// val retval = LargeByteBuffer.allocateMemoryBuffer(remaining(), ioConf) -//// val currentPosition = position() -//// retval.put(this) -//// position(currentPosition) -//// retval.clear() -//// retval -//// } -//// -//// -//// -//// // This is ONLY used for testing : that too as part of development of this and associated classes -//// // remove before contributing to spark. -//// def hexDump(): String = { -//// if (remaining() * 64 > Int.MaxValue) { -//// throw new UnsupportedOperationException("buffer too large " + remaining()) -//// } -//// -//// val sb = new StringBuilder((remaining() * 2).asInstanceOf[Int]) -//// -//// var perLine = 0 -//// var first = true -//// for (b <- toByteArray()) { -//// perLine += 1 -//// if (perLine % 8 == 0) { -//// sb.append('\n') -//// first = true -//// } -//// if (! first) sb.append(' ') -//// first = false -//// sb.append(java.lang.Integer.toHexString(b & 0xff)) -//// } -//// sb.append('\n') -//// sb.toString() -//// } -//// -//// override def toString: String = { -//// val sb: StringBuffer = new StringBuffer -//// sb.append(getClass.getName) -//// sb.append(' ') -//// sb.append(System.identityHashCode(this)) -//// sb.append("@[pos=") -//// sb.append(position()) -//// sb.append(" lim=") -//// sb.append(limit()) -//// sb.append(" cap=") -//// sb.append(capacity()) -//// sb.append("]") -//// sb.toString -//// } -//// -//// -//// -//// override def finalize(): Unit = { -//// var marked = false -//// if (containers ne null) { -//// if (containers.exists(container => container.isAcquired && container.requireRelease())) { -//// marked = true -//// logError("BUG: buffer was not released - and now going out of scope. " + -//// "Potential resource leak. Allocated at ", allocateLocationThrowable) -//// containers.foreach(_.release()) -//// } -//// if (containers.exists(container => !container.isFreed && container.requireFree())) { -//// if (!marked) { -//// logError("BUG: buffer was not freed - and now going out of scope. Potential resource leak", -//// allocateLocationThrowable) -//// } -//// else { -//// logError("BUG: buffer was not freed - and now going out of scope. Potential resource leak") -//// } -//// containers.foreach(_.free()) -//// } -//// } -//// super.finalize() -//// } -////} -//// -//// -////object LargeByteBuffer extends Logging { -//// -//// private val noopDisposeFunction = new BufferCleaner() { -//// protected def doClean(buffer: LargeByteBuffer) { -//// buffer.free(invokeCleaner = false) -//// } -//// } -//// -//// val enableExpensiveAssert = false -//// private val EMPTY_BYTEBUFFER = ByteBuffer.allocate(0) -//// val EMPTY_BUFFER = new LargeByteBuffer(ArrayBuffer( -//// new HeapByteBufferContainer(EMPTY_BYTEBUFFER, false)), false, false) -//// // Do not allow anyone else to override cleaner -//// EMPTY_BUFFER.overrideCleaner(noopDisposeFunction, allowOverride = false) -//// -//// // 8K sufficient ? -//// private val TEMP_ARRAY_SIZE = 8192 -//// -//// /** -//// * Create a LargeByteBuffer of specified size which is split across -//// * ByteBuffer's of size DiskStore.MAX_BLOCK_SIZE and backed by in memory -//// * ByteBuffer -//// * -//// */ -//// def allocateMemoryBuffer(totalSize: Long, ioConf: IOConfig): LargeByteBuffer = { -//// if (0 == totalSize) { -//// return EMPTY_BUFFER -//// } -//// -//// assert (totalSize > 0) -//// -//// val blockSize = ioConf.getMaxBlockSize(BufferType.MEMORY) -//// val numBlocks = ioConf.numBlocks(BufferType.MEMORY, totalSize) -//// val lastBlockSize = ioConf.lastBlockSize(BufferType.MEMORY, totalSize) -//// -//// assert (lastBlockSize > 0) -//// -//// val bufferArray = { -//// val arr = new ArrayBuffer[ByteBufferContainer](numBlocks) -//// for (index <- 0 until numBlocks - 1) { -//// val buff = ByteBuffer.allocate(blockSize) -//// // buff.clear() -//// arr += new HeapByteBufferContainer(buff, true) -//// } -//// arr += new HeapByteBufferContainer(ByteBuffer.allocate(lastBlockSize), true) -//// assert (arr.length == numBlocks) -//// arr -//// } -//// -//// new LargeByteBuffer(bufferArray, false, false) -//// } -//// -//// /** -//// * Create a LargeByteBuffer of specified size which is split across -//// * ByteBuffer's of size DiskStore.MAX_BLOCK_SIZE and backed by on disk -//// * -//// */ -//// private def allocateDiskBuffer(totalSize: Long, -//// blockManager: BlockManager): LargeByteBuffer = { -//// if (0 == totalSize) { -//// return EMPTY_BUFFER -//// } -//// -//// assert (totalSize > 0) -//// -//// // Create a file of the specified size. -//// val file = blockManager.diskBlockManager.createTempBlock()._2 -//// val raf = new RandomAccessFile(file, "rw") -//// try { -//// raf.setLength(totalSize) -//// } finally { -//// raf.close() -//// } -//// -//// readWriteDiskSegment(new FileSegment(file, 0, totalSize), -//// ephemeralDiskBacked = true, blockManager.ioConf) -//// } -//// -//// // The returned buffer takes up ownership of the underlying buffers -//// // (including dispos'ing that when done) -//// def fromBuffers(buffers: ByteBuffer*): LargeByteBuffer = { -//// val nonEmpty = buffers.filter(_.hasRemaining) -//// -//// // cleanup the empty buffers -//// buffers.filter(! _.hasRemaining).foreach(b => BlockManager.dispose(b)) -//// -//// -//// if (nonEmpty.isEmpty) { -//// return EMPTY_BUFFER -//// } -//// -//// // slice so that offsets match our requirement -//// new LargeByteBuffer(new ArrayBuffer() ++ nonEmpty.map(b => -//// new HeapByteBufferContainer(b.slice(), true)), false, false) -//// } -//// -//// def fromByteArrays(byteArrays: Array[Byte]*): LargeByteBuffer = { -//// // only non empty arrays -//// val arrays = byteArrays.filter(_.length > 0) -//// if (0 == arrays.length) return EMPTY_BUFFER -//// -//// new LargeByteBuffer(new ArrayBuffer() ++ arrays.map(arr => -//// new HeapByteBufferContainer(ByteBuffer.wrap(arr), true)), false, false) -//// } -//// -//// def fromLargeByteBuffers(canDispose: Boolean, inputBuffers: LargeByteBuffer*): LargeByteBuffer = { -//// -//// if (inputBuffers.isEmpty) return EMPTY_BUFFER -//// -//// if (! inputBuffers.exists(_.hasRemaining())) { -//// if (canDispose) inputBuffers.map(_.free()) -//// return EMPTY_BUFFER -//// } -//// -//// // release all temp resources acquired -//// inputBuffers.foreach(buff => buff.releasePendingContainers()) -//// // free current container if acquired. -//// inputBuffers.foreach(buff => if (buff.currentContainerIndex < buff.containers.length) { -//// buff.containers(buff.currentContainerIndex).release() -//// }) -//// // inputBuffers.foreach(b => b.doReleaseAll()) -//// -//// -//// // Dispose of any empty buffers -//// if (canDispose) inputBuffers.filter(! _.hasRemaining()).foreach(_.free()) -//// -//// // Find all containers we need. -//// val buffers = inputBuffers.filter(_.hasRemaining()).map(b => b.createSlice(b.remaining())) -//// -//// val containers = buffers.flatMap(_.containers) -//// assert (! containers.isEmpty) -//// // The in order containers of "buffers" seq constitute the required return value -//// val retval = new LargeByteBuffer(new ArrayBuffer() ++ containers, -//// // if you cant dispose, then we dont own the buffers : in which case, need duplicate -//// ! canDispose, inputBuffers.exists(_.ephemeralDiskBacked)) -//// -//// if (canDispose) { -//// // override dispose of all other buffers. -//// val disposeFunctions = inputBuffers.map { -//// buffer => { -//// (buffer, buffer.overrideCleaner(noopDisposeFunction)) -//// } -//// } -//// -//// val cleaner = retval.getCleaner() -//// val newCleaner = new BufferCleaner { -//// protected def doClean(buffer: LargeByteBuffer) { -//// -//// assert (retval == buffer) -//// // default cleaner. -//// cleaner.clean(retval) -//// // not required, since we are within clean anyway. -//// // retval.free(invokeCleaner = false) -//// -//// // retval.doDispose(needRelease = true) -//// -//// // This might actually call dispose twice on some (initially) empty buffers, -//// // which is fine since we now guard against that. -//// disposeFunctions.foreach(v => v._2.clean(v._1)) -//// // Call the free method too : so that buffers are marked free ... -//// disposeFunctions.foreach(v => v._1.free(invokeCleaner = false)) -//// } -//// } -//// -//// val prev = retval.overrideCleaner(newCleaner) -//// assert (prev == cleaner) -//// } -//// -//// retval -//// } -//// -//// private def checkOffsets(arr: Array[Byte], offset: Int, size: Int) { -//// if (arr == null) { -//// throw new NullPointerException -//// } else if (offset < 0 || size < 0 || offset + size > arr.length) { -//// throw new IndexOutOfBoundsException -//// } -//// } -//// -//// def allocateTransientBuffer(size: Long, blockManager: BlockManager) = { -//// if (size <= blockManager.ioConf.maxInMemSize) { -//// LargeByteBuffer.allocateMemoryBuffer(size, blockManager.ioConf) -//// } else { -//// LargeByteBuffer.allocateDiskBuffer(size, blockManager) -//// } -//// } -//// -//// def readFromDiskSegment(segment: FileSegment, ioConf: IOConfig, -//// ephemeralDiskBacked: Boolean): LargeByteBuffer = { -//// // Split the block into multiple of BlockStore.maxBlockSize -//// val segmentSize = segment.length -//// val blockSize = ioConf.getMaxBlockSize(BufferType.DISK).asInstanceOf[Long] -//// val numBlocks = ioConf.numBlocks(BufferType.DISK, segmentSize) -//// val lastBlockSize = ioConf.lastBlockSize(BufferType.DISK, segmentSize) -//// -//// val buffers = new ArrayBuffer[ByteBufferContainer](numBlocks) -//// -//// for (index <- 0 until numBlocks - 1) { -//// buffers += new ReadOnlyFileContainer(new FileSegment(segment.file, -//// segment.offset + index * blockSize, blockSize), ioConf) -//// } -//// -//// // Last block -//// buffers += new ReadOnlyFileContainer(new FileSegment(segment.file, -//// segment.offset + (numBlocks - 1) * blockSize, lastBlockSize), ioConf) -//// -//// new LargeByteBuffer(buffers, false, ephemeralDiskBacked) -//// } -//// -//// def readWriteDiskSegment(segment: FileSegment, ephemeralDiskBacked: Boolean, -//// ioConf: IOConfig): LargeByteBuffer = { -//// -//// // Split the block into multiple of BlockStore.maxBlockSize -//// val segmentSize = segment.length -//// val blockSize = ioConf.getMaxBlockSize(BufferType.DISK).asInstanceOf[Long] -//// val numBlocks = ioConf.numBlocks(BufferType.DISK, segmentSize) -//// val lastBlockSize = ioConf.lastBlockSize(BufferType.DISK, segmentSize) -//// -//// logInfo("readWriteDiskSegment = " + segment + ", numBlocks = " + numBlocks + -//// ", lastBlockSize = " + lastBlockSize) -//// val buffers = new ArrayBuffer[ByteBufferContainer](numBlocks) -//// -//// for (index <- 0 until numBlocks - 1) { -//// buffers += new ReadWriteFileContainer(new FileSegment(segment.file, -//// segment.offset + index * blockSize, blockSize), ephemeralDiskBacked, null) -//// } -//// -//// // Last block -//// buffers += new ReadWriteFileContainer(new FileSegment(segment.file, -//// segment.offset + (numBlocks - 1) * blockSize, lastBlockSize), ephemeralDiskBacked, null) -//// -//// new LargeByteBuffer(buffers, false, ephemeralDiskBacked) -//// } -////} diff --git a/network/common/src/test/java/org/apache/spark/network/ChunkFetchIntegrationSuite.java b/network/common/src/test/java/org/apache/spark/network/ChunkFetchIntegrationSuite.java index dfb7740344ed0..fde2b78d10a5d 100644 --- a/network/common/src/test/java/org/apache/spark/network/ChunkFetchIntegrationSuite.java +++ b/network/common/src/test/java/org/apache/spark/network/ChunkFetchIntegrationSuite.java @@ -37,9 +37,7 @@ import static org.junit.Assert.*; -import org.apache.spark.network.buffer.FileSegmentManagedBuffer; -import org.apache.spark.network.buffer.ManagedBuffer; -import org.apache.spark.network.buffer.NioManagedBuffer; +import org.apache.spark.network.buffer.*; import org.apache.spark.network.client.ChunkReceivedCallback; import org.apache.spark.network.client.RpcResponseCallback; import org.apache.spark.network.client.TransportClient; @@ -73,7 +71,8 @@ public static void setUp() throws Exception { buf.put((byte) i); } buf.flip(); - bufferChunk = new NioManagedBuffer(buf); + final LargeByteBuffer lBuf = LargeByteBufferHelper.asLargeByteBuffer(buf); + bufferChunk = new NioManagedBuffer(lBuf); testFile = File.createTempFile("shuffle-test-file", "txt"); testFile.deleteOnExit(); @@ -91,7 +90,7 @@ public static void setUp() throws Exception { public ManagedBuffer getChunk(long streamId, int chunkIndex) { assertEquals(STREAM_ID, streamId); if (chunkIndex == BUFFER_CHUNK_INDEX) { - return new NioManagedBuffer(buf); + return new NioManagedBuffer(lBuf); } else if (chunkIndex == FILE_CHUNK_INDEX) { return new FileSegmentManagedBuffer(conf, testFile, 10, testFile.length() - 25); } else { @@ -222,10 +221,10 @@ private void assertBufferListsEqual(List list0, List list0, List list throws Exception { assertEquals(list0.size(), list1.size()); for (int i = 0; i < list0.size(); i ++) { - assertBuffersEqual(list0.get(i), new NioManagedBuffer(ByteBuffer.wrap(list1.get(i)))); + assertBuffersEqual(list0.get(i), new NioManagedBuffer( + LargeByteBufferHelper.asLargeByteBuffer(list1.get(i)))); } } private void assertBuffersEqual(ManagedBuffer buffer0, ManagedBuffer buffer1) throws Exception { - ByteBuffer nio0 = buffer0.nioByteBuffer(); - ByteBuffer nio1 = buffer1.nioByteBuffer(); + LargeByteBuffer nio0 = buffer0.nioByteBuffer(); + LargeByteBuffer nio1 = buffer1.nioByteBuffer(); - int len = nio0.remaining(); + long len = nio0.remaining(); assertEquals(nio0.remaining(), nio1.remaining()); - for (int i = 0; i < len; i ++) { + for (long i = 0; i < len; i ++) { assertEquals(nio0.get(), nio1.get()); } } diff --git a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockFetcherSuite.java b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockFetcherSuite.java index 842741e3d354f..853d6c195bb85 100644 --- a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockFetcherSuite.java +++ b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockFetcherSuite.java @@ -24,6 +24,7 @@ import com.google.common.collect.Maps; import io.netty.buffer.Unpooled; +import org.apache.spark.network.buffer.LargeByteBufferHelper; import org.junit.Test; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; @@ -48,7 +49,7 @@ public class OneForOneBlockFetcherSuite { @Test public void testFetchOne() { LinkedHashMap blocks = Maps.newLinkedHashMap(); - blocks.put("shuffle_0_0_0", new NioManagedBuffer(ByteBuffer.wrap(new byte[0]))); + blocks.put("shuffle_0_0_0", new NioManagedBuffer(LargeByteBufferHelper.asLargeByteBuffer(new byte[0]))); BlockFetchingListener listener = fetchBlocks(blocks); @@ -58,8 +59,8 @@ public void testFetchOne() { @Test public void testFetchThree() { LinkedHashMap blocks = Maps.newLinkedHashMap(); - blocks.put("b0", new NioManagedBuffer(ByteBuffer.wrap(new byte[12]))); - blocks.put("b1", new NioManagedBuffer(ByteBuffer.wrap(new byte[23]))); + blocks.put("b0", new NioManagedBuffer(LargeByteBufferHelper.asLargeByteBuffer(new byte[12]))); + blocks.put("b1", new NioManagedBuffer(LargeByteBufferHelper.asLargeByteBuffer(new byte[23]))); blocks.put("b2", new NettyManagedBuffer(Unpooled.wrappedBuffer(new byte[23]))); BlockFetchingListener listener = fetchBlocks(blocks); @@ -72,7 +73,7 @@ public void testFetchThree() { @Test public void testFailure() { LinkedHashMap blocks = Maps.newLinkedHashMap(); - blocks.put("b0", new NioManagedBuffer(ByteBuffer.wrap(new byte[12]))); + blocks.put("b0", new NioManagedBuffer(LargeByteBufferHelper.asLargeByteBuffer(new byte[12]))); blocks.put("b1", null); blocks.put("b2", null); @@ -87,9 +88,9 @@ public void testFailure() { @Test public void testFailureAndSuccess() { LinkedHashMap blocks = Maps.newLinkedHashMap(); - blocks.put("b0", new NioManagedBuffer(ByteBuffer.wrap(new byte[12]))); + blocks.put("b0", new NioManagedBuffer(LargeByteBufferHelper.asLargeByteBuffer(new byte[12]))); blocks.put("b1", null); - blocks.put("b2", new NioManagedBuffer(ByteBuffer.wrap(new byte[21]))); + blocks.put("b2", new NioManagedBuffer(LargeByteBufferHelper.asLargeByteBuffer(new byte[21]))); BlockFetchingListener listener = fetchBlocks(blocks); diff --git a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/RetryingBlockFetcherSuite.java b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/RetryingBlockFetcherSuite.java index 1ad0d72ae5ec5..d2056254fbbf6 100644 --- a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/RetryingBlockFetcherSuite.java +++ b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/RetryingBlockFetcherSuite.java @@ -25,6 +25,7 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Sets; +import org.apache.spark.network.buffer.LargeByteBufferHelper; import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -47,9 +48,9 @@ */ public class RetryingBlockFetcherSuite { - ManagedBuffer block0 = new NioManagedBuffer(ByteBuffer.wrap(new byte[13])); - ManagedBuffer block1 = new NioManagedBuffer(ByteBuffer.wrap(new byte[7])); - ManagedBuffer block2 = new NioManagedBuffer(ByteBuffer.wrap(new byte[19])); + ManagedBuffer block0 = new NioManagedBuffer(LargeByteBufferHelper.asLargeByteBuffer(new byte[13])); + ManagedBuffer block1 = new NioManagedBuffer(LargeByteBufferHelper.asLargeByteBuffer(new byte[7])); + ManagedBuffer block2 = new NioManagedBuffer(LargeByteBufferHelper.asLargeByteBuffer(new byte[19])); @Before public void beforeEach() { From dcb46697d59fa77ac643e438b346eb28972d9e8f Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Fri, 27 Feb 2015 14:59:04 -0600 Subject: [PATCH 10/97] add real test case for uploading large blocks (failing now) --- .../netty/NettyBlockTransferSuite.scala | 79 ++++++++++++++++--- 1 file changed, 66 insertions(+), 13 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSuite.scala b/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSuite.scala index 2133b3286ff36..710bf2822fa9b 100644 --- a/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSuite.scala +++ b/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSuite.scala @@ -16,14 +16,17 @@ */ package org.apache.spark.network.netty +import java.nio.ByteBuffer import java.util.concurrent.TimeUnit import org.apache.commons.io.IOUtils import org.apache.spark.network.BlockDataManager -import org.apache.spark.network.buffer.{ManagedBuffer, LargeByteBufferHelper, NioManagedBuffer} +import org.apache.spark.network.buffer._ import org.apache.spark.network.shuffle.BlockFetchingListener -import org.apache.spark.storage.ShuffleBlockId -import org.apache.spark.{SecurityManager, SparkConf} +import org.apache.spark.storage.{BlockId, StorageLevel, RDDBlockId, ShuffleBlockId} +import org.apache.spark.{Logging, SecurityManager, SparkConf} +import org.mockito.ArgumentCaptor +import org.mockito.{Matchers => MockitoMatchers} import org.mockito.Mockito._ import org.scalatest.mock.MockitoSugar import org.scalatest.{Matchers, FunSuite} @@ -31,28 +34,23 @@ import org.scalatest.{Matchers, FunSuite} import scala.concurrent.duration.FiniteDuration import scala.concurrent.{Await, Promise} -class NettyBlockTransferSuite extends FunSuite with Matchers with MockitoSugar { +class NettyBlockTransferSuite extends FunSuite with Matchers with MockitoSugar with Logging { val conf = new SparkConf() .set("spark.app.id", "app-id") val securityManager = new SecurityManager(conf) - - test("simple fetch") { - + def fetchBlock(buf: LargeByteBuffer): ManagedBuffer = { val blockManager = mock[BlockDataManager] val blockId = ShuffleBlockId(0, 1, 2) - val buf = LargeByteBufferHelper.allocate(Integer.MAX_VALUE.toLong + 100l) val blockBuffer = new NioManagedBuffer(buf) when(blockManager.getBlockData(blockId)).thenReturn(blockBuffer) val from = new NettyBlockTransferService(conf, securityManager, numCores = 1) from.init(blockManager) - println("from: " + from.hostName + ":" + from.port) val to = new NettyBlockTransferService(conf, securityManager, numCores = 1) to.init(blockManager) - println("to: " + to.hostName + ":" + to.port) try { val promise = Promise[ManagedBuffer]() @@ -69,17 +67,72 @@ class NettyBlockTransferSuite extends FunSuite with Matchers with MockitoSugar { }) Await.ready(promise.future, FiniteDuration(100, TimeUnit.SECONDS)) - val v = promise.future.value.get.get -// IOUtils.toString(v.createInputStream()) should equal(blockString) - println(v.nioByteBuffer().limit()) + promise.future.value.get.get } finally { from.close() to.close() } + } + + ignore("simple fetch") { + val blockString = "Hello, world!" + val blockBuffer = LargeByteBufferHelper.asLargeByteBuffer(blockString.getBytes) + val fetched = fetchBlock(blockBuffer) + + IOUtils.toString(fetched.createInputStream()) should equal(blockString) + } + + + def uploadBlock(buf: LargeByteBuffer) { + + val fromBlockManager = mock[BlockDataManager] + val toBlockManager = mock[BlockDataManager] + val blockId = RDDBlockId(0, 1) + val blockBuffer = new NioManagedBuffer(buf) + val level = StorageLevel.DISK_ONLY //doesn't matter + + val from = new NettyBlockTransferService(conf, securityManager, numCores = 1) + from.init(fromBlockManager) + val to = new NettyBlockTransferService(conf, securityManager, numCores = 1) + logTrace("to block manager = " + toBlockManager) + to.init(toBlockManager) + + from.uploadBlock(to.hostName, to.port, "exec-1", blockId, blockBuffer, level) + //TODO how to get rid of this wait?? + Thread.sleep(1000) + val bufferCaptor = ArgumentCaptor.forClass(classOf[ManagedBuffer]) + verify(toBlockManager).putBlockData(MockitoMatchers.eq(blockId), bufferCaptor.capture(), + MockitoMatchers.eq(level)) + val putBuffer = bufferCaptor.getValue() + } + + test("simple upload") { + val buf = LargeByteBufferHelper.asLargeByteBuffer(Array[Byte](0,1,2,3)) + uploadBlock(buf) + } + + + test("giant upload") { + val parts = (0 until 2).map{_ => ByteBuffer.allocate(Integer.MAX_VALUE - 100)}.toArray + val buf = new WrappedLargeByteBuffer(parts) + uploadBlock(buf) + } + + + def equivalentBuffers(exp: ManagedBuffer, act: ManagedBuffer): Unit = { + equivalentBuffers(exp.nioByteBuffer(), act.nioByteBuffer()) } + def equivalentBuffers(exp: LargeByteBuffer, act: LargeByteBuffer): Unit = { + assert(exp.capacity() === act.capacity()) + assert(exp.remaining() === act.remaining()) + while (exp.remaining() > 0) { + assert(exp.get() === act.get()) + } + + } From 660f5e362439d79d8dfd000a805be0ad5181106c Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Fri, 27 Feb 2015 20:57:32 -0600 Subject: [PATCH 11/97] flesh out NettyBlockTransfer#uploadBlock --- .../spark/io/ChainedLargeByteBuffer.scala | 6 ++ .../network/netty/NettyBlockRpcServer.scala | 56 +++++++++++++-- .../netty/NettyBlockTransferService.scala | 72 +++++++++++++------ .../apache/spark/storage/BlockManager.scala | 4 +- .../netty/NettyBlockTransferSuite.scala | 36 ++++++---- .../rdd/LargePartitionCachingSuite.scala | 4 ++ .../spark/network/buffer/LargeByteBuffer.java | 4 ++ .../buffer/WrappedLargeByteBuffer.java | 8 +++ 8 files changed, 149 insertions(+), 41 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/io/ChainedLargeByteBuffer.scala b/core/src/main/scala/org/apache/spark/io/ChainedLargeByteBuffer.scala index eea1114ec35ca..a5af9618fde4a 100644 --- a/core/src/main/scala/org/apache/spark/io/ChainedLargeByteBuffer.scala +++ b/core/src/main/scala/org/apache/spark/io/ChainedLargeByteBuffer.scala @@ -19,6 +19,8 @@ package org.apache.spark.io import java.nio.ByteBuffer import java.nio.channels.WritableByteChannel +import scala.collection.JavaConverters._ + import org.apache.spark.network.buffer.LargeByteBuffer import org.apache.spark.util.collection.ChainedBuffer @@ -82,4 +84,8 @@ class ChainedLargeByteBuffer(private[io] val underlying: ChainedBuffer) extends override def firstByteBuffer(): ByteBuffer = { ByteBuffer.wrap(underlying.chunks(0)) } + + override def nioBuffers(): java.util.List[ByteBuffer] = { + underlying.chunks.map{bytes => ByteBuffer.wrap(bytes)}.asJava + } } diff --git a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala index 86df34920a666..0c67459382502 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala @@ -18,15 +18,16 @@ package org.apache.spark.network.netty import java.nio.ByteBuffer +import java.util.concurrent.ConcurrentHashMap -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import org.apache.spark.Logging import org.apache.spark.network.BlockDataManager -import org.apache.spark.network.buffer.{LargeByteBufferHelper, ManagedBuffer, NioManagedBuffer} +import org.apache.spark.network.buffer.{WrappedLargeByteBuffer, LargeByteBufferHelper, ManagedBuffer, NioManagedBuffer} import org.apache.spark.network.client.{RpcResponseCallback, TransportClient} import org.apache.spark.network.server.{OneForOneStreamManager, RpcHandler, StreamManager} -import org.apache.spark.network.shuffle.protocol.{BlockTransferMessage, OpenBlocks, StreamHandle, UploadBlock} +import org.apache.spark.network.shuffle.protocol._ import org.apache.spark.serializer.Serializer import org.apache.spark.storage.{BlockId, StorageLevel} @@ -44,6 +45,8 @@ class NettyBlockRpcServer( private val streamManager = new OneForOneStreamManager() + private val openRequests = new ConcurrentHashMap[String,PartialBlockUploadHandler]() + override def receive( client: TransportClient, messageBytes: Array[Byte], @@ -55,7 +58,7 @@ class NettyBlockRpcServer( case openBlocks: OpenBlocks => val blocks: Seq[ManagedBuffer] = openBlocks.blockIds.map(BlockId.apply).map(blockManager.getBlockData) - val streamId = streamManager.registerStream(blocks.iterator) + val streamId = streamManager.registerStream(blocks.iterator.asJava) logTrace(s"Registered streamId $streamId with ${blocks.size} buffers") responseContext.onSuccess(new StreamHandle(streamId, blocks.size).toByteArray) @@ -64,8 +67,53 @@ class NettyBlockRpcServer( val level: StorageLevel = serializer.newInstance().deserialize(ByteBuffer.wrap(uploadBlock.metadata)) val data = new NioManagedBuffer(LargeByteBufferHelper.asLargeByteBuffer(uploadBlock.blockData)) + logTrace("putting block into our block manager: " + blockManager) blockManager.putBlockData(BlockId(uploadBlock.blockId), data, level) responseContext.onSuccess(new Array[Byte](0)) + + case uploadPartialBock: UploadPartialBlock => + logTrace("received upload partial block: " + uploadPartialBock) + val storageLevel: StorageLevel = + serializer.newInstance().deserialize(ByteBuffer.wrap(uploadPartialBock.metadata)) + logTrace("open requests = " + openRequests) + openRequests.putIfAbsent(uploadPartialBock.blockId, + new PartialBlockUploadHandler(uploadPartialBock.blockId, storageLevel, + uploadPartialBock.nTotalBlockChunks)) + val handler = openRequests.get(uploadPartialBock.blockId) + handler.addPartialBlock(uploadPartialBock, storageLevel) + responseContext.onSuccess(new Array[Byte](0)) + } + } + + + private class PartialBlockUploadHandler( + val blockId: String, + val storageLevel: StorageLevel, + val nTotalBlockChunks: Int + ) { + val chunks = new Array[Array[Byte]](nTotalBlockChunks) + var nMissing = nTotalBlockChunks + + def addPartialBlock(partial: UploadPartialBlock, storageLevel: StorageLevel): Unit = { + if (partial.nTotalBlockChunks != nTotalBlockChunks) { + throw new IllegalArgumentException(s"received incompatible UploadPartialBlock: expecting " + + s"$nTotalBlockChunks total chunks, but new msg has ${partial.nTotalBlockChunks}") + } + if (storageLevel != this.storageLevel) { + throw new IllegalArgumentException(s"received incompatible UploadPartialBlock: expecting " + + s"${this.storageLevel}, but new message has $storageLevel") + } + logTrace("received partial msg") + chunks(partial.blockChunkIndex) = partial.blockData + nMissing -= 1 + logTrace("nmissing = " + nMissing) + if (nMissing == 0) { + //we've got all the blocks -- now we can insert into the block manager + logTrace("received all partial blocks for " + blockId) + val data = new NioManagedBuffer(new WrappedLargeByteBuffer(chunks.map{ByteBuffer.wrap})) + blockManager.putBlockData(BlockId(blockId), data, storageLevel) + openRequests.remove(blockId) + } } } diff --git a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala index 9824c7c38c188..3deecf3242d4a 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala @@ -18,6 +18,7 @@ package org.apache.spark.network.netty import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import scala.concurrent.{Future, Promise} import org.apache.spark.{SecurityManager, SparkConf} @@ -27,11 +28,13 @@ import org.apache.spark.network.client.{TransportClientBootstrap, RpcResponseCal import org.apache.spark.network.sasl.{SaslRpcHandler, SaslClientBootstrap} import org.apache.spark.network.server._ import org.apache.spark.network.shuffle.{RetryingBlockFetcher, BlockFetchingListener, OneForOneBlockFetcher} -import org.apache.spark.network.shuffle.protocol.UploadBlock +import org.apache.spark.network.shuffle.protocol.{UploadPartialBlock, UploadBlock} import org.apache.spark.serializer.JavaSerializer import org.apache.spark.storage.{BlockId, StorageLevel} import org.apache.spark.util.Utils +import scala.util.{Failure, Success} + /** * A BlockTransferService that uses Netty to fetch a set of blocks at at time. */ @@ -106,7 +109,6 @@ class NettyBlockTransferService(conf: SparkConf, securityManager: SecurityManage blockId: BlockId, blockData: ManagedBuffer, level: StorageLevel): Future[Unit] = { - val result = Promise[Unit]() val client = clientFactory.createClient(hostname, port) // StorageLevel is serialized as bytes using our JavaSerializer. Everything else is encoded @@ -114,27 +116,57 @@ class NettyBlockTransferService(conf: SparkConf, securityManager: SecurityManage val levelBytes = serializer.newInstance().serialize(level).array() // Convert or copy nio buffer into array in order to serialize it. - val nioBuffer = blockData.nioByteBuffer() - //TODO key change -- multiple uploads here - // this stub is not even efficient when the buffer actually is small - val array = new Array[Byte](nioBuffer.remaining().toInt) - nioBuffer.get(array, 0, nioBuffer.remaining().toInt) - - client.sendRpc(new UploadBlock(appId, execId, blockId.toString, levelBytes, array).toByteArray, - new RpcResponseCallback { - override def onSuccess(response: Array[Byte]): Unit = { - logTrace(s"Successfully uploaded block $blockId") - result.success() - } - override def onFailure(e: Throwable): Unit = { - logError(s"Error while uploading block $blockId", e) - result.failure(e) - } - }) + val largeByteBuffer = blockData.nioByteBuffer() + val bufferParts = largeByteBuffer.nioBuffers().asScala + val chunkOffsets: Seq[Long] = bufferParts.scanLeft(0l){case(offset, buf) => offset + buf.limit()} - result.future + performSequentially(bufferParts.zipWithIndex){case (buf,idx) => + val partialBlockArray = if (buf.hasArray) { + buf.array() + } else { + val arr = new Array[Byte](buf.limit()) + buf.get(arr) + arr + } + //Note: one major shortcoming of this is that it expects the incoming LargeByteBuffer to + // already be reasonably chunked -- in particular, the chunks cannot get too close to 2GB + // or else we'll still run into problems b/c there is some more overhead in the transfer + val msg = new UploadPartialBlock(appId, execId, blockId.toString, bufferParts.size, idx, + chunkOffsets(idx), levelBytes, partialBlockArray) + + val result = Promise[Unit]() + client.sendRpc(msg.toByteArray, + new RpcResponseCallback { + override def onSuccess(response: Array[Byte]): Unit = { + logTrace(s"Successfully uploaded partial block $blockId, part $idx (out of ${bufferParts.size})") + result.success() + } + + override def onFailure(e: Throwable): Unit = { + logError(s"Error while uploading partial block $blockId, part $idx (out of ${bufferParts.size})", e) + result.failure(e) + } + }) + result.future + } } + //thanks to our old friend @ryanlecompte: https://gist.github.com/squito/242f82ad6345e3f85a5b + private def performSequentially[A](items: Seq[A])(f: A => Future[Unit]): Future[Unit] = { + import scala.concurrent.ExecutionContext.Implicits.global + items.headOption match { + case Some(nextItem) => + val fut = f(nextItem) + fut.flatMap { _ => + performSequentially(items.tail)(f) + } + case None => + // nothing left to process + Future.successful(()) + } + } + + override def close(): Unit = { server.close() clientFactory.close() 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 be63f9cb03d29..92e9512146b35 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -947,8 +947,8 @@ private[spark] class BlockManager( //TODO //ACK! here we're stuck -- we can't replicate a large block until we figure out // how to deal w/ shuffling more than 2 gb -// blockTransferService.uploadBlockSync( -// peer.host, peer.port, peer.executorId, blockId, new NioManagedBuffer(data), tLevel) + blockTransferService.uploadBlockSync( + peer.host, peer.port, peer.executorId, blockId, new NioManagedBuffer(data), tLevel) logTrace(s"Replicated $blockId of ${data.limit()} bytes to $peer in %s ms" .format(System.currentTimeMillis - onePeerStartTime)) peersReplicatedTo += peer diff --git a/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSuite.scala b/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSuite.scala index 710bf2822fa9b..a484221aaa731 100644 --- a/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSuite.scala +++ b/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSuite.scala @@ -31,7 +31,7 @@ import org.mockito.Mockito._ import org.scalatest.mock.MockitoSugar import org.scalatest.{Matchers, FunSuite} -import scala.concurrent.duration.FiniteDuration +import scala.concurrent.duration.{Duration, FiniteDuration} import scala.concurrent.{Await, Promise} class NettyBlockTransferSuite extends FunSuite with Matchers with MockitoSugar with Logging { @@ -84,39 +84,49 @@ class NettyBlockTransferSuite extends FunSuite with Matchers with MockitoSugar w } - def uploadBlock(buf: LargeByteBuffer) { + def uploadBlock(buf: LargeByteBuffer, rddId: Int, timeout: Long) { val fromBlockManager = mock[BlockDataManager] val toBlockManager = mock[BlockDataManager] - val blockId = RDDBlockId(0, 1) + val blockId = RDDBlockId(rddId, rddId + 1) val blockBuffer = new NioManagedBuffer(buf) val level = StorageLevel.DISK_ONLY //doesn't matter val from = new NettyBlockTransferService(conf, securityManager, numCores = 1) from.init(fromBlockManager) val to = new NettyBlockTransferService(conf, securityManager, numCores = 1) - logTrace("to block manager = " + toBlockManager) to.init(toBlockManager) - from.uploadBlock(to.hostName, to.port, "exec-1", blockId, blockBuffer, level) - //TODO how to get rid of this wait?? - Thread.sleep(1000) + val uploadFuture = from.uploadBlock(to.hostName, to.port, "exec-1", blockId, blockBuffer, level) + Await.result(uploadFuture, Duration.apply(timeout, TimeUnit.MILLISECONDS)) val bufferCaptor = ArgumentCaptor.forClass(classOf[ManagedBuffer]) verify(toBlockManager).putBlockData(MockitoMatchers.eq(blockId), bufferCaptor.capture(), MockitoMatchers.eq(level)) val putBuffer = bufferCaptor.getValue() + logTrace("begin checking buffer equivalence") + equivalentBuffers(blockBuffer, putBuffer) + logTrace("finished checking buffer equivalence") } - test("simple upload") { + test("small one-part upload") { val buf = LargeByteBufferHelper.asLargeByteBuffer(Array[Byte](0,1,2,3)) - uploadBlock(buf) + uploadBlock(buf, 0, 100) } + test("small multi-part upload") { + val parts = (0 until 5).map{idx => + val arr = Array.tabulate[Byte](100){subIdx => (idx + subIdx).toByte} + ByteBuffer.wrap(arr) + }.toArray + val buf = new WrappedLargeByteBuffer(parts) + uploadBlock(buf, 1, 500) + } test("giant upload") { - val parts = (0 until 2).map{_ => ByteBuffer.allocate(Integer.MAX_VALUE - 100)}.toArray + //actually pretty close to max size due to overhead from the rest of the msg + val parts = (0 until 2).map{_ => ByteBuffer.allocate(Integer.MAX_VALUE - 200)}.toArray val buf = new WrappedLargeByteBuffer(parts) - uploadBlock(buf) + uploadBlock(buf, 2, 15 * 60 * 1000) } @@ -131,9 +141,5 @@ class NettyBlockTransferSuite extends FunSuite with Matchers with MockitoSugar w while (exp.remaining() > 0) { assert(exp.get() === act.get()) } - } - - - } diff --git a/core/src/test/scala/org/apache/spark/rdd/LargePartitionCachingSuite.scala b/core/src/test/scala/org/apache/spark/rdd/LargePartitionCachingSuite.scala index 21c6e5fe3ab50..1233f16978d32 100644 --- a/core/src/test/scala/org/apache/spark/rdd/LargePartitionCachingSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/LargePartitionCachingSuite.scala @@ -33,4 +33,8 @@ class LargePartitionCachingSuite extends FunSuite with SharedSparkContext { test("disk cache large partitions") { largePartitionRdd.persist(StorageLevel.DISK_ONLY).count() } + + test("disk cache large partitions with replications") { + pending + } } diff --git a/network/common/src/main/java/org/apache/spark/network/buffer/LargeByteBuffer.java b/network/common/src/main/java/org/apache/spark/network/buffer/LargeByteBuffer.java index 4997dcecc3370..d5a3c2cb476eb 100644 --- a/network/common/src/main/java/org/apache/spark/network/buffer/LargeByteBuffer.java +++ b/network/common/src/main/java/org/apache/spark/network/buffer/LargeByteBuffer.java @@ -19,6 +19,7 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.nio.channels.WritableByteChannel; +import java.util.List; public interface LargeByteBuffer { public long capacity(); @@ -59,4 +60,7 @@ public interface LargeByteBuffer { //TODO this should be deleted -- just to help me get going public ByteBuffer firstByteBuffer(); + //List b/c we need to know the size. Could also use Iterator w/ separate numBuffers method + public List nioBuffers(); + } diff --git a/network/common/src/main/java/org/apache/spark/network/buffer/WrappedLargeByteBuffer.java b/network/common/src/main/java/org/apache/spark/network/buffer/WrappedLargeByteBuffer.java index e948fa67581d0..34adcc98fa146 100644 --- a/network/common/src/main/java/org/apache/spark/network/buffer/WrappedLargeByteBuffer.java +++ b/network/common/src/main/java/org/apache/spark/network/buffer/WrappedLargeByteBuffer.java @@ -19,6 +19,9 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.nio.channels.WritableByteChannel; +import java.util.Arrays; +import java.util.Iterator; +import java.util.List; public class WrappedLargeByteBuffer implements LargeByteBuffer { @@ -142,4 +145,9 @@ public long writeTo(WritableByteChannel channel) throws IOException { public ByteBuffer firstByteBuffer() { return underlying[0]; } + + @Override + public List nioBuffers() { + return Arrays.asList(underlying); + } } From 4c228a07173e06f8da449db17b878d220e14dea0 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Fri, 27 Feb 2015 21:14:13 -0600 Subject: [PATCH 12/97] minor cleanup --- .../network/netty/NettyBlockRpcServer.scala | 1 - .../netty/NettyBlockTransferService.scala | 73 ++++++++----------- .../netty/NettyBlockTransferSuite.scala | 7 +- 3 files changed, 35 insertions(+), 46 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala index 0c67459382502..822934a323a08 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala @@ -75,7 +75,6 @@ class NettyBlockRpcServer( logTrace("received upload partial block: " + uploadPartialBock) val storageLevel: StorageLevel = serializer.newInstance().deserialize(ByteBuffer.wrap(uploadPartialBock.metadata)) - logTrace("open requests = " + openRequests) openRequests.putIfAbsent(uploadPartialBock.blockId, new PartialBlockUploadHandler(uploadPartialBock.blockId, storageLevel, uploadPartialBock.nTotalBlockChunks)) diff --git a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala index 3deecf3242d4a..8162b640b770b 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala @@ -115,58 +115,45 @@ class NettyBlockTransferService(conf: SparkConf, securityManager: SecurityManage // using our binary protocol. val levelBytes = serializer.newInstance().serialize(level).array() - // Convert or copy nio buffer into array in order to serialize it. val largeByteBuffer = blockData.nioByteBuffer() val bufferParts = largeByteBuffer.nioBuffers().asScala val chunkOffsets: Seq[Long] = bufferParts.scanLeft(0l){case(offset, buf) => offset + buf.limit()} - performSequentially(bufferParts.zipWithIndex){case (buf,idx) => - val partialBlockArray = if (buf.hasArray) { - buf.array() - } else { - val arr = new Array[Byte](buf.limit()) - buf.get(arr) - arr - } - //Note: one major shortcoming of this is that it expects the incoming LargeByteBuffer to - // already be reasonably chunked -- in particular, the chunks cannot get too close to 2GB - // or else we'll still run into problems b/c there is some more overhead in the transfer - val msg = new UploadPartialBlock(appId, execId, blockId.toString, bufferParts.size, idx, - chunkOffsets(idx), levelBytes, partialBlockArray) - - val result = Promise[Unit]() - client.sendRpc(msg.toByteArray, - new RpcResponseCallback { - override def onSuccess(response: Array[Byte]): Unit = { - logTrace(s"Successfully uploaded partial block $blockId, part $idx (out of ${bufferParts.size})") - result.success() - } - - override def onFailure(e: Throwable): Unit = { - logError(s"Error while uploading partial block $blockId, part $idx (out of ${bufferParts.size})", e) - result.failure(e) - } - }) - result.future - } - } - - //thanks to our old friend @ryanlecompte: https://gist.github.com/squito/242f82ad6345e3f85a5b - private def performSequentially[A](items: Seq[A])(f: A => Future[Unit]): Future[Unit] = { import scala.concurrent.ExecutionContext.Implicits.global - items.headOption match { - case Some(nextItem) => - val fut = f(nextItem) - fut.flatMap { _ => - performSequentially(items.tail)(f) + bufferParts.zipWithIndex.foldLeft(Future.successful(())){case (prevFuture,(buf,idx)) => + prevFuture.flatMap{_ => + // Convert or copy nio buffer into array in order to serialize it. + val partialBlockArray = if (buf.hasArray) { + buf.array() + } else { + val arr = new Array[Byte](buf.limit()) + buf.get(arr) + arr } - case None => - // nothing left to process - Future.successful(()) + //Note: one major shortcoming of this is that it expects the incoming LargeByteBuffer to + // already be reasonably chunked -- in particular, the chunks cannot get too close to 2GB + // or else we'll still run into problems b/c there is some more overhead in the transfer + val msg = new UploadPartialBlock(appId, execId, blockId.toString, bufferParts.size, idx, + chunkOffsets(idx), levelBytes, partialBlockArray) + + val result = Promise[Unit]() + client.sendRpc(msg.toByteArray, + new RpcResponseCallback { + override def onSuccess(response: Array[Byte]): Unit = { + logTrace(s"Successfully uploaded partial block $blockId, part $idx (out of ${bufferParts.size})") + result.success() + } + + override def onFailure(e: Throwable): Unit = { + logError(s"Error while uploading partial block $blockId, part $idx (out of ${bufferParts.size})", e) + result.failure(e) + } + }) + result.future + } } } - override def close(): Unit = { server.close() clientFactory.close() diff --git a/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSuite.scala b/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSuite.scala index a484221aaa731..b02b9d396c95c 100644 --- a/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSuite.scala +++ b/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSuite.scala @@ -123,12 +123,15 @@ class NettyBlockTransferSuite extends FunSuite with Matchers with MockitoSugar w } test("giant upload") { - //actually pretty close to max size due to overhead from the rest of the msg + // pretty close to max size due to overhead from the rest of the msg val parts = (0 until 2).map{_ => ByteBuffer.allocate(Integer.MAX_VALUE - 200)}.toArray val buf = new WrappedLargeByteBuffer(parts) - uploadBlock(buf, 2, 15 * 60 * 1000) + uploadBlock(buf, 2, 20 * 60 * 1000) // yup, takes this long ... } + test("cleanup partial uploads") { + pending + } def equivalentBuffers(exp: ManagedBuffer, act: ManagedBuffer): Unit = { From cf7c3a7067aaa61732782995984f17fa94a6cff7 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Fri, 27 Feb 2015 22:45:50 -0600 Subject: [PATCH 13/97] cleanup abandonded block uploads --- .../network/netty/NettyBlockRpcServer.scala | 53 +++++++++++++++++-- .../netty/NettyBlockTransferService.scala | 1 + .../netty/NettyBlockTransferSuite.scala | 25 +++++---- .../spark/network/TransportContext.java | 2 + .../spark/network/server/RpcHandler.java | 2 + 5 files changed, 69 insertions(+), 14 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala index 822934a323a08..9552b71e84de6 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala @@ -18,11 +18,13 @@ package org.apache.spark.network.netty import java.nio.ByteBuffer -import java.util.concurrent.ConcurrentHashMap +import java.util.concurrent.{TimeUnit, Executors, ConcurrentHashMap} + +import org.apache.spark.util.Utils import scala.collection.JavaConverters._ -import org.apache.spark.Logging +import org.apache.spark.{SparkException, Logging} import org.apache.spark.network.BlockDataManager import org.apache.spark.network.buffer.{WrappedLargeByteBuffer, LargeByteBufferHelper, ManagedBuffer, NioManagedBuffer} import org.apache.spark.network.client.{RpcResponseCallback, TransportClient} @@ -45,7 +47,22 @@ class NettyBlockRpcServer( private val streamManager = new OneForOneStreamManager() - private val openRequests = new ConcurrentHashMap[String,PartialBlockUploadHandler]() + private val openRequests = new ConcurrentHashMap[String, PartialBlockUploadHandler]() + // TODO from configuration. Might need to be really big ... + private val cleanupTime = 30 * 60 * 1000 + + //ideally, this should be empty, and it will contain a very small amount of data for abandoned + // requests -- so hopefully its OK to hold on to this forever + private val abandonedRequests = new ConcurrentHashMap[String,Object]() + + val cleaner = Executors.newSingleThreadScheduledExecutor( + Utils.namedThreadFactory("NettyBlockRPCServer cleanup")).scheduleWithFixedDelay( + new Runnable { + def run() { + dropAbandonedPartialUploads() + } + }, cleanupTime / 10, cleanupTime / 10, TimeUnit.MILLISECONDS + ) override def receive( client: TransportClient, @@ -75,6 +92,11 @@ class NettyBlockRpcServer( logTrace("received upload partial block: " + uploadPartialBock) val storageLevel: StorageLevel = serializer.newInstance().deserialize(ByteBuffer.wrap(uploadPartialBock.metadata)) + if (abandonedRequests.containsKey(uploadPartialBock.blockId)) { + val msg = s"Too much time passed between the msgs for this block -- the other msgs have" + + " already been dropped. Try increasing the timeout specified in XXX" + throw new SparkException(msg) + } openRequests.putIfAbsent(uploadPartialBock.blockId, new PartialBlockUploadHandler(uploadPartialBock.blockId, storageLevel, uploadPartialBock.nTotalBlockChunks)) @@ -92,8 +114,12 @@ class NettyBlockRpcServer( ) { val chunks = new Array[Array[Byte]](nTotalBlockChunks) var nMissing = nTotalBlockChunks + var lastUpdated = System.currentTimeMillis() - def addPartialBlock(partial: UploadPartialBlock, storageLevel: StorageLevel): Unit = { + def addPartialBlock( + partial: UploadPartialBlock, + storageLevel: StorageLevel + ): Unit = synchronized { if (partial.nTotalBlockChunks != nTotalBlockChunks) { throw new IllegalArgumentException(s"received incompatible UploadPartialBlock: expecting " + s"$nTotalBlockChunks total chunks, but new msg has ${partial.nTotalBlockChunks}") @@ -102,6 +128,7 @@ class NettyBlockRpcServer( throw new IllegalArgumentException(s"received incompatible UploadPartialBlock: expecting " + s"${this.storageLevel}, but new message has $storageLevel") } + lastUpdated = System.currentTimeMillis() logTrace("received partial msg") chunks(partial.blockChunkIndex) = partial.blockData nMissing -= 1 @@ -116,5 +143,23 @@ class NettyBlockRpcServer( } } + private def dropAbandonedPartialUploads(): Unit = { + logTrace("checking for abandoned uploads among: " + openRequests.keys().asScala.mkString(",")) + val itr = openRequests.entrySet.iterator + while (itr.hasNext()) { + val entry = itr.next() + if (System.currentTimeMillis() - entry.getValue().lastUpdated > cleanupTime) { + logWarning(s"never received all parts for block ${entry.getKey}; dropping this block") + abandonedRequests.putIfAbsent(entry.getKey, new Object()) + itr.remove() + } else { + logTrace(entry.getKey() + " OK") + } + } + } + + override def getStreamManager(): StreamManager = streamManager + + override def close(): Unit = {cleaner.cancel(false)} } diff --git a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala index 8162b640b770b..976a338eeba7f 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala @@ -157,5 +157,6 @@ class NettyBlockTransferService(conf: SparkConf, securityManager: SecurityManage override def close(): Unit = { server.close() clientFactory.close() + transportContext.close() } } diff --git a/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSuite.scala b/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSuite.scala index b02b9d396c95c..ba6e8a3079d10 100644 --- a/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSuite.scala +++ b/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSuite.scala @@ -97,15 +97,21 @@ class NettyBlockTransferSuite extends FunSuite with Matchers with MockitoSugar w val to = new NettyBlockTransferService(conf, securityManager, numCores = 1) to.init(toBlockManager) - val uploadFuture = from.uploadBlock(to.hostName, to.port, "exec-1", blockId, blockBuffer, level) - Await.result(uploadFuture, Duration.apply(timeout, TimeUnit.MILLISECONDS)) - val bufferCaptor = ArgumentCaptor.forClass(classOf[ManagedBuffer]) - verify(toBlockManager).putBlockData(MockitoMatchers.eq(blockId), bufferCaptor.capture(), - MockitoMatchers.eq(level)) - val putBuffer = bufferCaptor.getValue() - logTrace("begin checking buffer equivalence") - equivalentBuffers(blockBuffer, putBuffer) - logTrace("finished checking buffer equivalence") + try { + val uploadFuture = from.uploadBlock(to.hostName, to.port, "exec-1", blockId, blockBuffer, level) + Await.result(uploadFuture, Duration.apply(timeout, TimeUnit.MILLISECONDS)) + val bufferCaptor = ArgumentCaptor.forClass(classOf[ManagedBuffer]) + verify(toBlockManager).putBlockData(MockitoMatchers.eq(blockId), bufferCaptor.capture(), + MockitoMatchers.eq(level)) + val putBuffer = bufferCaptor.getValue() + logTrace("begin checking buffer equivalence") + equivalentBuffers(blockBuffer, putBuffer) + logTrace("finished checking buffer equivalence") + } finally { + from.close() + to.close() + } + } test("small one-part upload") { @@ -133,7 +139,6 @@ class NettyBlockTransferSuite extends FunSuite with Matchers with MockitoSugar w pending } - def equivalentBuffers(exp: ManagedBuffer, act: ManagedBuffer): Unit = { equivalentBuffers(exp.nioByteBuffer(), act.nioByteBuffer()) } diff --git a/network/common/src/main/java/org/apache/spark/network/TransportContext.java b/network/common/src/main/java/org/apache/spark/network/TransportContext.java index 5bc6e5a2418a9..2fa361edb44f6 100644 --- a/network/common/src/main/java/org/apache/spark/network/TransportContext.java +++ b/network/common/src/main/java/org/apache/spark/network/TransportContext.java @@ -131,4 +131,6 @@ private TransportChannelHandler createChannelHandler(Channel channel) { } public TransportConf getConf() { return conf; } + + public void close() { rpcHandler.close(); } } diff --git a/network/common/src/main/java/org/apache/spark/network/server/RpcHandler.java b/network/common/src/main/java/org/apache/spark/network/server/RpcHandler.java index 2ba92a40f8b0a..f8972072a2551 100644 --- a/network/common/src/main/java/org/apache/spark/network/server/RpcHandler.java +++ b/network/common/src/main/java/org/apache/spark/network/server/RpcHandler.java @@ -52,4 +52,6 @@ public abstract void receive( * No further requests will come from this client. */ public void connectionTerminated(TransportClient client) { } + + public void close() { } } From fe90fd682d71ce4c156dde9e6a016e7923e65aad Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Mon, 2 Mar 2015 09:46:19 -0600 Subject: [PATCH 14/97] crank up memory for tests --- pom.xml | 2 +- project/SparkBuild.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/pom.xml b/pom.xml index bb355bf735bee..b5c8f823f62de 100644 --- a/pom.xml +++ b/pom.xml @@ -1199,7 +1199,7 @@ ${project.build.directory}/surefire-reports . SparkTestSuite.txt - -ea -Xmx3g -XX:MaxPermSize=${MaxPermGen} -XX:ReservedCodeCacheSize=${CodeCacheSize} + -ea -Xmx16g -XX:MaxPermSize=${MaxPermGen} -XX:ReservedCodeCacheSize=${CodeCacheSize} updateCurrentBufferIfNeeded + comment --- .../spark/network/buffer/WrappedLargeByteBuffer.java | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/network/common/src/main/java/org/apache/spark/network/buffer/WrappedLargeByteBuffer.java b/network/common/src/main/java/org/apache/spark/network/buffer/WrappedLargeByteBuffer.java index 7c2af1d0be452..93c5c7ac2a37d 100644 --- a/network/common/src/main/java/org/apache/spark/network/buffer/WrappedLargeByteBuffer.java +++ b/network/common/src/main/java/org/apache/spark/network/buffer/WrappedLargeByteBuffer.java @@ -106,7 +106,7 @@ public void get(byte[] dest, int offset, int length) { int toRead = Math.min(length - moved, currentBuffer.remaining()); currentBuffer.get(dest, offset + moved, toRead); moved += toRead; - updateCurrentBuffer(); + updateCurrentBufferIfNeeded(); } _pos += moved; } @@ -147,11 +147,15 @@ public byte get() { } byte r = currentBuffer.get(); _pos += 1; - updateCurrentBuffer(); + updateCurrentBufferIfNeeded(); return r; } - private void updateCurrentBuffer() { + /** + * If we've read to the end of the current buffer, move on to the next one. Safe to call + * even if we haven't moved to the next buffer + */ + private void updateCurrentBufferIfNeeded() { while (currentBuffer != null && !currentBuffer.hasRemaining()) { currentBufferIdx += 1; currentBuffer = currentBufferIdx < underlying.length ? underlying[currentBufferIdx] : null; From b77bbe28e109db72775a592e99572c039b377517 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Mon, 1 Jun 2015 12:09:45 -0500 Subject: [PATCH 77/97] style --- .../buffer/LargeByteBufferInputStreamSuite.scala | 10 ++++++---- .../buffer/LargeByteBufferOutputStreamSuite.scala | 6 ++++-- .../util/io/ByteArrayChunkOutputStreamSuite.scala | 2 +- 3 files changed, 11 insertions(+), 7 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/network/buffer/LargeByteBufferInputStreamSuite.scala b/core/src/test/scala/org/apache/spark/network/buffer/LargeByteBufferInputStreamSuite.scala index 78e06c0381c78..1fa57ee7e77d9 100644 --- a/core/src/test/scala/org/apache/spark/network/buffer/LargeByteBufferInputStreamSuite.scala +++ b/core/src/test/scala/org/apache/spark/network/buffer/LargeByteBufferInputStreamSuite.scala @@ -16,16 +16,18 @@ */ package org.apache.spark.network.buffer -import java.io.{FileInputStream, FileOutputStream, OutputStream, File} +import java.io.{File, FileInputStream, FileOutputStream, OutputStream} import java.nio.channels.FileChannel.MapMode import org.junit.Assert._ -import org.scalatest.{FunSuite, Matchers} +import org.scalatest.Matchers -class LargeByteBufferInputStreamSuite extends FunSuite with Matchers { +import org.apache.spark.SparkFunSuite + +class LargeByteBufferInputStreamSuite extends SparkFunSuite with Matchers { test("read from large mapped file") { - val testFile = File.createTempFile("large-buffer-input-stream-test",".bin") + val testFile = File.createTempFile("large-buffer-input-stream-test", ".bin") try { val out: OutputStream = new FileOutputStream(testFile) diff --git a/core/src/test/scala/org/apache/spark/network/buffer/LargeByteBufferOutputStreamSuite.scala b/core/src/test/scala/org/apache/spark/network/buffer/LargeByteBufferOutputStreamSuite.scala index a55ef03480436..72c98b7feacab 100644 --- a/core/src/test/scala/org/apache/spark/network/buffer/LargeByteBufferOutputStreamSuite.scala +++ b/core/src/test/scala/org/apache/spark/network/buffer/LargeByteBufferOutputStreamSuite.scala @@ -18,9 +18,11 @@ package org.apache.spark.network.buffer import scala.util.Random -import org.scalatest.{FunSuite, Matchers} +import org.scalatest.Matchers -class LargeByteBufferOutputStreamSuite extends FunSuite with Matchers { +import org.apache.spark.SparkFunSuite + +class LargeByteBufferOutputStreamSuite extends SparkFunSuite with Matchers { test("merged buffers for < 2GB") { val out = new LargeByteBufferOutputStream(10) diff --git a/core/src/test/scala/org/apache/spark/util/io/ByteArrayChunkOutputStreamSuite.scala b/core/src/test/scala/org/apache/spark/util/io/ByteArrayChunkOutputStreamSuite.scala index a177365af9689..eaea83689588a 100644 --- a/core/src/test/scala/org/apache/spark/util/io/ByteArrayChunkOutputStreamSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/io/ByteArrayChunkOutputStreamSuite.scala @@ -119,7 +119,7 @@ class ByteArrayChunkOutputStreamSuite extends SparkFunSuite { } { withClue(s"start = $start; end = $end") { try { - assert(o.slice(start,end).toSeq === ref.slice(start,end)) + assert(o.slice(start, end).toSeq === ref.slice(start, end)) } catch { case ex => fail(ex) } From 6c2a115b970d31400cd77dc8f0236555e382a1af Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Mon, 1 Jun 2015 23:46:30 -0500 Subject: [PATCH 78/97] add tests that buffers position is independent from underyling bytebufs; some cleanup --- .../buffer/WrappedLargeByteBuffer.java | 7 ++--- .../buffer/WrappedLargeByteBufferSuite.java | 27 ++++++++++++++++++- 2 files changed, 28 insertions(+), 6 deletions(-) diff --git a/network/common/src/main/java/org/apache/spark/network/buffer/WrappedLargeByteBuffer.java b/network/common/src/main/java/org/apache/spark/network/buffer/WrappedLargeByteBuffer.java index 93c5c7ac2a37d..8c803fae86d12 100644 --- a/network/common/src/main/java/org/apache/spark/network/buffer/WrappedLargeByteBuffer.java +++ b/network/common/src/main/java/org/apache/spark/network/buffer/WrappedLargeByteBuffer.java @@ -211,11 +211,8 @@ public long remaining() { @Override public WrappedLargeByteBuffer duplicate() { - ByteBuffer[] duplicates = new ByteBuffer[underlying.length]; - for (int i = 0; i < underlying.length; i++) { - duplicates[i] = underlying[i].duplicate(); - } - WrappedLargeByteBuffer dup = new WrappedLargeByteBuffer(duplicates, subBufferSize); + // the constructor will duplicate the underlying buffers for us + WrappedLargeByteBuffer dup = new WrappedLargeByteBuffer(underlying, subBufferSize); dup.skip(position()); return dup; } diff --git a/network/common/src/test/java/org/apache/spark/network/buffer/WrappedLargeByteBufferSuite.java b/network/common/src/test/java/org/apache/spark/network/buffer/WrappedLargeByteBufferSuite.java index 5de3f68fe903b..2943705c40ecd 100644 --- a/network/common/src/test/java/org/apache/spark/network/buffer/WrappedLargeByteBufferSuite.java +++ b/network/common/src/test/java/org/apache/spark/network/buffer/WrappedLargeByteBufferSuite.java @@ -216,6 +216,13 @@ public void duplicate() { assertEquals(500 - initialPosition, dup.remaining()); assertConsistent(buf); assertConsistent(dup); + + // check positions of both buffers are independent + buf.skip(20); + assertEquals(initialPosition + 20, buf.position()); + assertEquals(initialPosition, dup.position()); + assertConsistent(buf); + assertConsistent(dup); } } @@ -224,6 +231,19 @@ public void testRequireAtLeastOneBuffer() { new WrappedLargeByteBuffer( new ByteBuffer[0]); } + @Test + public void positionIndependentOfInitialBuffers() { + ByteBuffer[] byteBufs = testDataBuf().underlying; + byteBufs[0].position(50); + for (int initialPosition: new int[]{0,20, 400}) { + WrappedLargeByteBuffer buf = new WrappedLargeByteBuffer(byteBufs, 50); + assertEquals(0L, buf.position()); + assertEquals(50, byteBufs[0].position()); + buf.skip(initialPosition); + assertEquals(initialPosition, buf.position()); + assertEquals(50, byteBufs[0].position()); + } + } private void assertConsistent(WrappedLargeByteBuffer buffer) { long pos = buffer.position(); @@ -245,7 +265,12 @@ private void assertConsistent(WrappedLargeByteBuffer buffer) { } } - private void assertSubArrayEquals(byte[] exp, int expOffset, byte[] act, int actOffset, int length) { + private void assertSubArrayEquals( + byte[] exp, + int expOffset, + byte[] act, + int actOffset, + int length) { byte[] expCopy = new byte[length]; byte[] actCopy = new byte[length]; System.arraycopy(exp, expOffset, expCopy, 0, length); From a9616e40e416f67f9a65f2fcf46e20c0510f7513 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Tue, 2 Jun 2015 14:38:36 -0500 Subject: [PATCH 79/97] better variable name; more chunks in tests --- .../apache/spark/util/io/ByteArrayChunkOutputStream.scala | 6 +++--- .../spark/util/io/ByteArrayChunkOutputStreamSuite.scala | 2 +- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/io/ByteArrayChunkOutputStream.scala b/core/src/main/scala/org/apache/spark/util/io/ByteArrayChunkOutputStream.scala index 907d2cbb7537e..43f2b30c9067e 100644 --- a/core/src/main/scala/org/apache/spark/util/io/ByteArrayChunkOutputStream.scala +++ b/core/src/main/scala/org/apache/spark/util/io/ByteArrayChunkOutputStream.scala @@ -112,11 +112,11 @@ class ByteArrayChunkOutputStream(chunkSize: Int) extends OutputStream { var foundStart = false val result = new Array[Byte](length) while (!foundStart) { - val nextSize = chunkStart + chunks(chunkIdx).size - if (nextSize > start) { + val nextChunkStart = chunkStart + chunks(chunkIdx).size + if (nextChunkStart > start) { foundStart = true } else { - chunkStart = nextSize + chunkStart = nextChunkStart chunkIdx += 1 } } diff --git a/core/src/test/scala/org/apache/spark/util/io/ByteArrayChunkOutputStreamSuite.scala b/core/src/test/scala/org/apache/spark/util/io/ByteArrayChunkOutputStreamSuite.scala index eaea83689588a..e77e17855d40a 100644 --- a/core/src/test/scala/org/apache/spark/util/io/ByteArrayChunkOutputStreamSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/io/ByteArrayChunkOutputStreamSuite.scala @@ -110,7 +110,7 @@ class ByteArrayChunkOutputStreamSuite extends SparkFunSuite { test("slice") { val ref = new Array[Byte](30) Random.nextBytes(ref) - val o = new ByteArrayChunkOutputStream(10) + val o = new ByteArrayChunkOutputStream(5) o.write(ref) for { From 0250ac557ba5b5e0fba3f0c2487437c4f5b42c34 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Tue, 2 Jun 2015 14:51:25 -0500 Subject: [PATCH 80/97] private, @VisibleForTesting --- .../network/buffer/LargeByteBufferInputStream.java | 4 ++-- .../buffer/LargeByteBufferOutputStream.java | 14 +++++++------- 2 files changed, 9 insertions(+), 9 deletions(-) diff --git a/core/src/main/java/org/apache/spark/network/buffer/LargeByteBufferInputStream.java b/core/src/main/java/org/apache/spark/network/buffer/LargeByteBufferInputStream.java index a313951990e81..af4e0c471dd47 100644 --- a/core/src/main/java/org/apache/spark/network/buffer/LargeByteBufferInputStream.java +++ b/core/src/main/java/org/apache/spark/network/buffer/LargeByteBufferInputStream.java @@ -26,8 +26,8 @@ */ public class LargeByteBufferInputStream extends InputStream { - LargeByteBuffer buffer; - final boolean dispose; + private LargeByteBuffer buffer; + private final boolean dispose; public LargeByteBufferInputStream(LargeByteBuffer buffer, boolean dispose) { this.buffer = buffer; diff --git a/core/src/main/java/org/apache/spark/network/buffer/LargeByteBufferOutputStream.java b/core/src/main/java/org/apache/spark/network/buffer/LargeByteBufferOutputStream.java index 739533992ecac..e6b8dc582cc47 100644 --- a/core/src/main/java/org/apache/spark/network/buffer/LargeByteBufferOutputStream.java +++ b/core/src/main/java/org/apache/spark/network/buffer/LargeByteBufferOutputStream.java @@ -20,15 +20,14 @@ import java.io.OutputStream; import java.nio.ByteBuffer; +import com.google.common.annotations.VisibleForTesting; import org.apache.spark.util.io.ByteArrayChunkOutputStream; public class LargeByteBufferOutputStream extends OutputStream { - private final int chunkSize; - final ByteArrayChunkOutputStream output; + private final ByteArrayChunkOutputStream output; public LargeByteBufferOutputStream(int chunkSize) { - this.chunkSize = chunkSize; output = new ByteArrayChunkOutputStream(chunkSize); } @@ -44,10 +43,11 @@ public LargeByteBuffer largeBuffer() { return largeBuffer(LargeByteBufferHelper.MAX_CHUNK_SIZE); } -/** - * exposed for testing. You don't really ever want to call this method -- the returned - * buffer will not implement {{asByteBuffer}} correctly. - */ + /** + * exposed for testing. You don't really ever want to call this method -- the returned + * buffer will not implement {{asByteBuffer}} correctly. + */ + @VisibleForTesting LargeByteBuffer largeBuffer(int maxChunk) { long totalSize = output.size(); int chunksNeeded = (int) ((totalSize + maxChunk - 1) / maxChunk); From b3b6363f49e4d581e6caf75738d1d5a8f4569763 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Tue, 2 Jun 2015 14:51:42 -0500 Subject: [PATCH 81/97] fix newlines --- .../apache/spark/network/buffer/LargeByteBufferInputStream.java | 1 - .../spark/network/buffer/LargeByteBufferOutputStream.java | 2 +- .../org/apache/spark/util/io/ByteArrayChunkOutputStream.scala | 2 -- .../spark/network/buffer/LargeByteBufferInputStreamSuite.scala | 2 -- .../apache/spark/util/io/ByteArrayChunkOutputStreamSuite.scala | 1 - .../java/org/apache/spark/network/buffer/LargeByteBuffer.java | 1 - .../org/apache/spark/network/buffer/LargeByteBufferHelper.java | 2 -- 7 files changed, 1 insertion(+), 10 deletions(-) diff --git a/core/src/main/java/org/apache/spark/network/buffer/LargeByteBufferInputStream.java b/core/src/main/java/org/apache/spark/network/buffer/LargeByteBufferInputStream.java index af4e0c471dd47..e57a7e0ad5ab6 100644 --- a/core/src/main/java/org/apache/spark/network/buffer/LargeByteBufferInputStream.java +++ b/core/src/main/java/org/apache/spark/network/buffer/LargeByteBufferInputStream.java @@ -38,7 +38,6 @@ public LargeByteBufferInputStream(LargeByteBuffer buffer) { this(buffer, false); } - public int read() { if (buffer == null || buffer.remaining() == 0) { return -1; diff --git a/core/src/main/java/org/apache/spark/network/buffer/LargeByteBufferOutputStream.java b/core/src/main/java/org/apache/spark/network/buffer/LargeByteBufferOutputStream.java index e6b8dc582cc47..b03de5fe810c3 100644 --- a/core/src/main/java/org/apache/spark/network/buffer/LargeByteBufferOutputStream.java +++ b/core/src/main/java/org/apache/spark/network/buffer/LargeByteBufferOutputStream.java @@ -66,4 +66,4 @@ LargeByteBuffer largeBuffer(int maxChunk) { public void close() throws IOException { output.close(); } -} \ No newline at end of file +} diff --git a/core/src/main/scala/org/apache/spark/util/io/ByteArrayChunkOutputStream.scala b/core/src/main/scala/org/apache/spark/util/io/ByteArrayChunkOutputStream.scala index 43f2b30c9067e..8101047618f2b 100644 --- a/core/src/main/scala/org/apache/spark/util/io/ByteArrayChunkOutputStream.scala +++ b/core/src/main/scala/org/apache/spark/util/io/ByteArrayChunkOutputStream.scala @@ -21,7 +21,6 @@ import java.io.OutputStream import scala.collection.mutable.ArrayBuffer - /** * An OutputStream that writes to fixed-size chunks of byte arrays. * @@ -135,5 +134,4 @@ class ByteArrayChunkOutputStream(chunkSize: Int) extends OutputStream { result } - } diff --git a/core/src/test/scala/org/apache/spark/network/buffer/LargeByteBufferInputStreamSuite.scala b/core/src/test/scala/org/apache/spark/network/buffer/LargeByteBufferInputStreamSuite.scala index 1fa57ee7e77d9..a20a3d48837e4 100644 --- a/core/src/test/scala/org/apache/spark/network/buffer/LargeByteBufferInputStreamSuite.scala +++ b/core/src/test/scala/org/apache/spark/network/buffer/LargeByteBufferInputStreamSuite.scala @@ -88,8 +88,6 @@ class LargeByteBufferInputStreamSuite extends SparkFunSuite with Matchers { (0 until 200).foreach{idx => arr(idx) should be (idx.toByte) } - } - } diff --git a/core/src/test/scala/org/apache/spark/util/io/ByteArrayChunkOutputStreamSuite.scala b/core/src/test/scala/org/apache/spark/util/io/ByteArrayChunkOutputStreamSuite.scala index e77e17855d40a..38bc24528f3a7 100644 --- a/core/src/test/scala/org/apache/spark/util/io/ByteArrayChunkOutputStreamSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/io/ByteArrayChunkOutputStreamSuite.scala @@ -21,7 +21,6 @@ import scala.util.Random import org.apache.spark.SparkFunSuite - class ByteArrayChunkOutputStreamSuite extends SparkFunSuite { test("empty output") { diff --git a/network/common/src/main/java/org/apache/spark/network/buffer/LargeByteBuffer.java b/network/common/src/main/java/org/apache/spark/network/buffer/LargeByteBuffer.java index fe03ec0a7e64c..84f8e0e60877b 100644 --- a/network/common/src/main/java/org/apache/spark/network/buffer/LargeByteBuffer.java +++ b/network/common/src/main/java/org/apache/spark/network/buffer/LargeByteBuffer.java @@ -91,7 +91,6 @@ public interface LargeByteBuffer { */ public LargeByteBuffer duplicate(); - public long remaining(); /** diff --git a/network/common/src/main/java/org/apache/spark/network/buffer/LargeByteBufferHelper.java b/network/common/src/main/java/org/apache/spark/network/buffer/LargeByteBufferHelper.java index cd85f3365df57..4941ed6559ea9 100644 --- a/network/common/src/main/java/org/apache/spark/network/buffer/LargeByteBufferHelper.java +++ b/network/common/src/main/java/org/apache/spark/network/buffer/LargeByteBufferHelper.java @@ -67,7 +67,6 @@ static LargeByteBuffer allocate(long size, int maxChunk) { return new WrappedLargeByteBuffer(chunks, maxChunk); } - public static LargeByteBuffer mapFile( FileChannel channel, FileChannel.MapMode mode, @@ -86,5 +85,4 @@ public static LargeByteBuffer mapFile( return new WrappedLargeByteBuffer(chunks); } - } From 112c49e7074dc3a53f7425f6b93fbff7608f6a23 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Tue, 2 Jun 2015 15:13:39 -0500 Subject: [PATCH 82/97] style --- .../network/buffer/LargeByteBufferInputStreamSuite.scala | 4 ++-- .../apache/spark/network/buffer/WrappedLargeByteBuffer.java | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/network/buffer/LargeByteBufferInputStreamSuite.scala b/core/src/test/scala/org/apache/spark/network/buffer/LargeByteBufferInputStreamSuite.scala index a20a3d48837e4..d8e48db32f78c 100644 --- a/core/src/test/scala/org/apache/spark/network/buffer/LargeByteBufferInputStreamSuite.scala +++ b/core/src/test/scala/org/apache/spark/network/buffer/LargeByteBufferInputStreamSuite.scala @@ -74,7 +74,7 @@ class LargeByteBufferInputStreamSuite extends SparkFunSuite with Matchers { test("io stream roundtrip") { val out = new LargeByteBufferOutputStream(128) - (0 until 200).foreach{idx => out.write(idx)} + (0 until 200).foreach { idx => out.write(idx) } out.close() val lb = out.largeBuffer(128) @@ -85,7 +85,7 @@ class LargeByteBufferInputStreamSuite extends SparkFunSuite with Matchers { val arr = new Array[Byte](500) val nRead = rawIn.read(arr, 0, 500) nRead should be (200) - (0 until 200).foreach{idx => + (0 until 200).foreach { idx => arr(idx) should be (idx.toByte) } } diff --git a/network/common/src/main/java/org/apache/spark/network/buffer/WrappedLargeByteBuffer.java b/network/common/src/main/java/org/apache/spark/network/buffer/WrappedLargeByteBuffer.java index 8c803fae86d12..ba1ff3d4d3467 100644 --- a/network/common/src/main/java/org/apache/spark/network/buffer/WrappedLargeByteBuffer.java +++ b/network/common/src/main/java/org/apache/spark/network/buffer/WrappedLargeByteBuffer.java @@ -84,7 +84,7 @@ public WrappedLargeByteBuffer(ByteBuffer[] underlying) { ByteBuffer b = underlying[i].duplicate(); b.position(0); this.underlying[i] = b; - if (i != underlying.length -1 && b.capacity() != subBufferSize) { + if (i != underlying.length - 1 && b.capacity() != subBufferSize) { throw new IllegalArgumentException("All buffers, except for the final one, must have " + "size = " + subBufferSize); } From 8ec2c5c25ff7439c6ecccd4967adb0c03b616ccb Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Tue, 2 Jun 2015 16:10:34 -0500 Subject: [PATCH 83/97] comment explaining check on subBufferSize --- .../apache/spark/network/buffer/WrappedLargeByteBuffer.java | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/network/common/src/main/java/org/apache/spark/network/buffer/WrappedLargeByteBuffer.java b/network/common/src/main/java/org/apache/spark/network/buffer/WrappedLargeByteBuffer.java index ba1ff3d4d3467..dbbbc0d0d7a9d 100644 --- a/network/common/src/main/java/org/apache/spark/network/buffer/WrappedLargeByteBuffer.java +++ b/network/common/src/main/java/org/apache/spark/network/buffer/WrappedLargeByteBuffer.java @@ -85,6 +85,11 @@ public WrappedLargeByteBuffer(ByteBuffer[] underlying) { b.position(0); this.underlying[i] = b; if (i != underlying.length - 1 && b.capacity() != subBufferSize) { + // this is to make sure that asByteBuffer() is implemented correctly. We need the first + // subBuffer to be LargeByteBufferHelper.MAX_CHUNK_SIZE. We don't *have* to check all the + // subBuffers, but I figure its makes it more consistent this way. (Also, this check + // really only serves a purpose when using the public constructor -- subBufferSize is a + // a parameter just to allow small tests.) throw new IllegalArgumentException("All buffers, except for the final one, must have " + "size = " + subBufferSize); } From d0605a18b3f022df9d139a27e65f9b887924536a Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 3 Jun 2015 14:28:51 -0500 Subject: [PATCH 84/97] get() return this; another version of get() which just takes dest array --- .../spark/network/buffer/LargeByteBuffer.java | 21 ++++++++++--- .../buffer/WrappedLargeByteBuffer.java | 9 +++++- .../buffer/WrappedLargeByteBufferSuite.java | 31 +++++++++++++++++-- 3 files changed, 54 insertions(+), 7 deletions(-) diff --git a/network/common/src/main/java/org/apache/spark/network/buffer/LargeByteBuffer.java b/network/common/src/main/java/org/apache/spark/network/buffer/LargeByteBuffer.java index 84f8e0e60877b..beeb007e2197e 100644 --- a/network/common/src/main/java/org/apache/spark/network/buffer/LargeByteBuffer.java +++ b/network/common/src/main/java/org/apache/spark/network/buffer/LargeByteBuffer.java @@ -47,15 +47,28 @@ public interface LargeByteBuffer { public byte get(); + + /** + * Bulk copy data from this buffer into the given array. First checks there is sufficient + * data in this buffer; if not, throws a {@link java.nio.BufferUnderflowException}. Behaves + * in the exact same way as get(dst, 0, dst.length) + * + * @param dst the destination array + * @return this buffer + */ + public LargeByteBuffer get(byte[] dst); + /** * Bulk copy data from this buffer into the given array. First checks there is sufficient * data in this buffer; if not, throws a {@link java.nio.BufferUnderflowException}. * - * @param dst - * @param offset - * @param length + * @param dst the destination array + * @param offset the offset within the destination array to write to + * @param length how many bytes to write + * @return this buffer */ - public void get(byte[] dst, int offset, int length); + public LargeByteBuffer get(byte[] dst, int offset, int length); + public LargeByteBuffer rewind(); diff --git a/network/common/src/main/java/org/apache/spark/network/buffer/WrappedLargeByteBuffer.java b/network/common/src/main/java/org/apache/spark/network/buffer/WrappedLargeByteBuffer.java index dbbbc0d0d7a9d..58a621249386f 100644 --- a/network/common/src/main/java/org/apache/spark/network/buffer/WrappedLargeByteBuffer.java +++ b/network/common/src/main/java/org/apache/spark/network/buffer/WrappedLargeByteBuffer.java @@ -101,8 +101,14 @@ public WrappedLargeByteBuffer(ByteBuffer[] underlying) { size = sum; } + @Override - public void get(byte[] dest, int offset, int length) { + public WrappedLargeByteBuffer get(byte[] dest) { + return get(dest, 0, dest.length); + } + + @Override + public WrappedLargeByteBuffer get(byte[] dest, int offset, int length) { if (length > remaining()) { throw new BufferUnderflowException(); } @@ -114,6 +120,7 @@ public void get(byte[] dest, int offset, int length) { updateCurrentBufferIfNeeded(); } _pos += moved; + return this; } @Override diff --git a/network/common/src/test/java/org/apache/spark/network/buffer/WrappedLargeByteBufferSuite.java b/network/common/src/test/java/org/apache/spark/network/buffer/WrappedLargeByteBufferSuite.java index 2943705c40ecd..3cbd2d8710304 100644 --- a/network/common/src/test/java/org/apache/spark/network/buffer/WrappedLargeByteBufferSuite.java +++ b/network/common/src/test/java/org/apache/spark/network/buffer/WrappedLargeByteBufferSuite.java @@ -138,6 +138,16 @@ public void skipAndGet() { assertConsistent(b); b.skip(position); assertConsistent(b); + + int copy2Length = Math.min(20, 500 - position); + byte[] copy2 = new byte[copy2Length]; + b.rewind(); + b.skip(position); + b.get(copy2); + assertSubArrayEquals(data, position, copy2, 0, copy2Length); + + b.rewind(); + b.skip(position); } } @@ -146,10 +156,18 @@ public void get() { WrappedLargeByteBuffer b = testDataBuf(); byte[] into = new byte[500]; for (int[] offsetAndLength: new int[][]{{0, 200}, {10,10}, {300, 20}, {30, 100}}) { + int offset = offsetAndLength[0]; + int length = offsetAndLength[1]; + b.rewind(); + b.get(into, offset, length); + assertConsistent(b); + assertSubArrayEquals(data, 0, into, offset, length); + + byte[] into2 = new byte[length]; b.rewind(); - b.get(into, offsetAndLength[0], offsetAndLength[1]); + b.get(into2); assertConsistent(b); - assertSubArrayEquals(data, 0, into, offsetAndLength[0], offsetAndLength[1]); + assertSubArrayEquals(data, 0, into2, 0, length); } try { @@ -159,6 +177,15 @@ public void get() { fail("expected exception"); } catch (BufferUnderflowException bue) { } + + try { + b.rewind(); + b.skip(1); + b.get(into); + fail("expected exception"); + } catch (BufferUnderflowException bue) { + } + b.rewind(); b.skip(495); assertEquals(data[495], b.get()); From b6620d0a5516519b073b49e6f191ff929c2a5152 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 3 Jun 2015 14:29:02 -0500 Subject: [PATCH 85/97] docs on LargeBBOutputStream --- .../buffer/LargeByteBufferOutputStream.java | 19 +++++++++++++++++++ 1 file changed, 19 insertions(+) diff --git a/core/src/main/java/org/apache/spark/network/buffer/LargeByteBufferOutputStream.java b/core/src/main/java/org/apache/spark/network/buffer/LargeByteBufferOutputStream.java index b03de5fe810c3..e8c8577f361cc 100644 --- a/core/src/main/java/org/apache/spark/network/buffer/LargeByteBufferOutputStream.java +++ b/core/src/main/java/org/apache/spark/network/buffer/LargeByteBufferOutputStream.java @@ -23,10 +23,22 @@ import com.google.common.annotations.VisibleForTesting; import org.apache.spark.util.io.ByteArrayChunkOutputStream; +/** + * An OutputStream that will write all data to memory. It supports writing over 2GB + * and the resulting data can be retrieved as a + * {@link org.apache.spark.network.buffer.LargeByteBuffer} + */ public class LargeByteBufferOutputStream extends OutputStream { private final ByteArrayChunkOutputStream output; + /** + * Create a new LargeByteBufferOutputStream which writes to byte arrays of the given size. Note + * that chunkSize has no effect on the LargeByteBuffer returned by + * {@link #largeBuffer()}. + * + * @param chunkSize size of the byte arrays used by this output stream, in bytes + */ public LargeByteBufferOutputStream(int chunkSize) { output = new ByteArrayChunkOutputStream(chunkSize); } @@ -39,6 +51,13 @@ public void write(byte[] bytes, int off, int len) { output.write(bytes, off, len); } + /** + * Get all of the data written to the stream so far as a LargeByteBuffer. This method can be + * called multiple times, and each returned buffer will be completely independent (the data + * is copied for each returned buffer). It does not close the stream. + * + * @return the data written to the stream as a LargeByteBuffer + */ public LargeByteBuffer largeBuffer() { return largeBuffer(LargeByteBufferHelper.MAX_CHUNK_SIZE); } From 54d09af36e66a0fb215a19bf6160d625cbabf8c8 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 3 Jun 2015 14:40:03 -0500 Subject: [PATCH 86/97] @Override --- .../spark/network/buffer/LargeByteBufferInputStream.java | 5 +++++ .../spark/network/buffer/LargeByteBufferOutputStream.java | 3 +++ 2 files changed, 8 insertions(+) diff --git a/core/src/main/java/org/apache/spark/network/buffer/LargeByteBufferInputStream.java b/core/src/main/java/org/apache/spark/network/buffer/LargeByteBufferInputStream.java index e57a7e0ad5ab6..a4b1e2571af56 100644 --- a/core/src/main/java/org/apache/spark/network/buffer/LargeByteBufferInputStream.java +++ b/core/src/main/java/org/apache/spark/network/buffer/LargeByteBufferInputStream.java @@ -38,6 +38,7 @@ public LargeByteBufferInputStream(LargeByteBuffer buffer) { this(buffer, false); } + @Override public int read() { if (buffer == null || buffer.remaining() == 0) { return -1; @@ -46,10 +47,12 @@ public int read() { } } + @Override public int read(byte[] dest) { return read(dest, 0, dest.length); } + @Override public int read(byte[] dest, int offset, int length) { if (buffer == null || buffer.remaining() == 0) { return -1; @@ -60,6 +63,7 @@ public int read(byte[] dest, int offset, int length) { } } + @Override public long skip(long toSkip) { if (buffer != null) { return buffer.skip(toSkip); @@ -75,6 +79,7 @@ public long skip(long toSkip) { /** * Clean up the buffer, and potentially dispose of it */ + @Override public void close() { if (buffer != null) { if (dispose) { diff --git a/core/src/main/java/org/apache/spark/network/buffer/LargeByteBufferOutputStream.java b/core/src/main/java/org/apache/spark/network/buffer/LargeByteBufferOutputStream.java index e8c8577f361cc..975de7b10f65c 100644 --- a/core/src/main/java/org/apache/spark/network/buffer/LargeByteBufferOutputStream.java +++ b/core/src/main/java/org/apache/spark/network/buffer/LargeByteBufferOutputStream.java @@ -43,10 +43,12 @@ public LargeByteBufferOutputStream(int chunkSize) { output = new ByteArrayChunkOutputStream(chunkSize); } + @Override public void write(int b) { output.write(b); } + @Override public void write(byte[] bytes, int off, int len) { output.write(bytes, off, len); } @@ -82,6 +84,7 @@ LargeByteBuffer largeBuffer(int maxChunk) { return new WrappedLargeByteBuffer(chunks, maxChunk); } + @Override public void close() throws IOException { output.close(); } From 31244c80b14685ebc88fcc43d99f06c3fbfe7717 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 3 Jun 2015 16:45:03 -0500 Subject: [PATCH 87/97] fix comment --- .../org/apache/spark/util/io/ByteArrayChunkOutputStream.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/util/io/ByteArrayChunkOutputStream.scala b/core/src/main/scala/org/apache/spark/util/io/ByteArrayChunkOutputStream.scala index 8101047618f2b..d48eb2f330321 100644 --- a/core/src/main/scala/org/apache/spark/util/io/ByteArrayChunkOutputStream.scala +++ b/core/src/main/scala/org/apache/spark/util/io/ByteArrayChunkOutputStream.scala @@ -97,7 +97,7 @@ class ByteArrayChunkOutputStream(chunkSize: Int) extends OutputStream { /** * Get a copy of the data between the two endpoints, start <= idx < until. Always returns - * an array of size (until - start). Throws an IllegalArgumentException if + * an array of size (until - start). Throws an IllegalArgumentException unless * 0 <= start <= until <= size */ def slice(start: Long, until: Long): Array[Byte] = { From 6cf204f57e976bef4e42431366dbc18f90f54bb5 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 10 Jun 2015 15:58:55 -0500 Subject: [PATCH 88/97] fix mistakes w/ merge --- .../org/apache/spark/storage/BlockManager.scala | 2 +- .../apache/spark/storage/ExternalBlockManager.scala | 6 ++++-- .../apache/spark/storage/ExternalBlockStore.scala | 9 +++++---- .../apache/spark/storage/TachyonBlockManager.scala | 12 ++++++++---- .../streaming/rdd/WriteAheadLogBackedBlockRDD.scala | 6 ++++-- .../streaming/receiver/ReceivedBlockHandler.scala | 2 +- .../spark/streaming/ReceivedBlockHandlerSuite.scala | 2 +- 7 files changed, 24 insertions(+), 15 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 0891624d55df5..6721c72baa62f 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -1220,7 +1220,7 @@ private[spark] class BlockManager( bytes: LargeByteBuffer, serializer: Serializer = defaultSerializer): Iterator[Any] = { bytes.rewind() - dataDeserializeStream(blockId, LargeByteBufferInputStream(bytes, true), serializer) + dataDeserializeStream(blockId, new LargeByteBufferInputStream(bytes, true), serializer) } /** diff --git a/core/src/main/scala/org/apache/spark/storage/ExternalBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/ExternalBlockManager.scala index f39325a12d244..d71253539e914 100644 --- a/core/src/main/scala/org/apache/spark/storage/ExternalBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/ExternalBlockManager.scala @@ -19,6 +19,8 @@ package org.apache.spark.storage import java.nio.ByteBuffer +import org.apache.spark.network.buffer.LargeByteBuffer + /** * An abstract class that the concrete external block manager has to inherit. * The class has to have a no-argument constructor, and will be initialized by init, @@ -75,7 +77,7 @@ private[spark] abstract class ExternalBlockManager { * * @throws java.io.IOException if there is any file system failure in putting the block. */ - def putBytes(blockId: BlockId, bytes: ByteBuffer): Unit + def putBytes(blockId: BlockId, bytes: LargeByteBuffer): Unit def putValues(blockId: BlockId, values: Iterator[_]): Unit = { val bytes = blockManager.dataSerialize(blockId, values) @@ -89,7 +91,7 @@ private[spark] abstract class ExternalBlockManager { * * @throws java.io.IOException if there is any file system failure in getting the block. */ - def getBytes(blockId: BlockId): Option[ByteBuffer] + def getBytes(blockId: BlockId): Option[LargeByteBuffer] /** * Retrieve the block data. diff --git a/core/src/main/scala/org/apache/spark/storage/ExternalBlockStore.scala b/core/src/main/scala/org/apache/spark/storage/ExternalBlockStore.scala index 291394ed34816..000c8788f9452 100644 --- a/core/src/main/scala/org/apache/spark/storage/ExternalBlockStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/ExternalBlockStore.scala @@ -22,6 +22,7 @@ import java.nio.ByteBuffer import scala.util.control.NonFatal import org.apache.spark.Logging +import org.apache.spark.network.buffer.LargeByteBuffer import org.apache.spark.util.Utils @@ -47,7 +48,7 @@ private[spark] class ExternalBlockStore(blockManager: BlockManager, executorId: } } - override def putBytes(blockId: BlockId, bytes: ByteBuffer, level: StorageLevel): PutResult = { + override def putBytes(blockId: BlockId, bytes: LargeByteBuffer, level: StorageLevel): PutResult = { putIntoExternalBlockStore(blockId, bytes, returnValues = true) } @@ -100,7 +101,7 @@ private[spark] class ExternalBlockStore(blockManager: BlockManager, executorId: private def putIntoExternalBlockStore( blockId: BlockId, - bytes: ByteBuffer, + bytes: LargeByteBuffer, returnValues: Boolean): PutResult = { logTrace(s"Attempting to put block $blockId into ExternalBlockStore") // we should never hit here if externalBlockManager is None. Handle it anyway for safety. @@ -110,7 +111,7 @@ private[spark] class ExternalBlockStore(blockManager: BlockManager, executorId: val byteBuffer = bytes.duplicate() byteBuffer.rewind() externalBlockManager.get.putBytes(blockId, byteBuffer) - val size = bytes.limit() + val size = bytes.size() val data = if (returnValues) { Right(bytes) } else { @@ -152,7 +153,7 @@ private[spark] class ExternalBlockStore(blockManager: BlockManager, executorId: } } - override def getBytes(blockId: BlockId): Option[ByteBuffer] = { + override def getBytes(blockId: BlockId): Option[LargeByteBuffer] = { try { externalBlockManager.flatMap(_.getBytes(blockId)) } catch { diff --git a/core/src/main/scala/org/apache/spark/storage/TachyonBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/TachyonBlockManager.scala index b53c86e89a273..68ea35f45db58 100644 --- a/core/src/main/scala/org/apache/spark/storage/TachyonBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/TachyonBlockManager.scala @@ -29,6 +29,7 @@ import com.google.common.io.ByteStreams import tachyon.client.{ReadType, WriteType, TachyonFS, TachyonFile} import tachyon.TachyonURI +import org.apache.spark.network.buffer.{BufferTooLargeException, LargeByteBufferHelper, LargeByteBuffer} import org.apache.spark.{SparkException, SparkConf, Logging} import org.apache.spark.executor.ExecutorExitCode import org.apache.spark.util.Utils @@ -94,12 +95,14 @@ private[spark] class TachyonBlockManager() extends ExternalBlockManager with Log fileExists(file) } - override def putBytes(blockId: BlockId, bytes: ByteBuffer): Unit = { + override def putBytes(blockId: BlockId, bytes: LargeByteBuffer): Unit = { val file = getFile(blockId) val os = file.getOutStream(WriteType.TRY_CACHE) try { - os.write(bytes.array()) + os.write(bytes.asByteBuffer().array()) } catch { + case tooLarge: BufferTooLargeException => + throw new TachyonBlockSizeLimitException(tooLarge) case NonFatal(e) => logWarning(s"Failed to put bytes of block $blockId into Tachyon", e) os.cancel() @@ -122,7 +125,7 @@ private[spark] class TachyonBlockManager() extends ExternalBlockManager with Log } } - override def getBytes(blockId: BlockId): Option[ByteBuffer] = { + override def getBytes(blockId: BlockId): Option[LargeByteBuffer] = { val file = getFile(blockId) if (file == null || file.getLocationHosts.size == 0) { return None @@ -130,9 +133,10 @@ private[spark] class TachyonBlockManager() extends ExternalBlockManager with Log val is = file.getInStream(ReadType.CACHE) try { val size = file.length + //TODO get tachyon to support large blocks val bs = new Array[Byte](size.asInstanceOf[Int]) ByteStreams.readFully(is, bs) - Some(ByteBuffer.wrap(bs)) + Some(LargeByteBufferHelper.asLargeByteBuffer(bs)) } catch { case NonFatal(e) => logWarning(s"Failed to get bytes of block $blockId from Tachyon", e) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDD.scala b/streaming/src/main/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDD.scala index 584332fb4454a..c87304feba1a7 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDD.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDD.scala @@ -158,11 +158,13 @@ class WriteAheadLogBackedBlockRDD[T: ClassTag]( logInfo(s"Read partition data of $this from write ahead log, record handle " + partition.walRecordHandle) if (storeInBlockManager) { - blockManager.putBytes(blockId, dataRead, storageLevel) + blockManager.putBytes(blockId, LargeByteBufferHelper.asLargeByteBuffer(dataRead), + storageLevel) logDebug(s"Stored partition data of $this into block manager with level $storageLevel") dataRead.rewind() } - blockManager.dataDeserialize(blockId, dataRead).asInstanceOf[Iterator[T]] + blockManager.dataDeserialize(blockId, LargeByteBufferHelper.asLargeByteBuffer(dataRead)) + .asInstanceOf[Iterator[T]] } if (partition.isBlockIdValid) { diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlockHandler.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlockHandler.scala index 14f43fcffe2fb..cf95721cf2ace 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlockHandler.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlockHandler.scala @@ -19,7 +19,7 @@ package org.apache.spark.streaming.receiver import scala.concurrent.{Await, ExecutionContext, Future} -import scala.concurrent.{Await, ExecutionContext, Future} +import scala.concurrent.duration._ import scala.language.{existentials, postfixOps} import org.apache.hadoop.conf.Configuration 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 f7d8f734bbdca..37b988de403fa 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala @@ -142,7 +142,7 @@ class ReceivedBlockHandlerSuite val loggedData = walSegments.flatMap { walSegment => val fileSegment = walSegment.asInstanceOf[FileBasedWriteAheadLogSegment] val reader = new FileBasedWriteAheadLogRandomReader(fileSegment.path, hadoopConf) - val bytes = LargeByteBufferHelper.asLargeByteBuffer(reader.read(segment)) + val bytes = LargeByteBufferHelper.asLargeByteBuffer(reader.read(fileSegment)) reader.close() blockManager.dataDeserialize(generateBlockId(), bytes).toList } From 160a4585c52c25127fe751d4ec2d1fd486c895f4 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 10 Jun 2015 16:14:36 -0500 Subject: [PATCH 89/97] more merge fixes --- .../spark/broadcast/TorrentBroadcast.scala | 5 +- .../apache/spark/storage/BlockManager.scala | 4 +- .../util/LargeByteBufferInputStream.scala | 93 ------------------- .../util/LargeByteBufferOutputStream.scala | 70 -------------- .../spark/storage/BlockManagerSuite.scala | 9 +- .../LargeByteBufferInputStreamSuite.scala | 85 ----------------- .../LargeByteBufferOutputStreamSuite.scala | 67 ------------- 7 files changed, 9 insertions(+), 324 deletions(-) delete mode 100644 core/src/main/scala/org/apache/spark/util/LargeByteBufferInputStream.scala delete mode 100644 core/src/main/scala/org/apache/spark/util/LargeByteBufferOutputStream.scala delete mode 100644 core/src/test/scala/org/apache/spark/util/LargeByteBufferInputStreamSuite.scala delete mode 100644 core/src/test/scala/org/apache/spark/util/LargeByteBufferOutputStreamSuite.scala 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 3c5b778aac4e5..8a516709dd2b8 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala @@ -19,17 +19,16 @@ package org.apache.spark.broadcast import java.io._ -import org.apache.spark.network.buffer.{LargeByteBufferHelper, LargeByteBuffer} - import scala.collection.JavaConversions.asJavaEnumeration import scala.reflect.ClassTag import scala.util.Random import org.apache.spark.{Logging, SparkConf, SparkEnv, SparkException} import org.apache.spark.io.CompressionCodec +import org.apache.spark.network.buffer.{LargeByteBufferInputStream, LargeByteBufferHelper, LargeByteBuffer} import org.apache.spark.serializer.Serializer import org.apache.spark.storage.{BroadcastBlockId, StorageLevel} -import org.apache.spark.util.{LargeByteBufferInputStream, Utils} +import org.apache.spark.util.Utils import org.apache.spark.util.io.ByteArrayChunkOutputStream /** 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 6721c72baa62f..dd2585c12e10d 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -1206,7 +1206,7 @@ private[spark] class BlockManager( blockId: BlockId, values: Iterator[Any], serializer: Serializer = defaultSerializer): LargeByteBuffer = { - val byteStream = new LargeByteBufferOutputStream() + val byteStream = new LargeByteBufferOutputStream(65536) dataSerializeStream(blockId, byteStream, values, serializer) byteStream.largeBuffer } @@ -1311,7 +1311,7 @@ object BlockSizeLimitException { def sizeMsg(cause: BufferTooLargeException): String = { s"that was ${Utils.bytesToString(cause.actualSize)} (too " + s"large by ${Utils.bytesToString(cause.extra)} / " + - s"${cause.actualSize.toDouble / LargeByteBufferHelper.MAX_CHUNK}x)." + s"${cause.actualSize.toDouble / LargeByteBufferHelper.MAX_CHUNK_SIZE}x)." } def sizeMsgAndAdvice(cause: BufferTooLargeException): String = { diff --git a/core/src/main/scala/org/apache/spark/util/LargeByteBufferInputStream.scala b/core/src/main/scala/org/apache/spark/util/LargeByteBufferInputStream.scala deleted file mode 100644 index 69bc4902d0aac..0000000000000 --- a/core/src/main/scala/org/apache/spark/util/LargeByteBufferInputStream.scala +++ /dev/null @@ -1,93 +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.util - -import java.io.InputStream - -import org.apache.spark.network.buffer.LargeByteBuffer -import org.apache.spark.storage.BlockManager - -/** - * Reads data from a LargeByteBuffer, and optionally cleans it up using buffer.dispose() - * at the end of the stream (e.g. to close a memory-mapped file). - */ -private[spark] -class LargeByteBufferInputStream(private var buffer: LargeByteBuffer, dispose: Boolean = false) - extends InputStream { - - override def read(): Int = { - if (buffer == null || buffer.remaining() == 0) { - cleanUp() - -1 - } else { - val r = buffer.get() & 0xFF - if (buffer.remaining() == 0) { - cleanUp() - } - r - } - } - - override def read(dest: Array[Byte]): Int = { - read(dest, 0, dest.length) - } - - override def read(dest: Array[Byte], offset: Int, length: Int): Int = { - if (buffer == null || buffer.remaining() == 0) { - cleanUp() - -1 - } else { - val amountToGet = math.min(buffer.remaining(), length).toInt - buffer.get(dest, offset, amountToGet) - // XXX I assume its not intentional that the stream is only disposed when you try to read - // *past* the end in ByteBufferInputStream, so we do a check here - if (buffer.remaining() == 0) { - cleanUp() - } - amountToGet - } - } - - override def skip(bytes: Long): Long = { - if (buffer != null) { - val skipped = buffer.skip(bytes) - if (buffer.remaining() == 0) { - cleanUp() - } - skipped - } else { - 0L - } - } - - // only for testing - private[util] var disposed = false - - /** - * Clean up the buffer, and potentially dispose of it - */ - private def cleanUp() { - if (buffer != null) { - if (dispose) { - buffer.dispose() - disposed = true - } - buffer = null - } - } -} diff --git a/core/src/main/scala/org/apache/spark/util/LargeByteBufferOutputStream.scala b/core/src/main/scala/org/apache/spark/util/LargeByteBufferOutputStream.scala deleted file mode 100644 index 04e685262f38c..0000000000000 --- a/core/src/main/scala/org/apache/spark/util/LargeByteBufferOutputStream.scala +++ /dev/null @@ -1,70 +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.util - -import java.io.OutputStream -import java.nio.ByteBuffer - -import org.apache.spark.network.buffer.{LargeByteBufferHelper, WrappedLargeByteBuffer, LargeByteBuffer} -import org.apache.spark.util.io.ByteArrayChunkOutputStream - -private[spark] -class LargeByteBufferOutputStream(chunkSize: Int = 65536) - extends OutputStream { - - private[util] val output = new ByteArrayChunkOutputStream(chunkSize) - - private var _pos = 0 - - override def write(b: Int): Unit = { - output.write(b) - } - - override def write(bytes: Array[Byte], offs: Int, len: Int): Unit = { - output.write(bytes, offs, len) - _pos += len - } - - def largeBuffer: LargeByteBuffer = { - largeBuffer(LargeByteBufferHelper.MAX_CHUNK) - } - - // exposed for testing - private[util] def largeBuffer(maxChunk: Int): LargeByteBuffer = { - // LargeByteBuffer is supposed to make a "best effort" to get all the data - // in one nio.ByteBuffer, so we want to try to merge the smaller chunks together - // as much as possible. This is necessary b/c there are a number of parts of spark that - // can only deal w/ one nio.ByteBuffer, and can't use a LargeByteBuffer yet. - val totalSize = output.size - val chunksNeeded = ((totalSize + maxChunk -1) / maxChunk).toInt - val chunks = new Array[Array[Byte]](chunksNeeded) - var remaining = totalSize - var pos = 0 - (0 until chunksNeeded).foreach{idx => - val nextSize = math.min(maxChunk, remaining).toInt - chunks(idx) = output.slice(pos, pos + nextSize) - pos += nextSize - remaining -= nextSize - } - new WrappedLargeByteBuffer(chunks.map{ByteBuffer.wrap}) - } - - override def close(): Unit = { - output.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 df05063a99c1f..5e66e0d75a09d 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -830,10 +830,11 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE val notMapped = diskStoreNotMapped.getBytes(blockId).get.asInstanceOf[WrappedLargeByteBuffer] // Not possible to do isInstanceOf due to visibility of HeapByteBuffer - assert(notMapped.nioBuffers().get(0).getClass.getName.endsWith("HeapByteBuffer"), - "Expected HeapByteBuffer for un-mapped read") - assert(mapped.nioBuffers().get(0).isInstanceOf[MappedByteBuffer], - "Expected MappedByteBuffer for mapped read") + // TODO fix me +// assert(notMapped.nioBuffers().get(0).getClass.getName.endsWith("HeapByteBuffer"), +// "Expected HeapByteBuffer for un-mapped read") +// assert(mapped.nioBuffers().get(0).isInstanceOf[MappedByteBuffer], +// "Expected MappedByteBuffer for mapped read") def arrayFromByteBuffer(in: LargeByteBuffer): Array[Byte] = { val array = new Array[Byte](in.remaining().toInt) diff --git a/core/src/test/scala/org/apache/spark/util/LargeByteBufferInputStreamSuite.scala b/core/src/test/scala/org/apache/spark/util/LargeByteBufferInputStreamSuite.scala deleted file mode 100644 index 209fb0d343fd0..0000000000000 --- a/core/src/test/scala/org/apache/spark/util/LargeByteBufferInputStreamSuite.scala +++ /dev/null @@ -1,85 +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.util - -import java.io.{FileInputStream, FileOutputStream, OutputStream, File} -import java.nio.channels.FileChannel.MapMode - -import org.junit.Assert._ -import org.scalatest.{FunSuite, Matchers} - -import org.apache.spark.network.buffer.{LargeByteBufferHelper, WrappedLargeByteBuffer} - -class LargeByteBufferInputStreamSuite extends FunSuite with Matchers { - - test("read from large mapped file") { - val testFile = File.createTempFile("large-buffer-input-stream-test",".bin") - testFile.deleteOnExit() - - val out: OutputStream = new FileOutputStream(testFile) - val buffer: Array[Byte] = new Array[Byte](1 << 16) - val len: Long = 3L << 30 - assertTrue(len > Integer.MAX_VALUE) - (0 until buffer.length).foreach { idx => - buffer(idx) = idx.toByte - } - (0 until (len / buffer.length).toInt).foreach { idx => - out.write(buffer) - } - out.close - - val channel = new FileInputStream(testFile).getChannel - val buf = LargeByteBufferHelper.mapFile(channel, MapMode.READ_ONLY, 0, len) - val in = new LargeByteBufferInputStream(buf, dispose = true) - - val read = new Array[Byte](buffer.length) - (0 until (len / buffer.length).toInt).foreach { idx => - in.disposed should be (false) - in.read(read) should be (read.length) - (0 until buffer.length).foreach { arrIdx => - assertEquals(buffer(arrIdx), read(arrIdx)) - } - } - // XXX I assume its *not* intentional that the stream is only disposed when you try to read - // *past* the end in ByteBufferInputStream? - in.disposed should be (true) - in.read(read) should be (-1) - in.disposed should be (true) - } - - test("io stream roundtrip") { - - val out = new LargeByteBufferOutputStream(128) - (0 until 200).foreach{idx => out.write(idx)} - out.close() - - val lb = out.largeBuffer(128) - //just make sure that we test reading from multiple chunks - lb.asInstanceOf[WrappedLargeByteBuffer].underlying.size should be > 1 - - val rawIn = new LargeByteBufferInputStream(lb) - val arr = new Array[Byte](500) - val nRead = rawIn.read(arr, 0, 500) - nRead should be (200) - (0 until 200).foreach{idx => - arr(idx) should be (idx.toByte) - } - - } - - -} diff --git a/core/src/test/scala/org/apache/spark/util/LargeByteBufferOutputStreamSuite.scala b/core/src/test/scala/org/apache/spark/util/LargeByteBufferOutputStreamSuite.scala deleted file mode 100644 index b25dfc5bbaaac..0000000000000 --- a/core/src/test/scala/org/apache/spark/util/LargeByteBufferOutputStreamSuite.scala +++ /dev/null @@ -1,67 +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.util - -import scala.util.Random - -import org.scalatest.{FunSuite, Matchers} - -import org.apache.spark.network.buffer.WrappedLargeByteBuffer - -class LargeByteBufferOutputStreamSuite extends FunSuite with Matchers { - - test("merged buffers for < 2GB") { - val out = new LargeByteBufferOutputStream(10) - val bytes = new Array[Byte](100) - Random.nextBytes(bytes) - out.write(bytes) - - val buffer = out.largeBuffer - buffer.position() should be (0) - buffer.size() should be (100) - val nioBuffer = buffer.asByteBuffer() - nioBuffer.position() should be (0) - nioBuffer.capacity() should be (100) - nioBuffer.limit() should be (100) - - val read = new Array[Byte](100) - buffer.get(read, 0, 100) - read should be (bytes) - - buffer.rewind() - nioBuffer.get(read) - read should be (bytes) - } - - test("chunking") { - val out = new LargeByteBufferOutputStream(10) - val bytes = new Array[Byte](100) - Random.nextBytes(bytes) - out.write(bytes) - - (10 to 100 by 10).foreach{chunkSize => - val buffer = out.largeBuffer(chunkSize).asInstanceOf[WrappedLargeByteBuffer] - buffer.position() should be (0) - buffer.size() should be (100) - val read = new Array[Byte](100) - buffer.get(read, 0, 100) - read should be (bytes) - } - - } - -} From 040a4615063aef46ce214fab4bd8b0eea7003c89 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 19 Aug 2015 10:47:44 -0500 Subject: [PATCH 90/97] use random numbers for test --- .../buffer/LargeByteBufferHelperSuite.java | 34 ++++++++++++++++--- 1 file changed, 30 insertions(+), 4 deletions(-) diff --git a/network/common/src/test/java/org/apache/spark/network/buffer/LargeByteBufferHelperSuite.java b/network/common/src/test/java/org/apache/spark/network/buffer/LargeByteBufferHelperSuite.java index d46acf342a79b..9e636fc032928 100644 --- a/network/common/src/test/java/org/apache/spark/network/buffer/LargeByteBufferHelperSuite.java +++ b/network/common/src/test/java/org/apache/spark/network/buffer/LargeByteBufferHelperSuite.java @@ -18,6 +18,7 @@ import java.io.*; import java.nio.channels.FileChannel; +import java.util.Random; import org.junit.Test; @@ -32,11 +33,10 @@ public void testMapFile() throws IOException { testFile.deleteOnExit(); OutputStream out = new FileOutputStream(testFile); byte[] buffer = new byte[1 << 16]; + Random rng = new XORShiftRandom(0L); long len = ((long)buffer.length) + Integer.MAX_VALUE + 1; - for (int i = 0; i < buffer.length; i++) { - buffer[i] = (byte) i; - } for (int i = 0; i < len / buffer.length; i++) { + rng.nextBytes(buffer); out.write(buffer); } out.close(); @@ -59,11 +59,14 @@ public void testMapFile() throws IOException { LargeByteBuffer buf = LargeByteBufferHelper.mapFile(in, FileChannel.MapMode.READ_ONLY, 0, len); assertEquals(len, buf.size()); byte[] read = new byte[buffer.length]; + byte[] expected = new byte[buffer.length]; + Random rngExpected = new XORShiftRandom(0L); for (int i = 0; i < len / buffer.length; i++) { buf.get(read, 0, buffer.length); // assertArrayEquals() is really slow + rngExpected.nextBytes(expected); for (int j = 0; j < buffer.length; j++) { - if (read[j] != (byte) (j)) + if (read[j] != expected[j]) fail("bad byte at (i,j) = (" + i + "," + j + ")"); } } @@ -81,4 +84,27 @@ public void testAllocate() { } assertEquals(5, buf.underlying[9].capacity()); } + + + private class XORShiftRandom extends Random { + + XORShiftRandom(long init) { + super(init); + seed = new Random(init).nextLong(); + } + + long seed; + + // we need to just override next - this will be called by nextInt, nextDouble, + // nextGaussian, nextLong, etc. + @Override + protected int next(int bits) { + long nextSeed = seed ^ (seed << 21); + nextSeed ^= (nextSeed >>> 35); + nextSeed ^= (nextSeed << 4); + seed = nextSeed; + return (int) (nextSeed & ((1L << bits) -1)); + } + } + } From 0fa150f7131afef9a445f9ad453856e94d3014ff Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 19 Aug 2015 11:13:35 -0500 Subject: [PATCH 91/97] bring up to date --- .../spark/scheduler/TaskResultGetter.scala | 2 - .../apache/spark/storage/BlockManager.scala | 232 +++++++++--------- .../spark/storage/DiskBlockObjectWriter.scala | 4 +- .../rdd/LargePartitionCachingSuite.scala | 2 +- .../buffer/FileSegmentManagedBuffer.java | 4 +- 5 files changed, 123 insertions(+), 121 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala index e59a8174d3bce..2c0b3f085ca76 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala @@ -20,8 +20,6 @@ package org.apache.spark.scheduler import java.nio.ByteBuffer import java.util.concurrent.RejectedExecutionException -import org.apache.spark.network.buffer.WrappedLargeByteBuffer - import scala.language.existentials import scala.util.control.NonFatal 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 fd3cfc5618368..dd2a198714bac 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -746,129 +746,133 @@ private[spark] class BlockManager( // 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 => - // Duplicate doesn't copy the bytes, but just creates a wrapper - val bufferView = try { - b.buffer.asByteBuffer() - } catch { - case ex: BufferTooLargeException => - throw new ReplicationBlockSizeLimitException(ex) - } - Future { - // This is a blocking action and should run in futureExecutionContext which is a cached - // thread pool - replicate(blockId, bufferView, putLevel) - }(futureExecutionContext) - case _ => null - } - - putBlockInfo.synchronized { - logTrace("Put for block %s took %s to get into synchronized block" - .format(blockId, Utils.getUsedTimeMs(startTimeMs))) - - var marked = false - try { - // returnValues - Whether to return the values put - // blockStore - The type of storage to put these values into - val (returnValues, blockStore: BlockStore) = { - 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. - (true, memoryStore) - } else if (putLevel.useOffHeap) { - // Use external block store - (false, externalBlockStore) - } else if (putLevel.useDisk) { - // Don't get back the bytes from put unless we replicate them - (putLevel.replication > 1, diskStore) - } else { - assert(putLevel == StorageLevel.NONE) - throw new BlockException( - blockId, s"Attempted to put block $blockId without specifying storage level!") + try { + // 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 => + // Duplicate doesn't copy the bytes, but just creates a wrapper + val bufferView = try { + b.buffer.asByteBuffer() + } catch { + case ex: BufferTooLargeException => + throw new ReplicationBlockSizeLimitException(ex) } - } - - // Actually put the values - val result = data match { - case IteratorValues(iterator) => - blockStore.putIterator(blockId, iterator, putLevel, returnValues) - case ArrayValues(array) => - blockStore.putArray(blockId, array, putLevel, returnValues) - case ByteBufferValues(bytes) => - bytes.rewind() - blockStore.putBytes(blockId, bytes, putLevel) - } - size = result.size - result.data match { - case Left (newIterator) if putLevel.useMemory => valuesAfterPut = newIterator - case Right (newBytes) => bytesAfterPut = newBytes - case _ => - } + Future { + // This is a blocking action and should run in futureExecutionContext which is a cached + // thread pool + replicate(blockId, bufferView, putLevel) + }(futureExecutionContext) + case _ => null + } - // Keep track of which blocks are dropped from memory - if (putLevel.useMemory) { - result.droppedBlocks.foreach { updatedBlocks += _ } - } + putBlockInfo.synchronized { + logTrace("Put for block %s took %s to get into synchronized block" + .format(blockId, Utils.getUsedTimeMs(startTimeMs))) + + var marked = false + try { + // returnValues - Whether to return the values put + // blockStore - The type of storage to put these values into + val (returnValues, blockStore: BlockStore) = { + 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. + (true, memoryStore) + } else if (putLevel.useOffHeap) { + // Use external block store + (false, externalBlockStore) + } else if (putLevel.useDisk) { + // Don't get back the bytes from put unless we replicate them + (putLevel.replication > 1, diskStore) + } else { + assert(putLevel == StorageLevel.NONE) + throw new BlockException( + blockId, s"Attempted to put block $blockId without specifying storage level!") + } + } - val putBlockStatus = getCurrentBlockStatus(blockId, putBlockInfo) - if (putBlockStatus.storageLevel != StorageLevel.NONE) { - // Now that the block is in either the memory, externalBlockStore, or disk store, - // let other threads read it, and tell the master about it. - marked = true - putBlockInfo.markReady(size) - if (tellMaster) { - reportBlockStatus(blockId, putBlockInfo, putBlockStatus) + // Actually put the values + val result = data match { + case IteratorValues(iterator) => + blockStore.putIterator(blockId, iterator, putLevel, returnValues) + case ArrayValues(array) => + blockStore.putArray(blockId, array, putLevel, returnValues) + case ByteBufferValues(bytes) => + bytes.rewind() + blockStore.putBytes(blockId, bytes, putLevel) + } + size = result.size + result.data match { + case Left(newIterator) if putLevel.useMemory => valuesAfterPut = newIterator + case Right(newBytes) => bytesAfterPut = newBytes + case _ => } - updatedBlocks += ((blockId, putBlockStatus)) - } - } finally { - // If we failed in putting the block to memory/disk, notify other possible readers - // that it has failed, and then remove it from the block info map. - if (!marked) { - // Note that the remove must happen before markFailure otherwise another thread - // could've inserted a new BlockInfo before we remove it. - blockInfo.remove(blockId) - putBlockInfo.markFailure() - logWarning(s"Putting block $blockId failed") - } - } - } - logDebug("Put block %s locally took %s".format(blockId, Utils.getUsedTimeMs(startTimeMs))) - // Either we're storing bytes and we asynchronously started replication, or we're storing - // values and need to serialize and replicate them now: - if (putLevel.replication > 1) { - data match { - case ByteBufferValues(bytes) => - if (replicationFuture != null) { - Await.ready(replicationFuture, Duration.Inf) + // Keep track of which blocks are dropped from memory + if (putLevel.useMemory) { + result.droppedBlocks.foreach { + updatedBlocks += _ + } } - case _ => - val remoteStartTime = System.currentTimeMillis - // Serialize the block if not already done - if (bytesAfterPut == null) { - if (valuesAfterPut == null) { - throw new SparkException( - "Underlying put returned neither an Iterator nor bytes! This shouldn't happen.") + + val putBlockStatus = getCurrentBlockStatus(blockId, putBlockInfo) + if (putBlockStatus.storageLevel != StorageLevel.NONE) { + // Now that the block is in either the memory, externalBlockStore, or disk store, + // let other threads read it, and tell the master about it. + marked = true + putBlockInfo.markReady(size) + if (tellMaster) { + reportBlockStatus(blockId, putBlockInfo, putBlockStatus) } - bytesAfterPut = dataSerialize(blockId, valuesAfterPut) + updatedBlocks += ((blockId, putBlockStatus)) } - try { - replicate(blockId, bytesAfterPut.asByteBuffer(), putLevel) - } catch { - case ex: BufferTooLargeException => - throw new ReplicationBlockSizeLimitException(ex) + } finally { + // If we failed in putting the block to memory/disk, notify other possible readers + // that it has failed, and then remove it from the block info map. + if (!marked) { + // Note that the remove must happen before markFailure otherwise another thread + // could've inserted a new BlockInfo before we remove it. + blockInfo.remove(blockId) + putBlockInfo.markFailure() + logWarning(s"Putting block $blockId failed") } - logDebug("Put block %s remotely took %s" - .format(blockId, Utils.getUsedTimeMs(remoteStartTime))) + } } - } + logDebug("Put block %s locally took %s".format(blockId, Utils.getUsedTimeMs(startTimeMs))) - if (bytesAfterPut != null) { - bytesAfterPut.dispose() + // Either we're storing bytes and we asynchronously started replication, or we're storing + // values and need to serialize and replicate them now: + if (putLevel.replication > 1) { + data match { + case ByteBufferValues(bytes) => + if (replicationFuture != null) { + Await.ready(replicationFuture, Duration.Inf) + } + case _ => + val remoteStartTime = System.currentTimeMillis + // Serialize the block if not already done + if (bytesAfterPut == null) { + if (valuesAfterPut == null) { + throw new SparkException( + "Underlying put returned neither an Iterator nor bytes! This shouldn't happen.") + } + bytesAfterPut = dataSerialize(blockId, valuesAfterPut) + } + try { + replicate(blockId, bytesAfterPut.asByteBuffer(), putLevel) + } catch { + case ex: BufferTooLargeException => + throw new ReplicationBlockSizeLimitException(ex) + } + logDebug("Put block %s remotely took %s" + .format(blockId, Utils.getUsedTimeMs(remoteStartTime))) + } + } + } finally { + if (bytesAfterPut != null) { + bytesAfterPut.dispose() + } } if (putLevel.replication > 1) { @@ -956,7 +960,7 @@ private[spark] class BlockManager( case Some(peer) => try { val onePeerStartTime = System.currentTimeMillis - data.position(0) + data.rewind() logTrace(s"Trying to replicate $blockId of ${data.limit()} bytes to $peer") blockTransferService.uploadBlockSync( peer.host, peer.port, peer.executorId, blockId, new NioManagedBuffer(data), tLevel) @@ -1339,7 +1343,7 @@ class TachyonBlockSizeLimitException(cause: BufferTooLargeException) class ShuffleBlockSizeLimitException(size: Long) extends SparkException("Spark cannot shuffle partitions that are greater than 2GB. " + "You tried to shuffle a block that was at least " + Utils.bytesToString(size) + ". " + - "You should try to increase the number of partitions of this shuffle, and / or increase the " + + "You should try to increase the number of partitions of this shuffle, and / or " + "figure out which stage created the partitions before the shuffle, and increase the number " + "of partitions for that stage. You may want to make both of these numbers easily " + "configurable parameters so you can continue to update as needed.") diff --git a/core/src/main/scala/org/apache/spark/storage/DiskBlockObjectWriter.scala b/core/src/main/scala/org/apache/spark/storage/DiskBlockObjectWriter.scala index ad758a44dc9b7..d571ce5e21e74 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskBlockObjectWriter.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockObjectWriter.scala @@ -110,7 +110,7 @@ private[spark] class DiskBlockObjectWriter( objOut.close() } - if (length > LargeByteBufferHelper.MAX_CHUNK) { + if (length > LargeByteBufferHelper.MAX_CHUNK_SIZE) { throw new ShuffleBlockSizeLimitException(length) } @@ -207,7 +207,7 @@ private[spark] class DiskBlockObjectWriter( if (numRecordsWritten % 32 == 0) { updateBytesWritten() - if (reportedPosition > LargeByteBufferHelper.MAX_CHUNK) { + if (reportedPosition > LargeByteBufferHelper.MAX_CHUNK_SIZE) { throw new ShuffleBlockSizeLimitException(reportedPosition) } } diff --git a/core/src/test/scala/org/apache/spark/rdd/LargePartitionCachingSuite.scala b/core/src/test/scala/org/apache/spark/rdd/LargePartitionCachingSuite.scala index baa28cceaab99..4859d0b4f47fc 100644 --- a/core/src/test/scala/org/apache/spark/rdd/LargePartitionCachingSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/LargePartitionCachingSuite.scala @@ -46,7 +46,7 @@ class LargePartitionCachingSuite extends FunSuite with SharedSparkContext with M .persist(StorageLevel.DISK_ONLY_2) myRDD.count() } - exc.getMessage() should include (classOf[ReplicationBlockSizeLimitException].getSimpleName) + exc.getCause() shouldBe a [ReplicationBlockSizeLimitException] } finally { clusterSc.stop() } diff --git a/network/common/src/main/java/org/apache/spark/network/buffer/FileSegmentManagedBuffer.java b/network/common/src/main/java/org/apache/spark/network/buffer/FileSegmentManagedBuffer.java index 5e318d8ca8b78..2d534f12abd62 100644 --- a/network/common/src/main/java/org/apache/spark/network/buffer/FileSegmentManagedBuffer.java +++ b/network/common/src/main/java/org/apache/spark/network/buffer/FileSegmentManagedBuffer.java @@ -73,8 +73,8 @@ public ByteBuffer nioByteBuffer() throws IOException { buf.flip(); return buf; } else { - if (length > LargeByteBufferHelper.MAX_CHUNK) { - throw new BufferTooLargeException(length); + if (length > LargeByteBufferHelper.MAX_CHUNK_SIZE) { + throw new BufferTooLargeException(length, LargeByteBufferHelper.MAX_CHUNK_SIZE); } return channel.map(FileChannel.MapMode.READ_ONLY, offset, length); } From fb733c5059982767437ee4b2dc2104ee1eb76867 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 19 Aug 2015 11:19:19 -0500 Subject: [PATCH 92/97] fix check for underlying ByteBuffer class --- .../buffer/LargeByteBufferTestHelper.scala | 30 +++++++++++++++++++ .../spark/storage/BlockManagerSuite.scala | 11 ++++--- 2 files changed, 35 insertions(+), 6 deletions(-) create mode 100644 core/src/test/scala/org/apache/spark/network/buffer/LargeByteBufferTestHelper.scala diff --git a/core/src/test/scala/org/apache/spark/network/buffer/LargeByteBufferTestHelper.scala b/core/src/test/scala/org/apache/spark/network/buffer/LargeByteBufferTestHelper.scala new file mode 100644 index 0000000000000..a04bb41fae366 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/network/buffer/LargeByteBufferTestHelper.scala @@ -0,0 +1,30 @@ +/* + * 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.network.buffer + +import java.nio.ByteBuffer +import java.util.{List => JList} + +/** + * cheat to access package-protected members in test + */ +object LargeByteBufferTestHelper { + def nioBuffers(wbb: WrappedLargeByteBuffer): JList[ByteBuffer] = { + wbb.nioBuffers() + } +} 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 5b38d28f0722c..d589234df7106 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.storage import java.nio.{ByteBuffer, MappedByteBuffer} import java.util.Arrays -import org.apache.spark.network.buffer.{WrappedLargeByteBuffer, LargeByteBufferHelper, LargeByteBuffer} +import org.apache.spark.network.buffer.{LargeByteBufferTestHelper, WrappedLargeByteBuffer, LargeByteBufferHelper, LargeByteBuffer} import scala.collection.mutable.ArrayBuffer import scala.concurrent.duration._ @@ -830,11 +830,10 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE val notMapped = diskStoreNotMapped.getBytes(blockId).get.asInstanceOf[WrappedLargeByteBuffer] // Not possible to do isInstanceOf due to visibility of HeapByteBuffer - // TODO fix me -// assert(notMapped.nioBuffers().get(0).getClass.getName.endsWith("HeapByteBuffer"), -// "Expected HeapByteBuffer for un-mapped read") -// assert(mapped.nioBuffers().get(0).isInstanceOf[MappedByteBuffer], -// "Expected MappedByteBuffer for mapped read") + assert(LargeByteBufferTestHelper.nioBuffers(mapped).get(0).getClass.getName + .endsWith("HeapByteBuffer"), "Expected HeapByteBuffer for un-mapped read") + assert(LargeByteBufferTestHelper.nioBuffers(notMapped).get(0).isInstanceOf[MappedByteBuffer], + "Expected MappedByteBuffer for mapped read") def arrayFromByteBuffer(in: LargeByteBuffer): Array[Byte] = { val array = new Array[Byte](in.remaining().toInt) From b921b0bd3e24f8a3e0ebc99d91b772299eed2851 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 19 Aug 2015 12:21:13 -0500 Subject: [PATCH 93/97] fix check, add test --- .../spark/storage/DiskBlockObjectWriter.scala | 3 ++- .../test/scala/org/apache/spark/ShuffleSuite.scala | 14 +++++++++++++- 2 files changed, 15 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/DiskBlockObjectWriter.scala b/core/src/main/scala/org/apache/spark/storage/DiskBlockObjectWriter.scala index d571ce5e21e74..61af603c351e1 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskBlockObjectWriter.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockObjectWriter.scala @@ -97,7 +97,6 @@ private[spark] class DiskBlockObjectWriter( override def close() { if (initialized) { - val length = channel.position() Utils.tryWithSafeFinally { if (syncWrites) { // Force outstanding writes to disk and track how long it takes @@ -110,6 +109,8 @@ private[spark] class DiskBlockObjectWriter( objOut.close() } + finalPosition = file.length() + val length = finalPosition - initialPosition if (length > LargeByteBufferHelper.MAX_CHUNK_SIZE) { throw new ShuffleBlockSizeLimitException(length) } diff --git a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala index 7e5640bcba0d5..8ba16e0639692 100644 --- a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala @@ -283,6 +283,18 @@ abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkC rdd.count() } + test("shuffle total > 2GB ok if each block is small") { + sc = new SparkContext("local", "test", conf) + val rdd = sc.parallelize(1 to 1e6.toInt, 1).map{ i => + val n = 3e3.toInt + val arr = new Array[Byte](n) + //need to make sure the array doesn't compress to something small + scala.util.Random.nextBytes(arr) + (i, arr) + } + rdd.partitionBy(new HashPartitioner(100)).count() + } + test("shuffle blocks > 2GB fail with sane exception") { // note that this *could* succeed in local mode, b/c local shuffles actually don't // have a limit at 2GB. BUT, we make them fail in any case, b/c its better to have @@ -301,7 +313,7 @@ abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkC rdd.partitionBy(new org.apache.spark.HashPartitioner(2)).count() } - exc.getMessage should include (classOf[ShuffleBlockSizeLimitException].getSimpleName) + exc.getCause shouldBe a[ShuffleBlockSizeLimitException] } test("metrics for shuffle without aggregation") { From e0f5130b90c6c6d5a2327f7362a46866a5f97e74 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 19 Aug 2015 14:22:32 -0500 Subject: [PATCH 94/97] fix check --- .../org/apache/spark/storage/DiskBlockObjectWriter.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/DiskBlockObjectWriter.scala b/core/src/main/scala/org/apache/spark/storage/DiskBlockObjectWriter.scala index 61af603c351e1..ee35281c93ee7 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskBlockObjectWriter.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockObjectWriter.scala @@ -208,8 +208,9 @@ private[spark] class DiskBlockObjectWriter( if (numRecordsWritten % 32 == 0) { updateBytesWritten() - if (reportedPosition > LargeByteBufferHelper.MAX_CHUNK_SIZE) { - throw new ShuffleBlockSizeLimitException(reportedPosition) + val length = reportedPosition - initialPosition + if (length > LargeByteBufferHelper.MAX_CHUNK_SIZE) { + throw new ShuffleBlockSizeLimitException(length) } } } From 331d517549fc9a3af804094b9e66a45452be35cd Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 19 Aug 2015 14:26:19 -0500 Subject: [PATCH 95/97] ignore the big tests for now --- core/src/test/scala/org/apache/spark/ShuffleSuite.scala | 4 ++-- .../org/apache/spark/rdd/LargePartitionCachingSuite.scala | 6 +++--- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala index 8ba16e0639692..92aeeb7de467d 100644 --- a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala @@ -283,7 +283,7 @@ abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkC rdd.count() } - test("shuffle total > 2GB ok if each block is small") { + ignore("shuffle total > 2GB ok if each block is small") { sc = new SparkContext("local", "test", conf) val rdd = sc.parallelize(1 to 1e6.toInt, 1).map{ i => val n = 3e3.toInt @@ -295,7 +295,7 @@ abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkC rdd.partitionBy(new HashPartitioner(100)).count() } - test("shuffle blocks > 2GB fail with sane exception") { + ignore("shuffle blocks > 2GB fail with sane exception") { // note that this *could* succeed in local mode, b/c local shuffles actually don't // have a limit at 2GB. BUT, we make them fail in any case, b/c its better to have // a consistent failure, and not have success depend on where tasks get scheduled diff --git a/core/src/test/scala/org/apache/spark/rdd/LargePartitionCachingSuite.scala b/core/src/test/scala/org/apache/spark/rdd/LargePartitionCachingSuite.scala index 4859d0b4f47fc..609336339deb8 100644 --- a/core/src/test/scala/org/apache/spark/rdd/LargePartitionCachingSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/LargePartitionCachingSuite.scala @@ -29,13 +29,13 @@ class LargePartitionCachingSuite extends FunSuite with SharedSparkContext with M largePartitionRdd.persist(StorageLevel.MEMORY_ONLY_SER).count() should be (1e6.toInt) } - test("disk cache large partitions") { + ignore("disk cache large partitions") { largePartitionRdd.persist(StorageLevel.DISK_ONLY).count() should be (1e6.toInt) } - test("disk cache large partitions with replications") { + ignore("disk cache large partitions with replications") { val conf = new SparkConf() - .setMaster("local-cluster[2, 1, 512]") + .setMaster("local-cluster[2, 1, 1024]") .setAppName("test-cluster") .set("spark.task.maxFailures", "1") .set("spark.akka.frameSize", "1") // set to 1MB to detect direct serialization of data From 06c8ffa6f576baad66b5897756eda990f114ade4 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 19 Aug 2015 15:54:10 -0500 Subject: [PATCH 96/97] style --- .../org/apache/spark/storage/ExternalBlockStore.scala | 5 ++++- .../org/apache/spark/storage/TachyonBlockManager.scala | 2 +- .../src/test/scala/org/apache/spark/ShuffleSuite.scala | 4 ++-- .../apache/spark/rdd/LargePartitionCachingSuite.scala | 10 ++++++---- 4 files changed, 13 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/ExternalBlockStore.scala b/core/src/main/scala/org/apache/spark/storage/ExternalBlockStore.scala index 6a54e51a0f169..c2271bccc8f4c 100644 --- a/core/src/main/scala/org/apache/spark/storage/ExternalBlockStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/ExternalBlockStore.scala @@ -48,7 +48,10 @@ private[spark] class ExternalBlockStore(blockManager: BlockManager, executorId: } } - override def putBytes(blockId: BlockId, bytes: LargeByteBuffer, level: StorageLevel): PutResult = { + override def putBytes( + blockId: BlockId, + bytes: LargeByteBuffer, + level: StorageLevel): PutResult = { putIntoExternalBlockStore(blockId, bytes, returnValues = true) } diff --git a/core/src/main/scala/org/apache/spark/storage/TachyonBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/TachyonBlockManager.scala index 7593dd41bf6b9..f806407c73ead 100644 --- a/core/src/main/scala/org/apache/spark/storage/TachyonBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/TachyonBlockManager.scala @@ -137,7 +137,7 @@ private[spark] class TachyonBlockManager() extends ExternalBlockManager with Log val is = file.getInStream(ReadType.CACHE) try { val size = file.length - //TODO get tachyon to support large blocks + // TODO get tachyon to support large blocks val bs = new Array[Byte](size.asInstanceOf[Int]) ByteStreams.readFully(is, bs) Some(LargeByteBufferHelper.asLargeByteBuffer(bs)) diff --git a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala index 92aeeb7de467d..e28818098b918 100644 --- a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala @@ -288,7 +288,7 @@ abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkC val rdd = sc.parallelize(1 to 1e6.toInt, 1).map{ i => val n = 3e3.toInt val arr = new Array[Byte](n) - //need to make sure the array doesn't compress to something small + // need to make sure the array doesn't compress to something small scala.util.Random.nextBytes(arr) (i, arr) } @@ -304,7 +304,7 @@ abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkC val rdd = sc.parallelize(1 to 1e6.toInt, 1).map{ i => val n = 3e3.toInt val arr = new Array[Byte](n) - //need to make sure the array doesn't compress to something small + // need to make sure the array doesn't compress to something small scala.util.Random.nextBytes(arr) (2 * i, arr) } diff --git a/core/src/test/scala/org/apache/spark/rdd/LargePartitionCachingSuite.scala b/core/src/test/scala/org/apache/spark/rdd/LargePartitionCachingSuite.scala index 609336339deb8..dfcc90df32a18 100644 --- a/core/src/test/scala/org/apache/spark/rdd/LargePartitionCachingSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/LargePartitionCachingSuite.scala @@ -18,13 +18,15 @@ package org.apache.spark.rdd import org.apache.spark._ import org.apache.spark.storage.{ReplicationBlockSizeLimitException, StorageLevel} -import org.scalatest.{Matchers, FunSuite} +import org.scalatest.Matchers -class LargePartitionCachingSuite extends FunSuite with SharedSparkContext with Matchers { +class LargePartitionCachingSuite extends SparkFunSuite with SharedSparkContext with Matchers { - def largePartitionRdd = sc.parallelize(1 to 1e6.toInt, 1).map{i => new Array[Byte](2.2e3.toInt)} + def largePartitionRdd: RDD[Array[Byte]] = { + sc.parallelize(1 to 1e6.toInt, 1).map{i => new Array[Byte](2.2e3.toInt)} + } - //just don't want to kill the test server + // just don't want to kill the test server ignore("memory serialized cache large partitions") { largePartitionRdd.persist(StorageLevel.MEMORY_ONLY_SER).count() should be (1e6.toInt) } From 200afdc33f59508a04172cfefcc4a40a3bdf6169 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 19 Aug 2015 19:38:47 -0500 Subject: [PATCH 97/97] whoops, swap cases --- .../scala/org/apache/spark/storage/BlockManagerSuite.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) 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 d589234df7106..d1d6778180a4c 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -830,9 +830,9 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE val notMapped = diskStoreNotMapped.getBytes(blockId).get.asInstanceOf[WrappedLargeByteBuffer] // Not possible to do isInstanceOf due to visibility of HeapByteBuffer - assert(LargeByteBufferTestHelper.nioBuffers(mapped).get(0).getClass.getName + assert(LargeByteBufferTestHelper.nioBuffers(notMapped).get(0).getClass.getName .endsWith("HeapByteBuffer"), "Expected HeapByteBuffer for un-mapped read") - assert(LargeByteBufferTestHelper.nioBuffers(notMapped).get(0).isInstanceOf[MappedByteBuffer], + assert(LargeByteBufferTestHelper.nioBuffers(mapped).get(0).isInstanceOf[MappedByteBuffer], "Expected MappedByteBuffer for mapped read") def arrayFromByteBuffer(in: LargeByteBuffer): Array[Byte] = {