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:
larsh 2012-11-13 06:54:02 +00:00
parent e7717f7bbc
commit fb934909a8
14 changed files with 133 additions and 135 deletions

View File

@ -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;

View File

@ -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.

View File

@ -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;
}

View File

@ -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.

View File

@ -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);
}
}
}

View File

@ -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);

View File

@ -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)

View File

@ -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;
}

View File

@ -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;

View File

@ -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 {

View File

@ -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(

View File

@ -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;;

View File

@ -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));
}
}

View File

@ -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(