HBASE-10752 Port HBASE-10270 'Remove DataBlockEncoding from BlockCacheKey' to trunk

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1579742 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Zhihong Yu 2014-03-20 21:16:05 +00:00
parent 39c1c2ee2d
commit 3a84ed22ee
16 changed files with 183 additions and 76 deletions

View File

@ -52,7 +52,7 @@ public interface BlockCache {
* @param repeat Whether this is a repeat lookup for the same block
* (used to avoid double counting cache misses when doing double-check locking)
* @return Block or null if block is not in 2 cache.
* @see HFileReaderV2#readBlock(long, long, boolean, boolean, boolean, BlockType)
* @see HFileReaderV2#readBlock(long, long, boolean, boolean, boolean, BlockType, DataBlockEncoding)
*/
Cacheable getBlock(BlockCacheKey cacheKey, boolean caching, boolean repeat);

View File

@ -19,7 +19,6 @@ package org.apache.hadoop.hbase.io.hfile;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.io.HeapSize;
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.ClassSize;
@ -31,39 +30,27 @@ public class BlockCacheKey implements HeapSize, java.io.Serializable {
private static final long serialVersionUID = -5199992013113130534L;
private final String hfileName;
private final long offset;
private final DataBlockEncoding encoding;
public BlockCacheKey(String file, long offset, DataBlockEncoding encoding,
BlockType blockType) {
this.hfileName = file;
this.offset = offset;
// We add encoding to the cache key only for data blocks. If the block type
// is unknown (this should never be the case in production), we just use
// the provided encoding, because it might be a data block.
this.encoding = (encoding != null && (blockType == null
|| blockType.isData())) ? encoding : DataBlockEncoding.NONE;
}
/**
* Construct a new BlockCacheKey
* @param file The name of the HFile this block belongs to.
* @param hfileName The name of the HFile this block belongs to.
* @param offset Offset of the block into the file
*/
public BlockCacheKey(String file, long offset) {
this(file, offset, DataBlockEncoding.NONE, null);
public BlockCacheKey(String hfileName, long offset) {
this.hfileName = hfileName;
this.offset = offset;
}
@Override
public int hashCode() {
return hfileName.hashCode() * 127 + (int) (offset ^ (offset >>> 32)) +
encoding.ordinal() * 17;
return hfileName.hashCode() * 127 + (int) (offset ^ (offset >>> 32));
}
@Override
public boolean equals(Object o) {
if (o instanceof BlockCacheKey) {
BlockCacheKey k = (BlockCacheKey) o;
return offset == k.offset && encoding == k.encoding
return offset == k.offset
&& (hfileName == null ? k.hfileName == null : hfileName
.equals(k.hfileName));
} else {
@ -73,18 +60,21 @@ public class BlockCacheKey implements HeapSize, java.io.Serializable {
@Override
public String toString() {
return hfileName + "_" + offset
+ (encoding == DataBlockEncoding.NONE ? "" : "_" + encoding);
return String.format("%s_%d", hfileName, offset);
}
public static final long FIXED_OVERHEAD = ClassSize.align(ClassSize.OBJECT +
ClassSize.REFERENCE + // this.hfileName
Bytes.SIZEOF_LONG); // this.offset
/**
* Strings have two bytes per character due to default Java Unicode encoding
* (hence length times 2).
*/
@Override
public long heapSize() {
return ClassSize.align(ClassSize.OBJECT + 2 * hfileName.length() +
Bytes.SIZEOF_LONG + 2 * ClassSize.REFERENCE);
return ClassSize.align(FIXED_OVERHEAD + ClassSize.STRING +
2 * hfileName.length());
}
// can't avoid this unfortunately
@ -95,10 +85,6 @@ public class BlockCacheKey implements HeapSize, java.io.Serializable {
return hfileName;
}
public DataBlockEncoding getDataBlockEncoding() {
return encoding;
}
public long getOffset() {
return offset;
}

View File

@ -438,9 +438,28 @@ public class HFile {
/** An abstraction used by the block index */
public interface CachingBlockReader {
/**
* Read in a file block.
* @param offset offset to read.
* @param onDiskBlockSize size of the block
* @param cacheBlock
* @param pread
* @param isCompaction is this block being read as part of a compaction
* @param expectedBlockType the block type we are expecting to read with this read operation, or
* null to read whatever block type is available and avoid checking (that might reduce
* caching efficiency of encoded data blocks)
* @param expectedDataBlockEncoding the data block encoding the caller is
* expecting data blocks to be in, or null to not perform this
* check and return the block irrespective of the encoding. This
* check only applies to data blocks and can be set to null when
* the caller is expecting to read a non-data block and has set
* expectedBlockType accordingly.
* @return Block wrapped in a ByteBuffer.
* @throws IOException
*/
HFileBlock readBlock(long offset, long onDiskBlockSize,
boolean cacheBlock, final boolean pread, final boolean isCompaction,
BlockType expectedBlockType)
BlockType expectedBlockType, DataBlockEncoding expectedDataBlockEncoding)
throws IOException;
}

View File

@ -167,17 +167,22 @@ public class HFileBlockIndex {
* @param key the key we are looking for
* @param keyOffset the offset of the key in its byte array
* @param keyLength the length of the key
* @param currentBlock the current block, to avoid re-reading the same
* block
* @param currentBlock the current block, to avoid re-reading the same block
* @param cacheBlocks
* @param pread
* @param isCompaction
* @param expectedDataBlockEncoding the data block encoding the caller is
* expecting the data block to be in, or null to not perform this
* check and return the block irrespective of the encoding
* @return reader a basic way to load blocks
* @throws IOException
*/
public HFileBlock seekToDataBlock(final byte[] key, int keyOffset,
int keyLength, HFileBlock currentBlock, boolean cacheBlocks,
boolean pread, boolean isCompaction)
boolean pread, boolean isCompaction, DataBlockEncoding expectedDataBlockEncoding)
throws IOException {
BlockWithScanInfo blockWithScanInfo = loadDataBlockWithScanInfo(key, keyOffset, keyLength,
currentBlock, cacheBlocks, pread, isCompaction);
currentBlock, cacheBlocks, pread, isCompaction, expectedDataBlockEncoding);
if (blockWithScanInfo == null) {
return null;
} else {
@ -198,13 +203,16 @@ public class HFileBlockIndex {
* @param cacheBlocks
* @param pread
* @param isCompaction
* @param expectedDataBlockEncoding the data block encoding the caller is
* expecting the data block to be in, or null to not perform this
* check and return the block irrespective of the encoding.
* @return the BlockWithScanInfo which contains the DataBlock with other scan info
* such as nextIndexedKey.
* @throws IOException
*/
public BlockWithScanInfo loadDataBlockWithScanInfo(final byte[] key, int keyOffset,
int keyLength, HFileBlock currentBlock, boolean cacheBlocks,
boolean pread, boolean isCompaction)
boolean pread, boolean isCompaction, DataBlockEncoding expectedDataBlockEncoding)
throws IOException {
int rootLevelIndex = rootBlockContainingKey(key, keyOffset, keyLength);
if (rootLevelIndex < 0 || rootLevelIndex >= blockOffsets.length) {
@ -252,7 +260,7 @@ public class HFileBlockIndex {
}
block = cachingBlockReader.readBlock(currentOffset,
currentOnDiskSize, shouldCache, pread, isCompaction,
expectedBlockType);
expectedBlockType, expectedDataBlockEncoding);
}
if (block == null) {
@ -328,7 +336,7 @@ public class HFileBlockIndex {
// Caching, using pread, assuming this is not a compaction.
HFileBlock midLeafBlock = cachingBlockReader.readBlock(
midLeafBlockOffset, midLeafBlockOnDiskSize, true, true, false,
BlockType.LEAF_INDEX);
BlockType.LEAF_INDEX, null);
ByteBuffer b = midLeafBlock.getBufferWithoutHeader();
int numDataBlocks = b.getInt();
@ -950,8 +958,7 @@ public class HFileBlockIndex {
if (blockCache != null) {
HFileBlock blockForCaching = blockWriter.getBlockForCaching();
blockCache.cacheBlock(new BlockCacheKey(nameForCaching,
beginOffset, DataBlockEncoding.NONE,
blockForCaching.getBlockType()), blockForCaching);
beginOffset), blockForCaching);
}
// Add intermediate index block size

View File

@ -66,6 +66,12 @@ public interface HFileDataBlockEncoder {
/** @return the data block encoding */
DataBlockEncoding getDataBlockEncoding();
/**
* @return the effective in-cache data block encoding, taking into account
* whether we are doing a compaction.
*/
public DataBlockEncoding getEffectiveEncodingInCache(boolean isCompaction);
/**
* Create an encoder specific encoding context object for writing. And the
* encoding context should also perform compression if compressionAlgorithm is

View File

@ -75,6 +75,20 @@ public class HFileDataBlockEncoderImpl implements HFileDataBlockEncoder {
return encoding;
}
public boolean useEncodedScanner(boolean isCompaction) {
if (isCompaction && encoding == DataBlockEncoding.NONE) {
return false;
}
return encoding != DataBlockEncoding.NONE;
}
@Override
public DataBlockEncoding getEffectiveEncodingInCache(boolean isCompaction) {
if (!useEncodedScanner(isCompaction)) {
return DataBlockEncoding.NONE;
}
return encoding;
}
/**
* Precondition: a non-encoded buffer. Postcondition: on-disk encoding.
*

View File

@ -209,6 +209,58 @@ public class HFileReaderV2 extends AbstractHFileReader {
return new ScannerV2(this, cacheBlocks, pread, isCompaction);
}
private HFileBlock getCachedBlock(BlockCacheKey cacheKey, boolean cacheBlock, boolean useLock,
boolean isCompaction, BlockType expectedBlockType,
DataBlockEncoding expectedDataBlockEncoding) throws IOException {
// Check cache for block. If found return.
if (cacheConf.isBlockCacheEnabled()) {
BlockCache cache = cacheConf.getBlockCache();
HFileBlock cachedBlock =
(HFileBlock) cache.getBlock(cacheKey, cacheBlock, useLock);
if (cachedBlock != null) {
validateBlockType(cachedBlock, expectedBlockType);
if (expectedDataBlockEncoding == null) {
return cachedBlock;
}
DataBlockEncoding actualDataBlockEncoding =
cachedBlock.getDataBlockEncoding();
// Block types other than data blocks always have
// DataBlockEncoding.NONE. To avoid false negative cache misses, only
// perform this check if cached block is a data block.
if (cachedBlock.getBlockType().isData() &&
!actualDataBlockEncoding.equals(expectedDataBlockEncoding)) {
// This mismatch may happen if a ScannerV2, which is used for say a
// compaction, tries to read an encoded block from the block cache.
// The reverse might happen when an EncodedScannerV2 tries to read
// un-encoded blocks which were cached earlier.
//
// Because returning a data block with an implicit BlockType mismatch
// will cause the requesting scanner to throw a disk read should be
// forced here. This will potentially cause a significant number of
// cache misses, so update so we should keep track of this as it might
// justify the work on a CompoundScannerV2.
if (!expectedDataBlockEncoding.equals(DataBlockEncoding.NONE) &&
!actualDataBlockEncoding.equals(DataBlockEncoding.NONE)) {
// If the block is encoded but the encoding does not match the
// expected encoding it is likely the encoding was changed but the
// block was not yet evicted. Evictions on file close happen async
// so blocks with the old encoding still linger in cache for some
// period of time. This event should be rare as it only happens on
// schema definition change.
LOG.info("Evicting cached block with key " + cacheKey +
" because of a data block encoding mismatch" +
"; expected: " + expectedDataBlockEncoding +
", actual: " + actualDataBlockEncoding);
cache.evictBlock(cacheKey);
}
return null;
}
return cachedBlock;
}
}
return null;
}
/**
* @param metaBlockName
* @param cacheBlock Add block to cache, if found
@ -239,13 +291,12 @@ public class HFileReaderV2 extends AbstractHFileReader {
synchronized (metaBlockIndexReader.getRootBlockKey(block)) {
// Check cache for block. If found return.
long metaBlockOffset = metaBlockIndexReader.getRootBlockOffset(block);
BlockCacheKey cacheKey = new BlockCacheKey(name, metaBlockOffset,
DataBlockEncoding.NONE, BlockType.META);
BlockCacheKey cacheKey = new BlockCacheKey(name, metaBlockOffset);
cacheBlock &= cacheConf.shouldCacheDataOnRead();
if (cacheConf.isBlockCacheEnabled()) {
HFileBlock cachedBlock =
(HFileBlock) cacheConf.getBlockCache().getBlock(cacheKey, cacheBlock, false);
HFileBlock cachedBlock = getCachedBlock(cacheKey, cacheBlock, false, false,
BlockType.META, null);
if (cachedBlock != null) {
// Return a distinct 'shallow copy' of the block,
// so pos does not get messed by the scanner
@ -271,7 +322,8 @@ public class HFileReaderV2 extends AbstractHFileReader {
}
/**
* Read in a file block.
* Read in a file block of the given {@link BlockType} and
* {@link DataBlockEncoding}.
* @param dataBlockOffset offset to read.
* @param onDiskBlockSize size of the block
* @param cacheBlock
@ -282,13 +334,19 @@ public class HFileReaderV2 extends AbstractHFileReader {
* read operation, or null to read whatever block type is available
* and avoid checking (that might reduce caching efficiency of
* encoded data blocks)
* @param expectedDataBlockEncoding the data block encoding the caller is
* expecting data blocks to be in, or null to not perform this
* check and return the block irrespective of the encoding. This
* check only applies to data blocks and can be set to null when
* the caller is expecting to read a non-data block and has set
* expectedBlockType accordingly.
* @return Block wrapped in a ByteBuffer.
* @throws IOException
*/
@Override
public HFileBlock readBlock(long dataBlockOffset, long onDiskBlockSize,
final boolean cacheBlock, boolean pread, final boolean isCompaction,
BlockType expectedBlockType)
BlockType expectedBlockType, DataBlockEncoding expectedDataBlockEncoding)
throws IOException {
if (dataBlockIndexReader == null) {
throw new IOException("Block index not loaded");
@ -306,9 +364,7 @@ public class HFileReaderV2 extends AbstractHFileReader {
// from doing).
BlockCacheKey cacheKey =
new BlockCacheKey(name, dataBlockOffset,
dataBlockEncoder.getDataBlockEncoding(),
expectedBlockType);
new BlockCacheKey(name, dataBlockOffset);
boolean useLock = false;
IdLock.Entry lockEntry = null;
@ -323,8 +379,8 @@ public class HFileReaderV2 extends AbstractHFileReader {
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);
HFileBlock cachedBlock = getCachedBlock(cacheKey, cacheBlock, useLock, isCompaction,
expectedBlockType, expectedDataBlockEncoding);
if (cachedBlock != null) {
validateBlockType(cachedBlock, expectedBlockType);
if (cachedBlock.getBlockType().isData()) {
@ -398,8 +454,7 @@ public class HFileReaderV2 extends AbstractHFileReader {
return;
}
BlockType actualBlockType = block.getBlockType();
if (actualBlockType == BlockType.ENCODED_DATA &&
expectedBlockType == BlockType.DATA) {
if (expectedBlockType.isData() && actualBlockType.isData()) {
// We consider DATA to match ENCODED_DATA for the purpose of this
// verification.
return;
@ -446,6 +501,10 @@ public class HFileReaderV2 extends AbstractHFileReader {
fsBlockReader.closeStreams();
}
public DataBlockEncoding getEffectiveEncodingInCache(boolean isCompaction) {
return dataBlockEncoder.getEffectiveEncodingInCache(isCompaction);
}
/** For testing */
@Override
HFileBlock.FSReader getUncachedBlockReader() {
@ -493,7 +552,7 @@ public class HFileReaderV2 extends AbstractHFileReader {
reader.getDataBlockIndexReader();
BlockWithScanInfo blockWithScanInfo =
indexReader.loadDataBlockWithScanInfo(key, offset, length, block,
cacheBlocks, pread, isCompaction);
cacheBlocks, pread, isCompaction, getEffectiveDataBlockEncoding());
if (blockWithScanInfo == null || blockWithScanInfo.getHFileBlock() == null) {
// This happens if the key e.g. falls before the beginning of the file.
return -1;
@ -547,7 +606,8 @@ public class HFileReaderV2 extends AbstractHFileReader {
throws IOException {
HFileBlock seekToBlock =
reader.getDataBlockIndexReader().seekToDataBlock(key, offset, length,
block, cacheBlocks, pread, isCompaction);
block, cacheBlocks, pread, isCompaction,
((HFileReaderV2)reader).getEffectiveEncodingInCache(isCompaction));
if (seekToBlock == null) {
return false;
}
@ -568,7 +628,7 @@ public class HFileReaderV2 extends AbstractHFileReader {
// figure out the size.
seekToBlock = reader.readBlock(previousBlockOffset,
seekToBlock.getOffset() - previousBlockOffset, cacheBlocks,
pread, isCompaction, BlockType.DATA);
pread, isCompaction, BlockType.DATA, getEffectiveDataBlockEncoding());
// TODO shortcut: seek forward in this block to the last key of the
// block.
}
@ -605,11 +665,15 @@ public class HFileReaderV2 extends AbstractHFileReader {
curBlock = reader.readBlock(curBlock.getOffset()
+ curBlock.getOnDiskSizeWithHeader(),
curBlock.getNextBlockOnDiskSizeWithHeader(), cacheBlocks, pread,
isCompaction, null);
isCompaction, null, getEffectiveDataBlockEncoding());
} while (!curBlock.getBlockType().isData());
return curBlock;
}
public DataBlockEncoding getEffectiveDataBlockEncoding() {
return ((HFileReaderV2)reader).getEffectiveEncodingInCache(isCompaction);
}
/**
* Compare the given key against the current key
* @param comparator
@ -763,7 +827,7 @@ public class HFileReaderV2 extends AbstractHFileReader {
}
block = reader.readBlock(firstDataBlockOffset, -1, cacheBlocks, pread,
isCompaction, BlockType.DATA);
isCompaction, BlockType.DATA, getEffectiveDataBlockEncoding());
if (block.getOffset() < 0) {
throw new IOException("Invalid block offset: " + block.getOffset());
}
@ -1050,7 +1114,7 @@ public class HFileReaderV2 extends AbstractHFileReader {
}
block = reader.readBlock(firstDataBlockOffset, -1, cacheBlocks, pread,
isCompaction, BlockType.DATA);
isCompaction, BlockType.DATA, getEffectiveDataBlockEncoding());
if (block.getOffset() < 0) {
throw new IOException("Invalid block offset: " + block.getOffset());
}

View File

@ -196,8 +196,7 @@ public class HFileWriterV2 extends AbstractHFileWriter {
private void doCacheOnWrite(long offset) {
HFileBlock cacheFormatBlock = fsBlockWriter.getBlockForCaching();
cacheConf.getBlockCache().cacheBlock(
new BlockCacheKey(name, offset, blockEncoder.getDataBlockEncoding(),
cacheFormatBlock.getBlockType()), cacheFormatBlock);
new BlockCacheKey(name, offset), cacheFormatBlock);
}
/**

View File

@ -347,10 +347,6 @@ public class LruBlockCache implements ResizableBlockCache, HeapSize {
/**
* Cache the block with the specified name and buffer.
* <p>
* It is assumed this will NEVER be called on an already cached block. If
* that is done, it is assumed that you are reinserting the same exact
* block due to a race condition and will update the buffer but not modify
* the size of the cache.
* @param cacheKey block's cache key
* @param buf block buffer
*/
@ -381,7 +377,7 @@ public class LruBlockCache implements ResizableBlockCache, HeapSize {
* @param repeat Whether this is a repeat lookup for the same block
* (used to avoid double counting cache misses when doing double-check locking)
* @return buffer of specified cache key, or null if not in cache
* @see HFileReaderV2#readBlock(long, long, boolean, boolean, boolean, BlockType)
* @see HFileReaderV2#readBlock(long, long, boolean, boolean, boolean, BlockType, DataBlockEncoding)
*/
@Override
public Cacheable getBlock(BlockCacheKey cacheKey, boolean caching, boolean repeat) {
@ -917,8 +913,9 @@ public class LruBlockCache implements ResizableBlockCache, HeapSize {
public Map<DataBlockEncoding, Integer> getEncodingCountsForTest() {
Map<DataBlockEncoding, Integer> counts =
new EnumMap<DataBlockEncoding, Integer>(DataBlockEncoding.class);
for (BlockCacheKey cacheKey : map.keySet()) {
DataBlockEncoding encoding = cacheKey.getDataBlockEncoding();
for (CachedBlock block : map.values()) {
DataBlockEncoding encoding =
((HFileBlock) block.getBuffer()).getDataBlockEncoding();
Integer count = counts.get(encoding);
counts.put(encoding, (count == null ? 0 : count) + 1);
}

View File

@ -68,6 +68,11 @@ public class NoOpDataBlockEncoder implements HFileDataBlockEncoder {
return DataBlockEncoding.NONE;
}
@Override
public DataBlockEncoding getEffectiveEncodingInCache(boolean isCompaction) {
return DataBlockEncoding.NONE;
}
@Override
public String toString() {
return getClass().getSimpleName();

View File

@ -98,7 +98,7 @@ public class CompoundBloomFilter extends CompoundBloomFilterBase
// We cache the block and use a positional read.
bloomBlock = reader.readBlock(index.getRootBlockOffset(block),
index.getRootBlockDataSize(block), true, true, false,
BlockType.BLOOM_CHUNK);
BlockType.BLOOM_CHUNK, null);
} catch (IOException ex) {
// The Bloom filter is broken, turn it off.
throw new IllegalArgumentException(

View File

@ -337,11 +337,10 @@ public class TestHeapSize {
assertEquals(expected, actual);
}
// Block cache key overhead
// Block cache key overhead. Only tests fixed overhead as estimating heap
// size of strings is hard.
cl = BlockCacheKey.class;
// Passing zero length file name, because estimateBase does not handle
// deep overhead.
actual = new BlockCacheKey("", 0).heapSize();
actual = BlockCacheKey.FIXED_OVERHEAD;
expected = ClassSize.estimateBase(cl, false);
if (expected != actual) {
ClassSize.estimateBase(cl, true);

View File

@ -245,9 +245,9 @@ public class TestCacheOnWrite {
// Flags: don't cache the block, use pread, this is not a compaction.
// Also, pass null for expected block type to avoid checking it.
HFileBlock block = reader.readBlock(offset, onDiskSize, false, true,
false, null);
false, null, encodingInCache);
BlockCacheKey blockCacheKey = new BlockCacheKey(reader.getName(),
offset, encodingInCache, block.getBlockType());
offset);
boolean isCached = blockCache.getBlock(blockCacheKey, true, false) != null;
boolean shouldBeCached = cowType.shouldBeCached(block.getBlockType());
if (shouldBeCached != isCached) {

View File

@ -48,6 +48,7 @@ import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.MediumTests;
import org.apache.hadoop.hbase.fs.HFileSystem;
import org.apache.hadoop.hbase.io.compress.Compression;
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
import org.apache.hadoop.hbase.io.hfile.HFileBlockIndex.BlockIndexChunk;
import org.apache.hadoop.hbase.io.hfile.HFileBlockIndex.BlockIndexReader;
import org.apache.hadoop.hbase.util.Bytes;
@ -164,7 +165,7 @@ public class TestHFileBlockIndex {
@Override
public HFileBlock readBlock(long offset, long onDiskSize,
boolean cacheBlock, boolean pread, boolean isCompaction,
BlockType expectedBlockType)
BlockType expectedBlockType, DataBlockEncoding expectedDataBlockEncoding)
throws IOException {
if (offset == prevOffset && onDiskSize == prevOnDiskSize &&
pread == prevPread) {
@ -214,7 +215,7 @@ public class TestHFileBlockIndex {
assertTrue(key != null);
assertTrue(indexReader != null);
HFileBlock b = indexReader.seekToDataBlock(key, 0, key.length, null,
true, true, false);
true, true, false, null);
if (Bytes.BYTES_RAWCOMPARATOR.compare(key, firstKeyInFile) < 0) {
assertTrue(b == null);
++i;

View File

@ -125,6 +125,15 @@ public class TestLruBlockCache {
assertEquals(buf.heapSize(), block.heapSize());
}
// Re-add same blocks and ensure nothing has changed
long expectedBlockCount = cache.getBlockCount();
for (CachedItem block : blocks) {
cache.cacheBlock(block.cacheKey, block);
}
assertEquals(
"Cache should ignore cache requests for blocks already in cache",
expectedBlockCount, cache.getBlockCount());
// Verify correctly calculated cache heap size
assertEquals(expectedCacheSize, cache.heapSize());

View File

@ -40,6 +40,7 @@ import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.MediumTests;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.fs.HFileSystem;
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
import org.apache.hadoop.hbase.io.hfile.BlockCache;
import org.apache.hadoop.hbase.io.hfile.BlockCacheKey;
import org.apache.hadoop.hbase.io.hfile.BlockType;
@ -233,7 +234,7 @@ public class TestCacheOnWriteInSchema {
// Flags: don't cache the block, use pread, this is not a compaction.
// Also, pass null for expected block type to avoid checking it.
HFileBlock block = reader.readBlock(offset, onDiskSize, false, true,
false, null);
false, null, DataBlockEncoding.NONE);
BlockCacheKey blockCacheKey = new BlockCacheKey(reader.getName(),
offset);
boolean isCached = cache.getBlock(blockCacheKey, true, false) != null;