From 9834942757b31d3392cd1e22e0251ea3da6de7fa Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Tue, 7 Apr 2015 13:25:06 -0500 Subject: [PATCH 01/29] ByteArrayChunkOutputStream.slice, for grabbing aribitrary portion of output --- .../util/io/ByteArrayChunkOutputStream.scala | 41 +++++++++++++++++++ .../io/ByteArrayChunkOutputStreamSuite.scala | 20 +++++++++ 2 files changed, 61 insertions(+) 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 daac6f971eb20..64f1ff7153fe4 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 @@ -43,10 +43,13 @@ class ByteArrayChunkOutputStream(chunkSize: Int) extends OutputStream { */ private var position = chunkSize + var size: Long = 0L + override def write(b: Int): Unit = { allocateNewChunkIfNeeded() chunks(lastChunkIndex)(position) = b.toByte position += 1 + size += 1 } override def write(bytes: Array[Byte], off: Int, len: Int): Unit = { @@ -58,6 +61,7 @@ class ByteArrayChunkOutputStream(chunkSize: Int) extends OutputStream { written += thisBatch position += thisBatch } + size += len } @inline @@ -91,4 +95,41 @@ class ByteArrayChunkOutputStream(chunkSize: Int) extends OutputStream { ret } } + + /** + * get a copy of the data between the two endpoints + */ + def slice(start: Long, until: Long): Array[Byte] = { + require((until - start) < Integer.MAX_VALUE, "max slice length = Integer.MAX_VALUE") + var chunkStart = 0L + var chunkIdx = 0 + var foundStart = false + val length = (until - start).toInt + val result = new Array[Byte](length) + while (!foundStart) { + val nextSize = chunkStart + chunks(chunkIdx).size + if (nextSize > start) { + foundStart = true + } + else { + chunkStart = nextSize + chunkIdx += 1 + } + } + + var remaining = length + var pos = 0 + var offsetInChunk = (start - chunkStart).toInt + while (remaining > 0) { + val lenToCopy = math.min(remaining, chunks(chunkIdx).size - offsetInChunk) + System.arraycopy(chunks(chunkIdx), offsetInChunk, result, pos, lenToCopy) + chunkIdx += 1 + offsetInChunk = 0 + pos += lenToCopy + remaining -= lenToCopy + } + result + } + + } 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 361ec95654f47..cacb3fbb0762b 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 @@ -106,4 +106,24 @@ class ByteArrayChunkOutputStreamSuite extends SparkFunSuite { assert(arrays(1).toSeq === ref.slice(10, 20)) assert(arrays(2).toSeq === ref.slice(20, 30)) } + + test("slice") { + val ref = new Array[Byte](30) + Random.nextBytes(ref) + val o = new ByteArrayChunkOutputStream(10) + o.write(ref) + + for { + start <- (0 until 30) + end <- (start to 30) + } { + withClue(s"start = $start; end = $end") { + try { + assert(o.slice(start,end).toSeq === ref.slice(start,end)) + } catch { + case ex => fail(ex) + } + } + } + } } From 4e1a842e99a612629dc0a422647da65484138a79 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Tue, 7 Apr 2015 13:19:08 -0500 Subject: [PATCH 02/29] LargeByteBuffer --- .../util/LargeByteBufferInputStream.scala | 93 +++++++ .../util/LargeByteBufferOutputStream.scala | 70 +++++ .../LargeByteBufferInputStreamSuite.scala | 85 ++++++ .../LargeByteBufferOutputStreamSuite.scala | 67 +++++ .../buffer/BufferTooLargeException.java | 32 +++ .../spark/network/buffer/LargeByteBuffer.java | 112 ++++++++ .../network/buffer/LargeByteBufferHelper.java | 73 +++++ .../buffer/WrappedLargeByteBuffer.java | 252 ++++++++++++++++++ .../buffer/LargeByteBufferHelperSuite.java | 71 +++++ .../buffer/WrappedLargeByteBufferSuite.java | 233 ++++++++++++++++ 10 files changed, 1088 insertions(+) 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/test/scala/org/apache/spark/util/LargeByteBufferInputStreamSuite.scala create mode 100644 core/src/test/scala/org/apache/spark/util/LargeByteBufferOutputStreamSuite.scala create mode 100644 network/common/src/main/java/org/apache/spark/network/buffer/BufferTooLargeException.java 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 create mode 100644 network/common/src/test/java/org/apache/spark/network/buffer/LargeByteBufferHelperSuite.java create mode 100644 network/common/src/test/java/org/apache/spark/network/buffer/WrappedLargeByteBufferSuite.java 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..69bc4902d0aac --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/LargeByteBufferInputStream.scala @@ -0,0 +1,93 @@ +/* + * 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 new file mode 100644 index 0000000000000..04e685262f38c --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/LargeByteBufferOutputStream.scala @@ -0,0 +1,70 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.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/util/LargeByteBufferInputStreamSuite.scala b/core/src/test/scala/org/apache/spark/util/LargeByteBufferInputStreamSuite.scala new file mode 100644 index 0000000000000..209fb0d343fd0 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/util/LargeByteBufferInputStreamSuite.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.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 new file mode 100644 index 0000000000000..b25dfc5bbaaac --- /dev/null +++ b/core/src/test/scala/org/apache/spark/util/LargeByteBufferOutputStreamSuite.scala @@ -0,0 +1,67 @@ +/* + * 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) + } + + } + +} diff --git a/network/common/src/main/java/org/apache/spark/network/buffer/BufferTooLargeException.java b/network/common/src/main/java/org/apache/spark/network/buffer/BufferTooLargeException.java new file mode 100644 index 0000000000000..158d29ad152ce --- /dev/null +++ b/network/common/src/main/java/org/apache/spark/network/buffer/BufferTooLargeException.java @@ -0,0 +1,32 @@ +/* + * 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; + +public class BufferTooLargeException extends IOException { + public final long actualSize; + public final long extra; + + public BufferTooLargeException(long actualSize) { + super("LargeByteBuffer is too large to convert. Size: " + actualSize + "; Size Limit: " + + LargeByteBufferHelper.MAX_CHUNK + " (" + + (actualSize - LargeByteBufferHelper.MAX_CHUNK) + " too big)"); + this.extra = actualSize - LargeByteBufferHelper.MAX_CHUNK; + this.actualSize = actualSize; + } +} 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..12d12647f3bb7 --- /dev/null +++ b/network/common/src/main/java/org/apache/spark/network/buffer/LargeByteBuffer.java @@ -0,0 +1,112 @@ +/* +* 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; +import java.util.List; + +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}. + * @param dst + * @param offset + * @param length + */ + public void get(byte[] dst,int offset, int length); + + public LargeByteBuffer rewind(); + + /** + * return a deep copy of this buffer. + * The returned buffer will have position == 0. The position + * of this buffer will not change as a result of copying. + * + * @return a new buffer with a full copy of this buffer's data + */ + public LargeByteBuffer deepCopy(); + + /** + * Advance the position in this buffer by up to n bytes. n may be + * positive or negative. It will move the full n unless that moves + * it past the end (or beginning) of the buffer, in which case it will move to the end + * (or beginning). + * + * @return the number of bytes moved forward (can be negative if n is negative) + */ + public long skip(long n); + + public long position(); + + /** + * Creates a new byte buffer that shares this buffer's content. + * + *

The content of the new buffer will be that of this buffer. Changes + * to this buffer's content will be visible in the new buffer, and vice + * versa; the two buffers' positions will be independent. + * + *

The new buffer's position will be identical to those of this buffer + * */ + public LargeByteBuffer duplicate(); + + + public long remaining(); + + /** + * the total number of bytes in this buffer + * @return + */ + public long size(); + + /** + * writes the data from the current position() to the end of this buffer + * to the given channel. The position() will be moved to the end of + * the buffer after this. + * + * Note that this method will continually attempt to push data to the given channel. If the + * channel cannot accept more data, this will continuously retry until the channel accepts + * the data. + * + * @param channel + * @return the number of bytes written to the channel + * @throws IOException + */ + public long writeTo(WritableByteChannel channel) throws IOException; + + /** + * get the entire contents of this as one ByteBuffer, if possible. The returned ByteBuffer + * will always have the position set 0, and the limit set to the end of the data. Each + * call will return a new ByteBuffer, but will not require copying the data (eg., it will + * use ByteBuffer#duplicate()). The returned byte buffer and this may or may not share data. + * + * @return + * @throws BufferTooLargeException if this buffer is too large to fit in one {@link ByteBuffer} + */ + public ByteBuffer asByteBuffer() throws BufferTooLargeException; + + /** + * Attempt to clean up if it is memory-mapped. This uses an *unsafe* Sun API that + * might cause errors if one attempts to read from the unmapped buffer, but it's better than + * waiting for the GC to find it because that could lead to huge numbers of open files. There's + * unfortunately no standard API to do this. + */ + public void dispose(); +} 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..9d40fc4f5c07d --- /dev/null +++ b/network/common/src/main/java/org/apache/spark/network/buffer/LargeByteBufferHelper.java @@ -0,0 +1,73 @@ +/* + * 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 { + + public static final int MAX_CHUNK = Integer.MAX_VALUE - 1000000; + + 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, 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, + long offset, + long length + ) throws IOException { + int maxChunk = MAX_CHUNK; + ArrayList offsets = new ArrayList(); + long curOffset = offset; + long end = offset + length; + while (curOffset < end) { + offsets.add(curOffset); + int chunkLength = (int) Math.min((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/WrappedLargeByteBuffer.java b/network/common/src/main/java/org/apache/spark/network/buffer/WrappedLargeByteBuffer.java new file mode 100644 index 0000000000000..8ead571dd102f --- /dev/null +++ b/network/common/src/main/java/org/apache/spark/network/buffer/WrappedLargeByteBuffer.java @@ -0,0 +1,252 @@ +/* +* 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 sun.nio.ch.DirectBuffer; + +import java.io.IOException; +import java.nio.BufferUnderflowException; +import java.nio.ByteBuffer; +import java.nio.MappedByteBuffer; +import java.nio.channels.WritableByteChannel; +import java.util.Arrays; +import java.util.List; + +public class WrappedLargeByteBuffer implements LargeByteBuffer { + + //only public for tests + public final ByteBuffer[] underlying; + + private final long size; + private long _pos; + private int currentBufferIdx; + private ByteBuffer currentBuffer; + + + public WrappedLargeByteBuffer(ByteBuffer[] underlying) { + this(underlying, findExpectedInitialPosition(underlying)); + } + + private static long findExpectedInitialPosition(ByteBuffer[] bufs) { + long sum = 0L; + for (ByteBuffer b: bufs) { + if (b.position() > 0) { + // this could still lead to a mix of positions half-way through buffers that + // would be inconsistent -- but we'll discover that in the constructor checks + sum += b.position(); + } else { + break; + } + } + return sum; + } + + private WrappedLargeByteBuffer(ByteBuffer[] underlying, long initialPosition) { + this.underlying = underlying; + long sum = 0L; + for (int i = 0; i < underlying.length; i++) { + ByteBuffer b = underlying[i]; + long nextSum = sum + b.capacity(); + int expectedPosition; + if (nextSum < initialPosition) { + expectedPosition = b.capacity(); + } else if (sum > initialPosition) { + expectedPosition = 0; + } else { + expectedPosition = (int) (initialPosition - sum); + } + if (b.position() != expectedPosition) { + throw new IllegalArgumentException("ByteBuffer[" + i + "]:" + b + " was expected to have" + + " position = " + expectedPosition + " to be consistent with the overall " + + "initialPosition = " + initialPosition); + } + sum = nextSum; + } + _pos = initialPosition; + currentBufferIdx = 0; + currentBuffer = underlying[0]; + size = sum; + } + + @Override + public void get(byte[] dest, int offset, int length) { + if (length > remaining()) + throw new BufferUnderflowException(); + int moved = 0; + while (moved < length) { + int toRead = Math.min(length - moved, currentBuffer.remaining()); + currentBuffer.get(dest, offset + moved, toRead); + moved += toRead; + updateCurrentBuffer(); + } + _pos += moved; + } + + @Override + public LargeByteBuffer rewind() { + while (currentBufferIdx > 0) { + if (currentBuffer != null) { + currentBuffer.rewind(); + } + currentBufferIdx -= 1; + currentBuffer = underlying[currentBufferIdx]; + } + currentBuffer.rewind(); + _pos = 0; + return this; + } + + @Override + public WrappedLargeByteBuffer deepCopy() { + ByteBuffer[] dataCopy = new ByteBuffer[underlying.length]; + for (int i = 0; i < underlying.length; i++) { + ByteBuffer b = underlying[i]; + dataCopy[i] = ByteBuffer.allocate(b.capacity()); + int originalPosition = b.position(); + b.position(0); + dataCopy[i].put(b); + dataCopy[i].position(0); + b.position(originalPosition); + } + return new WrappedLargeByteBuffer(dataCopy); + } + + @Override + public byte get() { + byte r = currentBuffer.get(); + _pos += 1; + updateCurrentBuffer(); + return r; + } + + private void updateCurrentBuffer() { + while (currentBuffer != null && !currentBuffer.hasRemaining()) { + currentBufferIdx += 1; + currentBuffer = currentBufferIdx < underlying.length ? underlying[currentBufferIdx] : null; + } + } + + @Override + public long position() { + return _pos; + } + + @Override + public long skip(long n) { + if (n < 0) { + final long moveTotal = Math.min(-n, _pos); + long toMove = moveTotal; + // move backwards -- set the position to 0 of every buffer's we go back + if (currentBuffer != null) { + currentBufferIdx += 1; + } + while (toMove > 0) { + currentBufferIdx -= 1; + currentBuffer = underlying[currentBufferIdx]; + int thisMove = (int) Math.min(toMove, currentBuffer.position()); + currentBuffer.position(currentBuffer.position() - thisMove); + toMove -= thisMove; + } + _pos -= moveTotal; + return -moveTotal; + } else if (n > 0) { + final long moveTotal = Math.min(n, remaining()); + long toMove = moveTotal; + // move forwards-- set the position to the end of every buffer as we go forwards + currentBufferIdx -= 1; + while (toMove > 0) { + currentBufferIdx += 1; + currentBuffer = underlying[currentBufferIdx]; + int thisMove = (int) Math.min(toMove, currentBuffer.remaining()); + currentBuffer.position(currentBuffer.position() + thisMove); + toMove -= thisMove; + } + _pos += moveTotal; + return moveTotal; + } else { + return 0; + } + } + + @Override + public long remaining() { + return size - _pos; + } + + @Override + public WrappedLargeByteBuffer duplicate() { + ByteBuffer[] duplicates = new ByteBuffer[underlying.length]; + for (int i = 0; i < underlying.length; i++) { + duplicates[i] = underlying[i].duplicate(); + } + return new WrappedLargeByteBuffer(duplicates, _pos); + } + + @Override + public long size() { + return size; + } + + @Override + public long writeTo(WritableByteChannel channel) throws IOException { + long written = 0l; + for (; currentBufferIdx < underlying.length; currentBufferIdx++) { + currentBuffer = underlying[currentBufferIdx]; + written += currentBuffer.remaining(); + while (currentBuffer.hasRemaining()) + channel.write(currentBuffer); + } + _pos = size(); + return written; + } + + @Override + public ByteBuffer asByteBuffer() throws BufferTooLargeException { + if (underlying.length > 1) { + throw new BufferTooLargeException(size()); + } + return underlying[0]; + } + + // only needed for tests + public List nioBuffers() { + return Arrays.asList(underlying); + } + + /** + * Attempt to clean up a ByteBuffer if it is memory-mapped. This uses an *unsafe* Sun API that + * might cause errors if one attempts to read from the unmapped buffer, but it's better than + * waiting for the GC to find it because that could lead to huge numbers of open files. There's + * unfortunately no standard API to do this. + */ + private static void dispose(ByteBuffer buffer) { + if (buffer != null && buffer instanceof MappedByteBuffer) { + DirectBuffer db = (DirectBuffer) buffer; + if (db.cleaner() != null) { + db.cleaner().clean(); + } + } + } + + @Override + public void dispose() { + for (ByteBuffer bb : underlying) { + dispose(bb); + } + } + +} 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 new file mode 100644 index 0000000000000..d56f216f11dc0 --- /dev/null +++ b/network/common/src/test/java/org/apache/spark/network/buffer/LargeByteBufferHelperSuite.java @@ -0,0 +1,71 @@ +/* + * 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 org.junit.Test; + +import java.io.*; +import java.nio.channels.FileChannel; + +import static org.junit.Assert.*; + +public class LargeByteBufferHelperSuite { + + @Test + public void testMapFile() throws IOException { + File testFile = File.createTempFile("large-byte-buffer-test", ".bin"); + testFile.deleteOnExit(); + OutputStream out = new FileOutputStream(testFile); + byte[] buffer = new byte[1 << 16]; + long len = 3L << 30; + assertTrue(len > Integer.MAX_VALUE); // its 1.5x Integer.MAX_VALUE, just a sanity check + for (int i = 0; i < buffer.length; i++) { + buffer[i] = (byte) i; + } + for (int i = 0; i < len / buffer.length; i++) { + out.write(buffer); + } + out.close(); + + FileChannel in = new FileInputStream(testFile).getChannel(); + + //fail quickly on bad bounds + try { + LargeByteBufferHelper.mapFile(in, FileChannel.MapMode.READ_ONLY, 0, len + 1); + fail("expected exception"); + } catch (IOException ioe) { + } + try { + LargeByteBufferHelper.mapFile(in, FileChannel.MapMode.READ_ONLY, -1, 10); + fail("expected exception"); + } catch (IllegalArgumentException iae) { + } + + //now try to read from the buffer + LargeByteBuffer buf = LargeByteBufferHelper.mapFile(in, FileChannel.MapMode.READ_ONLY, 0, len); + assertEquals(len, buf.size()); + byte[] read = new byte[buffer.length]; + for (int i = 0; i < len / buffer.length; i++) { + buf.get(read, 0, buffer.length); + // assertArrayEquals() is really slow + for (int j = 0; j < buffer.length; j++) { + if (read[j] != (byte)(j)) + fail("bad byte at (i,j) = (" + i + "," + j + ")"); + } + } + } +} 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 new file mode 100644 index 0000000000000..79398c6ae7ab1 --- /dev/null +++ b/network/common/src/test/java/org/apache/spark/network/buffer/WrappedLargeByteBufferSuite.java @@ -0,0 +1,233 @@ +/* + * 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.*; +import java.nio.BufferUnderflowException; +import java.nio.ByteBuffer; +import java.nio.channels.FileChannel; +import java.util.Arrays; +import java.util.Random; + +import org.junit.Test; + +import static org.junit.Assert.*; + +public class WrappedLargeByteBufferSuite { + + byte[] data = new byte[500]; + { + new Random(1234).nextBytes(data); + } + + private WrappedLargeByteBuffer testDataBuf() { + ByteBuffer[] bufs = new ByteBuffer[10]; + for (int i = 0; i < 10; i++) { + byte[] b = new byte[50]; + System.arraycopy(data, i * 50, b, 0, 50); + bufs[i] = ByteBuffer.wrap(b); + } + return new WrappedLargeByteBuffer(bufs); + } + + @Test + public void asByteBuffer() throws BufferTooLargeException { + //test that it works when buffer is small, and the right error when buffer is big + LargeByteBuffer buf = LargeByteBufferHelper.asLargeByteBuffer(new byte[100]); + ByteBuffer nioBuf = buf.asByteBuffer(); + assertEquals(100, nioBuf.remaining()); + + ByteBuffer[] bufs = new ByteBuffer[2]; + for (int i = 0; i < 2; i++) { + bufs[i] = ByteBuffer.allocate(10); + } + try { + new WrappedLargeByteBuffer(bufs).asByteBuffer(); + fail("expected an exception"); + } catch (BufferTooLargeException btl) { + } + } + + @Test + public void deepCopy() { + WrappedLargeByteBuffer b = testDataBuf(); + //intentionally move around sporadically + for (int initialPosition: new int[]{10,475, 0, 19, 58, 499, 498, 32, 234, 378}) { + b.rewind(); + b.skip(initialPosition); + WrappedLargeByteBuffer copy = b.deepCopy(); + assertEquals(0, copy.position()); + assertConsistent(copy); + assertConsistent(b); + assertEquals(b.size(), copy.size()); + assertEquals(initialPosition, b.position()); + byte[] copyData = new byte[500]; + copy.get(copyData, 0, 500); + assertArrayEquals(data, copyData); + } + } + + @Test + public void skipAndGet() { + WrappedLargeByteBuffer b = testDataBuf(); + int position = 0; + for (int move: new int[]{20, 50, 100, 0, -80, 0, 200, -175, 500, 0, -1000, 0}) { + long moved = b.skip(move); + assertConsistent(b); + long expMoved = move > 0 ? Math.min(move, 500 - position) : Math.max(move, -position); + position += moved; + assertEquals(expMoved, moved); + assertEquals(position, b.position()); + byte[] copyData = new byte[500 - position]; + b.get(copyData, 0, 500 - position); + assertConsistent(b); + byte[] dataSubset = new byte[500 - position]; + System.arraycopy(data, position, dataSubset, 0, 500 - position); + assertArrayEquals(dataSubset, copyData); + b.rewind(); + assertConsistent(b); + b.skip(position); + assertConsistent(b); + } + } + + private void assertConsistent(WrappedLargeByteBuffer buffer) { + long pos = buffer.position(); + long bufferStartPos = 0; + for (ByteBuffer p: buffer.nioBuffers()) { + if (pos < bufferStartPos) { + assertEquals(0, p.position()); + } else if (pos < bufferStartPos + p.capacity()) { + assertEquals(pos - bufferStartPos, p.position()); + } else { + assertEquals(p.capacity(), p.position()); + } + bufferStartPos += p.capacity(); + } + } + + @Test + public void get() { + WrappedLargeByteBuffer b = testDataBuf(); + byte[] into = new byte[500]; + for (int[] offsetAndLength: new int[][]{{0, 200}, {10,10}, {300, 20}, {30, 100}}) { + b.rewind(); + b.get(into, offsetAndLength[0], offsetAndLength[1]); + assertConsistent(b); + assertSubArrayEquals(data, 0, into, offsetAndLength[0], offsetAndLength[1]); + } + + try { + b.rewind(); + b.skip(400); + b.get(into, 0, 500); + fail("expected exception"); + } catch (BufferUnderflowException bue) { + } + } + + @Test + public void writeTo() throws IOException { + for (int initialPosition: new int[]{0,20, 400}) { + File testFile = File.createTempFile("WrappedLargeByteBuffer-writeTo-" + initialPosition,".bin"); + testFile.deleteOnExit(); + FileChannel channel = new FileOutputStream(testFile).getChannel(); + WrappedLargeByteBuffer buf = testDataBuf(); + buf.skip(initialPosition); + assertEquals(initialPosition, buf.position()); + int expN = 500 - initialPosition; + long bytesWritten = buf.writeTo(channel); + assertEquals(expN, bytesWritten); + channel.close(); + + byte[] fileBytes = new byte[expN]; + FileInputStream in = new FileInputStream(testFile); + int n = 0; + while (n < expN) { + n += in.read(fileBytes, n, expN - n); + } + assertEquals(-1, in.read()); + byte[] dataSlice = Arrays.copyOfRange(data, initialPosition, 500); + assertArrayEquals(dataSlice, fileBytes); + assertEquals(0, buf.remaining()); + assertEquals(500, buf.position()); + } + } + + @Test + public void duplicate() { + for (int initialPosition: new int[]{0,20, 400}) { + WrappedLargeByteBuffer buf = testDataBuf(); + buf.skip(initialPosition); + + WrappedLargeByteBuffer dup = buf.duplicate(); + assertEquals(initialPosition, buf.position()); + assertEquals(initialPosition, dup.position()); + assertEquals(500, buf.size()); + assertEquals(500, dup.size()); + assertEquals(500 - initialPosition, buf.remaining()); + assertEquals(500 - initialPosition, dup.remaining()); + assertConsistent(buf); + assertConsistent(dup); + } + } + + @Test + public void constructWithBuffersWithNonZeroPosition() { + ByteBuffer[] bufs = testDataBuf().underlying; + + bufs[0].position(50); + bufs[1].position(5); + + WrappedLargeByteBuffer b1 = new WrappedLargeByteBuffer(bufs); + assertEquals(55, b1.position()); + + + bufs[1].position(50); + bufs[2].position(50); + bufs[3].position(35); + WrappedLargeByteBuffer b2 = new WrappedLargeByteBuffer(bufs); + assertEquals(185, b2.position()); + + + bufs[5].position(16); + try { + new WrappedLargeByteBuffer(bufs); + fail("expected exception"); + } catch (IllegalArgumentException ex) { + } + + bufs[5].position(0); + bufs[0].position(49); + try { + new WrappedLargeByteBuffer(bufs); + fail("expected exception"); + } catch (IllegalArgumentException ex) { + } + + } + + 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); + System.arraycopy(act, actOffset, actCopy, 0, length); + assertArrayEquals(expCopy, actCopy); + } + + +} From 8bd56068d151908ff0c81f77d6f606a29edcf449 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Thu, 9 Apr 2015 09:26:14 -0500 Subject: [PATCH 03/29] update from PR feedback --- .../util/LargeByteBufferOutputStream.scala | 9 +- .../util/io/ByteArrayChunkOutputStream.scala | 3 +- .../LargeByteBufferOutputStreamSuite.scala | 2 +- .../buffer/BufferTooLargeException.java | 12 +- .../spark/network/buffer/LargeByteBuffer.java | 153 +++++++++--------- .../network/buffer/LargeByteBufferHelper.java | 85 +++++----- .../buffer/WrappedLargeByteBuffer.java | 85 +++++----- .../buffer/LargeByteBufferHelperSuite.java | 10 ++ .../buffer/WrappedLargeByteBufferSuite.java | 84 +++++++--- 9 files changed, 254 insertions(+), 189 deletions(-) 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 04e685262f38c..176825b7ec241 100644 --- a/core/src/main/scala/org/apache/spark/util/LargeByteBufferOutputStream.scala +++ b/core/src/main/scala/org/apache/spark/util/LargeByteBufferOutputStream.scala @@ -29,15 +29,12 @@ class LargeByteBufferOutputStream(chunkSize: Int = 65536) 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 = { @@ -51,17 +48,17 @@ class LargeByteBufferOutputStream(chunkSize: Int = 65536) // 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 chunksNeeded = ((totalSize + maxChunk - 1) / maxChunk).toInt val chunks = new Array[Array[Byte]](chunksNeeded) var remaining = totalSize var pos = 0 - (0 until chunksNeeded).foreach{idx => + (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}) + new WrappedLargeByteBuffer(chunks.map(ByteBuffer.wrap)) } override def close(): Unit = { 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 64f1ff7153fe4..34bb6ee6db906 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 @@ -110,8 +110,7 @@ class ByteArrayChunkOutputStream(chunkSize: Int) extends OutputStream { val nextSize = chunkStart + chunks(chunkIdx).size if (nextSize > start) { foundStart = true - } - else { + } else { chunkStart = nextSize chunkIdx += 1 } diff --git a/core/src/test/scala/org/apache/spark/util/LargeByteBufferOutputStreamSuite.scala b/core/src/test/scala/org/apache/spark/util/LargeByteBufferOutputStreamSuite.scala index b25dfc5bbaaac..4b28d383df908 100644 --- a/core/src/test/scala/org/apache/spark/util/LargeByteBufferOutputStreamSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/LargeByteBufferOutputStreamSuite.scala @@ -53,7 +53,7 @@ class LargeByteBufferOutputStreamSuite extends FunSuite with Matchers { Random.nextBytes(bytes) out.write(bytes) - (10 to 100 by 10).foreach{chunkSize => + (10 to 100 by 10).foreach { chunkSize => val buffer = out.largeBuffer(chunkSize).asInstanceOf[WrappedLargeByteBuffer] buffer.position() should be (0) buffer.size() should be (100) diff --git a/network/common/src/main/java/org/apache/spark/network/buffer/BufferTooLargeException.java b/network/common/src/main/java/org/apache/spark/network/buffer/BufferTooLargeException.java index 158d29ad152ce..4e1a85ba1f126 100644 --- a/network/common/src/main/java/org/apache/spark/network/buffer/BufferTooLargeException.java +++ b/network/common/src/main/java/org/apache/spark/network/buffer/BufferTooLargeException.java @@ -21,12 +21,14 @@ public class BufferTooLargeException extends IOException { public final long actualSize; public final long extra; + public final long maxSize; - public BufferTooLargeException(long actualSize) { - super("LargeByteBuffer is too large to convert. Size: " + actualSize + "; Size Limit: " - + LargeByteBufferHelper.MAX_CHUNK + " (" + - (actualSize - LargeByteBufferHelper.MAX_CHUNK) + " too big)"); - this.extra = actualSize - LargeByteBufferHelper.MAX_CHUNK; + public BufferTooLargeException(long actualSize, long maxSize) { + super(String.format("LargeByteBuffer is too large to convert. Size: %d; Size Limit: %d (%d " + + "too big)", actualSize, maxSize, + actualSize - maxSize)); + this.extra = actualSize - maxSize; this.actualSize = actualSize; + this.maxSize = maxSize; } } 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 12d12647f3bb7..bc48088cc8610 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,94 +19,95 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.nio.channels.WritableByteChannel; -import java.util.List; public interface LargeByteBuffer { - public byte get(); + 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}. - * @param dst - * @param offset - * @param length - */ - public void get(byte[] dst,int offset, int length); + /** + * 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 + */ + public void get(byte[] dst, int offset, int length); - public LargeByteBuffer rewind(); + public LargeByteBuffer rewind(); - /** - * return a deep copy of this buffer. - * The returned buffer will have position == 0. The position - * of this buffer will not change as a result of copying. - * - * @return a new buffer with a full copy of this buffer's data - */ - public LargeByteBuffer deepCopy(); + /** + * Return a deep copy of this buffer. + * The returned buffer will have position == 0. The position + * of this buffer will not change as a result of copying. + * + * @return a new buffer with a full copy of this buffer's data + */ + public LargeByteBuffer deepCopy(); - /** - * Advance the position in this buffer by up to n bytes. n may be - * positive or negative. It will move the full n unless that moves - * it past the end (or beginning) of the buffer, in which case it will move to the end - * (or beginning). - * - * @return the number of bytes moved forward (can be negative if n is negative) - */ - public long skip(long n); + /** + * Advance the position in this buffer by up to n bytes. n may be + * positive or negative. It will move the full n unless that moves + * it past the end (or beginning) of the buffer, in which case it will move to the end + * (or beginning). + * + * @return the number of bytes moved forward (can be negative if n is negative) + */ + public long skip(long n); - public long position(); + public long position(); - /** - * Creates a new byte buffer that shares this buffer's content. - * - *

The content of the new buffer will be that of this buffer. Changes - * to this buffer's content will be visible in the new buffer, and vice - * versa; the two buffers' positions will be independent. - * - *

The new buffer's position will be identical to those of this buffer - * */ - public LargeByteBuffer duplicate(); + /** + * Creates a new byte buffer that shares this buffer's content. + *

+ *

The content of the new buffer will be that of this buffer. Changes + * to this buffer's content will be visible in the new buffer, and vice + * versa; the two buffers' positions will be independent. + *

+ *

The new buffer's position will be identical to those of this buffer + */ + public LargeByteBuffer duplicate(); - public long remaining(); + public long remaining(); - /** - * the total number of bytes in this buffer - * @return - */ - public long size(); + /** + * Total number of bytes in this buffer + * + * @return + */ + public long size(); - /** - * writes the data from the current position() to the end of this buffer - * to the given channel. The position() will be moved to the end of - * the buffer after this. - * - * Note that this method will continually attempt to push data to the given channel. If the - * channel cannot accept more data, this will continuously retry until the channel accepts - * the data. - * - * @param channel - * @return the number of bytes written to the channel - * @throws IOException - */ - public long writeTo(WritableByteChannel channel) throws IOException; + /** + * Writes the data from the current position() to the end of this buffer + * to the given channel. The position() will be moved to the end of + * the buffer after this. + *

+ * Note that this method will continually attempt to push data to the given channel. If the + * channel cannot accept more data, this will continuously retry until the channel accepts + * the data. + * + * @param channel + * @return the number of bytes written to the channel + * @throws IOException + */ + public long writeTo(WritableByteChannel channel) throws IOException; - /** - * get the entire contents of this as one ByteBuffer, if possible. The returned ByteBuffer - * will always have the position set 0, and the limit set to the end of the data. Each - * call will return a new ByteBuffer, but will not require copying the data (eg., it will - * use ByteBuffer#duplicate()). The returned byte buffer and this may or may not share data. - * - * @return - * @throws BufferTooLargeException if this buffer is too large to fit in one {@link ByteBuffer} - */ - public ByteBuffer asByteBuffer() throws BufferTooLargeException; + /** + * Get the entire contents of this as one ByteBuffer, if possible. The returned ByteBuffer + * will always have the position set 0, and the limit set to the end of the data. Each + * call will return a new ByteBuffer, but will not require copying the data (eg., it will + * use ByteBuffer#duplicate()). The returned byte buffer and this may or may not share data. + * + * @return + * @throws BufferTooLargeException if this buffer is too large to fit in one {@link ByteBuffer} + */ + public ByteBuffer asByteBuffer() throws BufferTooLargeException; - /** - * Attempt to clean up if it is memory-mapped. This uses an *unsafe* Sun API that - * might cause errors if one attempts to read from the unmapped buffer, but it's better than - * waiting for the GC to find it because that could lead to huge numbers of open files. There's - * unfortunately no standard API to do this. - */ - public void dispose(); + /** + * Attempt to clean up if it is memory-mapped. This uses an *unsafe* Sun API that + * might cause errors if one attempts to read from the unmapped buffer, but it's better than + * waiting for the GC to find it because that could lead to huge numbers of open files. There's + * unfortunately no standard API to do this. + */ + public void dispose(); } 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 9d40fc4f5c07d..f762c686bde12 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 @@ -16,6 +16,8 @@ */ package org.apache.spark.network.buffer; +import com.google.common.annotations.VisibleForTesting; + import java.io.IOException; import java.nio.ByteBuffer; import java.nio.channels.FileChannel; @@ -23,51 +25,58 @@ public class LargeByteBufferHelper { - public static final int MAX_CHUNK = Integer.MAX_VALUE - 1000000; + public static final int MAX_CHUNK = Integer.MAX_VALUE - 1000000; - public static LargeByteBuffer asLargeByteBuffer(ByteBuffer buffer) { - return new WrappedLargeByteBuffer(new ByteBuffer[]{buffer}); - } + 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 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, MAX_CHUNK); - ByteBuffer next = ByteBuffer.allocate(nextSize); - remaining -= nextSize; - chunks.add(next); - } - return new WrappedLargeByteBuffer(chunks.toArray(new ByteBuffer[chunks.size()])); + public static LargeByteBuffer allocate(long size) { + return allocate(size, MAX_CHUNK); + } + + @VisibleForTesting + static LargeByteBuffer allocate(long size, int maxChunk) { + int chunksNeeded = (int) ((size + maxChunk - 1) / maxChunk); + ByteBuffer[] chunks = new ByteBuffer[chunksNeeded]; + long remaining = size; + for (int i = 0; i < chunksNeeded; i++) { + int nextSize = (int) Math.min(remaining, maxChunk); + ByteBuffer next = ByteBuffer.allocate(nextSize); + remaining -= nextSize; + chunks[i] = next; } + if (remaining != 0) throw new IllegalStateException("remaining = " + remaining); + return new WrappedLargeByteBuffer(chunks); + } - public static LargeByteBuffer mapFile( - FileChannel channel, - FileChannel.MapMode mode, - long offset, - long length - ) throws IOException { - int maxChunk = MAX_CHUNK; - ArrayList offsets = new ArrayList(); - long curOffset = offset; - long end = offset + length; - while (curOffset < end) { - offsets.add(curOffset); - int chunkLength = (int) Math.min((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); + public static LargeByteBuffer mapFile( + FileChannel channel, + FileChannel.MapMode mode, + long offset, + long length + ) throws IOException { + int maxChunk = MAX_CHUNK; + ArrayList offsets = new ArrayList(); + long curOffset = offset; + long end = offset + length; + while (curOffset < end) { + offsets.add(curOffset); + int chunkLength = (int) Math.min((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/WrappedLargeByteBuffer.java b/network/common/src/main/java/org/apache/spark/network/buffer/WrappedLargeByteBuffer.java index 8ead571dd102f..d5da70b9000de 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 @@ -16,6 +16,7 @@ */ package org.apache.spark.network.buffer; +import com.google.common.annotations.VisibleForTesting; import sun.nio.ch.DirectBuffer; import java.io.IOException; @@ -28,53 +29,36 @@ public class WrappedLargeByteBuffer implements LargeByteBuffer { - //only public for tests + @VisibleForTesting public final ByteBuffer[] underlying; private final long size; private long _pos; - private int currentBufferIdx; - private ByteBuffer currentBuffer; + @VisibleForTesting + int currentBufferIdx; + @VisibleForTesting + ByteBuffer currentBuffer; public WrappedLargeByteBuffer(ByteBuffer[] underlying) { - this(underlying, findExpectedInitialPosition(underlying)); - } - - private static long findExpectedInitialPosition(ByteBuffer[] bufs) { - long sum = 0L; - for (ByteBuffer b: bufs) { - if (b.position() > 0) { - // this could still lead to a mix of positions half-way through buffers that - // would be inconsistent -- but we'll discover that in the constructor checks - sum += b.position(); - } else { - break; - } + if (underlying.length == 0) { + throw new IllegalArgumentException("must wrap at least one ByteBuffer"); } - return sum; - } - - private WrappedLargeByteBuffer(ByteBuffer[] underlying, long initialPosition) { this.underlying = underlying; long sum = 0L; + boolean startFound = false; + long initialPosition = -1; for (int i = 0; i < underlying.length; i++) { ByteBuffer b = underlying[i]; - long nextSum = sum + b.capacity(); - int expectedPosition; - if (nextSum < initialPosition) { - expectedPosition = b.capacity(); - } else if (sum > initialPosition) { - expectedPosition = 0; - } else { - expectedPosition = (int) (initialPosition - sum); + if (startFound) { + if (b.position() != 0) { + throw new IllegalArgumentException("ByteBuffers have inconsistent positions"); + } + } else if (b.position() != b.capacity()) { + startFound = true; + initialPosition = sum + b.position(); } - if (b.position() != expectedPosition) { - throw new IllegalArgumentException("ByteBuffer[" + i + "]:" + b + " was expected to have" + - " position = " + expectedPosition + " to be consistent with the overall " + - "initialPosition = " + initialPosition); - } - sum = nextSum; + sum += b.capacity(); } _pos = initialPosition; currentBufferIdx = 0; @@ -84,8 +68,9 @@ private WrappedLargeByteBuffer(ByteBuffer[] underlying, long initialPosition) { @Override public void get(byte[] dest, int offset, int length) { - if (length > remaining()) + if (length > remaining()) { throw new BufferUnderflowException(); + } int moved = 0; while (moved < length) { int toRead = Math.min(length - moved, currentBuffer.remaining()); @@ -117,7 +102,7 @@ public WrappedLargeByteBuffer deepCopy() { ByteBuffer b = underlying[i]; dataCopy[i] = ByteBuffer.allocate(b.capacity()); int originalPosition = b.position(); - b.position(0); + b.rewind(); dataCopy[i].put(b); dataCopy[i].position(0); b.position(originalPosition); @@ -193,7 +178,7 @@ public WrappedLargeByteBuffer duplicate() { for (int i = 0; i < underlying.length; i++) { duplicates[i] = underlying[i].duplicate(); } - return new WrappedLargeByteBuffer(duplicates, _pos); + return new WrappedLargeByteBuffer(duplicates); } @Override @@ -216,13 +201,29 @@ public long writeTo(WritableByteChannel channel) throws IOException { @Override public ByteBuffer asByteBuffer() throws BufferTooLargeException { - if (underlying.length > 1) { - throw new BufferTooLargeException(size()); + return asByteBuffer(LargeByteBufferHelper.MAX_CHUNK); + } + + @VisibleForTesting + ByteBuffer asByteBuffer(int maxChunkSize) throws BufferTooLargeException { + if (underlying.length == 1) { + ByteBuffer b = underlying[0].duplicate(); + b.rewind(); + return b; + } else if (size() > maxChunkSize) { + throw new BufferTooLargeException(size(), maxChunkSize); + } else { + byte[] merged = new byte[(int) size()]; + long initialPosition = position(); + rewind(); + get(merged, 0, merged.length); + rewind(); + skip(initialPosition); + return ByteBuffer.wrap(merged); } - return underlying[0]; } - // only needed for tests + @VisibleForTesting public List nioBuffers() { return Arrays.asList(underlying); } @@ -234,7 +235,7 @@ public List nioBuffers() { * unfortunately no standard API to do this. */ private static void dispose(ByteBuffer buffer) { - if (buffer != null && buffer instanceof MappedByteBuffer) { + if (buffer != null && buffer instanceof DirectBuffer) { DirectBuffer db = (DirectBuffer) buffer; if (db.cleaner() != null) { db.cleaner().clean(); 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 d56f216f11dc0..4b15c42595e02 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 @@ -68,4 +68,14 @@ public void testMapFile() throws IOException { } } } + + @Test + public void testAllocate() { + WrappedLargeByteBuffer buf = (WrappedLargeByteBuffer) LargeByteBufferHelper.allocate(95,10); + assertEquals(10, buf.underlying.length); + for (int i = 0 ; i < 9; i++) { + assertEquals(10, buf.underlying[i].capacity()); + } + assertEquals(5, buf.underlying[9].capacity()); + } } 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 79398c6ae7ab1..62266afb500c9 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 @@ -46,17 +46,54 @@ private WrappedLargeByteBuffer testDataBuf() { @Test public void asByteBuffer() throws BufferTooLargeException { - //test that it works when buffer is small, and the right error when buffer is big + // test that it works when buffer is small LargeByteBuffer buf = LargeByteBufferHelper.asLargeByteBuffer(new byte[100]); ByteBuffer nioBuf = buf.asByteBuffer(); + assertEquals(0, nioBuf.position()); assertEquals(100, nioBuf.remaining()); + // if we move the large byte buffer, the nio.ByteBuffer we have doesn't change + buf.skip(10); + assertEquals(0, nioBuf.position()); + assertEquals(100, nioBuf.remaining()); + // if we grab another byte buffer while the large byte buffer's position != 0, + // the returned buffer still has position 0 + ByteBuffer nioBuf2 = buf.asByteBuffer(); + assertEquals(0, nioBuf2.position()); + assertEquals(100, nioBuf2.remaining()); + // the two byte buffers we grabbed are independent + nioBuf2.position(20); + assertEquals(0, nioBuf.position()); + assertEquals(100, nioBuf.remaining()); + assertEquals(20, nioBuf2.position()); + assertEquals(80, nioBuf2.remaining()); + // merges the data from multiple buffers ByteBuffer[] bufs = new ByteBuffer[2]; for (int i = 0; i < 2; i++) { - bufs[i] = ByteBuffer.allocate(10); + bufs[i] = ByteBuffer.allocate(250); + bufs[i].get(data, i * 250, 250); + bufs[i].rewind(); } + WrappedLargeByteBuffer wrappedBB = new WrappedLargeByteBuffer(bufs); + ByteBuffer mergedBuffer = wrappedBB.asByteBuffer(500); + assertConsistent(wrappedBB); + assertEquals(0, mergedBuffer.position()); + byte[] copyData = new byte[500]; + mergedBuffer.get(copyData); + mergedBuffer.rewind(); + assertArrayEquals(data, copyData); + wrappedBB.skip(20); + assertConsistent(wrappedBB); + ByteBuffer mergedBuffer2 = wrappedBB.asByteBuffer(500); + assertEquals(0, mergedBuffer2.position()); + mergedBuffer2.get(copyData); + assertArrayEquals(data, copyData); + assertEquals(0, mergedBuffer.position()); + assertEquals(20, wrappedBB.position()); + + // the right error when the buffer is too big try { - new WrappedLargeByteBuffer(bufs).asByteBuffer(); + wrappedBB.asByteBuffer(499); fail("expected an exception"); } catch (BufferTooLargeException btl) { } @@ -105,21 +142,6 @@ public void skipAndGet() { } } - private void assertConsistent(WrappedLargeByteBuffer buffer) { - long pos = buffer.position(); - long bufferStartPos = 0; - for (ByteBuffer p: buffer.nioBuffers()) { - if (pos < bufferStartPos) { - assertEquals(0, p.position()); - } else if (pos < bufferStartPos + p.capacity()) { - assertEquals(pos - bufferStartPos, p.position()); - } else { - assertEquals(p.capacity(), p.position()); - } - bufferStartPos += p.capacity(); - } - } - @Test public void get() { WrappedLargeByteBuffer b = testDataBuf(); @@ -218,7 +240,32 @@ public void constructWithBuffersWithNonZeroPosition() { fail("expected exception"); } catch (IllegalArgumentException ex) { } + } + @Test(expected=IllegalArgumentException.class) + public void testRequireAtLeastOneBuffer() { + new WrappedLargeByteBuffer( new ByteBuffer[0]); + } + + + private void assertConsistent(WrappedLargeByteBuffer buffer) { + long pos = buffer.position(); + long bufferStartPos = 0; + if (buffer.currentBufferIdx < buffer.underlying.length) { + assertEquals(buffer.currentBuffer, buffer.underlying[buffer.currentBufferIdx]); + } else { + assertNull(buffer.currentBuffer); + } + for (ByteBuffer p: buffer.nioBuffers()) { + if (pos < bufferStartPos) { + assertEquals(0, p.position()); + } else if (pos < bufferStartPos + p.capacity()) { + assertEquals(pos - bufferStartPos, p.position()); + } else { + assertEquals(p.capacity(), p.position()); + } + bufferStartPos += p.capacity(); + } } private void assertSubArrayEquals(byte[] exp, int expOffset, byte[] act, int actOffset, int length) { @@ -229,5 +276,4 @@ private void assertSubArrayEquals(byte[] exp, int expOffset, byte[] act, int act assertArrayEquals(expCopy, actCopy); } - } From ff9f968590d078086a0c9ec2b42c95a3b13f02f5 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Thu, 9 Apr 2015 09:40:13 -0500 Subject: [PATCH 04/29] LargeBBInputStream should dispose when we close the stream, not when we reach the end --- .../util/LargeByteBufferInputStream.scala | 20 +++---------------- .../LargeByteBufferInputStreamSuite.scala | 19 ++++++++++++++---- 2 files changed, 18 insertions(+), 21 deletions(-) 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 69bc4902d0aac..1f27f895ada59 100644 --- a/core/src/main/scala/org/apache/spark/util/LargeByteBufferInputStream.scala +++ b/core/src/main/scala/org/apache/spark/util/LargeByteBufferInputStream.scala @@ -24,7 +24,7 @@ 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). + * when the stream is closed (e.g. to close a memory-mapped file). */ private[spark] class LargeByteBufferInputStream(private var buffer: LargeByteBuffer, dispose: Boolean = false) @@ -32,14 +32,9 @@ class LargeByteBufferInputStream(private var buffer: LargeByteBuffer, dispose: B override def read(): Int = { if (buffer == null || buffer.remaining() == 0) { - cleanUp() -1 } else { - val r = buffer.get() & 0xFF - if (buffer.remaining() == 0) { - cleanUp() - } - r + buffer.get() & 0xFF } } @@ -49,16 +44,10 @@ class LargeByteBufferInputStream(private var buffer: LargeByteBuffer, dispose: B 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 } } @@ -66,9 +55,6 @@ class LargeByteBufferInputStream(private var buffer: LargeByteBuffer, dispose: B override def skip(bytes: Long): Long = { if (buffer != null) { val skipped = buffer.skip(bytes) - if (buffer.remaining() == 0) { - cleanUp() - } skipped } else { 0L @@ -81,7 +67,7 @@ class LargeByteBufferInputStream(private var buffer: LargeByteBuffer, dispose: B /** * Clean up the buffer, and potentially dispose of it */ - private def cleanUp() { + override def close() { if (buffer != null) { if (dispose) { buffer.dispose() diff --git a/core/src/test/scala/org/apache/spark/util/LargeByteBufferInputStreamSuite.scala b/core/src/test/scala/org/apache/spark/util/LargeByteBufferInputStreamSuite.scala index 209fb0d343fd0..ed8a86b62967e 100644 --- a/core/src/test/scala/org/apache/spark/util/LargeByteBufferInputStreamSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/LargeByteBufferInputStreamSuite.scala @@ -17,12 +17,13 @@ package org.apache.spark.util import java.io.{FileInputStream, FileOutputStream, OutputStream, File} +import java.nio.ByteBuffer import java.nio.channels.FileChannel.MapMode import org.junit.Assert._ import org.scalatest.{FunSuite, Matchers} -import org.apache.spark.network.buffer.{LargeByteBufferHelper, WrappedLargeByteBuffer} +import org.apache.spark.network.buffer.{LargeByteBuffer, LargeByteBufferHelper, WrappedLargeByteBuffer} class LargeByteBufferInputStreamSuite extends FunSuite with Matchers { @@ -54,10 +55,20 @@ class LargeByteBufferInputStreamSuite extends FunSuite with Matchers { 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.disposed should be (false) in.read(read) should be (-1) + in.disposed should be (false) + in.close() + in.disposed should be (true) + } + + test("dispose on close") { + //don't need to read to the end -- dispose anytime we close + val data = new Array[Byte](10) + val in = new LargeByteBufferInputStream(LargeByteBufferHelper.asLargeByteBuffer(data), + dispose = true) + in.disposed should be (false) + in.close() in.disposed should be (true) } From a3dc8110a27d56bced6c0c6005679a5cfc6ce626 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Thu, 9 Apr 2015 10:12:50 -0500 Subject: [PATCH 05/29] add bounds checks to ByteArrayChunkOutputStream.slice() --- .../spark/util/io/ByteArrayChunkOutputStream.scala | 11 ++++++++--- .../util/io/ByteArrayChunkOutputStreamSuite.scala | 6 ++++++ 2 files changed, 14 insertions(+), 3 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 34bb6ee6db906..b9b0243216fc1 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 @@ -43,7 +43,7 @@ class ByteArrayChunkOutputStream(chunkSize: Int) extends OutputStream { */ private var position = chunkSize - var size: Long = 0L + private[util] var size: Long = 0L override def write(b: Int): Unit = { allocateNewChunkIfNeeded() @@ -97,14 +97,19 @@ class ByteArrayChunkOutputStream(chunkSize: Int) extends OutputStream { } /** - * get a copy of the data between the two endpoints + * 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 + * 0 <= start <= until <= size */ def slice(start: Long, until: Long): Array[Byte] = { require((until - start) < Integer.MAX_VALUE, "max slice length = Integer.MAX_VALUE") + require(start >= 0 && start <= until, s"start ($start) must be >= 0 and <= until ($until)") + require(until >= start && until <= size, + s"until ($until) must be >= start ($start) and <= size ($size)") var chunkStart = 0L var chunkIdx = 0 - var foundStart = false val length = (until - start).toInt + var foundStart = false val result = new Array[Byte](length) while (!foundStart) { val nextSize = chunkStart + chunks(chunkIdx).size 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 cacb3fbb0762b..a177365af9689 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 @@ -125,5 +125,11 @@ class ByteArrayChunkOutputStreamSuite extends SparkFunSuite { } } } + + // errors on bad bounds + intercept[IllegalArgumentException]{o.slice(31, 31)} + intercept[IllegalArgumentException]{o.slice(-1, 10)} + intercept[IllegalArgumentException]{o.slice(10, 5)} + intercept[IllegalArgumentException]{o.slice(10, 35)} } } From da29c3debe8215a4b14c9abef4c87e071181dfa7 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Thu, 9 Apr 2015 10:32:12 -0500 Subject: [PATCH 06/29] cleanup rewind() a little --- .../spark/network/buffer/WrappedLargeByteBuffer.java | 10 +++++----- 1 file changed, 5 insertions(+), 5 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 d5da70b9000de..05366060b8285 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 @@ -83,14 +83,14 @@ public void get(byte[] dest, int offset, int length) { @Override public LargeByteBuffer rewind() { + if (currentBuffer != null) { + currentBuffer.rewind(); + } while (currentBufferIdx > 0) { - if (currentBuffer != null) { - currentBuffer.rewind(); - } currentBufferIdx -= 1; currentBuffer = underlying[currentBufferIdx]; + currentBuffer.rewind(); } - currentBuffer.rewind(); _pos = 0; return this; } @@ -151,7 +151,7 @@ public long skip(long n) { } else if (n > 0) { final long moveTotal = Math.min(n, remaining()); long toMove = moveTotal; - // move forwards-- set the position to the end of every buffer as we go forwards + // move forwards -- set the position to the end of every buffer as we go forwards currentBufferIdx -= 1; while (toMove > 0) { currentBufferIdx += 1; From 366f9212b128603406060064c0242dd711bf7df4 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Mon, 13 Apr 2015 14:49:57 -0500 Subject: [PATCH 07/29] WrappedLargeByteBuffer.asByteBuffer shouldnt copy -- it should just enforce underlying buffer sizes --- .../spark/network/buffer/LargeByteBuffer.java | 2 +- .../network/buffer/LargeByteBufferHelper.java | 2 +- .../buffer/WrappedLargeByteBuffer.java | 36 +++++++------ .../buffer/WrappedLargeByteBufferSuite.java | 53 +++++++++---------- 4 files changed, 47 insertions(+), 46 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 bc48088cc8610..34fe607edf2df 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 @@ -96,7 +96,7 @@ public interface LargeByteBuffer { * Get the entire contents of this as one ByteBuffer, if possible. The returned ByteBuffer * will always have the position set 0, and the limit set to the end of the data. Each * call will return a new ByteBuffer, but will not require copying the data (eg., it will - * use ByteBuffer#duplicate()). The returned byte buffer and this may or may not share data. + * use ByteBuffer#duplicate()). The returned byte buffer will share data with this buffer. * * @return * @throws BufferTooLargeException if this buffer is too large to fit in one {@link ByteBuffer} 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 f762c686bde12..ce2ff266649e8 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 @@ -51,7 +51,7 @@ static LargeByteBuffer allocate(long size, int maxChunk) { chunks[i] = next; } if (remaining != 0) throw new IllegalStateException("remaining = " + remaining); - return new WrappedLargeByteBuffer(chunks); + return new WrappedLargeByteBuffer(chunks, maxChunk); } 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 05366060b8285..bbcd9bcfeacbc 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 @@ -33,6 +33,13 @@ public class WrappedLargeByteBuffer implements LargeByteBuffer { public final ByteBuffer[] underlying; private final long size; + /** + * each sub-ByteBuffer (except for the last one) must be exactly this size. Note that this + * class *really* expects this to be LargeByteBufferHelper.MAX_CHUNK. The only reason it isn't + * is so that we can do tests without creating ginormous buffers. Public methods force it to + * be LargeByteBufferHelper.MAX_CHUNK + */ + private final int subBufferSize; private long _pos; @VisibleForTesting int currentBufferIdx; @@ -41,15 +48,25 @@ public class WrappedLargeByteBuffer implements LargeByteBuffer { public WrappedLargeByteBuffer(ByteBuffer[] underlying) { + this(underlying, LargeByteBufferHelper.MAX_CHUNK); + } + + @VisibleForTesting + WrappedLargeByteBuffer(ByteBuffer[] underlying, int subBufferSize) { if (underlying.length == 0) { throw new IllegalArgumentException("must wrap at least one ByteBuffer"); } this.underlying = underlying; + this.subBufferSize = subBufferSize; long sum = 0L; boolean startFound = false; long initialPosition = -1; for (int i = 0; i < underlying.length; i++) { ByteBuffer b = underlying[i]; + if (i != underlying.length -1 && b.capacity() != subBufferSize) { + throw new IllegalArgumentException("All buffers, except for the final one, must have " + + "size = " + subBufferSize); + } if (startFound) { if (b.position() != 0) { throw new IllegalArgumentException("ByteBuffers have inconsistent positions"); @@ -107,7 +124,7 @@ public WrappedLargeByteBuffer deepCopy() { dataCopy[i].position(0); b.position(originalPosition); } - return new WrappedLargeByteBuffer(dataCopy); + return new WrappedLargeByteBuffer(dataCopy, subBufferSize); } @Override @@ -178,7 +195,7 @@ public WrappedLargeByteBuffer duplicate() { for (int i = 0; i < underlying.length; i++) { duplicates[i] = underlying[i].duplicate(); } - return new WrappedLargeByteBuffer(duplicates); + return new WrappedLargeByteBuffer(duplicates, subBufferSize); } @Override @@ -201,25 +218,12 @@ public long writeTo(WritableByteChannel channel) throws IOException { @Override public ByteBuffer asByteBuffer() throws BufferTooLargeException { - return asByteBuffer(LargeByteBufferHelper.MAX_CHUNK); - } - - @VisibleForTesting - ByteBuffer asByteBuffer(int maxChunkSize) throws BufferTooLargeException { if (underlying.length == 1) { ByteBuffer b = underlying[0].duplicate(); b.rewind(); return b; - } else if (size() > maxChunkSize) { - throw new BufferTooLargeException(size(), maxChunkSize); } else { - byte[] merged = new byte[(int) size()]; - long initialPosition = position(); - rewind(); - get(merged, 0, merged.length); - rewind(); - skip(initialPosition); - return ByteBuffer.wrap(merged); + throw new BufferTooLargeException(size(), LargeByteBufferHelper.MAX_CHUNK); } } 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 62266afb500c9..4e3db7f1cd75c 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 @@ -41,7 +41,7 @@ private WrappedLargeByteBuffer testDataBuf() { System.arraycopy(data, i * 50, b, 0, 50); bufs[i] = ByteBuffer.wrap(b); } - return new WrappedLargeByteBuffer(bufs); + return new WrappedLargeByteBuffer(bufs, 50); } @Test @@ -67,38 +67,35 @@ public void asByteBuffer() throws BufferTooLargeException { assertEquals(20, nioBuf2.position()); assertEquals(80, nioBuf2.remaining()); - // merges the data from multiple buffers - ByteBuffer[] bufs = new ByteBuffer[2]; - for (int i = 0; i < 2; i++) { - bufs[i] = ByteBuffer.allocate(250); - bufs[i].get(data, i * 250, 250); - bufs[i].rewind(); - } - WrappedLargeByteBuffer wrappedBB = new WrappedLargeByteBuffer(bufs); - ByteBuffer mergedBuffer = wrappedBB.asByteBuffer(500); - assertConsistent(wrappedBB); - assertEquals(0, mergedBuffer.position()); - byte[] copyData = new byte[500]; - mergedBuffer.get(copyData); - mergedBuffer.rewind(); - assertArrayEquals(data, copyData); - wrappedBB.skip(20); - assertConsistent(wrappedBB); - ByteBuffer mergedBuffer2 = wrappedBB.asByteBuffer(500); - assertEquals(0, mergedBuffer2.position()); - mergedBuffer2.get(copyData); - assertArrayEquals(data, copyData); - assertEquals(0, mergedBuffer.position()); - assertEquals(20, wrappedBB.position()); - // the right error when the buffer is too big try { - wrappedBB.asByteBuffer(499); + WrappedLargeByteBuffer buf2 = new WrappedLargeByteBuffer( + new ByteBuffer[]{ByteBuffer.allocate(10), ByteBuffer.allocate(10)}, 10); + buf2.asByteBuffer(); fail("expected an exception"); } catch (BufferTooLargeException btl) { } } + @Test + public void checkSizesOfInternalBuffers() { + errorOnBuffersSized(10, new int[]{9,10}); + errorOnBuffersSized(10, new int[]{10,10,0,10}); + errorOnBuffersSized(20, new int[]{10,10,10,10}); + } + + private void errorOnBuffersSized(int chunkSize, int[] sizes) { + ByteBuffer[] bufs = new ByteBuffer[sizes.length]; + for (int i = 0; i < sizes.length; i++) { + bufs[i] = ByteBuffer.allocate(sizes[i]); + } + try { + new WrappedLargeByteBuffer(bufs, chunkSize); + fail("expected exception"); + } catch (IllegalArgumentException iae) { + } + } + @Test public void deepCopy() { WrappedLargeByteBuffer b = testDataBuf(); @@ -215,14 +212,14 @@ public void constructWithBuffersWithNonZeroPosition() { bufs[0].position(50); bufs[1].position(5); - WrappedLargeByteBuffer b1 = new WrappedLargeByteBuffer(bufs); + WrappedLargeByteBuffer b1 = new WrappedLargeByteBuffer(bufs, 50); assertEquals(55, b1.position()); bufs[1].position(50); bufs[2].position(50); bufs[3].position(35); - WrappedLargeByteBuffer b2 = new WrappedLargeByteBuffer(bufs); + WrappedLargeByteBuffer b2 = new WrappedLargeByteBuffer(bufs, 50); assertEquals(185, b2.position()); From 34c2131c7a08095f98a64dab63870643ce6740dd Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Tue, 14 Apr 2015 10:56:49 -0500 Subject: [PATCH 08/29] sigh, need to make the WrappedLargeBB constructor w/ chunkSize public for use in IOStream tests :/ --- .../spark/util/LargeByteBufferOutputStream.scala | 9 ++++++--- .../util/LargeByteBufferInputStreamSuite.scala | 2 +- .../network/buffer/WrappedLargeByteBuffer.java | 16 ++++++++++++++-- .../buffer/WrappedLargeByteBufferSuite.java | 2 ++ 4 files changed, 23 insertions(+), 6 deletions(-) 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 176825b7ec241..63bb6eaedd3ea 100644 --- a/core/src/main/scala/org/apache/spark/util/LargeByteBufferOutputStream.scala +++ b/core/src/main/scala/org/apache/spark/util/LargeByteBufferOutputStream.scala @@ -41,8 +41,11 @@ class LargeByteBufferOutputStream(chunkSize: Int = 65536) largeBuffer(LargeByteBufferHelper.MAX_CHUNK) } - // exposed for testing - private[util] def largeBuffer(maxChunk: Int): LargeByteBuffer = { + /** + * exposed for testing. You don't really ever want to call this method -- the returned + * buffer will not implement {{asByteBuffer}} correctly. + */ + private[util] def largeBuffer(maxChunk: Int): WrappedLargeByteBuffer = { // 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 @@ -58,7 +61,7 @@ class LargeByteBufferOutputStream(chunkSize: Int = 65536) pos += nextSize remaining -= nextSize } - new WrappedLargeByteBuffer(chunks.map(ByteBuffer.wrap)) + new WrappedLargeByteBuffer(chunks.map(ByteBuffer.wrap), maxChunk) } override def close(): Unit = { diff --git a/core/src/test/scala/org/apache/spark/util/LargeByteBufferInputStreamSuite.scala b/core/src/test/scala/org/apache/spark/util/LargeByteBufferInputStreamSuite.scala index ed8a86b62967e..4802069a7f67e 100644 --- a/core/src/test/scala/org/apache/spark/util/LargeByteBufferInputStreamSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/LargeByteBufferInputStreamSuite.scala @@ -63,7 +63,7 @@ class LargeByteBufferInputStreamSuite extends FunSuite with Matchers { } test("dispose on close") { - //don't need to read to the end -- dispose anytime we close + // don't need to read to the end -- dispose anytime we close val data = new Array[Byte](10) val in = new LargeByteBufferInputStream(LargeByteBufferHelper.asLargeByteBuffer(data), dispose = true) 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 bbcd9bcfeacbc..a8bc7afc6bdc1 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 @@ -51,8 +51,16 @@ public WrappedLargeByteBuffer(ByteBuffer[] underlying) { this(underlying, LargeByteBufferHelper.MAX_CHUNK); } + /** + * you do **not** want to call this version. It leads to a buffer which doesn't properly + * support {@link #asByteBuffer}. The only reason it exists is to we can have tests which + * don't require 2GB of memory + * + * @param underlying + * @param subBufferSize + */ @VisibleForTesting - WrappedLargeByteBuffer(ByteBuffer[] underlying, int subBufferSize) { + public WrappedLargeByteBuffer(ByteBuffer[] underlying, int subBufferSize) { if (underlying.length == 0) { throw new IllegalArgumentException("must wrap at least one ByteBuffer"); } @@ -223,7 +231,11 @@ public ByteBuffer asByteBuffer() throws BufferTooLargeException { b.rewind(); return b; } else { - throw new BufferTooLargeException(size(), LargeByteBufferHelper.MAX_CHUNK); + // NOTE: if subBufferSize != LargeByteBufferHelper.MAX_CAPACITY, in theory + // we could copy the data into a new buffer. But we don't want to do any copying. + // The only reason we allow smaller subBufferSize is so that we can have tests which + // don't require 2GB of memory + throw new BufferTooLargeException(size(), underlying[0].capacity()); } } 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 4e3db7f1cd75c..14ee9f19cbac4 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 @@ -71,6 +71,8 @@ public void asByteBuffer() throws BufferTooLargeException { try { WrappedLargeByteBuffer buf2 = new WrappedLargeByteBuffer( new ByteBuffer[]{ByteBuffer.allocate(10), ByteBuffer.allocate(10)}, 10); + // you really shouldn't ever construct a WrappedLargeByteBuffer with + // multiple small chunks, so this is somewhat contrived buf2.asByteBuffer(); fail("expected an exception"); } catch (BufferTooLargeException btl) { From bf4ec0aee447cc14269098b94870ec3992c10b0a Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Tue, 14 Apr 2015 11:24:28 -0500 Subject: [PATCH 09/29] style --- .../org/apache/spark/util/LargeByteBufferInputStreamSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/test/scala/org/apache/spark/util/LargeByteBufferInputStreamSuite.scala b/core/src/test/scala/org/apache/spark/util/LargeByteBufferInputStreamSuite.scala index 4802069a7f67e..67712cd70c856 100644 --- a/core/src/test/scala/org/apache/spark/util/LargeByteBufferInputStreamSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/LargeByteBufferInputStreamSuite.scala @@ -79,7 +79,7 @@ class LargeByteBufferInputStreamSuite extends FunSuite with Matchers { out.close() val lb = out.largeBuffer(128) - //just make sure that we test reading from multiple chunks + // just make sure that we test reading from multiple chunks lb.asInstanceOf[WrappedLargeByteBuffer].underlying.size should be > 1 val rawIn = new LargeByteBufferInputStream(lb) From 9d232d1b748f36bc3626ed2e6172cc7b610bdf78 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Tue, 14 Apr 2015 14:02:18 -0500 Subject: [PATCH 10/29] move package of LargeByteBufferIOStreams so that I can keep unsafe methods package-private at least --- .../buffer}/LargeByteBufferInputStream.scala | 7 ++----- .../buffer}/LargeByteBufferOutputStream.scala | 10 ++++------ .../spark/util/io/ByteArrayChunkOutputStream.scala | 2 +- .../buffer}/LargeByteBufferInputStreamSuite.scala | 2 +- .../buffer}/LargeByteBufferOutputStreamSuite.scala | 4 +--- .../spark/network/buffer/WrappedLargeByteBuffer.java | 2 +- 6 files changed, 10 insertions(+), 17 deletions(-) rename core/src/main/scala/org/apache/spark/{util => network/buffer}/LargeByteBufferInputStream.scala (92%) rename core/src/main/scala/org/apache/spark/{util => network/buffer}/LargeByteBufferOutputStream.scala (86%) rename core/src/test/scala/org/apache/spark/{util => network/buffer}/LargeByteBufferInputStreamSuite.scala (98%) rename core/src/test/scala/org/apache/spark/{util => network/buffer}/LargeByteBufferOutputStreamSuite.scala (95%) diff --git a/core/src/main/scala/org/apache/spark/util/LargeByteBufferInputStream.scala b/core/src/main/scala/org/apache/spark/network/buffer/LargeByteBufferInputStream.scala similarity index 92% rename from core/src/main/scala/org/apache/spark/util/LargeByteBufferInputStream.scala rename to core/src/main/scala/org/apache/spark/network/buffer/LargeByteBufferInputStream.scala index 1f27f895ada59..e4c4a83b2742f 100644 --- a/core/src/main/scala/org/apache/spark/util/LargeByteBufferInputStream.scala +++ b/core/src/main/scala/org/apache/spark/network/buffer/LargeByteBufferInputStream.scala @@ -15,13 +15,10 @@ * limitations under the License. */ -package org.apache.spark.util +package org.apache.spark.network.buffer 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() * when the stream is closed (e.g. to close a memory-mapped file). @@ -62,7 +59,7 @@ class LargeByteBufferInputStream(private var buffer: LargeByteBuffer, dispose: B } // only for testing - private[util] var disposed = false + private[buffer] var disposed = false /** * Clean up the buffer, and potentially dispose of it diff --git a/core/src/main/scala/org/apache/spark/util/LargeByteBufferOutputStream.scala b/core/src/main/scala/org/apache/spark/network/buffer/LargeByteBufferOutputStream.scala similarity index 86% rename from core/src/main/scala/org/apache/spark/util/LargeByteBufferOutputStream.scala rename to core/src/main/scala/org/apache/spark/network/buffer/LargeByteBufferOutputStream.scala index 63bb6eaedd3ea..5c228fb1c3aef 100644 --- a/core/src/main/scala/org/apache/spark/util/LargeByteBufferOutputStream.scala +++ b/core/src/main/scala/org/apache/spark/network/buffer/LargeByteBufferOutputStream.scala @@ -15,19 +15,17 @@ * limitations under the License. */ -package org.apache.spark.util +package org.apache.spark.network.buffer 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) +private[spark] class LargeByteBufferOutputStream(chunkSize: Int = 65536) extends OutputStream { - private[util] val output = new ByteArrayChunkOutputStream(chunkSize) + private[buffer] val output = new ByteArrayChunkOutputStream(chunkSize) override def write(b: Int): Unit = { output.write(b) @@ -45,7 +43,7 @@ class LargeByteBufferOutputStream(chunkSize: Int = 65536) * exposed for testing. You don't really ever want to call this method -- the returned * buffer will not implement {{asByteBuffer}} correctly. */ - private[util] def largeBuffer(maxChunk: Int): WrappedLargeByteBuffer = { + private[buffer] def largeBuffer(maxChunk: Int): WrappedLargeByteBuffer = { // 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 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 b9b0243216fc1..907d2cbb7537e 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 @@ -43,7 +43,7 @@ class ByteArrayChunkOutputStream(chunkSize: Int) extends OutputStream { */ private var position = chunkSize - private[util] var size: Long = 0L + private[spark] var size: Long = 0L override def write(b: Int): Unit = { allocateNewChunkIfNeeded() diff --git a/core/src/test/scala/org/apache/spark/util/LargeByteBufferInputStreamSuite.scala b/core/src/test/scala/org/apache/spark/network/buffer/LargeByteBufferInputStreamSuite.scala similarity index 98% rename from core/src/test/scala/org/apache/spark/util/LargeByteBufferInputStreamSuite.scala rename to core/src/test/scala/org/apache/spark/network/buffer/LargeByteBufferInputStreamSuite.scala index 67712cd70c856..be4cfd829f8b6 100644 --- a/core/src/test/scala/org/apache/spark/util/LargeByteBufferInputStreamSuite.scala +++ b/core/src/test/scala/org/apache/spark/network/buffer/LargeByteBufferInputStreamSuite.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.util +package org.apache.spark.network.buffer import java.io.{FileInputStream, FileOutputStream, OutputStream, File} import java.nio.ByteBuffer diff --git a/core/src/test/scala/org/apache/spark/util/LargeByteBufferOutputStreamSuite.scala b/core/src/test/scala/org/apache/spark/network/buffer/LargeByteBufferOutputStreamSuite.scala similarity index 95% rename from core/src/test/scala/org/apache/spark/util/LargeByteBufferOutputStreamSuite.scala rename to core/src/test/scala/org/apache/spark/network/buffer/LargeByteBufferOutputStreamSuite.scala index 4b28d383df908..a55ef03480436 100644 --- a/core/src/test/scala/org/apache/spark/util/LargeByteBufferOutputStreamSuite.scala +++ b/core/src/test/scala/org/apache/spark/network/buffer/LargeByteBufferOutputStreamSuite.scala @@ -14,14 +14,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.util +package org.apache.spark.network.buffer 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") { 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 a8bc7afc6bdc1..56fba1f179031 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 @@ -60,7 +60,7 @@ public WrappedLargeByteBuffer(ByteBuffer[] underlying) { * @param subBufferSize */ @VisibleForTesting - public WrappedLargeByteBuffer(ByteBuffer[] underlying, int subBufferSize) { + WrappedLargeByteBuffer(ByteBuffer[] underlying, int subBufferSize) { if (underlying.length == 0) { throw new IllegalArgumentException("must wrap at least one ByteBuffer"); } From 6b2f751a978c40b4f51431a6c8b5c6316cf14521 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Tue, 14 Apr 2015 17:02:27 -0500 Subject: [PATCH 11/29] fix typo --- .../java/org/apache/spark/network/buffer/LargeByteBuffer.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 34fe607edf2df..3abda80723cc1 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 @@ -94,7 +94,7 @@ public interface LargeByteBuffer { /** * Get the entire contents of this as one ByteBuffer, if possible. The returned ByteBuffer - * will always have the position set 0, and the limit set to the end of the data. Each + * will always have the position set to 0, and the limit set to the end of the data. Each * call will return a new ByteBuffer, but will not require copying the data (eg., it will * use ByteBuffer#duplicate()). The returned byte buffer will share data with this buffer. * From 4da46263355910aaddc5b14e083adfaf0def2fe7 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Mon, 1 Jun 2015 11:12:48 -0500 Subject: [PATCH 12/29] error handling for get(); comments, style --- .../buffer/LargeByteBufferInputStream.scala | 3 +- .../buffer/LargeByteBufferOutputStream.scala | 6 +- .../LargeByteBufferInputStreamSuite.scala | 59 ++++++++------- .../spark/network/buffer/LargeByteBuffer.java | 33 ++++++-- .../network/buffer/LargeByteBufferHelper.java | 48 ++++++------ .../buffer/WrappedLargeByteBuffer.java | 51 +++++++++---- .../buffer/LargeByteBufferHelperSuite.java | 75 ++++++++++--------- .../buffer/WrappedLargeByteBufferSuite.java | 19 ++++- 8 files changed, 179 insertions(+), 115 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/network/buffer/LargeByteBufferInputStream.scala b/core/src/main/scala/org/apache/spark/network/buffer/LargeByteBufferInputStream.scala index e4c4a83b2742f..f00899ead1ab0 100644 --- a/core/src/main/scala/org/apache/spark/network/buffer/LargeByteBufferInputStream.scala +++ b/core/src/main/scala/org/apache/spark/network/buffer/LargeByteBufferInputStream.scala @@ -51,8 +51,7 @@ class LargeByteBufferInputStream(private var buffer: LargeByteBuffer, dispose: B override def skip(bytes: Long): Long = { if (buffer != null) { - val skipped = buffer.skip(bytes) - skipped + buffer.skip(bytes) } else { 0L } diff --git a/core/src/main/scala/org/apache/spark/network/buffer/LargeByteBufferOutputStream.scala b/core/src/main/scala/org/apache/spark/network/buffer/LargeByteBufferOutputStream.scala index 5c228fb1c3aef..e9f13a6b27166 100644 --- a/core/src/main/scala/org/apache/spark/network/buffer/LargeByteBufferOutputStream.scala +++ b/core/src/main/scala/org/apache/spark/network/buffer/LargeByteBufferOutputStream.scala @@ -36,7 +36,7 @@ private[spark] class LargeByteBufferOutputStream(chunkSize: Int = 65536) } def largeBuffer: LargeByteBuffer = { - largeBuffer(LargeByteBufferHelper.MAX_CHUNK) + largeBuffer(LargeByteBufferHelper.MAX_CHUNK_SIZE) } /** @@ -44,10 +44,6 @@ private[spark] class LargeByteBufferOutputStream(chunkSize: Int = 65536) * buffer will not implement {{asByteBuffer}} correctly. */ private[buffer] def largeBuffer(maxChunk: Int): WrappedLargeByteBuffer = { - // 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) 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 be4cfd829f8b6..50051e803eb2f 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 @@ -29,37 +29,39 @@ 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)) + try { + val out: OutputStream = new FileOutputStream(testFile) + val buffer: Array[Byte] = new Array[Byte](1 << 16) + val len: Long = buffer.length.toLong + Integer.MAX_VALUE + 1 + (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)) + } } + in.disposed should be(false) + in.read(read) should be(-1) + in.disposed should be(false) + in.close() + in.disposed should be(true) + } finally { + testFile.delete() } - in.disposed should be (false) - in.read(read) should be (-1) - in.disposed should be (false) - in.close() - in.disposed should be (true) } test("dispose on close") { @@ -73,7 +75,6 @@ class LargeByteBufferInputStreamSuite extends FunSuite with Matchers { } test("io stream roundtrip") { - val out = new LargeByteBufferOutputStream(128) (0 until 200).foreach{idx => out.write(idx)} out.close() 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 3abda80723cc1..87f695638ede6 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 @@ -20,6 +20,30 @@ import java.nio.ByteBuffer; import java.nio.channels.WritableByteChannel; +/** + * A byte buffer which can hold over 2GB. + *

+ * This is roughly similar {@link java.nio.ByteBuffer}, with a limited set of operations relevant + * to use in Spark, and without the capacity restrictions of a ByteBuffer. + *

+ * Unlike ByteBuffers, this is read-only, and only supports reading bytes (with both single and bulk + * get methods). It supports random access via skip to move around the + * buffer. + *

+ * In general, implementations are expected to support O(1) random access. Furthermore, + * neighboring locations in the buffer are likely to be neighboring in memory, so sequential access + * will avoid cache-misses. However, these are only rough guidelines which may differ in + * implementations. + *

+ * Any code which expects a ByteBuffer can obtain one via {@link #asByteBuffer} when possible -- see + * that method for a full description of its limitations. + *

+ * Instances of this class can be created with + * {@link org.apache.spark.network.buffer.LargeByteBufferHelper}, + * with a LargeByteBufferOutputStream, + * or directly from the implementation + * {@link org.apache.spark.network.buffer.WrappedLargeByteBuffer}. + */ public interface LargeByteBuffer { public byte get(); @@ -59,11 +83,11 @@ public interface LargeByteBuffer { /** * Creates a new byte buffer that shares this buffer's content. *

- *

The content of the new buffer will be that of this buffer. Changes + * The content of the new buffer will be that of this buffer. Changes * to this buffer's content will be visible in the new buffer, and vice * versa; the two buffers' positions will be independent. *

- *

The new buffer's position will be identical to those of this buffer + * The new buffer's position will be identical to those of this buffer */ public LargeByteBuffer duplicate(); @@ -72,8 +96,6 @@ public interface LargeByteBuffer { /** * Total number of bytes in this buffer - * - * @return */ public long size(); @@ -98,13 +120,12 @@ public interface LargeByteBuffer { * call will return a new ByteBuffer, but will not require copying the data (eg., it will * use ByteBuffer#duplicate()). The returned byte buffer will share data with this buffer. * - * @return * @throws BufferTooLargeException if this buffer is too large to fit in one {@link ByteBuffer} */ public ByteBuffer asByteBuffer() throws BufferTooLargeException; /** - * Attempt to clean up if it is memory-mapped. This uses an *unsafe* Sun API that + * Attempt to clean this up if it is memory-mapped. This uses an *unsafe* Sun API that * might cause errors if one attempts to read from the unmapped buffer, but it's better than * waiting for the GC to find it because that could lead to huge numbers of open files. There's * unfortunately no standard API to do this. 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 ce2ff266649e8..221dd75c36681 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 @@ -16,27 +16,32 @@ */ package org.apache.spark.network.buffer; -import com.google.common.annotations.VisibleForTesting; - import java.io.IOException; import java.nio.ByteBuffer; import java.nio.channels.FileChannel; -import java.util.ArrayList; +import com.google.common.annotations.VisibleForTesting; + +/** + * Utils for creating {@link org.apache.spark.network.buffer.LargeByteBuffer}s, either from + * pre-allocated byte arrays, ByteBuffers, or by memory mapping a file. + */ public class LargeByteBufferHelper { - public static final int MAX_CHUNK = Integer.MAX_VALUE - 1000000; + // netty can't quite send msgs that are a full 2GB -- they need to be slightly smaller + // not sure what the exact limit is, but 200 seems OK. + public static final int MAX_CHUNK_SIZE = Integer.MAX_VALUE - 200; 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) } ); + return asLargeByteBuffer(ByteBuffer.wrap(bytes)); } public static LargeByteBuffer allocate(long size) { - return allocate(size, MAX_CHUNK); + return allocate(size, MAX_CHUNK_SIZE); } @VisibleForTesting @@ -50,30 +55,27 @@ static LargeByteBuffer allocate(long size, int maxChunk) { remaining -= nextSize; chunks[i] = next; } - if (remaining != 0) throw new IllegalStateException("remaining = " + remaining); + if (remaining != 0) { + throw new IllegalStateException("remaining = " + remaining); + } return new WrappedLargeByteBuffer(chunks, maxChunk); } public static LargeByteBuffer mapFile( - FileChannel channel, - FileChannel.MapMode mode, - long offset, - long length + FileChannel channel, + FileChannel.MapMode mode, + long offset, + long length ) throws IOException { - int maxChunk = MAX_CHUNK; - ArrayList offsets = new ArrayList(); - long curOffset = offset; + int chunksNeeded = (int) ((length - 1) / MAX_CHUNK_SIZE) + 1; + ByteBuffer[] chunks = new ByteBuffer[chunksNeeded]; + long curPos = offset; long end = offset + length; - while (curOffset < end) { - offsets.add(curOffset); - int chunkLength = (int) Math.min((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)); + for (int i = 0; i < chunksNeeded; i++) { + long nextPos = Math.min(curPos + MAX_CHUNK_SIZE, end); + chunks[i] = channel.map(mode, curPos, nextPos - curPos); + curPos = nextPos; } return new WrappedLargeByteBuffer(chunks); } 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 56fba1f179031..9038d914c0294 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 @@ -16,28 +16,34 @@ */ package org.apache.spark.network.buffer; -import com.google.common.annotations.VisibleForTesting; -import sun.nio.ch.DirectBuffer; - import java.io.IOException; import java.nio.BufferUnderflowException; import java.nio.ByteBuffer; -import java.nio.MappedByteBuffer; import java.nio.channels.WritableByteChannel; import java.util.Arrays; import java.util.List; +import com.google.common.annotations.VisibleForTesting; +import sun.nio.ch.DirectBuffer; + +/** + * A {@link org.apache.spark.network.buffer.LargeByteBuffer} which may contain multiple + * {@link java.nio.ByteBuffer}s. In order to support asByteBuffer, all + * of the underlying ByteBuffers must have size equal to + * {@link org.apache.spark.network.buffer.LargeByteBufferHelper#MAX_CHUNK_SIZE} (except that last + * one). The underlying ByteBuffers may be on-heap, direct, or memory-mapped. + */ public class WrappedLargeByteBuffer implements LargeByteBuffer { @VisibleForTesting - public final ByteBuffer[] underlying; + final ByteBuffer[] underlying; private final long size; /** * each sub-ByteBuffer (except for the last one) must be exactly this size. Note that this - * class *really* expects this to be LargeByteBufferHelper.MAX_CHUNK. The only reason it isn't + * class *really* expects this to be LargeByteBufferHelper.MAX_CHUNK_SIZE. The only reason it isn't * is so that we can do tests without creating ginormous buffers. Public methods force it to - * be LargeByteBufferHelper.MAX_CHUNK + * be LargeByteBufferHelper.MAX_CHUNK_SIZE */ private final int subBufferSize; private long _pos; @@ -46,9 +52,18 @@ public class WrappedLargeByteBuffer implements LargeByteBuffer { @VisibleForTesting ByteBuffer currentBuffer; - + /** + * Construct a WrappedLargeByteBuffer from the given ByteBuffers. Each of the ByteBuffers must + * have size equal to {@link org.apache.spark.network.buffer.LargeByteBufferHelper#MAX_CHUNK_SIZE} + * except for the final one. The buffers are duplicated, so the position of the + * given buffers and the returned buffer will be independent, though the underlying data will be + * shared. + *

+ * The position of the returned buffer is determined by the position of the given + * buffers. TODO + */ public WrappedLargeByteBuffer(ByteBuffer[] underlying) { - this(underlying, LargeByteBufferHelper.MAX_CHUNK); + this(underlying, LargeByteBufferHelper.MAX_CHUNK_SIZE); } /** @@ -69,6 +84,13 @@ public WrappedLargeByteBuffer(ByteBuffer[] underlying) { long sum = 0L; boolean startFound = false; long initialPosition = -1; + + // figure out the position in this LargeByteBuffer, by looking at the positions of the each + // of the given ByteBuffers. The ByteBuffers need to have positions that are consistent + // with each other. Eg., say we have 5 ByteBuffers, and the position is somewhere in the + // middle of ByteBuffer 2. Then ByteBuffers 0 & 1 must have position == capacity, + // and ByteBuffers 3 & 4 must have position == 0 + for (int i = 0; i < underlying.length; i++) { ByteBuffer b = underlying[i]; if (i != underlying.length -1 && b.capacity() != subBufferSize) { @@ -137,6 +159,9 @@ public WrappedLargeByteBuffer deepCopy() { @Override public byte get() { + if (remaining() < 1L) { + throw new BufferUnderflowException(); + } byte r = currentBuffer.get(); _pos += 1; updateCurrentBuffer(); @@ -160,7 +185,7 @@ public long skip(long n) { if (n < 0) { final long moveTotal = Math.min(-n, _pos); long toMove = moveTotal; - // move backwards -- set the position to 0 of every buffer's we go back + // move backwards and update the position of every buffer as we go if (currentBuffer != null) { currentBufferIdx += 1; } @@ -176,7 +201,7 @@ public long skip(long n) { } else if (n > 0) { final long moveTotal = Math.min(n, remaining()); long toMove = moveTotal; - // move forwards -- set the position to the end of every buffer as we go forwards + // move forwards and update the position of every buffer as we go currentBufferIdx -= 1; while (toMove > 0) { currentBufferIdx += 1; @@ -213,7 +238,7 @@ public long size() { @Override public long writeTo(WritableByteChannel channel) throws IOException { - long written = 0l; + long written = 0L; for (; currentBufferIdx < underlying.length; currentBufferIdx++) { currentBuffer = underlying[currentBufferIdx]; written += currentBuffer.remaining(); @@ -240,7 +265,7 @@ public ByteBuffer asByteBuffer() throws BufferTooLargeException { } @VisibleForTesting - public List nioBuffers() { + List nioBuffers() { return Arrays.asList(underlying); } 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 4b15c42595e02..d46acf342a79b 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 @@ -16,11 +16,11 @@ */ package org.apache.spark.network.buffer; -import org.junit.Test; - import java.io.*; import java.nio.channels.FileChannel; +import org.junit.Test; + import static org.junit.Assert.*; public class LargeByteBufferHelperSuite { @@ -28,44 +28,47 @@ public class LargeByteBufferHelperSuite { @Test public void testMapFile() throws IOException { File testFile = File.createTempFile("large-byte-buffer-test", ".bin"); - testFile.deleteOnExit(); - OutputStream out = new FileOutputStream(testFile); - byte[] buffer = new byte[1 << 16]; - long len = 3L << 30; - assertTrue(len > Integer.MAX_VALUE); // its 1.5x Integer.MAX_VALUE, just a sanity check - for (int i = 0; i < buffer.length; i++) { - buffer[i] = (byte) i; - } - for (int i = 0; i < len / buffer.length; i++) { - out.write(buffer); - } - out.close(); + try { + testFile.deleteOnExit(); + OutputStream out = new FileOutputStream(testFile); + byte[] buffer = new byte[1 << 16]; + 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++) { + out.write(buffer); + } + out.close(); - FileChannel in = new FileInputStream(testFile).getChannel(); + FileChannel in = new FileInputStream(testFile).getChannel(); - //fail quickly on bad bounds - try { - LargeByteBufferHelper.mapFile(in, FileChannel.MapMode.READ_ONLY, 0, len + 1); - fail("expected exception"); - } catch (IOException ioe) { - } - try { - LargeByteBufferHelper.mapFile(in, FileChannel.MapMode.READ_ONLY, -1, 10); - fail("expected exception"); - } catch (IllegalArgumentException iae) { - } + //fail quickly on bad bounds + try { + LargeByteBufferHelper.mapFile(in, FileChannel.MapMode.READ_ONLY, 0, len + 1); + fail("expected exception"); + } catch (IOException ioe) { + } + try { + LargeByteBufferHelper.mapFile(in, FileChannel.MapMode.READ_ONLY, -1, 10); + fail("expected exception"); + } catch (IllegalArgumentException iae) { + } - //now try to read from the buffer - LargeByteBuffer buf = LargeByteBufferHelper.mapFile(in, FileChannel.MapMode.READ_ONLY, 0, len); - assertEquals(len, buf.size()); - byte[] read = new byte[buffer.length]; - for (int i = 0; i < len / buffer.length; i++) { - buf.get(read, 0, buffer.length); - // assertArrayEquals() is really slow - for (int j = 0; j < buffer.length; j++) { - if (read[j] != (byte)(j)) - fail("bad byte at (i,j) = (" + i + "," + j + ")"); + //now try to read from the buffer + LargeByteBuffer buf = LargeByteBufferHelper.mapFile(in, FileChannel.MapMode.READ_ONLY, 0, len); + assertEquals(len, buf.size()); + byte[] read = new byte[buffer.length]; + for (int i = 0; i < len / buffer.length; i++) { + buf.get(read, 0, buffer.length); + // assertArrayEquals() is really slow + for (int j = 0; j < buffer.length; j++) { + if (read[j] != (byte) (j)) + fail("bad byte at (i,j) = (" + i + "," + j + ")"); + } } + } finally { + testFile.delete(); } } 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 14ee9f19cbac4..5aca657b59ac2 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 @@ -29,7 +29,7 @@ public class WrappedLargeByteBufferSuite { - byte[] data = new byte[500]; + private byte[] data = new byte[500]; { new Random(1234).nextBytes(data); } @@ -159,6 +159,18 @@ public void get() { fail("expected exception"); } catch (BufferUnderflowException bue) { } + b.rewind(); + b.skip(495); + assertEquals(data[495], b.get()); + assertEquals(data[496], b.get()); + assertEquals(data[497], b.get()); + assertEquals(data[498], b.get()); + assertEquals(data[499], b.get()); + try { + b.get(); + fail("expected exception"); + } catch (BufferUnderflowException bue) { + } } @Test @@ -241,6 +253,11 @@ public void constructWithBuffersWithNonZeroPosition() { } } + @Test + public void positionsIndependentAfterConstructor() { + fail("TODO"); + } + @Test(expected=IllegalArgumentException.class) public void testRequireAtLeastOneBuffer() { new WrappedLargeByteBuffer( new ByteBuffer[0]); From 2afb351a1e4615230863513e52b7a8a96f6b0a8b Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Mon, 1 Jun 2015 11:20:11 -0500 Subject: [PATCH 13/29] constructed WrappedLargeByteBuffer always has position==0, simplifies things a lot --- .../buffer/WrappedLargeByteBuffer.java | 35 +++++------------ .../buffer/WrappedLargeByteBufferSuite.java | 39 ------------------- 2 files changed, 10 insertions(+), 64 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 9038d914c0294..7c2af1d0be452 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 @@ -57,10 +57,7 @@ public class WrappedLargeByteBuffer implements LargeByteBuffer { * have size equal to {@link org.apache.spark.network.buffer.LargeByteBufferHelper#MAX_CHUNK_SIZE} * except for the final one. The buffers are duplicated, so the position of the * given buffers and the returned buffer will be independent, though the underlying data will be - * shared. - *

- * The position of the returned buffer is determined by the position of the given - * buffers. TODO + * shared. The constructed buffer will always have position == 0. */ public WrappedLargeByteBuffer(ByteBuffer[] underlying) { this(underlying, LargeByteBufferHelper.MAX_CHUNK_SIZE); @@ -79,37 +76,23 @@ public WrappedLargeByteBuffer(ByteBuffer[] underlying) { if (underlying.length == 0) { throw new IllegalArgumentException("must wrap at least one ByteBuffer"); } - this.underlying = underlying; + this.underlying = new ByteBuffer[underlying.length]; this.subBufferSize = subBufferSize; long sum = 0L; - boolean startFound = false; - long initialPosition = -1; - - // figure out the position in this LargeByteBuffer, by looking at the positions of the each - // of the given ByteBuffers. The ByteBuffers need to have positions that are consistent - // with each other. Eg., say we have 5 ByteBuffers, and the position is somewhere in the - // middle of ByteBuffer 2. Then ByteBuffers 0 & 1 must have position == capacity, - // and ByteBuffers 3 & 4 must have position == 0 for (int i = 0; i < underlying.length; i++) { - ByteBuffer b = underlying[i]; + ByteBuffer b = underlying[i].duplicate(); + b.position(0); + this.underlying[i] = b; if (i != underlying.length -1 && b.capacity() != subBufferSize) { throw new IllegalArgumentException("All buffers, except for the final one, must have " + "size = " + subBufferSize); } - if (startFound) { - if (b.position() != 0) { - throw new IllegalArgumentException("ByteBuffers have inconsistent positions"); - } - } else if (b.position() != b.capacity()) { - startFound = true; - initialPosition = sum + b.position(); - } sum += b.capacity(); } - _pos = initialPosition; + _pos = 0; currentBufferIdx = 0; - currentBuffer = underlying[0]; + currentBuffer = this.underlying[0]; size = sum; } @@ -228,7 +211,9 @@ public WrappedLargeByteBuffer duplicate() { for (int i = 0; i < underlying.length; i++) { duplicates[i] = underlying[i].duplicate(); } - return new WrappedLargeByteBuffer(duplicates, subBufferSize); + WrappedLargeByteBuffer dup = new WrappedLargeByteBuffer(duplicates, subBufferSize); + dup.skip(position()); + return dup; } @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 5aca657b59ac2..5de3f68fe903b 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 @@ -219,45 +219,6 @@ public void duplicate() { } } - @Test - public void constructWithBuffersWithNonZeroPosition() { - ByteBuffer[] bufs = testDataBuf().underlying; - - bufs[0].position(50); - bufs[1].position(5); - - WrappedLargeByteBuffer b1 = new WrappedLargeByteBuffer(bufs, 50); - assertEquals(55, b1.position()); - - - bufs[1].position(50); - bufs[2].position(50); - bufs[3].position(35); - WrappedLargeByteBuffer b2 = new WrappedLargeByteBuffer(bufs, 50); - assertEquals(185, b2.position()); - - - bufs[5].position(16); - try { - new WrappedLargeByteBuffer(bufs); - fail("expected exception"); - } catch (IllegalArgumentException ex) { - } - - bufs[5].position(0); - bufs[0].position(49); - try { - new WrappedLargeByteBuffer(bufs); - fail("expected exception"); - } catch (IllegalArgumentException ex) { - } - } - - @Test - public void positionsIndependentAfterConstructor() { - fail("TODO"); - } - @Test(expected=IllegalArgumentException.class) public void testRequireAtLeastOneBuffer() { new WrappedLargeByteBuffer( new ByteBuffer[0]); From 4042c1ad7f9370a78d9b88188861f4705f43f5e1 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Mon, 1 Jun 2015 11:39:02 -0500 Subject: [PATCH 14/29] move LargeBBIn/Out Streams to java --- .../buffer/LargeByteBufferInputStream.java} | 59 +++++++++------- .../buffer/LargeByteBufferOutputStream.java | 69 +++++++++++++++++++ .../buffer/LargeByteBufferOutputStream.scala | 64 ----------------- .../LargeByteBufferInputStreamSuite.scala | 8 +-- 4 files changed, 107 insertions(+), 93 deletions(-) rename core/src/main/{scala/org/apache/spark/network/buffer/LargeByteBufferInputStream.scala => java/org/apache/spark/network/buffer/LargeByteBufferInputStream.java} (55%) create mode 100644 core/src/main/java/org/apache/spark/network/buffer/LargeByteBufferOutputStream.java delete mode 100644 core/src/main/scala/org/apache/spark/network/buffer/LargeByteBufferOutputStream.scala diff --git a/core/src/main/scala/org/apache/spark/network/buffer/LargeByteBufferInputStream.scala b/core/src/main/java/org/apache/spark/network/buffer/LargeByteBufferInputStream.java similarity index 55% rename from core/src/main/scala/org/apache/spark/network/buffer/LargeByteBufferInputStream.scala rename to core/src/main/java/org/apache/spark/network/buffer/LargeByteBufferInputStream.java index f00899ead1ab0..a313951990e81 100644 --- a/core/src/main/scala/org/apache/spark/network/buffer/LargeByteBufferInputStream.scala +++ b/core/src/main/java/org/apache/spark/network/buffer/LargeByteBufferInputStream.java @@ -14,62 +14,75 @@ * See the License for the specific language governing permissions and * limitations under the License. */ +package org.apache.spark.network.buffer; -package org.apache.spark.network.buffer +import java.io.InputStream; -import java.io.InputStream +import com.google.common.annotations.VisibleForTesting; /** * Reads data from a LargeByteBuffer, and optionally cleans it up using buffer.dispose() * when the stream is closed (e.g. to close a memory-mapped file). */ -private[spark] -class LargeByteBufferInputStream(private var buffer: LargeByteBuffer, dispose: Boolean = false) - extends InputStream { +public class LargeByteBufferInputStream extends InputStream { - override def read(): Int = { + LargeByteBuffer buffer; + final boolean dispose; + + public LargeByteBufferInputStream(LargeByteBuffer buffer, boolean dispose) { + this.buffer = buffer; + this.dispose = dispose; + } + + public LargeByteBufferInputStream(LargeByteBuffer buffer) { + this(buffer, false); + } + + + public int read() { if (buffer == null || buffer.remaining() == 0) { - -1 + return -1; } else { - buffer.get() & 0xFF + return buffer.get() & 0xFF; } } - override def read(dest: Array[Byte]): Int = { - read(dest, 0, dest.length) + public int read(byte[] dest) { + return read(dest, 0, dest.length); } - override def read(dest: Array[Byte], offset: Int, length: Int): Int = { + public int read(byte[] dest, int offset, int length) { if (buffer == null || buffer.remaining() == 0) { - -1 + return -1; } else { - val amountToGet = math.min(buffer.remaining(), length).toInt - buffer.get(dest, offset, amountToGet) - amountToGet + int amountToGet = (int) Math.min(buffer.remaining(), length); + buffer.get(dest, offset, amountToGet); + return amountToGet; } } - override def skip(bytes: Long): Long = { + public long skip(long toSkip) { if (buffer != null) { - buffer.skip(bytes) + return buffer.skip(toSkip); } else { - 0L + return 0L; } } // only for testing - private[buffer] var disposed = false + @VisibleForTesting + boolean disposed = false; /** * Clean up the buffer, and potentially dispose of it */ - override def close() { + public void close() { if (buffer != null) { if (dispose) { - buffer.dispose() - disposed = true + buffer.dispose(); + disposed = true; } - buffer = null + buffer = null; } } } 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 new file mode 100644 index 0000000000000..739533992ecac --- /dev/null +++ b/core/src/main/java/org/apache/spark/network/buffer/LargeByteBufferOutputStream.java @@ -0,0 +1,69 @@ +/* + * 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.io.OutputStream; +import java.nio.ByteBuffer; + +import org.apache.spark.util.io.ByteArrayChunkOutputStream; + +public class LargeByteBufferOutputStream extends OutputStream { + + private final int chunkSize; + final ByteArrayChunkOutputStream output; + + public LargeByteBufferOutputStream(int chunkSize) { + this.chunkSize = chunkSize; + output = new ByteArrayChunkOutputStream(chunkSize); + } + + public void write(int b) { + output.write(b); + } + + public void write(byte[] bytes, int off, int len) { + output.write(bytes, off, len); + } + + 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. + */ + LargeByteBuffer largeBuffer(int maxChunk) { + long totalSize = output.size(); + int chunksNeeded = (int) ((totalSize + maxChunk - 1) / maxChunk); + ByteBuffer[] chunks = new ByteBuffer[chunksNeeded]; + long remaining = totalSize; + long pos = 0; + for (int idx = 0; idx < chunksNeeded; idx++) { + int nextSize = (int) Math.min(maxChunk, remaining); + chunks[idx] = ByteBuffer.wrap(output.slice(pos, pos + nextSize)); + pos += nextSize; + remaining -= nextSize; + } + return new WrappedLargeByteBuffer(chunks, maxChunk); + } + + public void close() throws IOException { + output.close(); + } +} \ No newline at end of file diff --git a/core/src/main/scala/org/apache/spark/network/buffer/LargeByteBufferOutputStream.scala b/core/src/main/scala/org/apache/spark/network/buffer/LargeByteBufferOutputStream.scala deleted file mode 100644 index e9f13a6b27166..0000000000000 --- a/core/src/main/scala/org/apache/spark/network/buffer/LargeByteBufferOutputStream.scala +++ /dev/null @@ -1,64 +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.io.OutputStream -import java.nio.ByteBuffer - -import org.apache.spark.util.io.ByteArrayChunkOutputStream - -private[spark] class LargeByteBufferOutputStream(chunkSize: Int = 65536) - extends OutputStream { - - private[buffer] val output = new ByteArrayChunkOutputStream(chunkSize) - - 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) - } - - def largeBuffer: LargeByteBuffer = { - 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. - */ - private[buffer] def largeBuffer(maxChunk: Int): WrappedLargeByteBuffer = { - 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), maxChunk) - } - - override def close(): Unit = { - output.close() - } -} 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 50051e803eb2f..78e06c0381c78 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 @@ -17,14 +17,11 @@ package org.apache.spark.network.buffer import java.io.{FileInputStream, FileOutputStream, OutputStream, File} -import java.nio.ByteBuffer import java.nio.channels.FileChannel.MapMode import org.junit.Assert._ import org.scalatest.{FunSuite, Matchers} -import org.apache.spark.network.buffer.{LargeByteBuffer, LargeByteBufferHelper, WrappedLargeByteBuffer} - class LargeByteBufferInputStreamSuite extends FunSuite with Matchers { test("read from large mapped file") { @@ -44,7 +41,7 @@ class LargeByteBufferInputStreamSuite extends FunSuite with Matchers { val channel = new FileInputStream(testFile).getChannel val buf = LargeByteBufferHelper.mapFile(channel, MapMode.READ_ONLY, 0, len) - val in = new LargeByteBufferInputStream(buf, dispose = true) + val in = new LargeByteBufferInputStream(buf, true) val read = new Array[Byte](buffer.length) (0 until (len / buffer.length).toInt).foreach { idx => @@ -67,8 +64,7 @@ class LargeByteBufferInputStreamSuite extends FunSuite with Matchers { test("dispose on close") { // don't need to read to the end -- dispose anytime we close val data = new Array[Byte](10) - val in = new LargeByteBufferInputStream(LargeByteBufferHelper.asLargeByteBuffer(data), - dispose = true) + val in = new LargeByteBufferInputStream(LargeByteBufferHelper.asLargeByteBuffer(data), true) in.disposed should be (false) in.close() in.disposed should be (true) From 3c599b2f01e0e4d1ac2a9d02cf6a8e8924da488d Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Mon, 1 Jun 2015 12:00:03 -0500 Subject: [PATCH 15/29] add comments for MAX_CHUNK_SIZE --- .../org/apache/spark/network/buffer/LargeByteBuffer.java | 4 +++- .../apache/spark/network/buffer/LargeByteBufferHelper.java | 6 ++++++ 2 files changed, 9 insertions(+), 1 deletion(-) 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 87f695638ede6..fe03ec0a7e64c 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 @@ -118,7 +118,9 @@ public interface LargeByteBuffer { * Get the entire contents of this as one ByteBuffer, if possible. The returned ByteBuffer * will always have the position set to 0, and the limit set to the end of the data. Each * call will return a new ByteBuffer, but will not require copying the data (eg., it will - * use ByteBuffer#duplicate()). The returned byte buffer will share data with this buffer. + * use ByteBuffer#duplicate()). The returned byte buffer will share data with this buffer. The + * returned buffers will never be larger than + * {@link org.apache.spark.network.buffer.LargeByteBufferHelper#MAX_CHUNK_SIZE} * * @throws BufferTooLargeException if this buffer is too large to fit in one {@link ByteBuffer} */ 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 221dd75c36681..cd85f3365df57 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 @@ -30,6 +30,12 @@ public class LargeByteBufferHelper { // netty can't quite send msgs that are a full 2GB -- they need to be slightly smaller // not sure what the exact limit is, but 200 seems OK. + /** + * The maximum size of any ByteBuffer. + * {@link org.apache.spark.network.buffer.LargeByteBuffer#asByteBuffer} will never return a + * ByteBuffer larger than this. This is close to the max ByteBuffer size (2GB), minus a small + * amount for message overhead. + */ public static final int MAX_CHUNK_SIZE = Integer.MAX_VALUE - 200; public static LargeByteBuffer asLargeByteBuffer(ByteBuffer buffer) { From 95588c25bb40a7971ea05d6b261a1a044548feb9 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Mon, 1 Jun 2015 12:05:42 -0500 Subject: [PATCH 16/29] updateCurrentBuffer --> 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 17/29] 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 18/29] 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 19/29] 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 20/29] 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 21/29] 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 22/29] 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 23/29] 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 24/29] 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 25/29] 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 26/29] @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 27/29] 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 040a4615063aef46ce214fab4bd8b0eea7003c89 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 19 Aug 2015 10:47:44 -0500 Subject: [PATCH 28/29] 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 3447bb995b53c4d93154328c7c7c06e08a5ec9b9 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 4 Nov 2015 11:56:11 -0600 Subject: [PATCH 29/29] review feedback --- .../buffer/LargeByteBufferInputStream.java | 5 ----- .../buffer/LargeByteBufferOutputStream.java | 2 +- .../LargeByteBufferInputStreamSuite.scala | 20 ++++++++++--------- .../io/ByteArrayChunkOutputStreamSuite.scala | 8 ++++---- .../spark/network/buffer/LargeByteBuffer.java | 2 -- .../buffer/WrappedLargeByteBuffer.java | 1 - .../buffer/LargeByteBufferHelperSuite.java | 2 +- 7 files changed, 17 insertions(+), 23 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 a4b1e2571af56..dc8da016db3ed 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 @@ -72,10 +72,6 @@ public long skip(long toSkip) { } } - // only for testing - @VisibleForTesting - boolean disposed = false; - /** * Clean up the buffer, and potentially dispose of it */ @@ -84,7 +80,6 @@ public void close() { if (buffer != null) { if (dispose) { buffer.dispose(); - disposed = true; } buffer = null; } 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 975de7b10f65c..81de016781400 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 @@ -65,7 +65,7 @@ public LargeByteBuffer largeBuffer() { } /** - * exposed for testing. You don't really ever want to call this method -- the returned + * You don't really ever want to call this method -- the returned * buffer will not implement {{asByteBuffer}} correctly. */ @VisibleForTesting 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 d8e48db32f78c..44b458d6555cf 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 @@ -20,11 +20,13 @@ import java.io.{File, FileInputStream, FileOutputStream, OutputStream} import java.nio.channels.FileChannel.MapMode import org.junit.Assert._ +import org.mockito.Mockito._ import org.scalatest.Matchers +import org.scalatest.mock.MockitoSugar import org.apache.spark.SparkFunSuite -class LargeByteBufferInputStreamSuite extends SparkFunSuite with Matchers { +class LargeByteBufferInputStreamSuite extends SparkFunSuite with Matchers with MockitoSugar { test("read from large mapped file") { val testFile = File.createTempFile("large-buffer-input-stream-test", ".bin") @@ -47,17 +49,13 @@ class LargeByteBufferInputStreamSuite extends SparkFunSuite with Matchers { 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)) } } - in.disposed should be(false) in.read(read) should be(-1) - in.disposed should be(false) in.close() - in.disposed should be(true) } finally { testFile.delete() } @@ -65,11 +63,15 @@ class LargeByteBufferInputStreamSuite extends SparkFunSuite with Matchers { test("dispose on close") { // don't need to read to the end -- dispose anytime we close - val data = new Array[Byte](10) - val in = new LargeByteBufferInputStream(LargeByteBufferHelper.asLargeByteBuffer(data), true) - in.disposed should be (false) + val mockBuffer = mock[LargeByteBuffer] + when(mockBuffer.remaining()).thenReturn(0) + val in = new LargeByteBufferInputStream(mockBuffer, true) + verify(mockBuffer, times(0)).dispose() + // reading to the end shouldn't auto-dispose + in.read() should be (-1) + verify(mockBuffer, times(0)).dispose() in.close() - in.disposed should be (true) + verify(mockBuffer, times(1)).dispose() } test("io stream roundtrip") { 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 38bc24528f3a7..2bb0df394a9c2 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 @@ -126,9 +126,9 @@ class ByteArrayChunkOutputStreamSuite extends SparkFunSuite { } // errors on bad bounds - intercept[IllegalArgumentException]{o.slice(31, 31)} - intercept[IllegalArgumentException]{o.slice(-1, 10)} - intercept[IllegalArgumentException]{o.slice(10, 5)} - intercept[IllegalArgumentException]{o.slice(10, 35)} + intercept[IllegalArgumentException] { o.slice(31, 31) } + intercept[IllegalArgumentException] { o.slice(-1, 10) } + intercept[IllegalArgumentException] { o.slice(10, 5) } + intercept[IllegalArgumentException] { o.slice(10, 35) } } } 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 beeb007e2197e..21d954fcaeca8 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,7 +47,6 @@ 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 @@ -69,7 +68,6 @@ public interface LargeByteBuffer { */ 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 58a621249386f..859ad30100413 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,7 +101,6 @@ public WrappedLargeByteBuffer(ByteBuffer[] underlying) { size = sum; } - @Override public WrappedLargeByteBuffer get(byte[] dest) { return get(dest, 0, dest.length); 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 9e636fc032928..d0eb61be933c9 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 @@ -77,7 +77,7 @@ public void testMapFile() throws IOException { @Test public void testAllocate() { - WrappedLargeByteBuffer buf = (WrappedLargeByteBuffer) LargeByteBufferHelper.allocate(95,10); + WrappedLargeByteBuffer buf = (WrappedLargeByteBuffer) LargeByteBufferHelper.allocate(95, 10); assertEquals(10, buf.underlying.length); for (int i = 0 ; i < 9; i++) { assertEquals(10, buf.underlying[i].capacity());