-
Notifications
You must be signed in to change notification settings - Fork 28.4k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[SPARK-6190][core] create LargeByteBuffer for eliminating 2GB block limit #5400
Changes from all commits
9834942
4e1a842
8bd5606
ff9f968
a3dc811
da29c3d
366f921
34c2131
bf4ec0a
9d232d1
6b2f751
4da4626
2afb351
4042c1a
3c599b2
95588c2
b77bbe2
6c2a115
a9616e4
0250ac5
b3b6363
112c49e
8ec2c5c
d0605a1
b6620d0
54d09af
31244c8
ca23763
36d1801
040a461
80c4032
3f701dc
3447bb9
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,87 @@ | ||
/* | ||
* 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.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). | ||
*/ | ||
public class LargeByteBufferInputStream extends InputStream { | ||
|
||
private LargeByteBuffer buffer; | ||
private final boolean dispose; | ||
|
||
public LargeByteBufferInputStream(LargeByteBuffer buffer, boolean dispose) { | ||
this.buffer = buffer; | ||
this.dispose = dispose; | ||
} | ||
|
||
public LargeByteBufferInputStream(LargeByteBuffer buffer) { | ||
this(buffer, false); | ||
} | ||
|
||
@Override | ||
public int read() { | ||
if (buffer == null || buffer.remaining() == 0) { | ||
return -1; | ||
} else { | ||
return buffer.get() & 0xFF; | ||
} | ||
} | ||
|
||
@Override | ||
public int read(byte[] dest) { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I think that we should be adding There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. good idea, I've added |
||
return read(dest, 0, dest.length); | ||
} | ||
|
||
@Override | ||
public int read(byte[] dest, int offset, int length) { | ||
if (buffer == null || buffer.remaining() == 0) { | ||
return -1; | ||
} else { | ||
int amountToGet = (int) Math.min(buffer.remaining(), length); | ||
buffer.get(dest, offset, amountToGet); | ||
return amountToGet; | ||
} | ||
} | ||
|
||
@Override | ||
public long skip(long toSkip) { | ||
if (buffer != null) { | ||
return buffer.skip(toSkip); | ||
} else { | ||
return 0L; | ||
} | ||
} | ||
|
||
/** | ||
* Clean up the buffer, and potentially dispose of it | ||
*/ | ||
@Override | ||
public void close() { | ||
if (buffer != null) { | ||
if (dispose) { | ||
buffer.dispose(); | ||
} | ||
buffer = null; | ||
} | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,91 @@ | ||
/* | ||
* 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 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 <code>chunkSize</code> has <b>no effect</b> 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) { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. long? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
I'll add some docs to this class, it isn't very clear right now. |
||
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); | ||
} | ||
|
||
/** | ||
* 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); | ||
} | ||
|
||
/** | ||
* 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); | ||
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); | ||
} | ||
|
||
@Override | ||
public void close() throws IOException { | ||
output.close(); | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,95 @@ | ||
/* | ||
* 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.{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 with MockitoSugar { | ||
|
||
test("read from large mapped file") { | ||
val testFile = File.createTempFile("large-buffer-input-stream-test", ".bin") | ||
|
||
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, true) | ||
|
||
val read = new Array[Byte](buffer.length) | ||
(0 until (len / buffer.length).toInt).foreach { idx => | ||
in.read(read) should be(read.length) | ||
(0 until buffer.length).foreach { arrIdx => | ||
assertEquals(buffer(arrIdx), read(arrIdx)) | ||
} | ||
} | ||
in.read(read) should be(-1) | ||
in.close() | ||
} finally { | ||
testFile.delete() | ||
} | ||
} | ||
|
||
test("dispose on close") { | ||
// don't need to read to the end -- dispose anytime we close | ||
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() | ||
verify(mockBuffer, times(1)).dispose() | ||
} | ||
|
||
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) | ||
} | ||
} | ||
|
||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
remove the null checking for
buffer
?There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
the null check is for after the stream has been closed. Looks like
InputStream
doesn't specify whatread
should do after the stream has been closed, but this is consistent with what we do elsewhere eg.ByteBufferInputStream