HBASE-20197 Review of ByteBufferWriterOutputStream.java (BELUGA BEHR)
Signed-off-by: Vasudevan <ramkrishna.s.vasudevan@gmail.com>
This commit is contained in:
parent
1170936f7b
commit
1687dc5ddb
|
@ -41,27 +41,51 @@ import org.apache.yetus.audience.InterfaceAudience;
|
||||||
public class ByteBufferWriterOutputStream extends OutputStream
|
public class ByteBufferWriterOutputStream extends OutputStream
|
||||||
implements ByteBufferWriter {
|
implements ByteBufferWriter {
|
||||||
|
|
||||||
private static final int TEMP_BUF_LENGTH = 4 * 1024;
|
private static final int DEFAULT_BUFFER_SIZE = 4096;
|
||||||
|
|
||||||
private final OutputStream os;
|
private final OutputStream os;
|
||||||
private byte[] tempBuf = null;
|
private final int bufSize;
|
||||||
|
private byte[] buf;
|
||||||
|
|
||||||
public ByteBufferWriterOutputStream(OutputStream os) {
|
public ByteBufferWriterOutputStream(OutputStream os) {
|
||||||
this.os = 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
|
@Override
|
||||||
public void write(ByteBuffer b, int off, int len) throws IOException {
|
public void write(ByteBuffer b, int off, int len) throws IOException {
|
||||||
byte[] buf = null;
|
// Lazily load in the event that this version of 'write' is not invoked
|
||||||
if (len > TEMP_BUF_LENGTH) {
|
if (this.buf == null) {
|
||||||
buf = new byte[len];
|
this.buf = new byte[this.bufSize];
|
||||||
} else {
|
}
|
||||||
if (this.tempBuf == null) {
|
int totalCopied = 0;
|
||||||
this.tempBuf = new byte[TEMP_BUF_LENGTH];
|
while (totalCopied < len) {
|
||||||
}
|
int bytesToCopy = Math.min((len - totalCopied), this.bufSize);
|
||||||
buf = this.tempBuf;
|
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
|
@Override
|
||||||
|
@ -75,7 +99,7 @@ public class ByteBufferWriterOutputStream extends OutputStream
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@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);
|
this.os.write(b, off, len);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -175,7 +175,9 @@ public class Reference {
|
||||||
in.mark(pblen);
|
in.mark(pblen);
|
||||||
byte [] pbuf = new byte[pblen];
|
byte [] pbuf = new byte[pblen];
|
||||||
int read = in.read(pbuf);
|
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!!!
|
// WATCHOUT! Return in middle of function!!!
|
||||||
if (ProtobufUtil.isPBMagicPrefix(pbuf)) return convert(FSProtos.Reference.parseFrom(in));
|
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.
|
// Else presume Writables. Need to reset the stream since it didn't start w/ pb.
|
||||||
|
|
Loading…
Reference in New Issue