Fix for zero-sized content throwing off toChannelBuffer(). Also
short-circuit writeTo(..) accordingly to avoid unnecessary work. Fixes #5543
This commit is contained in:
parent
374b633a4b
commit
ab3e22d17c
|
@ -84,30 +84,40 @@ public final class PagedBytesReference implements BytesReference {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void writeTo(OutputStream os) throws IOException {
|
public void writeTo(OutputStream os) throws IOException {
|
||||||
BytesRef ref = new BytesRef();
|
// nothing to do
|
||||||
int written = 0;
|
if (length == 0) {
|
||||||
|
return;
|
||||||
// are we a slice?
|
}
|
||||||
if (offset != 0) {
|
|
||||||
// remaining size of page fragment at offset
|
|
||||||
int fragmentSize = Math.min(length, PAGE_SIZE - (offset % PAGE_SIZE));
|
|
||||||
bytearray.get(offset, fragmentSize, ref);
|
|
||||||
os.write(ref.bytes, ref.offset, fragmentSize);
|
|
||||||
written += fragmentSize;
|
|
||||||
}
|
|
||||||
|
|
||||||
// handle remainder of pages + trailing fragment
|
BytesRef ref = new BytesRef();
|
||||||
while (written < length) {
|
int written = 0;
|
||||||
int remaining = length - written;
|
|
||||||
int bulkSize = (remaining > PAGE_SIZE) ? PAGE_SIZE : remaining;
|
// are we a slice?
|
||||||
bytearray.get(offset + written, bulkSize, ref);
|
if (offset != 0) {
|
||||||
os.write(ref.bytes, ref.offset, bulkSize);
|
// remaining size of page fragment at offset
|
||||||
written += bulkSize;
|
int fragmentSize = Math.min(length, PAGE_SIZE - (offset % PAGE_SIZE));
|
||||||
}
|
bytearray.get(offset, fragmentSize, ref);
|
||||||
|
os.write(ref.bytes, ref.offset, fragmentSize);
|
||||||
|
written += fragmentSize;
|
||||||
|
}
|
||||||
|
|
||||||
|
// handle remainder of pages + trailing fragment
|
||||||
|
while (written < length) {
|
||||||
|
int remaining = length - written;
|
||||||
|
int bulkSize = (remaining > PAGE_SIZE) ? PAGE_SIZE : remaining;
|
||||||
|
bytearray.get(offset + written, bulkSize, ref);
|
||||||
|
os.write(ref.bytes, ref.offset, bulkSize);
|
||||||
|
written += bulkSize;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void writeTo(GatheringByteChannel channel) throws IOException {
|
public void writeTo(GatheringByteChannel channel) throws IOException {
|
||||||
|
// nothing to do
|
||||||
|
if (length == 0) {
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
ByteBuffer[] buffers;
|
ByteBuffer[] buffers;
|
||||||
ByteBuffer currentBuffer = null;
|
ByteBuffer currentBuffer = null;
|
||||||
BytesRef ref = new BytesRef();
|
BytesRef ref = new BytesRef();
|
||||||
|
@ -201,6 +211,11 @@ public final class PagedBytesReference implements BytesReference {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public ChannelBuffer toChannelBuffer() {
|
public ChannelBuffer toChannelBuffer() {
|
||||||
|
// nothing to do
|
||||||
|
if (length == 0) {
|
||||||
|
return ChannelBuffers.EMPTY_BUFFER;
|
||||||
|
}
|
||||||
|
|
||||||
ChannelBuffer[] buffers;
|
ChannelBuffer[] buffers;
|
||||||
ChannelBuffer currentBuffer = null;
|
ChannelBuffer currentBuffer = null;
|
||||||
BytesRef ref = new BytesRef();
|
BytesRef ref = new BytesRef();
|
||||||
|
|
|
@ -19,9 +19,8 @@
|
||||||
|
|
||||||
package org.elasticsearch.common.io.stream;
|
package org.elasticsearch.common.io.stream;
|
||||||
|
|
||||||
import org.apache.lucene.util.BytesRef;
|
|
||||||
import org.elasticsearch.common.bytes.BytesArray;
|
|
||||||
import org.elasticsearch.common.bytes.BytesReference;
|
import org.elasticsearch.common.bytes.BytesReference;
|
||||||
|
import org.elasticsearch.common.bytes.PagedBytesReference;
|
||||||
import org.elasticsearch.common.io.BytesStream;
|
import org.elasticsearch.common.io.BytesStream;
|
||||||
import org.elasticsearch.common.util.BigArrays;
|
import org.elasticsearch.common.util.BigArrays;
|
||||||
import org.elasticsearch.common.util.ByteArray;
|
import org.elasticsearch.common.util.ByteArray;
|
||||||
|
@ -154,9 +153,7 @@ public class BytesStreamOutput extends StreamOutput implements BytesStream {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public BytesReference bytes() {
|
public BytesReference bytes() {
|
||||||
BytesRef bref = new BytesRef();
|
return new PagedBytesReference(bigarrays, bytes, count);
|
||||||
bytes.get(0, count, bref);
|
|
||||||
return new BytesArray(bref, false);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private void ensureCapacity(int offset) {
|
private void ensureCapacity(int offset) {
|
||||||
|
|
|
@ -283,6 +283,14 @@ public class PagedBytesReferenceTest extends ElasticsearchTestCase {
|
||||||
assertArrayEquals(pbr.toBytes(), bufferBytes);
|
assertArrayEquals(pbr.toBytes(), bufferBytes);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public void testEmptyToChannelBuffer() {
|
||||||
|
BytesReference pbr = getRandomizedPagedBytesReference(0);
|
||||||
|
ChannelBuffer cb = pbr.toChannelBuffer();
|
||||||
|
assertNotNull(cb);
|
||||||
|
assertEquals(0, pbr.length());
|
||||||
|
assertEquals(0, cb.capacity());
|
||||||
|
}
|
||||||
|
|
||||||
public void testSliceToChannelBuffer() {
|
public void testSliceToChannelBuffer() {
|
||||||
int length = randomIntBetween(10, PAGE_SIZE * randomIntBetween(2,8));
|
int length = randomIntBetween(10, PAGE_SIZE * randomIntBetween(2,8));
|
||||||
BytesReference pbr = getRandomizedPagedBytesReference(length);
|
BytesReference pbr = getRandomizedPagedBytesReference(length);
|
||||||
|
|
Loading…
Reference in New Issue