Repository: hbase Updated Branches: refs/heads/master 522b8075f -> cbd8b15e6
HBASE-20197 Review of ByteBufferWriterOutputStream.java (BELUGA BEHR) Signed-off-by: Vasudevan <ramkrishna.s.vasude...@gmail.com> Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/cbd8b15e Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/cbd8b15e Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/cbd8b15e Branch: refs/heads/master Commit: cbd8b15e6b65d891961df11ccde67ff705f1d210 Parents: 522b807 Author: Vasudevan <ramkrishna.s.vasude...@intel.com> Authored: Fri Mar 30 12:13:41 2018 +0530 Committer: Vasudevan <ramkrishna.s.vasude...@intel.com> Committed: Fri Mar 30 12:13:41 2018 +0530 ---------------------------------------------------------------------- .../hbase/io/ByteBufferWriterOutputStream.java | 50 +++++++++++++++----- .../org/apache/hadoop/hbase/io/Reference.java | 4 +- 2 files changed, 40 insertions(+), 14 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/hbase/blob/cbd8b15e/hbase-common/src/main/java/org/apache/hadoop/hbase/io/ByteBufferWriterOutputStream.java ---------------------------------------------------------------------- diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/ByteBufferWriterOutputStream.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/ByteBufferWriterOutputStream.java index 6d46fa8..83635cb 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/ByteBufferWriterOutputStream.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/ByteBufferWriterOutputStream.java @@ -41,27 +41,51 @@ import org.apache.yetus.audience.InterfaceAudience; public class ByteBufferWriterOutputStream extends OutputStream implements ByteBufferWriter { - private static final int TEMP_BUF_LENGTH = 4 * 1024; + private static final int DEFAULT_BUFFER_SIZE = 4096; + private final OutputStream os; - private byte[] tempBuf = null; + private final int bufSize; + private byte[] buf; public ByteBufferWriterOutputStream(OutputStream os) { + this(os, DEFAULT_BUFFER_SIZE); + } + + public ByteBufferWriterOutputStream(OutputStream os, int size) { this.os = os; + this.bufSize = size; + this.buf = null; } + /** + * Writes len bytes from the specified ByteBuffer starting at offset off to + * this OutputStream. If b is null, a NullPointerException is thrown. If off + * is negative or larger than the ByteBuffer then an ArrayIndexOutOfBoundsException + * is thrown. If len is greater than the length of the ByteBuffer, then an + * ArrayIndexOutOfBoundsException is thrown. This method does not change the + * position of the ByteBuffer. + * + * @param b the ByteBuffer + * @param off the start offset in the data + * @param len the number of bytes to write + * @throws IOException + * if an I/O error occurs. In particular, an IOException is thrown + * if the output stream is closed. + */ @Override public void write(ByteBuffer b, int off, int len) throws IOException { - byte[] buf = null; - if (len > TEMP_BUF_LENGTH) { - buf = new byte[len]; - } else { - if (this.tempBuf == null) { - this.tempBuf = new byte[TEMP_BUF_LENGTH]; - } - buf = this.tempBuf; + // Lazily load in the event that this version of 'write' is not invoked + if (this.buf == null) { + this.buf = new byte[this.bufSize]; + } + int totalCopied = 0; + while (totalCopied < len) { + int bytesToCopy = Math.min((len - totalCopied), this.bufSize); + ByteBufferUtils.copyFromBufferToArray(this.buf, b, off + totalCopied, 0, + bytesToCopy); + this.os.write(this.buf, 0, bytesToCopy); + totalCopied += bytesToCopy; } - ByteBufferUtils.copyFromBufferToArray(buf, b, off, 0, len); - this.os.write(buf, 0, len); } @Override @@ -75,7 +99,7 @@ public class ByteBufferWriterOutputStream extends OutputStream } @Override - public void write(byte b[], int off, int len) throws IOException { + public void write(byte[] b, int off, int len) throws IOException { this.os.write(b, off, len); } http://git-wip-us.apache.org/repos/asf/hbase/blob/cbd8b15e/hbase-server/src/main/java/org/apache/hadoop/hbase/io/Reference.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/Reference.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/Reference.java index 6dce132..5a03e6c 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/Reference.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/Reference.java @@ -175,7 +175,9 @@ public class Reference { in.mark(pblen); byte [] pbuf = new byte[pblen]; int read = in.read(pbuf); - if (read != pblen) throw new IOException("read=" + read + ", wanted=" + pblen); + if (read != pblen) { + throw new IOException("read=" + read + ", wanted=" + pblen); + } // WATCHOUT! Return in middle of function!!! if (ProtobufUtil.isPBMagicPrefix(pbuf)) return convert(FSProtos.Reference.parseFrom(in)); // Else presume Writables. Need to reset the stream since it didn't start w/ pb.