HBASE-14202 Reduce garbage we create.

This commit is contained in:
anoopsjohn 2015-08-11 21:51:17 +05:30
parent 7d4de20caf
commit d49b1f60f4
17 changed files with 166 additions and 55 deletions

View File

@ -44,7 +44,7 @@ import org.apache.hadoop.hbase.nio.ByteBuff;
import org.apache.hadoop.hbase.util.ByteBufferUtils;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.ClassSize;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.ObjectIntPair;
import org.apache.hadoop.io.WritableUtils;
/**
@ -127,10 +127,10 @@ abstract class BufferedDataBlockEncoder implements DataBlockEncoder {
protected KeyValue.KeyOnlyKeyValue currentKey = new KeyValue.KeyOnlyKeyValue();
// A temp pair object which will be reused by ByteBuff#asSubByteBuffer calls. This avoids too
// many object creations.
private final Pair<ByteBuffer, Integer> tmpPair;
private final ObjectIntPair<ByteBuffer> tmpPair;
private final boolean includeTags;
public SeekerState(Pair<ByteBuffer, Integer> tmpPair, boolean includeTags) {
public SeekerState(ObjectIntPair<ByteBuffer> tmpPair, boolean includeTags) {
this.tmpPair = tmpPair;
this.includeTags = includeTags;
}
@ -696,7 +696,7 @@ abstract class BufferedDataBlockEncoder implements DataBlockEncoder {
protected KeyValue.KeyOnlyKeyValue keyOnlyKV = new KeyValue.KeyOnlyKeyValue();
// A temp pair object which will be reused by ByteBuff#asSubByteBuffer calls. This avoids too
// many object creations.
protected final Pair<ByteBuffer, Integer> tmpPair = new Pair<ByteBuffer, Integer>();
protected final ObjectIntPair<ByteBuffer> tmpPair = new ObjectIntPair<ByteBuffer>();
protected STATE current, previous;
public BufferedEncodedSeeker(CellComparator comparator,

View File

@ -30,7 +30,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.nio.ByteBuff;
import org.apache.hadoop.hbase.util.ByteBufferUtils;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.ObjectIntPair;
/**
* Compress using:
@ -367,7 +367,7 @@ public class DiffKeyDeltaEncoder extends BufferedDataBlockEncoder {
private int rowLengthWithSize;
private long timestamp;
public DiffSeekerState(Pair<ByteBuffer, Integer> tmpPair, boolean includeTags) {
public DiffSeekerState(ObjectIntPair<ByteBuffer> tmpPair, boolean includeTags) {
super(tmpPair, includeTags);
}

View File

@ -30,7 +30,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.nio.ByteBuff;
import org.apache.hadoop.hbase.util.ByteBufferUtils;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.ObjectIntPair;
/**
* Encoder similar to {@link DiffKeyDeltaEncoder} but supposedly faster.
@ -379,7 +379,7 @@ public class FastDiffDeltaEncoder extends BufferedDataBlockEncoder {
private int rowLengthWithSize;
private int familyLengthWithSize;
public FastDiffSeekerState(Pair<ByteBuffer, Integer> tmpPair, boolean includeTags) {
public FastDiffSeekerState(ObjectIntPair<ByteBuffer> tmpPair, boolean includeTags) {
super(tmpPair, includeTags);
}

View File

@ -22,7 +22,7 @@ import java.nio.ByteBuffer;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.util.ByteBufferUtils;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.ObjectIntPair;
import org.apache.hadoop.io.WritableUtils;
/**
@ -119,7 +119,7 @@ public abstract class ByteBuff {
* the caller. Thus it avoids more object creations that would happen if the pair that is
* returned is created by this method every time.
*/
public abstract void asSubByteBuffer(int offset, int length, Pair<ByteBuffer, Integer> pair);
public abstract void asSubByteBuffer(int offset, int length, ObjectIntPair<ByteBuffer> pair);
/**
* Returns the number of elements between the current position and the

View File

@ -25,7 +25,7 @@ import java.nio.InvalidMarkException;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.util.ByteBufferUtils;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.ObjectIntPair;
/**
* Provides a unified view of all the underlying ByteBuffers and will look as if a bigger
@ -982,7 +982,7 @@ public class MultiByteBuff extends ByteBuff {
* set the values on the pair that is passed in by the caller
*/
@Override
public void asSubByteBuffer(int offset, int length, Pair<ByteBuffer, Integer> pair) {
public void asSubByteBuffer(int offset, int length, ObjectIntPair<ByteBuffer> pair) {
if (this.itemBeginPos[this.curItemIndex] <= offset) {
int relOffsetInCurItem = offset - this.itemBeginPos[this.curItemIndex];
if (this.curItem.limit() - relOffsetInCurItem >= length) {

View File

@ -21,7 +21,7 @@ import java.nio.ByteBuffer;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.util.ByteBufferUtils;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.ObjectIntPair;
/**
* An implementation of ByteBuff where a single BB backs the BBI. This just acts
@ -95,7 +95,7 @@ public class SingleByteBuff extends ByteBuff {
}
@Override
public void asSubByteBuffer(int offset, int length, Pair<ByteBuffer, Integer> pair) {
public void asSubByteBuffer(int offset, int length, ObjectIntPair<ByteBuffer> pair) {
// Just return the single BB that is available
pair.setFirst(this.buf);
pair.setSecond(offset);
@ -173,7 +173,7 @@ public class SingleByteBuff extends ByteBuff {
} else {
// TODO we can do some optimization here? Call to asSubByteBuffer might
// create a copy.
Pair<ByteBuffer, Integer> pair = new Pair<ByteBuffer, Integer>();
ObjectIntPair<ByteBuffer> pair = new ObjectIntPair<ByteBuffer>();
src.asSubByteBuffer(srcOffset, length, pair);
ByteBufferUtils.copyFromBufferToBuffer(pair.getFirst(), this.buf, pair.getSecond(), offset,
length);

View File

@ -0,0 +1,76 @@
/**
*
* 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.hadoop.hbase.util;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
/**
* A generic class for pair of an Object and and a primitive int value.
*/
@InterfaceAudience.Private
public class ObjectIntPair<T> {
private T first;
private int second;
public ObjectIntPair() {
}
public ObjectIntPair(T first, int second) {
this.setFirst(first);
this.setSecond(second);
}
public T getFirst() {
return first;
}
public void setFirst(T first) {
this.first = first;
}
public int getSecond() {
return second;
}
public void setSecond(int second) {
this.second = second;
}
@Override
public boolean equals(Object other) {
return other instanceof ObjectIntPair && equals(first, ((ObjectIntPair<?>) other).first)
&& (this.second == ((ObjectIntPair<?>) other).second);
}
private static boolean equals(Object x, Object y) {
return (x == null && y == null) || (x != null && x.equals(y));
}
@Override
public int hashCode() {
return first == null ? 0 : (first.hashCode() * 17) + 13 * second;
}
@Override
public String toString() {
return "{" + getFirst() + "," + getSecond() + "}";
}
}

View File

@ -32,7 +32,7 @@ import org.apache.hadoop.hbase.testclassification.MiscTests;
import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.apache.hadoop.hbase.util.ByteBufferUtils;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.ObjectIntPair;
import org.junit.Test;
import org.junit.experimental.categories.Category;
@ -243,11 +243,11 @@ public class TestMultiByteBuff {
assertFalse(bb2 == sub);
assertEquals(l2, ByteBufferUtils.toLong(sub, sub.position()));
multi.rewind();
Pair<ByteBuffer, Integer> p = new Pair<ByteBuffer, Integer>();
ObjectIntPair<ByteBuffer> p = new ObjectIntPair<ByteBuffer>();
multi.asSubByteBuffer(8, Bytes.SIZEOF_LONG, p);
assertFalse(bb1 == p.getFirst());
assertFalse(bb2 == p.getFirst());
assertEquals(0, p.getSecond().intValue());
assertEquals(0, p.getSecond());
assertEquals(l2, ByteBufferUtils.toLong(sub, p.getSecond()));
}

View File

@ -49,7 +49,7 @@ import org.apache.hadoop.hbase.io.hfile.HFile.CachingBlockReader;
import org.apache.hadoop.hbase.nio.ByteBuff;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.ClassSize;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.ObjectIntPair;
import org.apache.hadoop.io.WritableUtils;
import org.apache.hadoop.util.StringUtils;
@ -730,7 +730,7 @@ public class HFileBlockIndex {
// keys[numEntries] = Infinity, then we are maintaining an invariant that
// keys[low - 1] < key < keys[high + 1] while narrowing down the range.
ByteBufferedKeyOnlyKeyValue nonRootIndexkeyOnlyKV = new ByteBufferedKeyOnlyKeyValue();
Pair<ByteBuffer, Integer> pair = new Pair<ByteBuffer, Integer>();
ObjectIntPair<ByteBuffer> pair = new ObjectIntPair<ByteBuffer>();
while (low <= high) {
mid = (low + high) >>> 1;

View File

@ -57,7 +57,7 @@ import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.util.ByteBufferUtils;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.IdLock;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.ObjectIntPair;
import org.apache.hadoop.io.WritableUtils;
import org.apache.htrace.Trace;
import org.apache.htrace.TraceScope;
@ -470,7 +470,7 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
// buffer backed keyonlyKV
private ByteBufferedKeyOnlyKeyValue bufBackedKeyOnlyKv = new ByteBufferedKeyOnlyKeyValue();
// A pair for reusing in blockSeek() so that we don't garbage lot of objects
final Pair<ByteBuffer, Integer> pair = new Pair<ByteBuffer, Integer>();
final ObjectIntPair<ByteBuffer> pair = new ObjectIntPair<ByteBuffer>();
/**
* The next indexed key is to keep track of the indexed key of the next data block.
@ -945,7 +945,7 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
public Cell getKey() {
assertSeeked();
// Create a new object so that this getKey is cached as firstKey, lastKey
Pair<ByteBuffer, Integer> keyPair = new Pair<ByteBuffer, Integer>();
ObjectIntPair<ByteBuffer> keyPair = new ObjectIntPair<ByteBuffer>();
blockBuffer.asSubByteBuffer(blockBuffer.position() + KEY_VALUE_LEN_SIZE, currKeyLen, keyPair);
ByteBuffer keyBuf = keyPair.getFirst();
if (keyBuf.hasArray()) {
@ -1001,7 +1001,7 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
public ByteBuffer getValue() {
assertSeeked();
// Okie to create new Pair. Not used in hot path
Pair<ByteBuffer, Integer> valuePair = new Pair<ByteBuffer, Integer>();
ObjectIntPair<ByteBuffer> valuePair = new ObjectIntPair<ByteBuffer>();
this.blockBuffer.asSubByteBuffer(blockBuffer.position() + KEY_VALUE_LEN_SIZE + currKeyLen,
currValueLen, valuePair);
ByteBuffer valBuf = valuePair.getFirst().duplicate();

View File

@ -69,7 +69,6 @@ import org.apache.hadoop.hbase.util.ConcurrentIndex;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.HasThread;
import org.apache.hadoop.hbase.util.IdLock;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.util.StringUtils;
import com.google.common.annotations.VisibleForTesting;
@ -423,17 +422,14 @@ public class BucketCache implements BlockCache, HeapSize {
// TODO : change this area - should be removed after server cells and
// 12295 are available
int len = bucketEntry.getLength();
Pair<ByteBuff, MemoryType> pair = ioEngine.read(bucketEntry.offset(), len);
ByteBuff bb = pair.getFirst();
CacheableDeserializer<Cacheable> deserializer =
bucketEntry.deserializerReference(this.deserialiserMap);
Cacheable cachedBlock = deserializer.deserialize(bb, true, pair.getSecond());
Cacheable cachedBlock = ioEngine.read(bucketEntry.offset(), len,
bucketEntry.deserializerReference(this.deserialiserMap));
long timeTaken = System.nanoTime() - start;
if (updateCacheMetrics) {
cacheStats.hit(caching);
cacheStats.ioHit(timeTaken);
}
if (pair.getSecond() == MemoryType.SHARED) {
if (cachedBlock.getMemoryType() == MemoryType.SHARED) {
bucketEntry.refCount.incrementAndGet();
}
bucketEntry.access(accessCount.incrementAndGet());

View File

@ -22,10 +22,11 @@ import java.io.IOException;
import java.nio.ByteBuffer;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.io.hfile.Cacheable;
import org.apache.hadoop.hbase.io.hfile.CacheableDeserializer;
import org.apache.hadoop.hbase.io.hfile.Cacheable.MemoryType;
import org.apache.hadoop.hbase.nio.ByteBuff;
import org.apache.hadoop.hbase.util.ByteBufferArray;
import org.apache.hadoop.hbase.util.Pair;
/**
* IO engine that stores data in memory using an array of ByteBuffers
@ -67,7 +68,8 @@ public class ByteBufferIOEngine implements IOEngine {
}
@Override
public Pair<ByteBuff, MemoryType> read(long offset, int length) throws IOException {
public Cacheable read(long offset, int length, CacheableDeserializer<Cacheable> deserializer)
throws IOException {
ByteBuff dstBuffer = bufferArray.asSubByteBuff(offset, length);
// Here the buffer that is created directly refers to the buffer in the actual buckets.
// When any cell is referring to the blocks created out of these buckets then it means that
@ -75,7 +77,7 @@ public class ByteBufferIOEngine implements IOEngine {
// lead to corruption of results. Hence we set the type of the buffer as SHARED_MEMORY
// so that the readers using this block are aware of this fact and do the necessary action
// to prevent eviction till the results are either consumed or copied
return new Pair<ByteBuff, MemoryType>(dstBuffer, MemoryType.SHARED);
return deserializer.deserialize(dstBuffer, true, MemoryType.SHARED);
}
/**

View File

@ -26,10 +26,11 @@ import java.nio.channels.FileChannel;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.io.hfile.Cacheable;
import org.apache.hadoop.hbase.io.hfile.CacheableDeserializer;
import org.apache.hadoop.hbase.io.hfile.Cacheable.MemoryType;
import org.apache.hadoop.hbase.nio.ByteBuff;
import org.apache.hadoop.hbase.nio.SingleByteBuff;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.util.StringUtils;
/**
@ -90,7 +91,8 @@ public class FileIOEngine implements IOEngine {
* @throws IOException
*/
@Override
public Pair<ByteBuff, MemoryType> read(long offset, int length) throws IOException {
public Cacheable read(long offset, int length, CacheableDeserializer<Cacheable> deserializer)
throws IOException {
ByteBuffer dstBuffer = ByteBuffer.allocate(length);
fileChannel.read(dstBuffer, offset);
// The buffer created out of the fileChannel is formed by copying the data from the file
@ -101,7 +103,7 @@ public class FileIOEngine implements IOEngine {
throw new RuntimeException("Only " + dstBuffer.limit() + " bytes read, " + length
+ " expected");
}
return new Pair<ByteBuff, MemoryType>(new SingleByteBuff(dstBuffer), MemoryType.EXCLUSIVE);
return deserializer.deserialize(new SingleByteBuff(dstBuffer), true, MemoryType.EXCLUSIVE);
}
/**

View File

@ -22,9 +22,9 @@ import java.io.IOException;
import java.nio.ByteBuffer;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.io.hfile.Cacheable.MemoryType;
import org.apache.hadoop.hbase.io.hfile.Cacheable;
import org.apache.hadoop.hbase.io.hfile.CacheableDeserializer;
import org.apache.hadoop.hbase.nio.ByteBuff;
import org.apache.hadoop.hbase.util.Pair;
/**
* A class implementing IOEngine interface supports data services for
@ -38,14 +38,16 @@ public interface IOEngine {
boolean isPersistent();
/**
* Transfers data from IOEngine to a byte buffer
* Transfers data from IOEngine to a Cacheable object.
* @param length How many bytes to be read from the offset
* @param offset The offset in the IO engine where the first byte to be read
* @return Pair of ByteBuffer where data is read and its MemoryType ({@link MemoryType})
* @param deserializer The deserializer to be used to make a Cacheable from the data.
* @return Cacheable
* @throws IOException
* @throws RuntimeException when the length of the ByteBuff read is less than 'len'
*/
Pair<ByteBuff, MemoryType> read(long offset, int length) throws IOException;
Cacheable read(long offset, int length, CacheableDeserializer<Cacheable> deserializer)
throws IOException;
/**
* Transfers data from the given byte buffer to IOEngine

View File

@ -25,7 +25,7 @@ import org.apache.hadoop.hbase.KeyValue.Type;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.testclassification.IOTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.ObjectIntPair;
import org.junit.Test;
import org.junit.experimental.categories.Category;
@ -50,7 +50,7 @@ public class TestBufferedDataBlockEncoder {
@Test
public void testEnsureSpaceForKey() {
BufferedDataBlockEncoder.SeekerState state =
new BufferedDataBlockEncoder.SeekerState(new Pair<ByteBuffer, Integer>(), false);
new BufferedDataBlockEncoder.SeekerState(new ObjectIntPair<ByteBuffer>(), false);
for (int i = 1; i <= 65536; ++i) {
state.keyLength = i;
state.ensureSpaceForKey();

View File

@ -20,14 +20,15 @@ package org.apache.hadoop.hbase.io.hfile.bucket;
import static org.junit.Assert.assertTrue;
import java.io.IOException;
import java.nio.ByteBuffer;
import org.apache.hadoop.hbase.io.hfile.Cacheable;
import org.apache.hadoop.hbase.io.hfile.Cacheable.MemoryType;
import org.apache.hadoop.hbase.io.hfile.CacheableDeserializer;
import org.apache.hadoop.hbase.nio.ByteBuff;
import org.apache.hadoop.hbase.nio.MultiByteBuff;
import org.apache.hadoop.hbase.testclassification.IOTests;
import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.apache.hadoop.hbase.util.Pair;
import org.junit.Test;
import org.junit.experimental.categories.Category;
@ -67,8 +68,9 @@ public class TestByteBufferIOEngine {
offset = (int) (Math.random() * (capacity - maxBlockSize));
}
ioEngine.write(srcBuffer, offset);
Pair<ByteBuff, MemoryType> pair = ioEngine.read(offset, blockSize);
ByteBuff dstBuffer = pair.getFirst();
BufferGrabbingDeserializer deserializer = new BufferGrabbingDeserializer();
ioEngine.read(offset, blockSize, deserializer);
ByteBuff dstBuffer = deserializer.buf;
for (int j = 0; j < byteArray.length; ++j) {
assertTrue(byteArray[j] == dstBuffer.get(j));
}
@ -77,6 +79,35 @@ public class TestByteBufferIOEngine {
assert testOffsetAtEndNum == 0;
}
/**
* A CacheableDeserializer implementation which just store reference to the {@link ByteBuff} to be
* deserialized. Use {@link #getDeserializedByteBuff()} to get this reference.
*/
static class BufferGrabbingDeserializer implements CacheableDeserializer<Cacheable> {
private ByteBuff buf;
@Override
public Cacheable deserialize(ByteBuff b) throws IOException {
return null;
}
@Override
public Cacheable deserialize(final ByteBuff b, boolean reuse, MemoryType memType)
throws IOException {
this.buf = b;
return null;
}
@Override
public int getDeserialiserIdentifier() {
return 0;
}
public ByteBuff getDeserializedByteBuff() {
return this.buf;
}
}
@Test
public void testByteBufferIOEngineWithMBB() throws Exception {
int capacity = 32 * 1024 * 1024; // 32 MB
@ -107,9 +138,11 @@ public class TestByteBufferIOEngine {
offset = (int) (Math.random() * (capacity - maxBlockSize));
}
ioEngine.write(srcBuffer, offset);
Pair<ByteBuff, MemoryType> read = ioEngine.read(offset, blockSize);
BufferGrabbingDeserializer deserializer = new BufferGrabbingDeserializer();
ioEngine.read(offset, blockSize, deserializer);
ByteBuff dstBuffer = deserializer.buf;
for (int j = 0; j < byteArray.length; ++j) {
assertTrue(srcBuffer.get(j) == read.getFirst().get(j));
assertTrue(srcBuffer.get(j) == dstBuffer.get(j));
}
}
assert testOffsetAtStartNum == 0;

View File

@ -24,11 +24,10 @@ import java.io.File;
import java.io.IOException;
import java.nio.ByteBuffer;
import org.apache.hadoop.hbase.io.hfile.Cacheable.MemoryType;
import org.apache.hadoop.hbase.io.hfile.bucket.TestByteBufferIOEngine.BufferGrabbingDeserializer;
import org.apache.hadoop.hbase.nio.ByteBuff;
import org.apache.hadoop.hbase.testclassification.IOTests;
import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.apache.hadoop.hbase.util.Pair;
import org.junit.Test;
import org.junit.experimental.categories.Category;
@ -51,10 +50,11 @@ public class TestFileIOEngine {
data1[j] = (byte) (Math.random() * 255);
}
fileIOEngine.write(ByteBuffer.wrap(data1), offset);
Pair<ByteBuff, MemoryType> pair = fileIOEngine.read(offset, len);
byte[] data2 = pair.getFirst().array();
BufferGrabbingDeserializer deserializer = new BufferGrabbingDeserializer();
fileIOEngine.read(offset, len, deserializer);
ByteBuff data2 = deserializer.getDeserializedByteBuff();
for (int j = 0; j < data1.length; ++j) {
assertTrue(data1[j] == data2[j]);
assertTrue(data1[j] == data2.get(j));
}
}
} finally {