Extract AbstractBytesReferenceTestCase (#19141)

We have a ton of tests for PagedBytesReference but not really many for the other
implementation of BytesReference. This change factors out a basic AbstractBytesReferenceTestCase
that simplifies testing other implementations. It also caught a couple of bug here and there like
a missing mask when reading bytes as ints in PagedBytesReference.
This commit is contained in:
Simon Willnauer 2016-06-29 14:45:54 +02:00 committed by GitHub
parent 39d38e513d
commit 819fe40d61
10 changed files with 646 additions and 485 deletions

View File

@ -27,7 +27,7 @@ import java.io.OutputStream;
import java.nio.charset.StandardCharsets;
import java.util.Arrays;
public class BytesArray implements BytesReference {
public final class BytesArray implements BytesReference {
public static final BytesArray EMPTY = new BytesArray(BytesRef.EMPTY_BYTES, 0, 0);

View File

@ -160,7 +160,7 @@ public interface BytesReference {
*/
default BytesRefIterator iterator() {
return new BytesRefIterator() {
BytesRef ref = toBytesRef();
BytesRef ref = length() == 0 ? null : toBytesRef();
@Override
public BytesRef next() throws IOException {
BytesRef r = ref;

View File

@ -297,7 +297,7 @@ public class PagedBytesReference implements BytesReference {
@Override
public int read() throws IOException {
return (pos < length) ? bytearray.get(offset + pos++) : -1;
return (pos < length) ? Byte.toUnsignedInt(bytearray.get(offset + pos++)) : -1;
}
@Override

View File

@ -51,7 +51,7 @@ final class ChannelBufferBytesReference implements BytesReference {
@Override
public BytesReference slice(int from, int length) {
return new ChannelBufferBytesReference(buffer.slice(from, length));
return new ChannelBufferBytesReference(buffer.slice(buffer.readerIndex() + from, length));
}
@Override

View File

@ -132,7 +132,7 @@ public class ChannelBufferStreamInput extends StreamInput {
public void readBytes(byte[] b, int offset, int len) throws IOException {
int read = read(b, offset, len);
if (read < len) {
throw new EOFException();
throw new IndexOutOfBoundsException();
}
}

View File

@ -0,0 +1,509 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch 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.elasticsearch.common.bytes;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
import org.apache.lucene.util.BytesRefIterator;
import org.elasticsearch.common.io.stream.BytesStreamOutput;
import org.elasticsearch.common.io.stream.ReleasableBytesStreamOutput;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.common.util.ByteArray;
import org.elasticsearch.indices.breaker.NoneCircuitBreakerService;
import org.elasticsearch.test.ESTestCase;
import org.hamcrest.Matchers;
import java.io.EOFException;
import java.io.IOException;
import java.util.Arrays;
public abstract class AbstractBytesReferenceTestCase extends ESTestCase {
protected static final int PAGE_SIZE = BigArrays.BYTE_PAGE_SIZE;
protected final BigArrays bigarrays = new BigArrays(null, new NoneCircuitBreakerService(), false);
public void testGet() throws IOException {
int length = randomIntBetween(1, PAGE_SIZE * 3);
BytesReference pbr = newBytesReference(length);
int sliceOffset = randomIntBetween(0, length / 2);
int sliceLength = Math.max(1, length - sliceOffset - 1);
BytesReference slice = pbr.slice(sliceOffset, sliceLength);
assertEquals(pbr.get(sliceOffset), slice.get(0));
assertEquals(pbr.get(sliceOffset + sliceLength - 1), slice.get(sliceLength - 1));
}
public void testLength() throws IOException {
int[] sizes = {0, randomInt(PAGE_SIZE), PAGE_SIZE, randomInt(PAGE_SIZE * 3)};
for (int i = 0; i < sizes.length; i++) {
BytesReference pbr = newBytesReference(sizes[i]);
assertEquals(sizes[i], pbr.length());
}
}
public void testSlice() throws IOException {
int length = randomInt(PAGE_SIZE * 3);
BytesReference pbr = newBytesReference(length);
int sliceOffset = randomIntBetween(0, length / 2);
int sliceLength = Math.max(0, length - sliceOffset - 1);
BytesReference slice = pbr.slice(sliceOffset, sliceLength);
assertEquals(sliceLength, slice.length());
if (slice.hasArray()) {
assertEquals(sliceOffset, slice.arrayOffset());
} else {
expectThrows(IllegalStateException.class, () ->
slice.arrayOffset());
}
}
public void testStreamInput() throws IOException {
int length = randomIntBetween(10, scaledRandomIntBetween(PAGE_SIZE * 2, PAGE_SIZE * 20));
BytesReference pbr = newBytesReference(length);
StreamInput si = pbr.streamInput();
assertNotNull(si);
// read single bytes one by one
assertEquals(pbr.get(0), si.readByte());
assertEquals(pbr.get(1), si.readByte());
assertEquals(pbr.get(2), si.readByte());
// reset the stream for bulk reading
si.reset();
// buffer for bulk reads
byte[] origBuf = new byte[length];
random().nextBytes(origBuf);
byte[] targetBuf = Arrays.copyOf(origBuf, origBuf.length);
// bulk-read 0 bytes: must not modify buffer
si.readBytes(targetBuf, 0, 0);
assertEquals(origBuf[0], targetBuf[0]);
si.reset();
// read a few few bytes as ints
int bytesToRead = randomIntBetween(1, length / 2);
for (int i = 0; i < bytesToRead; i++) {
int b = si.read();
assertEquals(pbr.get(i) & 0xff, b);
}
si.reset();
// bulk-read all
si.readFully(targetBuf);
assertArrayEquals(pbr.toBytes(), targetBuf);
// continuing to read should now fail with EOFException
try {
si.readByte();
fail("expected EOF");
} catch (EOFException | IndexOutOfBoundsException eof) {
// yay
}
// try to read more than the stream contains
si.reset();
expectThrows(IndexOutOfBoundsException.class, () ->
si.readBytes(targetBuf, 0, length * 2));
}
public void testStreamInputBulkReadWithOffset() throws IOException {
final int length = randomIntBetween(10, scaledRandomIntBetween(PAGE_SIZE * 2, PAGE_SIZE * 20));
BytesReference pbr = newBytesReference(length);
StreamInput si = pbr.streamInput();
assertNotNull(si);
// read a bunch of single bytes one by one
int offset = randomIntBetween(1, length / 2);
for (int i = 0; i < offset; i++) {
assertEquals(si.available(), length - i);
assertEquals(pbr.get(i), si.readByte());
}
// now do NOT reset the stream - keep the stream's offset!
// buffer to compare remaining bytes against bulk read
byte[] pbrBytesWithOffset = Arrays.copyOfRange(pbr.toBytes(), offset, length);
// randomized target buffer to ensure no stale slots
byte[] targetBytes = new byte[pbrBytesWithOffset.length];
random().nextBytes(targetBytes);
// bulk-read all
si.readFully(targetBytes);
assertArrayEquals(pbrBytesWithOffset, targetBytes);
assertEquals(si.available(), 0);
}
public void testRandomReads() throws IOException {
int length = randomIntBetween(10, scaledRandomIntBetween(PAGE_SIZE * 2, PAGE_SIZE * 20));
BytesReference pbr = newBytesReference(length);
StreamInput streamInput = pbr.streamInput();
BytesRefBuilder target = new BytesRefBuilder();
while (target.length() < pbr.length()) {
switch (randomIntBetween(0, 10)) {
case 6:
case 5:
target.append(new BytesRef(new byte[]{streamInput.readByte()}));
break;
case 4:
case 3:
BytesRef bytesRef = streamInput.readBytesRef(scaledRandomIntBetween(1, pbr.length() - target.length()));
target.append(bytesRef);
break;
default:
byte[] buffer = new byte[scaledRandomIntBetween(1, pbr.length() - target.length())];
int offset = scaledRandomIntBetween(0, buffer.length - 1);
int read = streamInput.read(buffer, offset, buffer.length - offset);
target.append(new BytesRef(buffer, offset, read));
break;
}
}
assertEquals(pbr.length(), target.length());
BytesRef targetBytes = target.get();
assertArrayEquals(pbr.toBytes(), Arrays.copyOfRange(targetBytes.bytes, targetBytes.offset, targetBytes.length));
}
public void testSliceStreamInput() throws IOException {
int length = randomIntBetween(10, scaledRandomIntBetween(PAGE_SIZE * 2, PAGE_SIZE * 20));
BytesReference pbr = newBytesReference(length);
// test stream input over slice (upper half of original)
int sliceOffset = randomIntBetween(1, length / 2);
int sliceLength = length - sliceOffset;
BytesReference slice = pbr.slice(sliceOffset, sliceLength);
StreamInput sliceInput = slice.streamInput();
assertEquals(sliceInput.available(), sliceLength);
// single reads
assertEquals(slice.get(0), sliceInput.readByte());
assertEquals(slice.get(1), sliceInput.readByte());
assertEquals(slice.get(2), sliceInput.readByte());
assertEquals(sliceInput.available(), sliceLength - 3);
// reset the slice stream for bulk reading
sliceInput.reset();
assertEquals(sliceInput.available(), sliceLength);
// bulk read
byte[] sliceBytes = new byte[sliceLength];
sliceInput.readFully(sliceBytes);
assertEquals(sliceInput.available(), 0);
// compare slice content with upper half of original
byte[] pbrSliceBytes = Arrays.copyOfRange(pbr.toBytes(), sliceOffset, length);
assertArrayEquals(pbrSliceBytes, sliceBytes);
// compare slice bytes with bytes read from slice via streamInput :D
byte[] sliceToBytes = slice.toBytes();
assertEquals(sliceBytes.length, sliceToBytes.length);
assertArrayEquals(sliceBytes, sliceToBytes);
sliceInput.reset();
assertEquals(sliceInput.available(), sliceLength);
byte[] buffer = new byte[sliceLength + scaledRandomIntBetween(1, 100)];
int offset = scaledRandomIntBetween(0, Math.max(1, buffer.length - sliceLength - 1));
int read = sliceInput.read(buffer, offset, sliceLength / 2);
assertEquals(sliceInput.available(), sliceLength - read);
sliceInput.read(buffer, offset + read, sliceLength - read);
assertArrayEquals(sliceBytes, Arrays.copyOfRange(buffer, offset, offset + sliceLength));
assertEquals(sliceInput.available(), 0);
}
public void testWriteToOutputStream() throws IOException {
int length = randomIntBetween(10, PAGE_SIZE * 4);
BytesReference pbr = newBytesReference(length);
BytesStreamOutput out = new BytesStreamOutput();
pbr.writeTo(out);
assertEquals(pbr.length(), out.size());
assertArrayEquals(pbr.toBytes(), out.bytes().toBytes());
out.close();
}
public void testSliceWriteToOutputStream() throws IOException {
int length = randomIntBetween(10, PAGE_SIZE * randomIntBetween(2, 5));
BytesReference pbr = newBytesReference(length);
int sliceOffset = randomIntBetween(1, length / 2);
int sliceLength = length - sliceOffset;
BytesReference slice = pbr.slice(sliceOffset, sliceLength);
BytesStreamOutput sliceOut = new BytesStreamOutput(sliceLength);
slice.writeTo(sliceOut);
assertEquals(slice.length(), sliceOut.size());
assertArrayEquals(slice.toBytes(), sliceOut.bytes().toBytes());
sliceOut.close();
}
public void testToBytes() throws IOException {
int[] sizes = {0, randomInt(PAGE_SIZE), PAGE_SIZE, randomIntBetween(2, PAGE_SIZE * randomIntBetween(2, 5))};
for (int i = 0; i < sizes.length; i++) {
BytesReference pbr = newBytesReference(sizes[i]);
byte[] bytes = pbr.toBytes();
assertEquals(sizes[i], bytes.length);
}
}
public void testToBytesArraySharedPage() throws IOException {
int length = randomIntBetween(10, PAGE_SIZE);
BytesReference pbr = newBytesReference(length);
BytesArray ba = pbr.toBytesArray();
BytesArray ba2 = pbr.toBytesArray();
assertNotNull(ba);
assertNotNull(ba2);
assertEquals(pbr.length(), ba.length());
assertEquals(ba.length(), ba2.length());
// single-page optimization
assertSame(ba.array(), ba2.array());
}
public void testToBytesArrayMaterializedPages() throws IOException {
// we need a length != (n * pagesize) to avoid page sharing at boundaries
int length = 0;
while ((length % PAGE_SIZE) == 0) {
length = randomIntBetween(PAGE_SIZE, PAGE_SIZE * randomIntBetween(2, 5));
}
BytesReference pbr = newBytesReference(length);
BytesArray ba = pbr.toBytesArray();
BytesArray ba2 = pbr.toBytesArray();
assertNotNull(ba);
assertNotNull(ba2);
assertEquals(pbr.length(), ba.length());
assertEquals(ba.length(), ba2.length());
}
public void testCopyBytesArray() throws IOException {
// small PBR which would normally share the first page
int length = randomIntBetween(10, PAGE_SIZE);
BytesReference pbr = newBytesReference(length);
BytesArray ba = pbr.copyBytesArray();
BytesArray ba2 = pbr.copyBytesArray();
assertNotNull(ba);
assertNotSame(ba, ba2);
assertNotSame(ba.array(), ba2.array());
}
public void testSliceCopyBytesArray() throws IOException {
int length = randomIntBetween(10, PAGE_SIZE * randomIntBetween(2, 8));
BytesReference pbr = newBytesReference(length);
int sliceOffset = randomIntBetween(0, pbr.length());
int sliceLength = randomIntBetween(pbr.length() - sliceOffset, pbr.length() - sliceOffset);
BytesReference slice = pbr.slice(sliceOffset, sliceLength);
BytesArray ba1 = slice.copyBytesArray();
BytesArray ba2 = slice.copyBytesArray();
assertNotNull(ba1);
assertNotNull(ba2);
assertNotSame(ba1.array(), ba2.array());
assertArrayEquals(slice.toBytes(), ba1.array());
assertArrayEquals(slice.toBytes(), ba2.array());
assertArrayEquals(ba1.array(), ba2.array());
}
public void testEmptyToBytesRefIterator() throws IOException {
BytesReference pbr = newBytesReference(0);
assertNull(pbr.iterator().next());
}
public void testIterator() throws IOException {
int length = randomIntBetween(10, PAGE_SIZE * randomIntBetween(2, 8));
BytesReference pbr = newBytesReference(length);
BytesRefIterator iterator = pbr.iterator();
BytesRef ref;
BytesRefBuilder builder = new BytesRefBuilder();
while((ref = iterator.next()) != null) {
builder.append(ref);
}
assertArrayEquals(pbr.toBytes(), BytesRef.deepCopyOf(builder.toBytesRef()).bytes);
}
public void testSliceIterator() throws IOException {
int length = randomIntBetween(10, PAGE_SIZE * randomIntBetween(2, 8));
BytesReference pbr = newBytesReference(length);
int sliceOffset = randomIntBetween(0, pbr.length());
int sliceLength = randomIntBetween(pbr.length() - sliceOffset, pbr.length() - sliceOffset);
BytesReference slice = pbr.slice(sliceOffset, sliceLength);
BytesRefIterator iterator = slice.iterator();
BytesRef ref = null;
BytesRefBuilder builder = new BytesRefBuilder();
while((ref = iterator.next()) != null) {
builder.append(ref);
}
assertArrayEquals(slice.toBytes(), BytesRef.deepCopyOf(builder.toBytesRef()).bytes);
}
public void testIteratorRandom() throws IOException {
int length = randomIntBetween(10, PAGE_SIZE * randomIntBetween(2, 8));
BytesReference pbr = newBytesReference(length);
if (randomBoolean()) {
int sliceOffset = randomIntBetween(0, pbr.length());
int sliceLength = randomIntBetween(pbr.length() - sliceOffset, pbr.length() - sliceOffset);
pbr = pbr.slice(sliceOffset, sliceLength);
}
if (randomBoolean()) {
pbr = pbr.toBytesArray();
}
BytesRefIterator iterator = pbr.iterator();
BytesRef ref = null;
BytesRefBuilder builder = new BytesRefBuilder();
while((ref = iterator.next()) != null) {
builder.append(ref);
}
assertArrayEquals(pbr.toBytes(), BytesRef.deepCopyOf(builder.toBytesRef()).bytes);
}
public void testArray() throws IOException {
int[] sizes = {0, randomInt(PAGE_SIZE), PAGE_SIZE, randomIntBetween(2, PAGE_SIZE * randomIntBetween(2, 5))};
for (int i = 0; i < sizes.length; i++) {
BytesReference pbr = newBytesReference(sizes[i]);
byte[] array = pbr.array();
assertNotNull(array);
assertEquals(sizes[i], array.length);
assertSame(array, pbr.array());
}
}
public void testArrayOffset() throws IOException {
int length = randomInt(PAGE_SIZE * randomIntBetween(2, 5));
BytesReference pbr = newBytesReference(length);
if (pbr.hasArray()) {
assertEquals(0, pbr.arrayOffset());
} else {
expectThrows(IllegalStateException.class, () ->
pbr.arrayOffset());
}
}
public void testSliceArrayOffset() throws IOException {
int length = randomInt(PAGE_SIZE * randomIntBetween(2, 5));
BytesReference pbr = newBytesReference(length);
int sliceOffset = randomIntBetween(0, pbr.length());
int sliceLength = randomIntBetween(pbr.length() - sliceOffset, pbr.length() - sliceOffset);
BytesReference slice = pbr.slice(sliceOffset, sliceLength);
if (slice.hasArray()) {
assertEquals(sliceOffset, slice.arrayOffset());
} else {
expectThrows(IllegalStateException.class, () ->
slice.arrayOffset());
}
}
public void testToUtf8() throws IOException {
// test empty
BytesReference pbr = newBytesReference(0);
assertEquals("", pbr.toUtf8());
// TODO: good way to test?
}
public void testToBytesRef() throws IOException {
int length = randomIntBetween(0, PAGE_SIZE);
BytesReference pbr = newBytesReference(length);
BytesRef ref = pbr.toBytesRef();
assertNotNull(ref);
assertEquals(pbr.arrayOffset(), ref.offset);
assertEquals(pbr.length(), ref.length);
}
public void testSliceToBytesRef() throws IOException {
int length = randomIntBetween(0, PAGE_SIZE);
BytesReference pbr = newBytesReference(length);
// get a BytesRef from a slice
int sliceOffset = randomIntBetween(0, pbr.length());
int sliceLength = randomIntBetween(pbr.length() - sliceOffset, pbr.length() - sliceOffset);
BytesRef sliceRef = pbr.slice(sliceOffset, sliceLength).toBytesRef();
// note that these are only true if we have <= than a page, otherwise offset/length are shifted
assertEquals(sliceOffset, sliceRef.offset);
assertEquals(sliceLength, sliceRef.length);
}
public void testCopyBytesRef() throws IOException {
int length = randomIntBetween(0, PAGE_SIZE * randomIntBetween(2, 5));
BytesReference pbr = newBytesReference(length);
BytesRef ref = pbr.copyBytesRef();
assertNotNull(ref);
assertEquals(pbr.length(), ref.length);
}
public void testHashCode() throws IOException {
// empty content must have hash 1 (JDK compat)
BytesReference pbr = newBytesReference(0);
assertEquals(Arrays.hashCode(BytesRef.EMPTY_BYTES), pbr.hashCode());
// test with content
pbr = newBytesReference(randomIntBetween(0, PAGE_SIZE * randomIntBetween(2, 5)));
int jdkHash = Arrays.hashCode(pbr.toBytes());
int pbrHash = pbr.hashCode();
assertEquals(jdkHash, pbrHash);
// test hashes of slices
int sliceFrom = randomIntBetween(0, pbr.length());
int sliceLength = randomIntBetween(pbr.length() - sliceFrom, pbr.length() - sliceFrom);
BytesReference slice = pbr.slice(sliceFrom, sliceLength);
int sliceJdkHash = Arrays.hashCode(slice.toBytes());
int sliceHash = slice.hashCode();
assertEquals(sliceJdkHash, sliceHash);
}
public void testEquals() {
int length = randomIntBetween(100, PAGE_SIZE * randomIntBetween(2, 5));
ByteArray ba1 = bigarrays.newByteArray(length, false);
ByteArray ba2 = bigarrays.newByteArray(length, false);
// copy contents
for (long i = 0; i < length; i++) {
ba2.set(i, ba1.get(i));
}
// get refs & compare
BytesReference pbr = new PagedBytesReference(bigarrays, ba1, length);
BytesReference pbr2 = new PagedBytesReference(bigarrays, ba2, length);
assertEquals(pbr, pbr2);
}
public void testEqualsPeerClass() throws IOException {
int length = randomIntBetween(100, PAGE_SIZE * randomIntBetween(2, 5));
BytesReference pbr = newBytesReference(length);
BytesReference ba = new BytesArray(pbr.toBytes());
assertEquals(pbr, ba);
}
public void testSliceEquals() {
int length = randomIntBetween(100, PAGE_SIZE * randomIntBetween(2, 5));
ByteArray ba1 = bigarrays.newByteArray(length, false);
BytesReference pbr = new PagedBytesReference(bigarrays, ba1, length);
// test equality of slices
int sliceFrom = randomIntBetween(0, pbr.length());
int sliceLength = randomIntBetween(pbr.length() - sliceFrom, pbr.length() - sliceFrom);
BytesReference slice1 = pbr.slice(sliceFrom, sliceLength);
BytesReference slice2 = pbr.slice(sliceFrom, sliceLength);
assertArrayEquals(slice1.toBytes(), slice2.toBytes());
// test a slice with same offset but different length,
// unless randomized testing gave us a 0-length slice.
if (sliceLength > 0) {
BytesReference slice3 = pbr.slice(sliceFrom, sliceLength / 2);
assertFalse(Arrays.equals(slice1.toBytes(), slice3.toBytes()));
}
}
protected abstract BytesReference newBytesReference(int length) throws IOException;
}

View File

@ -0,0 +1,41 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch 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.elasticsearch.common.bytes;
import org.elasticsearch.common.io.stream.BytesStreamOutput;
import org.hamcrest.Matchers;
import java.io.IOException;
public class BytesArrayTests extends AbstractBytesReferenceTestCase {
@Override
protected BytesReference newBytesReference(int length) throws IOException {
// we know bytes stream output always creates a paged bytes reference, we use it to create randomized content
final BytesStreamOutput out = new BytesStreamOutput(length);
for (int i = 0; i < length; i++) {
out.writeByte((byte) random().nextInt(1 << 8));
}
assertEquals(length, out.size());
BytesArray ref = out.bytes().toBytesArray();
assertEquals(length, ref.length());
assertTrue(ref instanceof BytesArray);
assertThat(ref.length(), Matchers.equalTo(length));
return ref;
}
}

View File

@ -30,263 +30,24 @@ import org.elasticsearch.common.util.ByteArray;
import org.elasticsearch.indices.breaker.NoneCircuitBreakerService;
import org.elasticsearch.test.ESTestCase;
import org.hamcrest.Matchers;
import org.junit.After;
import org.junit.Before;
import java.io.EOFException;
import java.io.IOException;
import java.util.Arrays;
public class PagedBytesReferenceTests extends ESTestCase {
public class PagedBytesReferenceTests extends AbstractBytesReferenceTestCase {
private static final int PAGE_SIZE = BigArrays.BYTE_PAGE_SIZE;
private BigArrays bigarrays = new BigArrays(null, new NoneCircuitBreakerService(), false);
public void testGet() throws IOException {
int length = randomIntBetween(1, PAGE_SIZE * 3);
BytesReference pbr = getRandomizedPagedBytesReference(length);
int sliceOffset = randomIntBetween(0, length / 2);
int sliceLength = Math.max(1, length - sliceOffset - 1);
BytesReference slice = pbr.slice(sliceOffset, sliceLength);
assertEquals(pbr.get(sliceOffset), slice.get(0));
assertEquals(pbr.get(sliceOffset + sliceLength - 1), slice.get(sliceLength - 1));
}
public void testLength() throws IOException {
int[] sizes = {0, randomInt(PAGE_SIZE), PAGE_SIZE, randomInt(PAGE_SIZE * 3)};
for (int i = 0; i < sizes.length; i++) {
BytesReference pbr = getRandomizedPagedBytesReference(sizes[i]);
assertEquals(sizes[i], pbr.length());
protected BytesReference newBytesReference(int length) throws IOException {
// we know bytes stream output always creates a paged bytes reference, we use it to create randomized content
ReleasableBytesStreamOutput out = new ReleasableBytesStreamOutput(length, bigarrays);
for (int i = 0; i < length; i++) {
out.writeByte((byte) random().nextInt(1 << 8));
}
}
public void testSlice() throws IOException {
int length = randomInt(PAGE_SIZE * 3);
BytesReference pbr = getRandomizedPagedBytesReference(length);
int sliceOffset = randomIntBetween(0, length / 2);
int sliceLength = Math.max(0, length - sliceOffset - 1);
BytesReference slice = pbr.slice(sliceOffset, sliceLength);
assertEquals(sliceLength, slice.length());
if (slice.hasArray()) {
assertEquals(sliceOffset, slice.arrayOffset());
} else {
try {
slice.arrayOffset();
fail("expected IllegalStateException");
} catch (IllegalStateException ise) {
// expected
}
}
}
public void testStreamInput() throws IOException {
int length = randomIntBetween(10, scaledRandomIntBetween(PAGE_SIZE * 2, PAGE_SIZE * 20));
BytesReference pbr = getRandomizedPagedBytesReference(length);
StreamInput si = pbr.streamInput();
assertNotNull(si);
// read single bytes one by one
assertEquals(pbr.get(0), si.readByte());
assertEquals(pbr.get(1), si.readByte());
assertEquals(pbr.get(2), si.readByte());
// reset the stream for bulk reading
si.reset();
// buffer for bulk reads
byte[] origBuf = new byte[length];
random().nextBytes(origBuf);
byte[] targetBuf = Arrays.copyOf(origBuf, origBuf.length);
// bulk-read 0 bytes: must not modify buffer
si.readBytes(targetBuf, 0, 0);
assertEquals(origBuf[0], targetBuf[0]);
si.reset();
// read a few few bytes as ints
int bytesToRead = randomIntBetween(1, length / 2);
for (int i = 0; i < bytesToRead; i++) {
int b = si.read();
assertEquals(pbr.get(i), b);
}
si.reset();
// bulk-read all
si.readFully(targetBuf);
assertArrayEquals(pbr.toBytes(), targetBuf);
// continuing to read should now fail with EOFException
try {
si.readByte();
fail("expected EOF");
} catch (EOFException eof) {
// yay
}
// try to read more than the stream contains
si.reset();
try {
si.readBytes(targetBuf, 0, length * 2);
fail("expected IndexOutOfBoundsException: le > stream.length");
} catch (IndexOutOfBoundsException ioob) {
// expected
}
}
public void testStreamInputBulkReadWithOffset() throws IOException {
final int length = randomIntBetween(10, scaledRandomIntBetween(PAGE_SIZE * 2, PAGE_SIZE * 20));
BytesReference pbr = getRandomizedPagedBytesReference(length);
StreamInput si = pbr.streamInput();
assertNotNull(si);
// read a bunch of single bytes one by one
int offset = randomIntBetween(1, length / 2);
for (int i = 0; i < offset; i++) {
assertEquals(si.available(), length - i);
assertEquals(pbr.get(i), si.readByte());
}
// now do NOT reset the stream - keep the stream's offset!
// buffer to compare remaining bytes against bulk read
byte[] pbrBytesWithOffset = Arrays.copyOfRange(pbr.toBytes(), offset, length);
// randomized target buffer to ensure no stale slots
byte[] targetBytes = new byte[pbrBytesWithOffset.length];
random().nextBytes(targetBytes);
// bulk-read all
si.readFully(targetBytes);
assertArrayEquals(pbrBytesWithOffset, targetBytes);
assertEquals(si.available(), 0);
}
public void testRandomReads() throws IOException {
int length = randomIntBetween(10, scaledRandomIntBetween(PAGE_SIZE * 2, PAGE_SIZE * 20));
BytesReference pbr = getRandomizedPagedBytesReference(length);
StreamInput streamInput = pbr.streamInput();
BytesRefBuilder target = new BytesRefBuilder();
while (target.length() < pbr.length()) {
switch (randomIntBetween(0, 10)) {
case 6:
case 5:
target.append(new BytesRef(new byte[]{streamInput.readByte()}));
break;
case 4:
case 3:
BytesRef bytesRef = streamInput.readBytesRef(scaledRandomIntBetween(1, pbr.length() - target.length()));
target.append(bytesRef);
break;
default:
byte[] buffer = new byte[scaledRandomIntBetween(1, pbr.length() - target.length())];
int offset = scaledRandomIntBetween(0, buffer.length - 1);
int read = streamInput.read(buffer, offset, buffer.length - offset);
target.append(new BytesRef(buffer, offset, read));
break;
}
}
assertEquals(pbr.length(), target.length());
BytesRef targetBytes = target.get();
assertArrayEquals(pbr.toBytes(), Arrays.copyOfRange(targetBytes.bytes, targetBytes.offset, targetBytes.length));
}
public void testSliceStreamInput() throws IOException {
int length = randomIntBetween(10, scaledRandomIntBetween(PAGE_SIZE * 2, PAGE_SIZE * 20));
BytesReference pbr = getRandomizedPagedBytesReference(length);
// test stream input over slice (upper half of original)
int sliceOffset = randomIntBetween(1, length / 2);
int sliceLength = length - sliceOffset;
BytesReference slice = pbr.slice(sliceOffset, sliceLength);
StreamInput sliceInput = slice.streamInput();
assertEquals(sliceInput.available(), sliceLength);
// single reads
assertEquals(slice.get(0), sliceInput.readByte());
assertEquals(slice.get(1), sliceInput.readByte());
assertEquals(slice.get(2), sliceInput.readByte());
assertEquals(sliceInput.available(), sliceLength - 3);
// reset the slice stream for bulk reading
sliceInput.reset();
assertEquals(sliceInput.available(), sliceLength);
// bulk read
byte[] sliceBytes = new byte[sliceLength];
sliceInput.readFully(sliceBytes);
assertEquals(sliceInput.available(), 0);
// compare slice content with upper half of original
byte[] pbrSliceBytes = Arrays.copyOfRange(pbr.toBytes(), sliceOffset, length);
assertArrayEquals(pbrSliceBytes, sliceBytes);
// compare slice bytes with bytes read from slice via streamInput :D
byte[] sliceToBytes = slice.toBytes();
assertEquals(sliceBytes.length, sliceToBytes.length);
assertArrayEquals(sliceBytes, sliceToBytes);
sliceInput.reset();
assertEquals(sliceInput.available(), sliceLength);
byte[] buffer = new byte[sliceLength + scaledRandomIntBetween(1, 100)];
int offset = scaledRandomIntBetween(0, Math.max(1, buffer.length - sliceLength - 1));
int read = sliceInput.read(buffer, offset, sliceLength / 2);
assertEquals(sliceInput.available(), sliceLength - read);
sliceInput.read(buffer, offset + read, sliceLength);
assertArrayEquals(sliceBytes, Arrays.copyOfRange(buffer, offset, offset + sliceLength));
assertEquals(sliceInput.available(), 0);
}
public void testWriteToOutputStream() throws IOException {
int length = randomIntBetween(10, PAGE_SIZE * 4);
BytesReference pbr = getRandomizedPagedBytesReference(length);
BytesStreamOutput out = new BytesStreamOutput();
pbr.writeTo(out);
assertEquals(pbr.length(), out.size());
assertArrayEquals(pbr.toBytes(), out.bytes().toBytes());
out.close();
}
public void testSliceWriteToOutputStream() throws IOException {
int length = randomIntBetween(10, PAGE_SIZE * randomIntBetween(2, 5));
BytesReference pbr = getRandomizedPagedBytesReference(length);
int sliceOffset = randomIntBetween(1, length / 2);
int sliceLength = length - sliceOffset;
BytesReference slice = pbr.slice(sliceOffset, sliceLength);
BytesStreamOutput sliceOut = new BytesStreamOutput(sliceLength);
slice.writeTo(sliceOut);
assertEquals(slice.length(), sliceOut.size());
assertArrayEquals(slice.toBytes(), sliceOut.bytes().toBytes());
sliceOut.close();
}
public void testToBytes() throws IOException {
int[] sizes = {0, randomInt(PAGE_SIZE), PAGE_SIZE, randomIntBetween(2, PAGE_SIZE * randomIntBetween(2, 5))};
for (int i = 0; i < sizes.length; i++) {
BytesReference pbr = getRandomizedPagedBytesReference(sizes[i]);
byte[] bytes = pbr.toBytes();
assertEquals(sizes[i], bytes.length);
// verify that toBytes() is cheap for small payloads
if (sizes[i] <= PAGE_SIZE) {
assertSame(bytes, pbr.toBytes());
} else {
assertNotSame(bytes, pbr.toBytes());
}
}
}
public void testToBytesArraySharedPage() throws IOException {
int length = randomIntBetween(10, PAGE_SIZE);
BytesReference pbr = getRandomizedPagedBytesReference(length);
BytesArray ba = pbr.toBytesArray();
BytesArray ba2 = pbr.toBytesArray();
assertNotNull(ba);
assertNotNull(ba2);
assertEquals(pbr.length(), ba.length());
assertEquals(ba.length(), ba2.length());
// single-page optimization
assertSame(ba.array(), ba2.array());
assertThat(out.size(), Matchers.equalTo(length));
BytesReference ref = out.bytes();
assertThat(ref.length(), Matchers.equalTo(length));
assertThat(ref, Matchers.instanceOf(PagedBytesReference.class));
return ref;
}
public void testToBytesArrayMaterializedPages() throws IOException {
@ -295,7 +56,7 @@ public class PagedBytesReferenceTests extends ESTestCase {
while ((length % PAGE_SIZE) == 0) {
length = randomIntBetween(PAGE_SIZE, PAGE_SIZE * randomIntBetween(2, 5));
}
BytesReference pbr = getRandomizedPagedBytesReference(length);
BytesReference pbr = newBytesReference(length);
BytesArray ba = pbr.toBytesArray();
BytesArray ba2 = pbr.toBytesArray();
assertNotNull(ba);
@ -306,99 +67,11 @@ public class PagedBytesReferenceTests extends ESTestCase {
assertNotSame(ba.array(), ba2.array());
}
public void testCopyBytesArray() throws IOException {
// small PBR which would normally share the first page
int length = randomIntBetween(10, PAGE_SIZE);
BytesReference pbr = getRandomizedPagedBytesReference(length);
BytesArray ba = pbr.copyBytesArray();
BytesArray ba2 = pbr.copyBytesArray();
assertNotNull(ba);
assertNotSame(ba, ba2);
assertNotSame(ba.array(), ba2.array());
}
public void testSliceCopyBytesArray() throws IOException {
int length = randomIntBetween(10, PAGE_SIZE * randomIntBetween(2, 8));
BytesReference pbr = getRandomizedPagedBytesReference(length);
int sliceOffset = randomIntBetween(0, pbr.length());
int sliceLength = randomIntBetween(pbr.length() - sliceOffset, pbr.length() - sliceOffset);
BytesReference slice = pbr.slice(sliceOffset, sliceLength);
BytesArray ba1 = slice.copyBytesArray();
BytesArray ba2 = slice.copyBytesArray();
assertNotNull(ba1);
assertNotNull(ba2);
assertNotSame(ba1.array(), ba2.array());
assertArrayEquals(slice.toBytes(), ba1.array());
assertArrayEquals(slice.toBytes(), ba2.array());
assertArrayEquals(ba1.array(), ba2.array());
}
public void testEmptyToBytesRefIterator() throws IOException {
BytesReference pbr = getRandomizedPagedBytesReference(0);
assertNull(pbr.iterator().next());
}
public void testIterator() throws IOException {
int length = randomIntBetween(10, PAGE_SIZE * randomIntBetween(2, 8));
BytesReference pbr = getRandomizedPagedBytesReference(length);
BytesRefIterator iterator = pbr.iterator();
BytesRef ref;
BytesRefBuilder builder = new BytesRefBuilder();
while((ref = iterator.next()) != null) {
builder.append(ref);
}
assertArrayEquals(pbr.toBytes(), BytesRef.deepCopyOf(builder.toBytesRef()).bytes);
}
public void testSliceIterator() throws IOException {
int length = randomIntBetween(10, PAGE_SIZE * randomIntBetween(2, 8));
BytesReference pbr = getRandomizedPagedBytesReference(length);
int sliceOffset = randomIntBetween(0, pbr.length());
int sliceLength = randomIntBetween(pbr.length() - sliceOffset, pbr.length() - sliceOffset);
BytesReference slice = pbr.slice(sliceOffset, sliceLength);
BytesRefIterator iterator = slice.iterator();
BytesRef ref = null;
BytesRefBuilder builder = new BytesRefBuilder();
while((ref = iterator.next()) != null) {
builder.append(ref);
}
assertArrayEquals(slice.toBytes(), BytesRef.deepCopyOf(builder.toBytesRef()).bytes);
}
public void testIteratorRandom() throws IOException {
int length = randomIntBetween(10, PAGE_SIZE * randomIntBetween(2, 8));
BytesReference pbr = getRandomizedPagedBytesReference(length);
if (randomBoolean()) {
int sliceOffset = randomIntBetween(0, pbr.length());
int sliceLength = randomIntBetween(pbr.length() - sliceOffset, pbr.length() - sliceOffset);
pbr = pbr.slice(sliceOffset, sliceLength);
}
if (randomBoolean()) {
pbr = pbr.toBytesArray();
}
BytesRefIterator iterator = pbr.iterator();
BytesRef ref = null;
BytesRefBuilder builder = new BytesRefBuilder();
while((ref = iterator.next()) != null) {
builder.append(ref);
}
assertArrayEquals(pbr.toBytes(), BytesRef.deepCopyOf(builder.toBytesRef()).bytes);
}
public void testHasArray() throws IOException {
int length = randomIntBetween(10, PAGE_SIZE * randomIntBetween(1, 3));
BytesReference pbr = getRandomizedPagedBytesReference(length);
// must return true for <= pagesize
assertEquals(length <= PAGE_SIZE, pbr.hasArray());
}
public void testArray() throws IOException {
int[] sizes = {0, randomInt(PAGE_SIZE), PAGE_SIZE, randomIntBetween(2, PAGE_SIZE * randomIntBetween(2, 5))};
for (int i = 0; i < sizes.length; i++) {
BytesReference pbr = getRandomizedPagedBytesReference(sizes[i]);
BytesReference pbr = newBytesReference(sizes[i]);
// verify that array() is cheap for small payloads
if (sizes[i] <= PAGE_SIZE) {
byte[] array = pbr.array();
@ -416,149 +89,27 @@ public class PagedBytesReferenceTests extends ESTestCase {
}
}
public void testArrayOffset() throws IOException {
int length = randomInt(PAGE_SIZE * randomIntBetween(2, 5));
BytesReference pbr = getRandomizedPagedBytesReference(length);
if (pbr.hasArray()) {
assertEquals(0, pbr.arrayOffset());
} else {
try {
pbr.arrayOffset();
fail("expected IllegalStateException");
} catch (IllegalStateException ise) {
// expected
public void testToBytes() throws IOException {
int[] sizes = {0, randomInt(PAGE_SIZE), PAGE_SIZE, randomIntBetween(2, PAGE_SIZE * randomIntBetween(2, 5))};
for (int i = 0; i < sizes.length; i++) {
BytesReference pbr = newBytesReference(sizes[i]);
byte[] bytes = pbr.toBytes();
assertEquals(sizes[i], bytes.length);
// verify that toBytes() is cheap for small payloads
if (sizes[i] <= PAGE_SIZE) {
assertSame(bytes, pbr.toBytes());
} else {
assertNotSame(bytes, pbr.toBytes());
}
}
}
public void testSliceArrayOffset() throws IOException {
int length = randomInt(PAGE_SIZE * randomIntBetween(2, 5));
BytesReference pbr = getRandomizedPagedBytesReference(length);
int sliceOffset = randomIntBetween(0, pbr.length());
int sliceLength = randomIntBetween(pbr.length() - sliceOffset, pbr.length() - sliceOffset);
BytesReference slice = pbr.slice(sliceOffset, sliceLength);
if (slice.hasArray()) {
assertEquals(sliceOffset, slice.arrayOffset());
} else {
try {
slice.arrayOffset();
fail("expected IllegalStateException");
} catch (IllegalStateException ise) {
// expected
}
}
}
public void testToUtf8() throws IOException {
// test empty
BytesReference pbr = getRandomizedPagedBytesReference(0);
assertEquals("", pbr.toUtf8());
// TODO: good way to test?
}
public void testToBytesRef() throws IOException {
int length = randomIntBetween(0, PAGE_SIZE);
BytesReference pbr = getRandomizedPagedBytesReference(length);
BytesRef ref = pbr.toBytesRef();
assertNotNull(ref);
assertEquals(pbr.arrayOffset(), ref.offset);
assertEquals(pbr.length(), ref.length);
}
public void testSliceToBytesRef() throws IOException {
int length = randomIntBetween(0, PAGE_SIZE);
BytesReference pbr = getRandomizedPagedBytesReference(length);
// get a BytesRef from a slice
int sliceOffset = randomIntBetween(0, pbr.length());
int sliceLength = randomIntBetween(pbr.length() - sliceOffset, pbr.length() - sliceOffset);
BytesRef sliceRef = pbr.slice(sliceOffset, sliceLength).toBytesRef();
// note that these are only true if we have <= than a page, otherwise offset/length are shifted
assertEquals(sliceOffset, sliceRef.offset);
assertEquals(sliceLength, sliceRef.length);
}
public void testCopyBytesRef() throws IOException {
int length = randomIntBetween(0, PAGE_SIZE * randomIntBetween(2, 5));
BytesReference pbr = getRandomizedPagedBytesReference(length);
BytesRef ref = pbr.copyBytesRef();
assertNotNull(ref);
assertEquals(pbr.length(), ref.length);
}
public void testHashCode() throws IOException {
// empty content must have hash 1 (JDK compat)
BytesReference pbr = getRandomizedPagedBytesReference(0);
assertEquals(Arrays.hashCode(BytesRef.EMPTY_BYTES), pbr.hashCode());
// test with content
pbr = getRandomizedPagedBytesReference(randomIntBetween(0, PAGE_SIZE * randomIntBetween(2, 5)));
int jdkHash = Arrays.hashCode(pbr.toBytes());
int pbrHash = pbr.hashCode();
assertEquals(jdkHash, pbrHash);
// test hashes of slices
int sliceFrom = randomIntBetween(0, pbr.length());
int sliceLength = randomIntBetween(pbr.length() - sliceFrom, pbr.length() - sliceFrom);
BytesReference slice = pbr.slice(sliceFrom, sliceLength);
int sliceJdkHash = Arrays.hashCode(slice.toBytes());
int sliceHash = slice.hashCode();
assertEquals(sliceJdkHash, sliceHash);
}
public void testEquals() {
int length = randomIntBetween(100, PAGE_SIZE * randomIntBetween(2, 5));
ByteArray ba1 = bigarrays.newByteArray(length, false);
ByteArray ba2 = bigarrays.newByteArray(length, false);
// copy contents
for (long i = 0; i < length; i++) {
ba2.set(i, ba1.get(i));
}
// get refs & compare
BytesReference pbr = new PagedBytesReference(bigarrays, ba1, length);
BytesReference pbr2 = new PagedBytesReference(bigarrays, ba2, length);
assertEquals(pbr, pbr2);
}
public void testEqualsPeerClass() throws IOException {
int length = randomIntBetween(100, PAGE_SIZE * randomIntBetween(2, 5));
BytesReference pbr = getRandomizedPagedBytesReference(length);
BytesReference ba = new BytesArray(pbr.toBytes());
assertEquals(pbr, ba);
}
public void testSliceEquals() {
int length = randomIntBetween(100, PAGE_SIZE * randomIntBetween(2, 5));
ByteArray ba1 = bigarrays.newByteArray(length, false);
BytesReference pbr = new PagedBytesReference(bigarrays, ba1, length);
// test equality of slices
int sliceFrom = randomIntBetween(0, pbr.length());
int sliceLength = randomIntBetween(pbr.length() - sliceFrom, pbr.length() - sliceFrom);
BytesReference slice1 = pbr.slice(sliceFrom, sliceLength);
BytesReference slice2 = pbr.slice(sliceFrom, sliceLength);
assertArrayEquals(slice1.toBytes(), slice2.toBytes());
// test a slice with same offset but different length,
// unless randomized testing gave us a 0-length slice.
if (sliceLength > 0) {
BytesReference slice3 = pbr.slice(sliceFrom, sliceLength / 2);
assertFalse(Arrays.equals(slice1.toBytes(), slice3.toBytes()));
}
}
private BytesReference getRandomizedPagedBytesReference(int length) throws IOException {
// we know bytes stream output always creates a paged bytes reference, we use it to create randomized content
ReleasableBytesStreamOutput out = new ReleasableBytesStreamOutput(length, bigarrays);
for (int i = 0; i < length; i++) {
out.writeByte((byte) random().nextInt(1 << 8));
}
assertThat(out.size(), Matchers.equalTo(length));
BytesReference ref = out.bytes();
assertThat(ref.length(), Matchers.equalTo(length));
assertThat(ref, Matchers.instanceOf(PagedBytesReference.class));
return ref;
public void testHasArray() throws IOException {
int length = randomIntBetween(10, PAGE_SIZE * randomIntBetween(1, 3));
BytesReference pbr = newBytesReference(length);
// must return true for <= pagesize
assertEquals(length <= PAGE_SIZE, pbr.hasArray());
}
}

View File

@ -0,0 +1,61 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch 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.elasticsearch.common.netty;
import org.elasticsearch.common.bytes.AbstractBytesReferenceTestCase;
import org.elasticsearch.common.bytes.BytesArray;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.io.stream.ReleasableBytesStreamOutput;
import org.jboss.netty.buffer.ChannelBuffer;
import org.jboss.netty.buffer.ChannelBuffers;
import java.io.IOException;
public class ChannelBufferBytesReferenceTests extends AbstractBytesReferenceTestCase {
@Override
protected BytesReference newBytesReference(int length) throws IOException {
ReleasableBytesStreamOutput out = new ReleasableBytesStreamOutput(length, bigarrays);
for (int i = 0; i < length; i++) {
out.writeByte((byte) random().nextInt(1 << 8));
}
assertEquals(out.size(), length);
BytesReference ref = out.bytes();
assertEquals(ref.length(), length);
BytesArray bytesArray = ref.toBytesArray();
return NettyUtils.toBytesReference(ChannelBuffers.wrappedBuffer(bytesArray.array(), bytesArray.arrayOffset(),
bytesArray.length()));
}
public void testSliceOnAdvancedBuffer() throws IOException {
BytesReference bytesReference = newBytesReference(randomIntBetween(10, 3 * PAGE_SIZE));
BytesArray bytesArray = bytesReference.toBytesArray();
ChannelBuffer channelBuffer = ChannelBuffers.wrappedBuffer(bytesArray.array(), bytesArray.arrayOffset(),
bytesArray.length());
int numBytesToRead = randomIntBetween(1, 5);
for (int i = 0; i < numBytesToRead; i++) {
channelBuffer.readByte();
}
BytesReference other = NettyUtils.toBytesReference(channelBuffer);
BytesReference slice = bytesReference.slice(numBytesToRead, bytesReference.length() - numBytesToRead);
assertEquals(other, slice);
assertEquals(other.slice(3, 1), slice.slice(3, 1));
}
}

View File

@ -73,7 +73,6 @@ public class NettyUtilsTests extends ESTestCase {
}
private BytesReference getRandomizedBytesReference(int length) throws IOException {
// TODO we should factor out a BaseBytesReferenceTestCase
// we know bytes stream output always creates a paged bytes reference, we use it to create randomized content
ReleasableBytesStreamOutput out = new ReleasableBytesStreamOutput(length, bigarrays);
for (int i = 0; i < length; i++) {