HBASE-20447 Only fail cacheBlock if block collisions aren't related to next block metadata
When we pread, we don't force the read to read all of the next block header. However, when we get into a race condition where two opener threads try to cache the same block and one thread read all of the next block header and the other one didn't, it will fail the open process. This is especially important in a splitting case where it will potentially fail the split process. Instead, in the caches, we should only fail if the required blocks are different. Signed-off-by: Andrew Purtell <apurtell@apache.org>
This commit is contained in:
parent
92a58016ae
commit
d2daada970
|
@ -263,7 +263,7 @@ public class MemcachedBlockCache implements BlockCache {
|
|||
@Override
|
||||
public CachedData encode(HFileBlock block) {
|
||||
ByteBuffer bb = ByteBuffer.allocate(block.getSerializedLength());
|
||||
block.serialize(bb);
|
||||
block.serialize(bb, true);
|
||||
return new CachedData(0, bb.array(), CachedData.MAX_SIZE);
|
||||
}
|
||||
|
||||
|
|
|
@ -33,6 +33,8 @@ import org.apache.yetus.audience.InterfaceAudience;
|
|||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.metrics.impl.FastLongHistogram;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
/**
|
||||
* Utilty for aggregating counts in CachedBlocks and toString/toJSON CachedBlocks and BlockCaches.
|
||||
|
@ -41,6 +43,7 @@ import org.apache.hadoop.hbase.util.Bytes;
|
|||
@InterfaceAudience.Private
|
||||
public class BlockCacheUtil {
|
||||
|
||||
private static final Logger LOG = LoggerFactory.getLogger(BlockCacheUtil.class);
|
||||
|
||||
public static final long NANOS_PER_SECOND = 1000000000;
|
||||
|
||||
|
@ -173,15 +176,44 @@ public class BlockCacheUtil {
|
|||
return cbsbf;
|
||||
}
|
||||
|
||||
public static int compareCacheBlock(Cacheable left, Cacheable right) {
|
||||
private static int compareCacheBlock(Cacheable left, Cacheable right,
|
||||
boolean includeNextBlockMetadata) {
|
||||
ByteBuffer l = ByteBuffer.allocate(left.getSerializedLength());
|
||||
left.serialize(l);
|
||||
left.serialize(l, includeNextBlockMetadata);
|
||||
ByteBuffer r = ByteBuffer.allocate(right.getSerializedLength());
|
||||
right.serialize(r);
|
||||
right.serialize(r, includeNextBlockMetadata);
|
||||
return Bytes.compareTo(l.array(), l.arrayOffset(), l.limit(),
|
||||
r.array(), r.arrayOffset(), r.limit());
|
||||
}
|
||||
|
||||
/**
|
||||
* Validate that the existing and newBlock are the same without including the nextBlockMetadata,
|
||||
* if not, throw an exception. If they are the same without the nextBlockMetadata,
|
||||
* return the comparison.
|
||||
*
|
||||
* @param existing block that is existing in the cache.
|
||||
* @param newBlock block that is trying to be cached.
|
||||
* @param cacheKey the cache key of the blocks.
|
||||
* @return comparison of the existing block to the newBlock.
|
||||
*/
|
||||
public static int validateBlockAddition(Cacheable existing, Cacheable newBlock,
|
||||
BlockCacheKey cacheKey) {
|
||||
int comparison = compareCacheBlock(existing, newBlock, true);
|
||||
if (comparison != 0) {
|
||||
LOG.warn("Cached block contents differ, trying to just compare the block contents " +
|
||||
"without the next block. CacheKey: " + cacheKey);
|
||||
|
||||
// compare the contents, if they are not equal, we are in big trouble
|
||||
int comparisonWithoutNextBlockMetadata = compareCacheBlock(existing, newBlock, false);
|
||||
|
||||
if (comparisonWithoutNextBlockMetadata != 0) {
|
||||
throw new RuntimeException("Cached block contents differ, which should not have happened."
|
||||
+ "cacheKey:" + cacheKey);
|
||||
}
|
||||
}
|
||||
return comparison;
|
||||
}
|
||||
|
||||
/**
|
||||
* Use one of these to keep a running account of cached blocks by file. Throw it away when done.
|
||||
* This is different than metrics in that it is stats on current state of a cache.
|
||||
|
|
|
@ -46,8 +46,9 @@ public interface Cacheable extends HeapSize {
|
|||
/**
|
||||
* Serializes its data into destination.
|
||||
* @param destination Where to serialize to
|
||||
* @param includeNextBlockMetadata Whether to include nextBlockMetadata in the Cache block.
|
||||
*/
|
||||
void serialize(ByteBuffer destination);
|
||||
void serialize(ByteBuffer destination, boolean includeNextBlockMetadata);
|
||||
|
||||
/**
|
||||
* Returns CacheableDeserializer instance which reconstructs original object from ByteBuffer.
|
||||
|
|
|
@ -95,8 +95,8 @@ import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
|
|||
* Caches cache whole blocks with trailing checksums if any. We then tag on some metadata, the
|
||||
* content of BLOCK_METADATA_SPACE which will be flag on if we are doing 'hbase'
|
||||
* checksums and then the offset into the file which is needed when we re-make a cache key
|
||||
* when we return the block to the cache as 'done'. See {@link Cacheable#serialize(ByteBuffer)} and
|
||||
* {@link Cacheable#getDeserializer()}.
|
||||
* when we return the block to the cache as 'done'.
|
||||
* See {@link Cacheable#serialize(ByteBuffer, boolean)} and {@link Cacheable#getDeserializer()}.
|
||||
*
|
||||
* <p>TODO: Should we cache the checksums? Down in Writer#getBlockForCaching(CacheConfig) where
|
||||
* we make a block to cache-on-write, there is an attempt at turning off checksums. This is not the
|
||||
|
@ -325,7 +325,6 @@ public class HFileBlock implements Cacheable {
|
|||
* Creates a new {@link HFile} block from the given fields. This constructor
|
||||
* is used only while writing blocks and caching,
|
||||
* and is sitting in a byte buffer and we want to stuff the block into cache.
|
||||
* See {@link Writer#getBlockForCaching(CacheConfig)}.
|
||||
*
|
||||
* <p>TODO: The caller presumes no checksumming
|
||||
* required of this block instance since going into cache; checksum already verified on
|
||||
|
@ -341,9 +340,11 @@ public class HFileBlock implements Cacheable {
|
|||
* @param onDiskDataSizeWithHeader see {@link #onDiskDataSizeWithHeader}
|
||||
* @param fileContext HFile meta data
|
||||
*/
|
||||
HFileBlock(BlockType blockType, int onDiskSizeWithoutHeader, int uncompressedSizeWithoutHeader,
|
||||
long prevBlockOffset, ByteBuffer b, boolean fillHeader, long offset,
|
||||
final int nextBlockOnDiskSize, int onDiskDataSizeWithHeader, HFileContext fileContext) {
|
||||
@VisibleForTesting
|
||||
public HFileBlock(BlockType blockType, int onDiskSizeWithoutHeader,
|
||||
int uncompressedSizeWithoutHeader, long prevBlockOffset, ByteBuffer b, boolean fillHeader,
|
||||
long offset, final int nextBlockOnDiskSize, int onDiskDataSizeWithHeader,
|
||||
HFileContext fileContext) {
|
||||
init(blockType, onDiskSizeWithoutHeader, uncompressedSizeWithoutHeader,
|
||||
prevBlockOffset, offset, onDiskDataSizeWithHeader, nextBlockOnDiskSize, fileContext);
|
||||
this.buf = new SingleByteBuff(b);
|
||||
|
@ -620,6 +621,7 @@ public class HFileBlock implements Cacheable {
|
|||
.append(", buf=[").append(buf).append("]")
|
||||
.append(", dataBeginsWith=").append(dataBegin)
|
||||
.append(", fileContext=").append(fileContext)
|
||||
.append(", nextBlockOnDiskSize=").append(nextBlockOnDiskSize)
|
||||
.append("]");
|
||||
return sb.toString();
|
||||
}
|
||||
|
@ -1893,11 +1895,9 @@ public class HFileBlock implements Cacheable {
|
|||
|
||||
// Cacheable implementation
|
||||
@Override
|
||||
public void serialize(ByteBuffer destination) {
|
||||
// BE CAREFUL!! There is a custom version of this serialization over in BucketCache#doDrain.
|
||||
// Make sure any changes in here are reflected over there.
|
||||
public void serialize(ByteBuffer destination, boolean includeNextBlockMetadata) {
|
||||
this.buf.get(destination, 0, getSerializedLength() - BLOCK_METADATA_SPACE);
|
||||
destination = addMetaData(destination);
|
||||
destination = addMetaData(destination, includeNextBlockMetadata);
|
||||
|
||||
// Make it ready for reading. flip sets position to zero and limit to current position which
|
||||
// is what we want if we do not want to serialize the block plus checksums if present plus
|
||||
|
@ -1910,7 +1910,7 @@ public class HFileBlock implements Cacheable {
|
|||
*/
|
||||
public ByteBuffer getMetaData() {
|
||||
ByteBuffer bb = ByteBuffer.allocate(BLOCK_METADATA_SPACE);
|
||||
bb = addMetaData(bb);
|
||||
bb = addMetaData(bb, true);
|
||||
bb.flip();
|
||||
return bb;
|
||||
}
|
||||
|
@ -1919,10 +1919,12 @@ public class HFileBlock implements Cacheable {
|
|||
* Adds metadata at current position (position is moved forward). Does not flip or reset.
|
||||
* @return The passed <code>destination</code> with metadata added.
|
||||
*/
|
||||
private ByteBuffer addMetaData(final ByteBuffer destination) {
|
||||
private ByteBuffer addMetaData(final ByteBuffer destination, boolean includeNextBlockMetadata) {
|
||||
destination.put(this.fileContext.isUseHBaseChecksum() ? (byte) 1 : (byte) 0);
|
||||
destination.putLong(this.offset);
|
||||
destination.putInt(this.nextBlockOnDiskSize);
|
||||
if (includeNextBlockMetadata) {
|
||||
destination.putInt(this.nextBlockOnDiskSize);
|
||||
}
|
||||
return destination;
|
||||
}
|
||||
|
||||
|
|
|
@ -380,15 +380,20 @@ public class LruBlockCache implements ResizableBlockCache, HeapSize {
|
|||
|
||||
LruCachedBlock cb = map.get(cacheKey);
|
||||
if (cb != null) {
|
||||
// compare the contents, if they are not equal, we are in big trouble
|
||||
if (BlockCacheUtil.compareCacheBlock(buf, cb.getBuffer()) != 0) {
|
||||
throw new RuntimeException("Cached block contents differ, which should not have happened."
|
||||
+ "cacheKey:" + cacheKey);
|
||||
int comparison = BlockCacheUtil.validateBlockAddition(cb.getBuffer(), buf, cacheKey);
|
||||
if (comparison != 0) {
|
||||
if (comparison < 0) {
|
||||
LOG.warn("Cached block contents differ by nextBlockOnDiskSize. Keeping cached block.");
|
||||
return;
|
||||
} else {
|
||||
LOG.warn("Cached block contents differ by nextBlockOnDiskSize. Caching new block.");
|
||||
}
|
||||
} else {
|
||||
String msg = "Cached an already cached block: " + cacheKey + " cb:" + cb.getCacheKey();
|
||||
msg += ". This is harmless and can happen in rare cases (see HBASE-8547)";
|
||||
LOG.warn(msg);
|
||||
return;
|
||||
}
|
||||
String msg = "Cached an already cached block: " + cacheKey + " cb:" + cb.getCacheKey();
|
||||
msg += ". This is harmless and can happen in rare cases (see HBASE-8547)";
|
||||
LOG.warn(msg);
|
||||
return;
|
||||
}
|
||||
long currentSize = size.get();
|
||||
long currentAcceptableSize = acceptableSize();
|
||||
|
|
|
@ -425,21 +425,28 @@ public class BucketCache implements BlockCache, HeapSize {
|
|||
return;
|
||||
}
|
||||
|
||||
if (backingMap.containsKey(cacheKey)) {
|
||||
if (backingMap.containsKey(cacheKey) || ramCache.containsKey(cacheKey)) {
|
||||
Cacheable existingBlock = getBlock(cacheKey, false, false, false);
|
||||
|
||||
try {
|
||||
if (BlockCacheUtil.compareCacheBlock(cachedItem, existingBlock) != 0) {
|
||||
throw new RuntimeException("Cached block contents differ, which should not have happened."
|
||||
+ "cacheKey:" + cacheKey);
|
||||
int comparison = BlockCacheUtil.validateBlockAddition(existingBlock, cachedItem, cacheKey);
|
||||
if (comparison != 0) {
|
||||
if (comparison < 0) {
|
||||
LOG.warn("Cached block contents differ by nextBlockOnDiskSize. Keeping cached block.");
|
||||
return;
|
||||
} else {
|
||||
LOG.warn("Cached block contents differ by nextBlockOnDiskSize. Caching new block.");
|
||||
}
|
||||
} else {
|
||||
String msg = "Caching an already cached block: " + cacheKey;
|
||||
msg += ". This is harmless and can happen in rare cases (see HBASE-8547)";
|
||||
LOG.warn(msg);
|
||||
return;
|
||||
}
|
||||
String msg = "Caching an already cached block: " + cacheKey;
|
||||
msg += ". This is harmless and can happen in rare cases (see HBASE-8547)";
|
||||
LOG.warn(msg);
|
||||
} finally {
|
||||
// return the block since we need to decrement the count
|
||||
returnBlock(cacheKey, existingBlock);
|
||||
}
|
||||
return;
|
||||
}
|
||||
|
||||
/*
|
||||
|
@ -1505,7 +1512,7 @@ public class BucketCache implements BlockCache, HeapSize {
|
|||
ioEngine.write(metadata, offset + len - metadata.limit());
|
||||
} else {
|
||||
ByteBuffer bb = ByteBuffer.allocate(len);
|
||||
data.serialize(bb);
|
||||
data.serialize(bb, true);
|
||||
ioEngine.write(bb, offset);
|
||||
}
|
||||
} catch (IOException ioe) {
|
||||
|
|
|
@ -311,7 +311,7 @@ public class CacheTestUtils {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void serialize(ByteBuffer destination) {
|
||||
public void serialize(ByteBuffer destination, boolean includeNextBlockMetadata) {
|
||||
destination.putInt(buf.length);
|
||||
Thread.yield();
|
||||
destination.put(buf);
|
||||
|
@ -398,4 +398,15 @@ public class CacheTestUtils {
|
|||
return this.block;
|
||||
}
|
||||
}
|
||||
|
||||
public static void getBlockAndAssertEquals(BlockCache cache, BlockCacheKey key,
|
||||
Cacheable blockToCache, ByteBuffer destBuffer,
|
||||
ByteBuffer expectedBuffer) {
|
||||
destBuffer.clear();
|
||||
cache.cacheBlock(key, blockToCache);
|
||||
Cacheable actualBlock = cache.getBlock(key, false, false, false);
|
||||
actualBlock.serialize(destBuffer, true);
|
||||
assertEquals(expectedBuffer, destBuffer);
|
||||
cache.returnBlock(key, actualBlock);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -136,7 +136,7 @@ public class TestCacheConfig {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void serialize(ByteBuffer destination) {
|
||||
public void serialize(ByteBuffer destination, boolean includeNextBlockMetadata) {
|
||||
LOG.info("Serialized " + this + " to " + destination);
|
||||
}
|
||||
|
||||
|
|
|
@ -131,7 +131,7 @@ public class TestCachedBlockQueue extends TestCase {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void serialize(ByteBuffer destination) {
|
||||
public void serialize(ByteBuffer destination, boolean includeNextBlockMetadata) {
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -468,7 +468,7 @@ public class TestHFileBlock {
|
|||
// test serialized blocks
|
||||
for (boolean reuseBuffer : new boolean[] { false, true }) {
|
||||
ByteBuffer serialized = ByteBuffer.allocate(blockFromHFile.getSerializedLength());
|
||||
blockFromHFile.serialize(serialized);
|
||||
blockFromHFile.serialize(serialized, true);
|
||||
HFileBlock deserialized =
|
||||
(HFileBlock) blockFromHFile.getDeserializer().deserialize(
|
||||
new SingleByteBuff(serialized), reuseBuffer, MemoryType.EXCLUSIVE);
|
||||
|
@ -858,4 +858,27 @@ public class TestHFileBlock {
|
|||
block.heapSize());
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSerializeWithoutNextBlockMetadata() {
|
||||
int size = 100;
|
||||
int length = HConstants.HFILEBLOCK_HEADER_SIZE + size;
|
||||
byte[] byteArr = new byte[length];
|
||||
ByteBuffer buf = ByteBuffer.wrap(byteArr, 0, size);
|
||||
HFileContext meta = new HFileContextBuilder().build();
|
||||
HFileBlock blockWithNextBlockMetadata = new HFileBlock(BlockType.DATA, size, size, -1, buf,
|
||||
HFileBlock.FILL_HEADER, -1, 52, -1, meta);
|
||||
HFileBlock blockWithoutNextBlockMetadata = new HFileBlock(BlockType.DATA, size, size, -1, buf,
|
||||
HFileBlock.FILL_HEADER, -1, -1, -1, meta);
|
||||
ByteBuffer buff1 = ByteBuffer.allocate(length);
|
||||
ByteBuffer buff2 = ByteBuffer.allocate(length);
|
||||
blockWithNextBlockMetadata.serialize(buff1, true);
|
||||
blockWithoutNextBlockMetadata.serialize(buff2, true);
|
||||
assertNotEquals(buff1, buff2);
|
||||
buff1.clear();
|
||||
buff2.clear();
|
||||
blockWithNextBlockMetadata.serialize(buff1, false);
|
||||
blockWithoutNextBlockMetadata.serialize(buff2, false);
|
||||
assertEquals(buff1, buff2);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -31,6 +31,7 @@ import java.util.concurrent.atomic.AtomicInteger;
|
|||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.Waiter;
|
||||
import org.apache.hadoop.hbase.Waiter.ExplainingPredicate;
|
||||
import org.apache.hadoop.hbase.io.HeapSize;
|
||||
|
@ -803,6 +804,59 @@ public class TestLruBlockCache {
|
|||
assertEquals(0.5, stats.getHitCachingRatioPastNPeriods(), delta);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCacheBlockNextBlockMetadataMissing() {
|
||||
long maxSize = 100000;
|
||||
long blockSize = calculateBlockSize(maxSize, 10);
|
||||
int size = 100;
|
||||
int length = HConstants.HFILEBLOCK_HEADER_SIZE + size;
|
||||
byte[] byteArr = new byte[length];
|
||||
ByteBuffer buf = ByteBuffer.wrap(byteArr, 0, size);
|
||||
HFileContext meta = new HFileContextBuilder().build();
|
||||
HFileBlock blockWithNextBlockMetadata = new HFileBlock(BlockType.DATA, size, size, -1, buf,
|
||||
HFileBlock.FILL_HEADER, -1, 52, -1, meta);
|
||||
HFileBlock blockWithoutNextBlockMetadata = new HFileBlock(BlockType.DATA, size, size, -1, buf,
|
||||
HFileBlock.FILL_HEADER, -1, -1, -1, meta);
|
||||
|
||||
LruBlockCache cache = new LruBlockCache(maxSize, blockSize, false,
|
||||
(int)Math.ceil(1.2*maxSize/blockSize),
|
||||
LruBlockCache.DEFAULT_LOAD_FACTOR,
|
||||
LruBlockCache.DEFAULT_CONCURRENCY_LEVEL,
|
||||
0.66f, // min
|
||||
0.99f, // acceptable
|
||||
0.33f, // single
|
||||
0.33f, // multi
|
||||
0.34f, // memory
|
||||
1.2f, // limit
|
||||
false,
|
||||
1024);
|
||||
|
||||
BlockCacheKey key = new BlockCacheKey("key1", 0);
|
||||
ByteBuffer actualBuffer = ByteBuffer.allocate(length);
|
||||
ByteBuffer block1Buffer = ByteBuffer.allocate(length);
|
||||
ByteBuffer block2Buffer = ByteBuffer.allocate(length);
|
||||
blockWithNextBlockMetadata.serialize(block1Buffer, true);
|
||||
blockWithoutNextBlockMetadata.serialize(block2Buffer, true);
|
||||
|
||||
//Add blockWithNextBlockMetadata, expect blockWithNextBlockMetadata back.
|
||||
CacheTestUtils.getBlockAndAssertEquals(cache, key, blockWithNextBlockMetadata, actualBuffer,
|
||||
block1Buffer);
|
||||
|
||||
//Add blockWithoutNextBlockMetada, expect blockWithNextBlockMetadata back.
|
||||
CacheTestUtils.getBlockAndAssertEquals(cache, key, blockWithoutNextBlockMetadata, actualBuffer,
|
||||
block1Buffer);
|
||||
|
||||
//Clear and add blockWithoutNextBlockMetadata
|
||||
cache.clearCache();
|
||||
assertNull(cache.getBlock(key, false, false, false));
|
||||
CacheTestUtils.getBlockAndAssertEquals(cache, key, blockWithoutNextBlockMetadata, actualBuffer,
|
||||
block2Buffer);
|
||||
|
||||
//Add blockWithNextBlockMetadata, expect blockWithNextBlockMetadata to replace.
|
||||
CacheTestUtils.getBlockAndAssertEquals(cache, key, blockWithNextBlockMetadata, actualBuffer,
|
||||
block1Buffer);
|
||||
}
|
||||
|
||||
private CachedItem [] generateFixedBlocks(int numBlocks, int size, String pfx) {
|
||||
CachedItem [] blocks = new CachedItem[numBlocks];
|
||||
for(int i=0;i<numBlocks;i++) {
|
||||
|
@ -887,7 +941,7 @@ public class TestLruBlockCache {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void serialize(ByteBuffer destination) {
|
||||
public void serialize(ByteBuffer destination, boolean includeNextBlockMetadata) {
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -19,10 +19,12 @@ package org.apache.hadoop.hbase.io.hfile.bucket;
|
|||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertNull;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import java.io.FileNotFoundException;
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
|
@ -35,10 +37,15 @@ import org.apache.hadoop.fs.Path;
|
|||
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.io.hfile.BlockCacheKey;
|
||||
import org.apache.hadoop.hbase.io.hfile.BlockType;
|
||||
import org.apache.hadoop.hbase.io.hfile.CacheTestUtils;
|
||||
import org.apache.hadoop.hbase.io.hfile.CacheTestUtils.HFileBlockPair;
|
||||
import org.apache.hadoop.hbase.io.hfile.Cacheable;
|
||||
import org.apache.hadoop.hbase.io.hfile.HFileBlock;
|
||||
import org.apache.hadoop.hbase.io.hfile.HFileContext;
|
||||
import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
|
||||
import org.apache.hadoop.hbase.io.hfile.bucket.BucketAllocator.BucketSizeInfo;
|
||||
import org.apache.hadoop.hbase.io.hfile.bucket.BucketAllocator.IndexStatistics;
|
||||
import org.apache.hadoop.hbase.testclassification.IOTests;
|
||||
|
@ -110,17 +117,11 @@ public class TestBucketCache {
|
|||
|
||||
@Override
|
||||
public void cacheBlock(BlockCacheKey cacheKey, Cacheable buf, boolean inMemory) {
|
||||
if (super.getBlock(cacheKey, true, false, true) != null) {
|
||||
throw new RuntimeException("Cached an already cached block");
|
||||
}
|
||||
super.cacheBlock(cacheKey, buf, inMemory);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void cacheBlock(BlockCacheKey cacheKey, Cacheable buf) {
|
||||
if (super.getBlock(cacheKey, true, false, true) != null) {
|
||||
throw new RuntimeException("Cached an already cached block");
|
||||
}
|
||||
super.cacheBlock(cacheKey, buf);
|
||||
}
|
||||
}
|
||||
|
@ -406,4 +407,42 @@ public class TestBucketCache {
|
|||
BucketCache.BucketEntry bucketEntry = new BucketCache.BucketEntry(testValue, 10, 10L, true);
|
||||
assertEquals(testValue, bucketEntry.offset());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCacheBlockNextBlockMetadataMissing() {
|
||||
int size = 100;
|
||||
int length = HConstants.HFILEBLOCK_HEADER_SIZE + size;
|
||||
byte[] byteArr = new byte[length];
|
||||
ByteBuffer buf = ByteBuffer.wrap(byteArr, 0, size);
|
||||
HFileContext meta = new HFileContextBuilder().build();
|
||||
HFileBlock blockWithNextBlockMetadata = new HFileBlock(BlockType.DATA, size, size, -1, buf,
|
||||
HFileBlock.FILL_HEADER, -1, 52, -1, meta);
|
||||
HFileBlock blockWithoutNextBlockMetadata = new HFileBlock(BlockType.DATA, size, size, -1, buf,
|
||||
HFileBlock.FILL_HEADER, -1, -1, -1, meta);
|
||||
|
||||
BlockCacheKey key = new BlockCacheKey("key1", 0);
|
||||
ByteBuffer actualBuffer = ByteBuffer.allocate(length);
|
||||
ByteBuffer block1Buffer = ByteBuffer.allocate(length);
|
||||
ByteBuffer block2Buffer = ByteBuffer.allocate(length);
|
||||
blockWithNextBlockMetadata.serialize(block1Buffer, true);
|
||||
blockWithoutNextBlockMetadata.serialize(block2Buffer, true);
|
||||
|
||||
//Add blockWithNextBlockMetadata, expect blockWithNextBlockMetadata back.
|
||||
CacheTestUtils.getBlockAndAssertEquals(cache, key, blockWithNextBlockMetadata, actualBuffer,
|
||||
block1Buffer);
|
||||
|
||||
//Add blockWithoutNextBlockMetada, expect blockWithNextBlockMetadata back.
|
||||
CacheTestUtils.getBlockAndAssertEquals(cache, key, blockWithoutNextBlockMetadata, actualBuffer,
|
||||
block1Buffer);
|
||||
|
||||
//Clear and add blockWithoutNextBlockMetadata
|
||||
cache.evictBlock(key);
|
||||
assertNull(cache.getBlock(key, false, false, false));
|
||||
CacheTestUtils.getBlockAndAssertEquals(cache, key, blockWithoutNextBlockMetadata, actualBuffer,
|
||||
block2Buffer);
|
||||
|
||||
//Add blockWithNextBlockMetadata, expect blockWithNextBlockMetadata to replace.
|
||||
CacheTestUtils.getBlockAndAssertEquals(cache, key, blockWithNextBlockMetadata, actualBuffer,
|
||||
block1Buffer);
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue