HBASE-14202 Reduce garbage we create.
This commit is contained in:
parent
7d4de20caf
commit
d49b1f60f4
|
@ -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,
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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() + "}";
|
||||
}
|
||||
}
|
|
@ -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()));
|
||||
}
|
||||
|
||||
|
|
|
@ -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;
|
||||
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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());
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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 {
|
||||
|
|
Loading…
Reference in New Issue