HBASE-5898 Consider double-checked locking for block cache lock (Todd, Elliot, LarsH)
git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1408620 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
e7717f7bbc
commit
fb934909a8
|
@ -90,10 +90,6 @@ public abstract class AbstractHFileReader implements HFile.Reader {
|
|||
/** Block cache configuration. */
|
||||
protected final CacheConfig cacheConf;
|
||||
|
||||
protected AtomicLong cacheHits = new AtomicLong();
|
||||
protected AtomicLong blockLoads = new AtomicLong();
|
||||
protected AtomicLong metaLoads = new AtomicLong();
|
||||
|
||||
/** Path of file */
|
||||
protected final Path path;
|
||||
|
||||
|
|
|
@ -49,9 +49,12 @@ public interface BlockCache {
|
|||
* Fetch block from cache.
|
||||
* @param cacheKey Block to fetch.
|
||||
* @param caching Whether this request has caching enabled (used for stats)
|
||||
* @param repeat Whether this is a repeat lookup for the same block
|
||||
* (used to avoid double counting cache misses when doing double-check locking)
|
||||
* {@see HFileReaderV2#readBlock(long, long, boolean, boolean, boolean, BlockType)}
|
||||
* @return Block or null if block is not in 2 cache.
|
||||
*/
|
||||
public Cacheable getBlock(BlockCacheKey cacheKey, boolean caching);
|
||||
public Cacheable getBlock(BlockCacheKey cacheKey, boolean caching, boolean repeat);
|
||||
|
||||
/**
|
||||
* Evict block from cache.
|
||||
|
|
|
@ -91,14 +91,14 @@ public class DoubleBlockCache implements BlockCache, HeapSize {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Cacheable getBlock(BlockCacheKey cacheKey, boolean caching) {
|
||||
public Cacheable getBlock(BlockCacheKey cacheKey, boolean caching, boolean repeat) {
|
||||
Cacheable cachedBlock;
|
||||
|
||||
if ((cachedBlock = onHeapCache.getBlock(cacheKey, caching)) != null) {
|
||||
if ((cachedBlock = onHeapCache.getBlock(cacheKey, caching, repeat)) != null) {
|
||||
stats.hit(caching);
|
||||
return cachedBlock;
|
||||
|
||||
} else if ((cachedBlock = offHeapCache.getBlock(cacheKey, caching)) != null) {
|
||||
} else if ((cachedBlock = offHeapCache.getBlock(cacheKey, caching, repeat)) != null) {
|
||||
if (caching) {
|
||||
onHeapCache.cacheBlock(cacheKey, cachedBlock);
|
||||
}
|
||||
|
@ -106,7 +106,7 @@ public class DoubleBlockCache implements BlockCache, HeapSize {
|
|||
return cachedBlock;
|
||||
}
|
||||
|
||||
stats.miss(caching);
|
||||
if (!repeat) stats.miss(caching);
|
||||
return null;
|
||||
}
|
||||
|
||||
|
|
|
@ -225,15 +225,12 @@ public class HFileReaderV1 extends AbstractHFileReader {
|
|||
|
||||
// Per meta key from any given file, synchronize reads for said block
|
||||
synchronized (metaBlockIndexReader.getRootBlockKey(block)) {
|
||||
metaLoads.incrementAndGet();
|
||||
|
||||
// Check cache for block. If found return.
|
||||
if (cacheConf.isBlockCacheEnabled()) {
|
||||
HFileBlock cachedBlock =
|
||||
(HFileBlock) cacheConf.getBlockCache().getBlock(cacheKey,
|
||||
cacheConf.shouldCacheBlockOnRead(effectiveCategory));
|
||||
cacheConf.shouldCacheBlockOnRead(effectiveCategory), false);
|
||||
if (cachedBlock != null) {
|
||||
cacheHits.incrementAndGet();
|
||||
return cachedBlock.getBufferWithoutHeader();
|
||||
}
|
||||
// Cache Miss, please load.
|
||||
|
@ -285,15 +282,12 @@ public class HFileReaderV1 extends AbstractHFileReader {
|
|||
// the other choice is to duplicate work (which the cache would prevent you
|
||||
// from doing).
|
||||
synchronized (dataBlockIndexReader.getRootBlockKey(block)) {
|
||||
blockLoads.incrementAndGet();
|
||||
|
||||
// Check cache for block. If found return.
|
||||
if (cacheConf.isBlockCacheEnabled()) {
|
||||
HFileBlock cachedBlock =
|
||||
(HFileBlock) cacheConf.getBlockCache().getBlock(cacheKey,
|
||||
cacheConf.shouldCacheDataOnRead());
|
||||
cacheConf.shouldCacheDataOnRead(), false);
|
||||
if (cachedBlock != null) {
|
||||
cacheHits.incrementAndGet();
|
||||
return cachedBlock.getBufferWithoutHeader();
|
||||
}
|
||||
// Carry on, please load.
|
||||
|
|
|
@ -34,7 +34,6 @@ import org.apache.hadoop.hbase.KeyValue;
|
|||
import org.apache.hadoop.hbase.fs.HFileSystem;
|
||||
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoder;
|
||||
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
|
||||
import org.apache.hadoop.hbase.io.hfile.BlockType.BlockCategory;
|
||||
import org.apache.hadoop.hbase.io.hfile.HFile.FileInfo;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.IdLock;
|
||||
|
@ -211,8 +210,6 @@ public class HFileReaderV2 extends AbstractHFileReader {
|
|||
// is OK to do for meta blocks because the meta block index is always
|
||||
// single-level.
|
||||
synchronized (metaBlockIndexReader.getRootBlockKey(block)) {
|
||||
metaLoads.incrementAndGet();
|
||||
|
||||
// Check cache for block. If found return.
|
||||
long metaBlockOffset = metaBlockIndexReader.getRootBlockOffset(block);
|
||||
BlockCacheKey cacheKey = new BlockCacheKey(name, metaBlockOffset,
|
||||
|
@ -221,11 +218,10 @@ public class HFileReaderV2 extends AbstractHFileReader {
|
|||
cacheBlock &= cacheConf.shouldCacheDataOnRead();
|
||||
if (cacheConf.isBlockCacheEnabled()) {
|
||||
HFileBlock cachedBlock =
|
||||
(HFileBlock) cacheConf.getBlockCache().getBlock(cacheKey, cacheBlock);
|
||||
(HFileBlock) cacheConf.getBlockCache().getBlock(cacheKey, cacheBlock, false);
|
||||
if (cachedBlock != null) {
|
||||
// Return a distinct 'shallow copy' of the block,
|
||||
// so pos does not get messed by the scanner
|
||||
cacheHits.incrementAndGet();
|
||||
return cachedBlock.getBufferWithoutHeader();
|
||||
}
|
||||
// Cache Miss, please load.
|
||||
|
@ -286,66 +282,72 @@ public class HFileReaderV2 extends AbstractHFileReader {
|
|||
new BlockCacheKey(name, dataBlockOffset,
|
||||
dataBlockEncoder.getEffectiveEncodingInCache(isCompaction),
|
||||
expectedBlockType);
|
||||
IdLock.Entry lockEntry = offsetLock.getLockEntry(dataBlockOffset);
|
||||
|
||||
boolean useLock = false;
|
||||
IdLock.Entry lockEntry = null;
|
||||
try {
|
||||
blockLoads.incrementAndGet();
|
||||
while (true) {
|
||||
|
||||
// Check cache for block. If found return.
|
||||
if (cacheConf.isBlockCacheEnabled()) {
|
||||
HFileBlock cachedBlock = (HFileBlock)
|
||||
cacheConf.getBlockCache().getBlock(cacheKey, cacheBlock);
|
||||
if (cachedBlock != null) {
|
||||
BlockCategory blockCategory =
|
||||
cachedBlock.getBlockType().getCategory();
|
||||
cacheHits.incrementAndGet();
|
||||
|
||||
|
||||
if (cachedBlock.getBlockType() == BlockType.DATA) {
|
||||
HFile.dataBlockReadCnt.incrementAndGet();
|
||||
}
|
||||
|
||||
validateBlockType(cachedBlock, expectedBlockType);
|
||||
|
||||
// Validate encoding type for encoded blocks. We include encoding
|
||||
// type in the cache key, and we expect it to match on a cache hit.
|
||||
if (cachedBlock.getBlockType() == BlockType.ENCODED_DATA &&
|
||||
cachedBlock.getDataBlockEncoding() !=
|
||||
dataBlockEncoder.getEncodingInCache()) {
|
||||
throw new IOException("Cached block under key " + cacheKey + " " +
|
||||
"has wrong encoding: " + cachedBlock.getDataBlockEncoding() +
|
||||
" (expected: " + dataBlockEncoder.getEncodingInCache() + ")");
|
||||
}
|
||||
return cachedBlock;
|
||||
if (useLock) {
|
||||
lockEntry = offsetLock.getLockEntry(dataBlockOffset);
|
||||
}
|
||||
// Carry on, please load.
|
||||
|
||||
// Check cache for block. If found return.
|
||||
if (cacheConf.isBlockCacheEnabled()) {
|
||||
// Try and get the block from the block cache. If the useLock variable is true then this
|
||||
// is the second time through the loop and it should not be counted as a block cache miss.
|
||||
HFileBlock cachedBlock = (HFileBlock) cacheConf.getBlockCache().getBlock(cacheKey,
|
||||
cacheBlock, useLock);
|
||||
if (cachedBlock != null) {
|
||||
if (cachedBlock.getBlockType() == BlockType.DATA) {
|
||||
HFile.dataBlockReadCnt.incrementAndGet();
|
||||
}
|
||||
|
||||
validateBlockType(cachedBlock, expectedBlockType);
|
||||
|
||||
// Validate encoding type for encoded blocks. We include encoding
|
||||
// type in the cache key, and we expect it to match on a cache hit.
|
||||
if (cachedBlock.getBlockType() == BlockType.ENCODED_DATA
|
||||
&& cachedBlock.getDataBlockEncoding() != dataBlockEncoder.getEncodingInCache()) {
|
||||
throw new IOException("Cached block under key " + cacheKey + " "
|
||||
+ "has wrong encoding: " + cachedBlock.getDataBlockEncoding() + " (expected: "
|
||||
+ dataBlockEncoder.getEncodingInCache() + ")");
|
||||
}
|
||||
return cachedBlock;
|
||||
}
|
||||
// Carry on, please load.
|
||||
}
|
||||
if (!useLock) {
|
||||
// check cache again with lock
|
||||
useLock = true;
|
||||
continue;
|
||||
}
|
||||
|
||||
// Load block from filesystem.
|
||||
long startTimeNs = System.nanoTime();
|
||||
HFileBlock hfileBlock = fsBlockReader.readBlockData(dataBlockOffset, onDiskBlockSize, -1,
|
||||
pread);
|
||||
hfileBlock = dataBlockEncoder.diskToCacheFormat(hfileBlock, isCompaction);
|
||||
validateBlockType(hfileBlock, expectedBlockType);
|
||||
|
||||
final long delta = System.nanoTime() - startTimeNs;
|
||||
HFile.offerReadLatency(delta, pread);
|
||||
|
||||
// Cache the block if necessary
|
||||
if (cacheBlock && cacheConf.shouldCacheBlockOnRead(hfileBlock.getBlockType().getCategory())) {
|
||||
cacheConf.getBlockCache().cacheBlock(cacheKey, hfileBlock, cacheConf.isInMemory());
|
||||
}
|
||||
|
||||
if (hfileBlock.getBlockType() == BlockType.DATA) {
|
||||
HFile.dataBlockReadCnt.incrementAndGet();
|
||||
}
|
||||
|
||||
return hfileBlock;
|
||||
}
|
||||
|
||||
// Load block from filesystem.
|
||||
long startTimeNs = System.nanoTime();
|
||||
HFileBlock hfileBlock = fsBlockReader.readBlockData(dataBlockOffset,
|
||||
onDiskBlockSize, -1, pread);
|
||||
hfileBlock = dataBlockEncoder.diskToCacheFormat(hfileBlock,
|
||||
isCompaction);
|
||||
validateBlockType(hfileBlock, expectedBlockType);
|
||||
BlockCategory blockCategory = hfileBlock.getBlockType().getCategory();
|
||||
|
||||
final long delta = System.nanoTime() - startTimeNs;
|
||||
HFile.offerReadLatency(delta, pread);
|
||||
|
||||
// Cache the block if necessary
|
||||
if (cacheBlock && cacheConf.shouldCacheBlockOnRead(
|
||||
hfileBlock.getBlockType().getCategory())) {
|
||||
cacheConf.getBlockCache().cacheBlock(cacheKey, hfileBlock,
|
||||
cacheConf.isInMemory());
|
||||
}
|
||||
|
||||
if (hfileBlock.getBlockType() == BlockType.DATA) {
|
||||
HFile.dataBlockReadCnt.incrementAndGet();
|
||||
}
|
||||
|
||||
return hfileBlock;
|
||||
} finally {
|
||||
offsetLock.releaseLockEntry(lockEntry);
|
||||
if (lockEntry != null) {
|
||||
offsetLock.releaseLockEntry(lockEntry);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -332,13 +332,16 @@ public class LruBlockCache implements BlockCache, HeapSize {
|
|||
* Get the buffer of the block with the specified name.
|
||||
* @param cacheKey block's cache key
|
||||
* @param caching true if the caller caches blocks on cache misses
|
||||
* @param repeat Whether this is a repeat lookup for the same block
|
||||
* (used to avoid double counting cache misses when doing double-check locking)
|
||||
* {@see HFileReaderV2#readBlock(long, long, boolean, boolean, boolean, BlockType)}
|
||||
* @return buffer of specified cache key, or null if not in cache
|
||||
*/
|
||||
@Override
|
||||
public Cacheable getBlock(BlockCacheKey cacheKey, boolean caching) {
|
||||
public Cacheable getBlock(BlockCacheKey cacheKey, boolean caching, boolean repeat) {
|
||||
CachedBlock cb = map.get(cacheKey);
|
||||
if(cb == null) {
|
||||
stats.miss(caching);
|
||||
if (!repeat) stats.miss(caching);
|
||||
return null;
|
||||
}
|
||||
stats.hit(caching);
|
||||
|
|
|
@ -69,7 +69,7 @@ public class SimpleBlockCache implements BlockCache {
|
|||
return cache.size();
|
||||
}
|
||||
|
||||
public synchronized Cacheable getBlock(BlockCacheKey cacheKey, boolean caching) {
|
||||
public synchronized Cacheable getBlock(BlockCacheKey cacheKey, boolean caching, boolean repeat) {
|
||||
processQueue(); // clear out some crap.
|
||||
Ref ref = cache.get(cacheKey);
|
||||
if (ref == null)
|
||||
|
|
|
@ -153,10 +153,10 @@ public class SingleSizeCache implements BlockCache, HeapSize {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Cacheable getBlock(BlockCacheKey key, boolean caching) {
|
||||
public Cacheable getBlock(BlockCacheKey key, boolean caching, boolean repeat) {
|
||||
CacheablePair contentBlock = backingMap.get(key);
|
||||
if (contentBlock == null) {
|
||||
stats.miss(caching);
|
||||
if (!repeat) stats.miss(caching);
|
||||
return null;
|
||||
}
|
||||
|
||||
|
|
|
@ -229,23 +229,23 @@ public class SlabCache implements SlabItemActionWatcher, BlockCache, HeapSize {
|
|||
|
||||
/**
|
||||
* Get the buffer of the block with the specified name.
|
||||
*
|
||||
* @param key
|
||||
* @param caching
|
||||
* @param key
|
||||
*
|
||||
* @return buffer of specified block name, or null if not in cache
|
||||
*/
|
||||
public Cacheable getBlock(BlockCacheKey key, boolean caching) {
|
||||
public Cacheable getBlock(BlockCacheKey key, boolean caching, boolean repeat) {
|
||||
SingleSizeCache cachedBlock = backingStore.get(key);
|
||||
if (cachedBlock == null) {
|
||||
stats.miss(caching);
|
||||
if (!repeat) stats.miss(caching);
|
||||
return null;
|
||||
}
|
||||
|
||||
Cacheable contentBlock = cachedBlock.getBlock(key, caching);
|
||||
Cacheable contentBlock = cachedBlock.getBlock(key, caching, false);
|
||||
|
||||
if (contentBlock != null) {
|
||||
stats.hit(caching);
|
||||
} else {
|
||||
} else if (!repeat) {
|
||||
stats.miss(caching);
|
||||
}
|
||||
return contentBlock;
|
||||
|
|
|
@ -90,12 +90,12 @@ public class CacheTestUtils {
|
|||
}
|
||||
toBeTested.cacheBlock(ourBlock.blockName, ourBlock.block);
|
||||
Cacheable retrievedBlock = toBeTested.getBlock(ourBlock.blockName,
|
||||
false);
|
||||
false, false);
|
||||
if (retrievedBlock != null) {
|
||||
assertEquals(ourBlock.block, retrievedBlock);
|
||||
toBeTested.evictBlock(ourBlock.blockName);
|
||||
hits.incrementAndGet();
|
||||
assertNull(toBeTested.getBlock(ourBlock.blockName, false));
|
||||
assertNull(toBeTested.getBlock(ourBlock.blockName, false, false));
|
||||
} else {
|
||||
miss.incrementAndGet();
|
||||
}
|
||||
|
@ -123,7 +123,7 @@ public class CacheTestUtils {
|
|||
HFileBlockPair[] blocks = generateHFileBlocks(numBlocks, blockSize);
|
||||
// Confirm empty
|
||||
for (HFileBlockPair block : blocks) {
|
||||
assertNull(toBeTested.getBlock(block.blockName, true));
|
||||
assertNull(toBeTested.getBlock(block.blockName, true, false));
|
||||
}
|
||||
|
||||
// Add blocks
|
||||
|
@ -136,7 +136,7 @@ public class CacheTestUtils {
|
|||
// MapMaker makes no guarantees when it will evict, so neither can we.
|
||||
|
||||
for (HFileBlockPair block : blocks) {
|
||||
HFileBlock buf = (HFileBlock) toBeTested.getBlock(block.blockName, true);
|
||||
HFileBlock buf = (HFileBlock) toBeTested.getBlock(block.blockName, true, false);
|
||||
if (buf != null) {
|
||||
assertEquals(block.block, buf);
|
||||
}
|
||||
|
@ -147,7 +147,7 @@ public class CacheTestUtils {
|
|||
|
||||
for (HFileBlockPair block : blocks) {
|
||||
try {
|
||||
if (toBeTested.getBlock(block.blockName, true) != null) {
|
||||
if (toBeTested.getBlock(block.blockName, true, false) != null) {
|
||||
toBeTested.cacheBlock(block.blockName, block.block);
|
||||
fail("Cache should not allow re-caching a block");
|
||||
}
|
||||
|
@ -177,7 +177,7 @@ public class CacheTestUtils {
|
|||
@Override
|
||||
public void doAnAction() throws Exception {
|
||||
ByteArrayCacheable returned = (ByteArrayCacheable) toBeTested
|
||||
.getBlock(key, false);
|
||||
.getBlock(key, false, false);
|
||||
assertArrayEquals(buf, returned.buf);
|
||||
totalQueries.incrementAndGet();
|
||||
}
|
||||
|
@ -216,7 +216,7 @@ public class CacheTestUtils {
|
|||
final ByteArrayCacheable bac = new ByteArrayCacheable(buf);
|
||||
|
||||
ByteArrayCacheable gotBack = (ByteArrayCacheable) toBeTested
|
||||
.getBlock(key, true);
|
||||
.getBlock(key, true, false);
|
||||
if (gotBack != null) {
|
||||
assertArrayEquals(gotBack.buf, bac.buf);
|
||||
} else {
|
||||
|
|
|
@ -239,7 +239,7 @@ public class TestCacheOnWrite {
|
|||
false, null);
|
||||
BlockCacheKey blockCacheKey = new BlockCacheKey(reader.getName(),
|
||||
offset, encodingInCache, block.getBlockType());
|
||||
boolean isCached = blockCache.getBlock(blockCacheKey, true) != null;
|
||||
boolean isCached = blockCache.getBlock(blockCacheKey, true, false) != null;
|
||||
boolean shouldBeCached = cowType.shouldBeCached(block.getBlockType());
|
||||
if (shouldBeCached != isCached) {
|
||||
throw new AssertionError(
|
||||
|
|
|
@ -87,7 +87,7 @@ public class TestHFileDataBlockEncoder {
|
|||
BlockCacheKey cacheKey = new BlockCacheKey("test", 0);
|
||||
blockCache.cacheBlock(cacheKey, cacheBlock);
|
||||
|
||||
HeapSize heapSize = blockCache.getBlock(cacheKey, false);
|
||||
HeapSize heapSize = blockCache.getBlock(cacheKey, false, false);
|
||||
assertTrue(heapSize instanceof HFileBlock);
|
||||
|
||||
HFileBlock returnedBlock = (HFileBlock) heapSize;;
|
||||
|
|
|
@ -99,7 +99,7 @@ public class TestLruBlockCache {
|
|||
|
||||
// Confirm empty
|
||||
for (CachedItem block : blocks) {
|
||||
assertTrue(cache.getBlock(block.cacheKey, true) == null);
|
||||
assertTrue(cache.getBlock(block.cacheKey, true, false) == null);
|
||||
}
|
||||
|
||||
// Add blocks
|
||||
|
@ -113,7 +113,7 @@ public class TestLruBlockCache {
|
|||
|
||||
// Check if all blocks are properly cached and retrieved
|
||||
for (CachedItem block : blocks) {
|
||||
HeapSize buf = cache.getBlock(block.cacheKey, true);
|
||||
HeapSize buf = cache.getBlock(block.cacheKey, true, false);
|
||||
assertTrue(buf != null);
|
||||
assertEquals(buf.heapSize(), block.heapSize());
|
||||
}
|
||||
|
@ -133,7 +133,7 @@ public class TestLruBlockCache {
|
|||
|
||||
// Check if all blocks are properly cached and retrieved
|
||||
for (CachedItem block : blocks) {
|
||||
HeapSize buf = cache.getBlock(block.cacheKey, true);
|
||||
HeapSize buf = cache.getBlock(block.cacheKey, true, false);
|
||||
assertTrue(buf != null);
|
||||
assertEquals(buf.heapSize(), block.heapSize());
|
||||
}
|
||||
|
@ -178,9 +178,9 @@ public class TestLruBlockCache {
|
|||
(maxSize * LruBlockCache.DEFAULT_ACCEPTABLE_FACTOR));
|
||||
|
||||
// All blocks except block 0 should be in the cache
|
||||
assertTrue(cache.getBlock(blocks[0].cacheKey, true) == null);
|
||||
assertTrue(cache.getBlock(blocks[0].cacheKey, true, false) == null);
|
||||
for(int i=1;i<blocks.length;i++) {
|
||||
assertEquals(cache.getBlock(blocks[i].cacheKey, true),
|
||||
assertEquals(cache.getBlock(blocks[i].cacheKey, true, false),
|
||||
blocks[i]);
|
||||
}
|
||||
}
|
||||
|
@ -202,7 +202,7 @@ public class TestLruBlockCache {
|
|||
for (CachedItem block : multiBlocks) {
|
||||
cache.cacheBlock(block.cacheKey, block);
|
||||
expectedCacheSize += block.cacheBlockHeapSize();
|
||||
assertEquals(cache.getBlock(block.cacheKey, true), block);
|
||||
assertEquals(cache.getBlock(block.cacheKey, true, false), block);
|
||||
}
|
||||
|
||||
// Add the single blocks (no get)
|
||||
|
@ -232,14 +232,14 @@ public class TestLruBlockCache {
|
|||
// This test makes multi go barely over its limit, in-memory
|
||||
// empty, and the rest in single. Two single evictions and
|
||||
// one multi eviction expected.
|
||||
assertTrue(cache.getBlock(singleBlocks[0].cacheKey, true) == null);
|
||||
assertTrue(cache.getBlock(multiBlocks[0].cacheKey, true) == null);
|
||||
assertTrue(cache.getBlock(singleBlocks[0].cacheKey, true, false) == null);
|
||||
assertTrue(cache.getBlock(multiBlocks[0].cacheKey, true, false) == null);
|
||||
|
||||
// And all others to be cached
|
||||
for(int i=1;i<4;i++) {
|
||||
assertEquals(cache.getBlock(singleBlocks[i].cacheKey, true),
|
||||
assertEquals(cache.getBlock(singleBlocks[i].cacheKey, true, false),
|
||||
singleBlocks[i]);
|
||||
assertEquals(cache.getBlock(multiBlocks[i].cacheKey, true),
|
||||
assertEquals(cache.getBlock(multiBlocks[i].cacheKey, true, false),
|
||||
multiBlocks[i]);
|
||||
}
|
||||
}
|
||||
|
@ -277,7 +277,7 @@ public class TestLruBlockCache {
|
|||
// Add and get multi blocks
|
||||
cache.cacheBlock(multiBlocks[i].cacheKey, multiBlocks[i]);
|
||||
expectedCacheSize += multiBlocks[i].cacheBlockHeapSize();
|
||||
cache.getBlock(multiBlocks[i].cacheKey, true);
|
||||
cache.getBlock(multiBlocks[i].cacheKey, true, false);
|
||||
|
||||
// Add memory blocks as such
|
||||
cache.cacheBlock(memoryBlocks[i].cacheKey, memoryBlocks[i], true);
|
||||
|
@ -299,10 +299,10 @@ public class TestLruBlockCache {
|
|||
assertEquals(1, cache.getEvictedCount());
|
||||
|
||||
// Verify oldest single block is the one evicted
|
||||
assertEquals(null, cache.getBlock(singleBlocks[0].cacheKey, true));
|
||||
assertEquals(null, cache.getBlock(singleBlocks[0].cacheKey, true, false));
|
||||
|
||||
// Change the oldest remaining single block to a multi
|
||||
cache.getBlock(singleBlocks[1].cacheKey, true);
|
||||
cache.getBlock(singleBlocks[1].cacheKey, true, false);
|
||||
|
||||
// Insert another single block
|
||||
cache.cacheBlock(singleBlocks[4].cacheKey, singleBlocks[4]);
|
||||
|
@ -312,7 +312,7 @@ public class TestLruBlockCache {
|
|||
assertEquals(2, cache.getEvictedCount());
|
||||
|
||||
// Oldest multi block should be evicted now
|
||||
assertEquals(null, cache.getBlock(multiBlocks[0].cacheKey, true));
|
||||
assertEquals(null, cache.getBlock(multiBlocks[0].cacheKey, true, false));
|
||||
|
||||
// Insert another memory block
|
||||
cache.cacheBlock(memoryBlocks[3].cacheKey, memoryBlocks[3], true);
|
||||
|
@ -322,7 +322,7 @@ public class TestLruBlockCache {
|
|||
assertEquals(3, cache.getEvictedCount());
|
||||
|
||||
// Oldest memory block should be evicted now
|
||||
assertEquals(null, cache.getBlock(memoryBlocks[0].cacheKey, true));
|
||||
assertEquals(null, cache.getBlock(memoryBlocks[0].cacheKey, true, false));
|
||||
|
||||
// Add a block that is twice as big (should force two evictions)
|
||||
CachedItem [] bigBlocks = generateFixedBlocks(3, blockSize*3, "big");
|
||||
|
@ -333,12 +333,12 @@ public class TestLruBlockCache {
|
|||
assertEquals(6, cache.getEvictedCount());
|
||||
|
||||
// Expect three remaining singles to be evicted
|
||||
assertEquals(null, cache.getBlock(singleBlocks[2].cacheKey, true));
|
||||
assertEquals(null, cache.getBlock(singleBlocks[3].cacheKey, true));
|
||||
assertEquals(null, cache.getBlock(singleBlocks[4].cacheKey, true));
|
||||
assertEquals(null, cache.getBlock(singleBlocks[2].cacheKey, true, false));
|
||||
assertEquals(null, cache.getBlock(singleBlocks[3].cacheKey, true, false));
|
||||
assertEquals(null, cache.getBlock(singleBlocks[4].cacheKey, true, false));
|
||||
|
||||
// Make the big block a multi block
|
||||
cache.getBlock(bigBlocks[0].cacheKey, true);
|
||||
cache.getBlock(bigBlocks[0].cacheKey, true, false);
|
||||
|
||||
// Cache another single big block
|
||||
cache.cacheBlock(bigBlocks[1].cacheKey, bigBlocks[1]);
|
||||
|
@ -348,9 +348,9 @@ public class TestLruBlockCache {
|
|||
assertEquals(9, cache.getEvictedCount());
|
||||
|
||||
// Expect three remaining multis to be evicted
|
||||
assertEquals(null, cache.getBlock(singleBlocks[1].cacheKey, true));
|
||||
assertEquals(null, cache.getBlock(multiBlocks[1].cacheKey, true));
|
||||
assertEquals(null, cache.getBlock(multiBlocks[2].cacheKey, true));
|
||||
assertEquals(null, cache.getBlock(singleBlocks[1].cacheKey, true, false));
|
||||
assertEquals(null, cache.getBlock(multiBlocks[1].cacheKey, true, false));
|
||||
assertEquals(null, cache.getBlock(multiBlocks[2].cacheKey, true, false));
|
||||
|
||||
// Cache a big memory block
|
||||
cache.cacheBlock(bigBlocks[2].cacheKey, bigBlocks[2], true);
|
||||
|
@ -360,9 +360,9 @@ public class TestLruBlockCache {
|
|||
assertEquals(12, cache.getEvictedCount());
|
||||
|
||||
// Expect three remaining in-memory to be evicted
|
||||
assertEquals(null, cache.getBlock(memoryBlocks[1].cacheKey, true));
|
||||
assertEquals(null, cache.getBlock(memoryBlocks[2].cacheKey, true));
|
||||
assertEquals(null, cache.getBlock(memoryBlocks[3].cacheKey, true));
|
||||
assertEquals(null, cache.getBlock(memoryBlocks[1].cacheKey, true, false));
|
||||
assertEquals(null, cache.getBlock(memoryBlocks[2].cacheKey, true, false));
|
||||
assertEquals(null, cache.getBlock(memoryBlocks[3].cacheKey, true, false));
|
||||
|
||||
|
||||
}
|
||||
|
@ -390,7 +390,7 @@ public class TestLruBlockCache {
|
|||
// Add 5 multi blocks
|
||||
for (CachedItem block : multiBlocks) {
|
||||
cache.cacheBlock(block.cacheKey, block);
|
||||
cache.getBlock(block.cacheKey, true);
|
||||
cache.getBlock(block.cacheKey, true, false);
|
||||
}
|
||||
|
||||
// Add 5 single blocks
|
||||
|
@ -405,10 +405,10 @@ public class TestLruBlockCache {
|
|||
assertEquals(4, cache.getEvictedCount());
|
||||
|
||||
// Should have been taken off equally from single and multi
|
||||
assertEquals(null, cache.getBlock(singleBlocks[0].cacheKey, true));
|
||||
assertEquals(null, cache.getBlock(singleBlocks[1].cacheKey, true));
|
||||
assertEquals(null, cache.getBlock(multiBlocks[0].cacheKey, true));
|
||||
assertEquals(null, cache.getBlock(multiBlocks[1].cacheKey, true));
|
||||
assertEquals(null, cache.getBlock(singleBlocks[0].cacheKey, true, false));
|
||||
assertEquals(null, cache.getBlock(singleBlocks[1].cacheKey, true, false));
|
||||
assertEquals(null, cache.getBlock(multiBlocks[0].cacheKey, true, false));
|
||||
assertEquals(null, cache.getBlock(multiBlocks[1].cacheKey, true, false));
|
||||
|
||||
// Let's keep "scanning" by adding single blocks. From here on we only
|
||||
// expect evictions from the single bucket.
|
||||
|
@ -459,7 +459,7 @@ public class TestLruBlockCache {
|
|||
|
||||
// Add and get multi blocks
|
||||
cache.cacheBlock(multiBlocks[i].cacheKey, multiBlocks[i]);
|
||||
cache.getBlock(multiBlocks[i].cacheKey, true);
|
||||
cache.getBlock(multiBlocks[i].cacheKey, true, false);
|
||||
|
||||
// Add memory blocks as such
|
||||
cache.cacheBlock(memoryBlocks[i].cacheKey, memoryBlocks[i], true);
|
||||
|
@ -479,16 +479,16 @@ public class TestLruBlockCache {
|
|||
|
||||
// And the oldest 5 blocks from each category should be gone
|
||||
for(int i=0;i<5;i++) {
|
||||
assertEquals(null, cache.getBlock(singleBlocks[i].cacheKey, true));
|
||||
assertEquals(null, cache.getBlock(multiBlocks[i].cacheKey, true));
|
||||
assertEquals(null, cache.getBlock(memoryBlocks[i].cacheKey, true));
|
||||
assertEquals(null, cache.getBlock(singleBlocks[i].cacheKey, true, false));
|
||||
assertEquals(null, cache.getBlock(multiBlocks[i].cacheKey, true, false));
|
||||
assertEquals(null, cache.getBlock(memoryBlocks[i].cacheKey, true, false));
|
||||
}
|
||||
|
||||
// And the newest 5 blocks should still be accessible
|
||||
for(int i=5;i<10;i++) {
|
||||
assertEquals(singleBlocks[i], cache.getBlock(singleBlocks[i].cacheKey, true));
|
||||
assertEquals(multiBlocks[i], cache.getBlock(multiBlocks[i].cacheKey, true));
|
||||
assertEquals(memoryBlocks[i], cache.getBlock(memoryBlocks[i].cacheKey, true));
|
||||
assertEquals(singleBlocks[i], cache.getBlock(singleBlocks[i].cacheKey, true, false));
|
||||
assertEquals(multiBlocks[i], cache.getBlock(multiBlocks[i].cacheKey, true, false));
|
||||
assertEquals(memoryBlocks[i], cache.getBlock(memoryBlocks[i].cacheKey, true, false));
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -213,7 +213,7 @@ public class TestCacheOnWriteInSchema {
|
|||
false, null);
|
||||
BlockCacheKey blockCacheKey = new BlockCacheKey(reader.getName(),
|
||||
offset);
|
||||
boolean isCached = cache.getBlock(blockCacheKey, true) != null;
|
||||
boolean isCached = cache.getBlock(blockCacheKey, true, false) != null;
|
||||
boolean shouldBeCached = cowType.shouldBeCached(block.getBlockType());
|
||||
if (shouldBeCached != isCached) {
|
||||
throw new AssertionError(
|
||||
|
|
Loading…
Reference in New Issue