HBASE-12295 Prevent block eviction under us if reads are in progress from

the BBs (Ram)
This commit is contained in:
ramkrishna 2015-07-21 21:15:32 +05:30
parent 3b6db26863
commit ccb22bd80d
39 changed files with 2380 additions and 355 deletions

View File

@ -0,0 +1,36 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
/**
* A cell implementing this interface would mean that the memory area backing this cell will refer
* to a memory area that could be part of a larger common memory area used by the
* RegionServer. If an exclusive instance is required, use the {@link #cloneToCell()} to have the
* contents of the cell copied to an exclusive memory area.
*/
@InterfaceAudience.Private
public interface ShareableMemory {
/**
* Does a deep copy of the contents to a new memory area and
* returns it in the form of a cell.
* @return Cell the deep cloned cell
*/
public Cell cloneToCell();
}

View File

@ -283,7 +283,7 @@ abstract class BufferedDataBlockEncoder implements DataBlockEncoder {
/** /**
* Copies only the key part of the keybuffer by doing a deep copy and passes the * Copies only the key part of the keybuffer by doing a deep copy and passes the
* seeker state members for taking a clone. * seeker state members for taking a clone.
* Note that the value byte[] part is still pointing to the currentBuffer and the * Note that the value byte[] part is still pointing to the currentBuffer and
* represented by the valueOffset and valueLength * represented by the valueOffset and valueLength
*/ */
// We return this as a Cell to the upper layers of read flow and might try setting a new SeqId // We return this as a Cell to the upper layers of read flow and might try setting a new SeqId

View File

@ -56,6 +56,7 @@ public class HFileContext implements HeapSize, Cloneable {
/** Encryption algorithm and key used */ /** Encryption algorithm and key used */
private Encryption.Context cryptoContext = Encryption.Context.NONE; private Encryption.Context cryptoContext = Encryption.Context.NONE;
private long fileCreateTime; private long fileCreateTime;
private String hfileName;
//Empty constructor. Go with setters //Empty constructor. Go with setters
public HFileContext() { public HFileContext() {
@ -77,12 +78,13 @@ public class HFileContext implements HeapSize, Cloneable {
this.encoding = context.encoding; this.encoding = context.encoding;
this.cryptoContext = context.cryptoContext; this.cryptoContext = context.cryptoContext;
this.fileCreateTime = context.fileCreateTime; this.fileCreateTime = context.fileCreateTime;
this.hfileName = context.hfileName;
} }
public HFileContext(boolean useHBaseChecksum, boolean includesMvcc, boolean includesTags, HFileContext(boolean useHBaseChecksum, boolean includesMvcc, boolean includesTags,
Compression.Algorithm compressAlgo, boolean compressTags, ChecksumType checksumType, Compression.Algorithm compressAlgo, boolean compressTags, ChecksumType checksumType,
int bytesPerChecksum, int blockSize, DataBlockEncoding encoding, int bytesPerChecksum, int blockSize, DataBlockEncoding encoding,
Encryption.Context cryptoContext, long fileCreateTime) { Encryption.Context cryptoContext, long fileCreateTime, String hfileName) {
this.usesHBaseChecksum = useHBaseChecksum; this.usesHBaseChecksum = useHBaseChecksum;
this.includesMvcc = includesMvcc; this.includesMvcc = includesMvcc;
this.includesTags = includesTags; this.includesTags = includesTags;
@ -96,6 +98,7 @@ public class HFileContext implements HeapSize, Cloneable {
} }
this.cryptoContext = cryptoContext; this.cryptoContext = cryptoContext;
this.fileCreateTime = fileCreateTime; this.fileCreateTime = fileCreateTime;
this.hfileName = hfileName;
} }
/** /**
@ -119,10 +122,6 @@ public class HFileContext implements HeapSize, Cloneable {
return compressAlgo; return compressAlgo;
} }
public void setCompression(Compression.Algorithm compressAlgo) {
this.compressAlgo = compressAlgo;
}
public boolean isUseHBaseChecksum() { public boolean isUseHBaseChecksum() {
return usesHBaseChecksum; return usesHBaseChecksum;
} }
@ -175,10 +174,6 @@ public class HFileContext implements HeapSize, Cloneable {
return encoding; return encoding;
} }
public void setDataBlockEncoding(DataBlockEncoding encoding) {
this.encoding = encoding;
}
public Encryption.Context getEncryptionContext() { public Encryption.Context getEncryptionContext() {
return cryptoContext; return cryptoContext;
} }
@ -187,6 +182,10 @@ public class HFileContext implements HeapSize, Cloneable {
this.cryptoContext = cryptoContext; this.cryptoContext = cryptoContext;
} }
public String getHFileName() {
return this.hfileName;
}
/** /**
* HeapSize implementation * HeapSize implementation
* NOTE : The heapsize should be altered as and when new state variable are added * NOTE : The heapsize should be altered as and when new state variable are added
@ -196,11 +195,14 @@ public class HFileContext implements HeapSize, Cloneable {
public long heapSize() { public long heapSize() {
long size = ClassSize.align(ClassSize.OBJECT + long size = ClassSize.align(ClassSize.OBJECT +
// Algorithm reference, encodingon, checksumtype, Encryption.Context reference // Algorithm reference, encodingon, checksumtype, Encryption.Context reference
4 * ClassSize.REFERENCE + 5 * ClassSize.REFERENCE +
2 * Bytes.SIZEOF_INT + 2 * Bytes.SIZEOF_INT +
// usesHBaseChecksum, includesMvcc, includesTags and compressTags // usesHBaseChecksum, includesMvcc, includesTags and compressTags
4 * Bytes.SIZEOF_BOOLEAN + 4 * Bytes.SIZEOF_BOOLEAN +
Bytes.SIZEOF_LONG); Bytes.SIZEOF_LONG);
if (this.hfileName != null) {
size += ClassSize.STRING + this.hfileName.length();
}
return size; return size;
} }
@ -227,6 +229,10 @@ public class HFileContext implements HeapSize, Cloneable {
sb.append(" compressAlgo="); sb.append(compressAlgo); sb.append(" compressAlgo="); sb.append(compressAlgo);
sb.append(" compressTags="); sb.append(compressTags); sb.append(" compressTags="); sb.append(compressTags);
sb.append(" cryptoContext=[ "); sb.append(cryptoContext); sb.append(" ]"); sb.append(" cryptoContext=[ "); sb.append(cryptoContext); sb.append(" ]");
if (hfileName != null) {
sb.append(" name=");
sb.append(hfileName);
}
sb.append(" ]"); sb.append(" ]");
return sb.toString(); return sb.toString();
} }

View File

@ -53,6 +53,8 @@ public class HFileContextBuilder {
private Encryption.Context cryptoContext = Encryption.Context.NONE; private Encryption.Context cryptoContext = Encryption.Context.NONE;
private long fileCreateTime = 0; private long fileCreateTime = 0;
private String hfileName = null;
public HFileContextBuilder withHBaseCheckSum(boolean useHBaseCheckSum) { public HFileContextBuilder withHBaseCheckSum(boolean useHBaseCheckSum) {
this.usesHBaseChecksum = useHBaseCheckSum; this.usesHBaseChecksum = useHBaseCheckSum;
return this; return this;
@ -108,9 +110,14 @@ public class HFileContextBuilder {
return this; return this;
} }
public HFileContextBuilder withHFileName(String name) {
this.hfileName = name;
return this;
}
public HFileContext build() { public HFileContext build() {
return new HFileContext(usesHBaseChecksum, includesMvcc, includesTags, compression, return new HFileContext(usesHBaseChecksum, includesMvcc, includesTags, compression,
compressTags, checksumType, bytesPerChecksum, blocksize, encoding, cryptoContext, compressTags, checksumType, bytesPerChecksum, blocksize, encoding, cryptoContext,
fileCreateTime); fileCreateTime, hfileName);
} }
} }

View File

@ -21,6 +21,7 @@ package org.apache.hadoop.hbase.io.hfile;
import java.util.Iterator; import java.util.Iterator;
import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.io.hfile.Cacheable.MemoryType;
/** /**
* Block cache interface. Anything that implements the {@link Cacheable} * Block cache interface. Anything that implements the {@link Cacheable}
@ -116,4 +117,16 @@ public interface BlockCache extends Iterable<CachedBlock> {
* @return The list of sub blockcaches that make up this one; returns null if no sub caches. * @return The list of sub blockcaches that make up this one; returns null if no sub caches.
*/ */
BlockCache [] getBlockCaches(); BlockCache [] getBlockCaches();
/**
* Called when the scanner using the block decides to return the block once its usage
* is over.
* This API should be called after the block is used, failing to do so may have adverse effects
* by preventing the blocks from being evicted because of which it will prevent new hot blocks
* from getting added to the block cache. The implementation of the BlockCache will decide
* on what to be done with the block based on the memory type of the block's {@link MemoryType}.
* @param cacheKey the cache key of the block
* @param block the hfileblock to be returned
*/
void returnBlock(BlockCacheKey cacheKey, Cacheable block);
} }

View File

@ -60,4 +60,19 @@ public interface Cacheable extends HeapSize {
* @return the block type of this cached HFile block * @return the block type of this cached HFile block
*/ */
BlockType getBlockType(); BlockType getBlockType();
/**
* @return the {@code MemoryType} of this Cacheable
*/
MemoryType getMemoryType();
/**
* SHARED means when this Cacheable is read back from cache it refers to the same memory area as
* used by the cache for caching it.
* EXCLUSIVE means when this Cacheable is read back from cache, the data was copied to an
* exclusive memory area of this Cacheable.
*/
public static enum MemoryType {
SHARED, EXCLUSIVE;
}
} }

View File

@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.io.hfile;
import java.io.IOException; import java.io.IOException;
import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.io.hfile.Cacheable.MemoryType;
import org.apache.hadoop.hbase.nio.ByteBuff; import org.apache.hadoop.hbase.nio.ByteBuff;
/** /**
@ -36,14 +37,14 @@ public interface CacheableDeserializer<T extends Cacheable> {
T deserialize(ByteBuff b) throws IOException; T deserialize(ByteBuff b) throws IOException;
/** /**
*
* @param b * @param b
* @param reuse true if Cacheable object can use the given buffer as its * @param reuse true if Cacheable object can use the given buffer as its
* content * content
* @param memType the {@link MemoryType} of the buffer
* @return T the deserialized object. * @return T the deserialized object.
* @throws IOException * @throws IOException
*/ */
T deserialize(ByteBuff b, boolean reuse) throws IOException; T deserialize(ByteBuff b, boolean reuse, MemoryType memType) throws IOException;
/** /**
* Get the identifier of this deserialiser. Identifier is unique for each * Get the identifier of this deserialiser. Identifier is unique for each

View File

@ -25,6 +25,8 @@ import org.apache.hadoop.hbase.io.HeapSize;
import org.apache.hadoop.hbase.io.hfile.BlockType.BlockCategory; import org.apache.hadoop.hbase.io.hfile.BlockType.BlockCategory;
import org.apache.hadoop.hbase.io.hfile.bucket.BucketCache; import org.apache.hadoop.hbase.io.hfile.bucket.BucketCache;
import com.google.common.annotations.VisibleForTesting;
/** /**
* CombinedBlockCache is an abstraction layer that combines * CombinedBlockCache is an abstraction layer that combines
@ -219,4 +221,16 @@ public class CombinedBlockCache implements ResizableBlockCache, HeapSize {
public void setMaxSize(long size) { public void setMaxSize(long size) {
this.lruCache.setMaxSize(size); this.lruCache.setMaxSize(size);
} }
@Override
public void returnBlock(BlockCacheKey cacheKey, Cacheable block) {
// A noop
this.lruCache.returnBlock(cacheKey, block);
this.l2Cache.returnBlock(cacheKey, block);
}
@VisibleForTesting
public int getRefCount(BlockCacheKey cacheKey) {
return ((BucketCache) this.l2Cache).getRefCount(cacheKey);
}
} }

View File

@ -119,11 +119,15 @@ public class CompoundBloomFilter extends CompoundBloomFilterBase
"Failed to load Bloom block for key " "Failed to load Bloom block for key "
+ Bytes.toStringBinary(key, keyOffset, keyLength), ex); + Bytes.toStringBinary(key, keyOffset, keyLength), ex);
} }
try {
ByteBuff bloomBuf = bloomBlock.getBufferReadOnly(); ByteBuff bloomBuf = bloomBlock.getBufferReadOnly();
result = BloomFilterUtil.contains(key, keyOffset, keyLength, result =
bloomBuf, bloomBlock.headerSize(), BloomFilterUtil.contains(key, keyOffset, keyLength, bloomBuf, bloomBlock.headerSize(),
bloomBlock.getUncompressedSizeWithoutHeader(), hash, hashCount); bloomBlock.getUncompressedSizeWithoutHeader(), hash, hashCount);
} finally {
// After the use return back the block if it was served from a cache.
reader.returnBlock(bloomBlock);
}
} }
if (numQueriesPerChunk != null && block >= 0) { if (numQueriesPerChunk != null && block >= 0) {

View File

@ -54,7 +54,6 @@ import org.apache.hadoop.hbase.fs.HFileSystem;
import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper; import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
import org.apache.hadoop.hbase.io.compress.Compression; import org.apache.hadoop.hbase.io.compress.Compression;
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding; import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
import org.apache.hadoop.hbase.nio.ByteBuff;
import org.apache.hadoop.hbase.protobuf.ProtobufMagic; import org.apache.hadoop.hbase.protobuf.ProtobufMagic;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos; import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
@ -374,6 +373,12 @@ public class HFile {
final boolean updateCacheMetrics, BlockType expectedBlockType, final boolean updateCacheMetrics, BlockType expectedBlockType,
DataBlockEncoding expectedDataBlockEncoding) DataBlockEncoding expectedDataBlockEncoding)
throws IOException; throws IOException;
/**
* Return the given block back to the cache, if it was obtained from cache.
* @param block Block to be returned.
*/
void returnBlock(HFileBlock block);
} }
/** An interface used by clients to open and iterate an {@link HFile}. */ /** An interface used by clients to open and iterate an {@link HFile}. */
@ -389,7 +394,7 @@ public class HFile {
HFileScanner getScanner(boolean cacheBlocks, final boolean pread, final boolean isCompaction); HFileScanner getScanner(boolean cacheBlocks, final boolean pread, final boolean isCompaction);
ByteBuff getMetaBlock(String metaBlockName, boolean cacheBlock) throws IOException; HFileBlock getMetaBlock(String metaBlockName, boolean cacheBlock) throws IOException;
Map<byte[], byte[]> loadFileInfo() throws IOException; Map<byte[], byte[]> loadFileInfo() throws IOException;

View File

@ -121,7 +121,8 @@ public class HFileBlock implements Cacheable {
static final CacheableDeserializer<Cacheable> blockDeserializer = static final CacheableDeserializer<Cacheable> blockDeserializer =
new CacheableDeserializer<Cacheable>() { new CacheableDeserializer<Cacheable>() {
public HFileBlock deserialize(ByteBuff buf, boolean reuse) throws IOException{ public HFileBlock deserialize(ByteBuff buf, boolean reuse, MemoryType memType)
throws IOException {
buf.limit(buf.limit() - HFileBlock.EXTRA_SERIALIZATION_SPACE).rewind(); buf.limit(buf.limit() - HFileBlock.EXTRA_SERIALIZATION_SPACE).rewind();
ByteBuff newByteBuffer; ByteBuff newByteBuffer;
if (reuse) { if (reuse) {
@ -135,7 +136,7 @@ public class HFileBlock implements Cacheable {
buf.position(buf.limit()); buf.position(buf.limit());
buf.limit(buf.limit() + HFileBlock.EXTRA_SERIALIZATION_SPACE); buf.limit(buf.limit() + HFileBlock.EXTRA_SERIALIZATION_SPACE);
boolean usesChecksum = buf.get() == (byte)1; boolean usesChecksum = buf.get() == (byte)1;
HFileBlock hFileBlock = new HFileBlock(newByteBuffer, usesChecksum); HFileBlock hFileBlock = new HFileBlock(newByteBuffer, usesChecksum, memType);
hFileBlock.offset = buf.getLong(); hFileBlock.offset = buf.getLong();
hFileBlock.nextBlockOnDiskSizeWithHeader = buf.getInt(); hFileBlock.nextBlockOnDiskSizeWithHeader = buf.getInt();
if (hFileBlock.hasNextBlockHeader()) { if (hFileBlock.hasNextBlockHeader()) {
@ -152,7 +153,7 @@ public class HFileBlock implements Cacheable {
@Override @Override
public HFileBlock deserialize(ByteBuff b) throws IOException { public HFileBlock deserialize(ByteBuff b) throws IOException {
// Used only in tests // Used only in tests
return deserialize(b, false); return deserialize(b, false, MemoryType.EXCLUSIVE);
} }
}; };
private static final int deserializerIdentifier; private static final int deserializerIdentifier;
@ -198,6 +199,8 @@ public class HFileBlock implements Cacheable {
*/ */
private int nextBlockOnDiskSizeWithHeader = -1; private int nextBlockOnDiskSizeWithHeader = -1;
private MemoryType memType = MemoryType.EXCLUSIVE;
/** /**
* Creates a new {@link HFile} block from the given fields. This constructor * Creates a new {@link HFile} block from the given fields. This constructor
* is mostly used when the block data has already been read and uncompressed, * is mostly used when the block data has already been read and uncompressed,
@ -255,15 +258,24 @@ public class HFileBlock implements Cacheable {
HFileBlock(ByteBuffer b, boolean usesHBaseChecksum) throws IOException { HFileBlock(ByteBuffer b, boolean usesHBaseChecksum) throws IOException {
this(new SingleByteBuff(b), usesHBaseChecksum); this(new SingleByteBuff(b), usesHBaseChecksum);
} }
/** /**
* Creates a block from an existing buffer starting with a header. Rewinds * Creates a block from an existing buffer starting with a header. Rewinds
* and takes ownership of the buffer. By definition of rewind, ignores the * and takes ownership of the buffer. By definition of rewind, ignores the
* buffer position, but if you slice the buffer beforehand, it will rewind * buffer position, but if you slice the buffer beforehand, it will rewind
* to that point. The reason this has a minorNumber and not a majorNumber is * to that point.
* because majorNumbers indicate the format of a HFile whereas minorNumbers
* indicate the format inside a HFileBlock.
*/ */
HFileBlock(ByteBuff b, boolean usesHBaseChecksum) throws IOException { HFileBlock(ByteBuff b, boolean usesHBaseChecksum) throws IOException {
this(b, usesHBaseChecksum, MemoryType.EXCLUSIVE);
}
/**
* Creates a block from an existing buffer starting with a header. Rewinds
* and takes ownership of the buffer. By definition of rewind, ignores the
* buffer position, but if you slice the buffer beforehand, it will rewind
* to that point.
*/
HFileBlock(ByteBuff b, boolean usesHBaseChecksum, MemoryType memType) throws IOException {
b.rewind(); b.rewind();
blockType = BlockType.read(b); blockType = BlockType.read(b);
onDiskSizeWithoutHeader = b.getInt(); onDiskSizeWithoutHeader = b.getInt();
@ -282,6 +294,7 @@ public class HFileBlock implements Cacheable {
HConstants.HFILEBLOCK_HEADER_SIZE_NO_CHECKSUM; HConstants.HFILEBLOCK_HEADER_SIZE_NO_CHECKSUM;
} }
this.fileContext = contextBuilder.build(); this.fileContext = contextBuilder.build();
this.memType = memType;
buf = b; buf = b;
buf.rewind(); buf.rewind();
} }
@ -650,8 +663,8 @@ public class HFileBlock implements Cacheable {
public long heapSize() { public long heapSize() {
long size = ClassSize.align( long size = ClassSize.align(
ClassSize.OBJECT + ClassSize.OBJECT +
// Block type, multi byte buffer and meta references // Block type, multi byte buffer, MemoryType and meta references
3 * ClassSize.REFERENCE + 4 * ClassSize.REFERENCE +
// On-disk size, uncompressed size, and next block's on-disk size // On-disk size, uncompressed size, and next block's on-disk size
// bytePerChecksum and onDiskDataSize // bytePerChecksum and onDiskDataSize
4 * Bytes.SIZEOF_INT + 4 * Bytes.SIZEOF_INT +
@ -1885,6 +1898,11 @@ public class HFileBlock implements Cacheable {
return this.fileContext; return this.fileContext;
} }
@Override
public MemoryType getMemoryType() {
return this.memType;
}
/** /**
* Convert the contents of the block header into a human readable string. * Convert the contents of the block header into a human readable string.
* This is mostly helpful for debugging. This assumes that the block * This is mostly helpful for debugging. This assumes that the block

View File

@ -32,7 +32,6 @@ import java.util.concurrent.atomic.AtomicReference;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.Cell;
@ -42,6 +41,7 @@ import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValue.KeyOnlyKeyValue; import org.apache.hadoop.hbase.KeyValue.KeyOnlyKeyValue;
import org.apache.hadoop.hbase.KeyValueUtil; import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.io.HeapSize; import org.apache.hadoop.hbase.io.HeapSize;
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding; import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
import org.apache.hadoop.hbase.io.hfile.HFile.CachingBlockReader; import org.apache.hadoop.hbase.io.hfile.HFile.CachingBlockReader;
@ -295,12 +295,12 @@ public class HFileBlockIndex {
int lookupLevel = 1; // How many levels deep we are in our lookup. int lookupLevel = 1; // How many levels deep we are in our lookup.
int index = -1; int index = -1;
HFileBlock block; HFileBlock block = null;
boolean dataBlock = false;
KeyOnlyKeyValue tmpNextIndexKV = new KeyValue.KeyOnlyKeyValue(); KeyOnlyKeyValue tmpNextIndexKV = new KeyValue.KeyOnlyKeyValue();
while (true) { while (true) {
try {
if (currentBlock != null && currentBlock.getOffset() == currentOffset) if (currentBlock != null && currentBlock.getOffset() == currentOffset) {
{
// Avoid reading the same block again, even with caching turned off. // Avoid reading the same block again, even with caching turned off.
// This is crucial for compaction-type workload which might have // This is crucial for compaction-type workload which might have
// caching turned off. This is like a one-block cache inside the // caching turned off. This is like a one-block cache inside the
@ -319,26 +319,27 @@ public class HFileBlockIndex {
// this also accounts for ENCODED_DATA // this also accounts for ENCODED_DATA
expectedBlockType = BlockType.DATA; expectedBlockType = BlockType.DATA;
} }
block = cachingBlockReader.readBlock(currentOffset, block =
currentOnDiskSize, shouldCache, pread, isCompaction, true, cachingBlockReader.readBlock(currentOffset, currentOnDiskSize, shouldCache, pread,
expectedBlockType, expectedDataBlockEncoding); isCompaction, true, expectedBlockType, expectedDataBlockEncoding);
} }
if (block == null) { if (block == null) {
throw new IOException("Failed to read block at offset " + throw new IOException("Failed to read block at offset " + currentOffset
currentOffset + ", onDiskSize=" + currentOnDiskSize); + ", onDiskSize=" + currentOnDiskSize);
} }
// Found a data block, break the loop and check our level in the tree. // Found a data block, break the loop and check our level in the tree.
if (block.getBlockType().isData()) { if (block.getBlockType().isData()) {
dataBlock = true;
break; break;
} }
// Not a data block. This must be a leaf-level or intermediate-level // Not a data block. This must be a leaf-level or intermediate-level
// index block. We don't allow going deeper than searchTreeLevel. // index block. We don't allow going deeper than searchTreeLevel.
if (++lookupLevel > searchTreeLevel) { if (++lookupLevel > searchTreeLevel) {
throw new IOException("Search Tree Level overflow: lookupLevel="+ throw new IOException("Search Tree Level overflow: lookupLevel=" + lookupLevel
lookupLevel + ", searchTreeLevel=" + searchTreeLevel); + ", searchTreeLevel=" + searchTreeLevel);
} }
// Locate the entry corresponding to the given key in the non-root // Locate the entry corresponding to the given key in the non-root
@ -351,8 +352,7 @@ public class HFileBlockIndex {
KeyValue kv = KeyValueUtil.ensureKeyValue(key); KeyValue kv = KeyValueUtil.ensureKeyValue(key);
throw new IOException("The key " throw new IOException("The key "
+ Bytes.toStringBinary(kv.getKey(), kv.getKeyOffset(), kv.getKeyLength()) + Bytes.toStringBinary(kv.getKey(), kv.getKeyOffset(), kv.getKeyLength())
+ " is before the" + " first key of the non-root index block " + " is before the" + " first key of the non-root index block " + block);
+ block);
} }
currentOffset = buffer.getLong(); currentOffset = buffer.getLong();
@ -364,9 +364,21 @@ public class HFileBlockIndex {
tmpNextIndexKV.setKey(nonRootIndexedKey, 0, nonRootIndexedKey.length); tmpNextIndexKV.setKey(nonRootIndexedKey, 0, nonRootIndexedKey.length);
nextIndexedKey = tmpNextIndexKV; nextIndexedKey = tmpNextIndexKV;
} }
} finally {
if (!dataBlock) {
// Return the block immediately if it is not the
// data block
cachingBlockReader.returnBlock(block);
}
}
} }
if (lookupLevel != searchTreeLevel) { if (lookupLevel != searchTreeLevel) {
assert dataBlock == true;
// Though we have retrieved a data block we have found an issue
// in the retrieved data block. Hence returned the block so that
// the ref count can be decremented
cachingBlockReader.returnBlock(block);
throw new IOException("Reached a data block at level " + lookupLevel + throw new IOException("Reached a data block at level " + lookupLevel +
" but the number of levels is " + searchTreeLevel); " but the number of levels is " + searchTreeLevel);
} }
@ -396,16 +408,19 @@ public class HFileBlockIndex {
HFileBlock midLeafBlock = cachingBlockReader.readBlock( HFileBlock midLeafBlock = cachingBlockReader.readBlock(
midLeafBlockOffset, midLeafBlockOnDiskSize, true, true, false, true, midLeafBlockOffset, midLeafBlockOnDiskSize, true, true, false, true,
BlockType.LEAF_INDEX, null); BlockType.LEAF_INDEX, null);
try {
ByteBuff b = midLeafBlock.getBufferWithoutHeader(); ByteBuff b = midLeafBlock.getBufferWithoutHeader();
int numDataBlocks = b.getIntAfterPosition(0); int numDataBlocks = b.getIntAfterPosition(0);
int keyRelOffset = b.getIntAfterPosition(Bytes.SIZEOF_INT * (midKeyEntry + 1)); int keyRelOffset = b.getIntAfterPosition(Bytes.SIZEOF_INT * (midKeyEntry + 1));
int keyLen = b.getIntAfterPosition(Bytes.SIZEOF_INT * (midKeyEntry + 2)) - int keyLen = b.getIntAfterPosition(Bytes.SIZEOF_INT * (midKeyEntry + 2)) - keyRelOffset;
keyRelOffset; int keyOffset =
int keyOffset = Bytes.SIZEOF_INT * (numDataBlocks + 2) + keyRelOffset Bytes.SIZEOF_INT * (numDataBlocks + 2) + keyRelOffset
+ SECONDARY_INDEX_ENTRY_OVERHEAD; + SECONDARY_INDEX_ENTRY_OVERHEAD;
byte[] bytes = b.toBytes(keyOffset, keyLen); byte[] bytes = b.toBytes(keyOffset, keyLen);
targetMidKey = new KeyValue.KeyOnlyKeyValue(bytes, 0, bytes.length); targetMidKey = new KeyValue.KeyOnlyKeyValue(bytes, 0, bytes.length);
} finally {
cachingBlockReader.returnBlock(midLeafBlock);
}
} else { } else {
// The middle of the root-level index. // The middle of the root-level index.
targetMidKey = blockKeys[rootCount / 2]; targetMidKey = blockKeys[rootCount / 2];

View File

@ -34,10 +34,11 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellComparator; import org.apache.hadoop.hbase.CellComparator;
import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.ShareableMemory;
import org.apache.hadoop.hbase.SizeCachedKeyValue; import org.apache.hadoop.hbase.SizeCachedKeyValue;
import org.apache.hadoop.hbase.SizeCachedNoTagsKeyValue;
import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.SizeCachedNoTagsKeyValue;
import org.apache.hadoop.hbase.fs.HFileSystem; import org.apache.hadoop.hbase.fs.HFileSystem;
import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper; import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
import org.apache.hadoop.hbase.io.compress.Compression; import org.apache.hadoop.hbase.io.compress.Compression;
@ -46,6 +47,7 @@ import org.apache.hadoop.hbase.io.crypto.Encryption;
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoder; import org.apache.hadoop.hbase.io.encoding.DataBlockEncoder;
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding; import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
import org.apache.hadoop.hbase.io.encoding.HFileBlockDecodingContext; import org.apache.hadoop.hbase.io.encoding.HFileBlockDecodingContext;
import org.apache.hadoop.hbase.io.hfile.Cacheable.MemoryType;
import org.apache.hadoop.hbase.io.hfile.HFile.FileInfo; import org.apache.hadoop.hbase.io.hfile.HFile.FileInfo;
import org.apache.hadoop.hbase.nio.ByteBuff; import org.apache.hadoop.hbase.nio.ByteBuff;
import org.apache.hadoop.hbase.security.EncryptionUtil; import org.apache.hadoop.hbase.security.EncryptionUtil;
@ -256,6 +258,11 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
} }
HFileBlock block = readBlock(offset, onDiskSize, true, false, false, false, HFileBlock block = readBlock(offset, onDiskSize, true, false, false, false,
null, null); null, null);
// Need not update the current block. Ideally here the readBlock won't find the
// block in cache. We call this readBlock so that block data is read from FS and
// cached in BC. So there is no reference count increment that happens here.
// The return will ideally be a noop because the block is not of MemoryType SHARED.
returnBlock(block);
prevBlock = block; prevBlock = block;
offset += block.getOnDiskSizeWithHeader(); offset += block.getOnDiskSizeWithHeader();
} }
@ -337,6 +344,15 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
return fileSize; return fileSize;
} }
@Override
public void returnBlock(HFileBlock block) {
BlockCache blockCache = this.cacheConf.getBlockCache();
if (blockCache != null && block != null) {
BlockCacheKey cacheKey = new BlockCacheKey(this.getFileContext().getHFileName(),
block.getOffset());
blockCache.returnBlock(cacheKey, block);
}
}
/** /**
* @return the first key in the file. May be null if file has no entries. Note * @return the first key in the file. May be null if file has no entries. Note
* that this is not the first row key, but rather the byte form of the * that this is not the first row key, but rather the byte form of the
@ -449,7 +465,6 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
protected final HFile.Reader reader; protected final HFile.Reader reader;
private int currTagsLen; private int currTagsLen;
private KeyValue.KeyOnlyKeyValue keyOnlyKv = new KeyValue.KeyOnlyKeyValue(); private KeyValue.KeyOnlyKeyValue keyOnlyKv = new KeyValue.KeyOnlyKeyValue();
protected HFileBlock block;
// A pair for reusing in blockSeek() so that we don't garbage lot of objects // A pair for reusing in blockSeek() so that we don't garbage lot of objects
final Pair<ByteBuffer, Integer> pair = new Pair<ByteBuffer, Integer>(); final Pair<ByteBuffer, Integer> pair = new Pair<ByteBuffer, Integer>();
@ -461,6 +476,10 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
* If the nextIndexedKey is null, it means the nextIndexedKey has not been loaded yet. * If the nextIndexedKey is null, it means the nextIndexedKey has not been loaded yet.
*/ */
protected Cell nextIndexedKey; protected Cell nextIndexedKey;
// Current block being used
protected HFileBlock curBlock;
// Previous blocks that were used in the course of the read
protected final ArrayList<HFileBlock> prevBlocks = new ArrayList<HFileBlock>();
public HFileScannerImpl(final HFile.Reader reader, final boolean cacheBlocks, public HFileScannerImpl(final HFile.Reader reader, final boolean cacheBlocks,
final boolean pread, final boolean isCompaction) { final boolean pread, final boolean isCompaction) {
@ -470,6 +489,41 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
this.isCompaction = isCompaction; this.isCompaction = isCompaction;
} }
void updateCurrBlockRef(HFileBlock block) {
if (block != null && this.curBlock != null &&
block.getOffset() == this.curBlock.getOffset()) {
return;
}
if (this.curBlock != null) {
prevBlocks.add(this.curBlock);
}
this.curBlock = block;
}
void reset() {
if (this.curBlock != null) {
this.prevBlocks.add(this.curBlock);
}
this.curBlock = null;
}
private void returnBlockToCache(HFileBlock block) {
if (LOG.isTraceEnabled()) {
LOG.trace("Returning the block : " + block);
}
this.reader.returnBlock(block);
}
private void returnBlocks(boolean returnAll) {
for (int i = 0; i < this.prevBlocks.size(); i++) {
returnBlockToCache(this.prevBlocks.get(i));
}
this.prevBlocks.clear();
if (returnAll && this.curBlock != null) {
returnBlockToCache(this.curBlock);
this.curBlock = null;
}
}
@Override @Override
public boolean isSeeked(){ public boolean isSeeked(){
return blockBuffer != null; return blockBuffer != null;
@ -498,6 +552,11 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
return kvBufSize; return kvBufSize;
} }
@Override
public void close() {
this.returnBlocks(true);
}
protected int getNextCellStartPosition() { protected int getNextCellStartPosition() {
int nextKvPos = blockBuffer.position() + KEY_VALUE_LEN_SIZE + currKeyLen + currValueLen int nextKvPos = blockBuffer.position() + KEY_VALUE_LEN_SIZE + currKeyLen + currValueLen
+ currMemstoreTSLen; + currMemstoreTSLen;
@ -536,7 +595,8 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
private final void checkTagsLen() { private final void checkTagsLen() {
if (checkLen(this.currTagsLen)) { if (checkLen(this.currTagsLen)) {
throw new IllegalStateException("Invalid currTagsLen " + this.currTagsLen + throw new IllegalStateException("Invalid currTagsLen " + this.currTagsLen +
". Block offset: " + block.getOffset() + ", block length: " + this.blockBuffer.limit() + ". Block offset: " + curBlock.getOffset() + ", block length: " +
this.blockBuffer.limit() +
", position: " + this.blockBuffer.position() + " (without header)."); ", position: " + this.blockBuffer.position() + " (without header).");
} }
} }
@ -610,7 +670,7 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
|| vlen > blockBuffer.limit()) { || vlen > blockBuffer.limit()) {
throw new IllegalStateException("Invalid klen " + klen + " or vlen " throw new IllegalStateException("Invalid klen " + klen + " or vlen "
+ vlen + ". Block offset: " + vlen + ". Block offset: "
+ block.getOffset() + ", block length: " + blockBuffer.limit() + ", position: " + curBlock.getOffset() + ", block length: " + blockBuffer.limit() + ", position: "
+ blockBuffer.position() + " (without header)."); + blockBuffer.position() + " (without header).");
} }
offsetFromPos += Bytes.SIZEOF_LONG; offsetFromPos += Bytes.SIZEOF_LONG;
@ -626,7 +686,7 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
^ (blockBuffer.getByteAfterPosition(offsetFromPos + 1) & 0xff); ^ (blockBuffer.getByteAfterPosition(offsetFromPos + 1) & 0xff);
if (tlen < 0 || tlen > blockBuffer.limit()) { if (tlen < 0 || tlen > blockBuffer.limit()) {
throw new IllegalStateException("Invalid tlen " + tlen + ". Block offset: " throw new IllegalStateException("Invalid tlen " + tlen + ". Block offset: "
+ block.getOffset() + ", block length: " + blockBuffer.limit() + ", position: " + curBlock.getOffset() + ", block length: " + blockBuffer.limit() + ", position: "
+ blockBuffer.position() + " (without header)."); + blockBuffer.position() + " (without header).");
} }
// add the two bytes read for the tags. // add the two bytes read for the tags.
@ -641,8 +701,8 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
if (lastKeyValueSize < 0) { if (lastKeyValueSize < 0) {
throw new IllegalStateException("blockSeek with seekBefore " throw new IllegalStateException("blockSeek with seekBefore "
+ "at the first key of the block: key=" + CellUtil.getCellKeyAsString(key) + "at the first key of the block: key=" + CellUtil.getCellKeyAsString(key)
+ ", blockOffset=" + block.getOffset() + ", onDiskSize=" + ", blockOffset=" + curBlock.getOffset() + ", onDiskSize="
+ block.getOnDiskSizeWithHeader()); + curBlock.getOnDiskSizeWithHeader());
} }
blockBuffer.moveBack(lastKeyValueSize); blockBuffer.moveBack(lastKeyValueSize);
readKeyValueLen(); readKeyValueLen();
@ -709,8 +769,10 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
// smaller than // smaller than
// the next indexed key or the current data block is the last data // the next indexed key or the current data block is the last data
// block. // block.
return loadBlockAndSeekToKey(this.block, nextIndexedKey, false, key, false); return loadBlockAndSeekToKey(this.curBlock, nextIndexedKey, false, key,
false);
} }
} }
} }
// Don't rewind on a reseek operation, because reseek implies that we are // Don't rewind on a reseek operation, because reseek implies that we are
@ -734,10 +796,11 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
*/ */
public int seekTo(Cell key, boolean rewind) throws IOException { public int seekTo(Cell key, boolean rewind) throws IOException {
HFileBlockIndex.BlockIndexReader indexReader = reader.getDataBlockIndexReader(); HFileBlockIndex.BlockIndexReader indexReader = reader.getDataBlockIndexReader();
BlockWithScanInfo blockWithScanInfo = indexReader.loadDataBlockWithScanInfo(key, block, BlockWithScanInfo blockWithScanInfo = indexReader.loadDataBlockWithScanInfo(key, curBlock,
cacheBlocks, pread, isCompaction, getEffectiveDataBlockEncoding()); cacheBlocks, pread, isCompaction, getEffectiveDataBlockEncoding());
if (blockWithScanInfo == null || blockWithScanInfo.getHFileBlock() == null) { if (blockWithScanInfo == null || blockWithScanInfo.getHFileBlock() == null) {
// This happens if the key e.g. falls before the beginning of the file. // This happens if the key e.g. falls before the beginning of the
// file.
return -1; return -1;
} }
return loadBlockAndSeekToKey(blockWithScanInfo.getHFileBlock(), return loadBlockAndSeekToKey(blockWithScanInfo.getHFileBlock(),
@ -746,7 +809,7 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
@Override @Override
public boolean seekBefore(Cell key) throws IOException { public boolean seekBefore(Cell key) throws IOException {
HFileBlock seekToBlock = reader.getDataBlockIndexReader().seekToDataBlock(key, block, HFileBlock seekToBlock = reader.getDataBlockIndexReader().seekToDataBlock(key, curBlock,
cacheBlocks, pread, isCompaction, reader.getEffectiveEncodingInCache(isCompaction)); cacheBlocks, pread, isCompaction, reader.getEffectiveEncodingInCache(isCompaction));
if (seekToBlock == null) { if (seekToBlock == null) {
return false; return false;
@ -761,6 +824,10 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
return false; return false;
} }
// The first key in the current block 'seekToBlock' is greater than the given
// seekBefore key. We will go ahead by reading the next block that satisfies the
// given key. Return the current block before reading the next one.
reader.returnBlock(seekToBlock);
// It is important that we compute and pass onDiskSize to the block // It is important that we compute and pass onDiskSize to the block
// reader so that it does not have to read the header separately to // reader so that it does not have to read the header separately to
// figure out the size. // figure out the size.
@ -783,28 +850,33 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
*/ */
protected HFileBlock readNextDataBlock() throws IOException { protected HFileBlock readNextDataBlock() throws IOException {
long lastDataBlockOffset = reader.getTrailer().getLastDataBlockOffset(); long lastDataBlockOffset = reader.getTrailer().getLastDataBlockOffset();
if (block == null) if (curBlock == null)
return null; return null;
HFileBlock curBlock = block; HFileBlock block = this.curBlock;
do { do {
if (curBlock.getOffset() >= lastDataBlockOffset) if (block.getOffset() >= lastDataBlockOffset)
return null; return null;
if (curBlock.getOffset() < 0) { if (block.getOffset() < 0) {
throw new IOException("Invalid block file offset: " + block); throw new IOException("Invalid block file offset: " + block);
} }
// We are reading the next block without block type validation, because // We are reading the next block without block type validation, because
// it might turn out to be a non-data block. // it might turn out to be a non-data block.
curBlock = reader.readBlock(curBlock.getOffset() block = reader.readBlock(block.getOffset()
+ curBlock.getOnDiskSizeWithHeader(), + block.getOnDiskSizeWithHeader(),
curBlock.getNextBlockOnDiskSizeWithHeader(), cacheBlocks, pread, block.getNextBlockOnDiskSizeWithHeader(), cacheBlocks, pread,
isCompaction, true, null, getEffectiveDataBlockEncoding()); isCompaction, true, null, getEffectiveDataBlockEncoding());
} while (!curBlock.getBlockType().isData()); if (block != null && !block.getBlockType().isData()) {
// Whatever block we read we will be returning it unless
// it is a datablock. Just in case the blocks are non data blocks
reader.returnBlock(block);
}
} while (!block.getBlockType().isData());
return curBlock; return block;
} }
public DataBlockEncoding getEffectiveDataBlockEncoding() { public DataBlockEncoding getEffectiveDataBlockEncoding() {
@ -817,13 +889,27 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
return null; return null;
KeyValue ret; KeyValue ret;
// TODO : reduce the varieties of KV here. Check if based on a boolean
// we can handle the 'no tags' case
// TODO : Handle MBB here
if (currTagsLen > 0) { if (currTagsLen > 0) {
if (this.curBlock.getMemoryType() == MemoryType.SHARED) {
ret = new ShareableMemoryKeyValue(blockBuffer.array(), blockBuffer.arrayOffset()
+ blockBuffer.position(), getCellBufSize());
} else {
ret = new SizeCachedKeyValue(blockBuffer.array(), blockBuffer.arrayOffset() ret = new SizeCachedKeyValue(blockBuffer.array(), blockBuffer.arrayOffset()
+ blockBuffer.position(), getCellBufSize()); + blockBuffer.position(), getCellBufSize());
}
} else {
if (this.curBlock.getMemoryType() == MemoryType.SHARED) {
ret = new ShareableMemoryNoTagsKeyValue(blockBuffer.array(), blockBuffer.arrayOffset()
+ blockBuffer.position(), getCellBufSize());
} else { } else {
ret = new SizeCachedNoTagsKeyValue(blockBuffer.array(), blockBuffer.arrayOffset() ret = new SizeCachedNoTagsKeyValue(blockBuffer.array(), blockBuffer.arrayOffset()
+ blockBuffer.position(), getCellBufSize()); + blockBuffer.position(), getCellBufSize());
} }
}
if (this.reader.shouldIncludeMemstoreTS()) { if (this.reader.shouldIncludeMemstoreTS()) {
ret.setSequenceId(currMemstoreTS); ret.setSequenceId(currMemstoreTS);
} }
@ -838,6 +924,32 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
+ KEY_VALUE_LEN_SIZE, currKeyLen); + KEY_VALUE_LEN_SIZE, currKeyLen);
} }
private static class ShareableMemoryKeyValue extends SizeCachedKeyValue implements
ShareableMemory {
public ShareableMemoryKeyValue(byte[] bytes, int offset, int length) {
super(bytes, offset, length);
}
@Override
public Cell cloneToCell() {
byte[] copy = Bytes.copy(this.bytes, this.offset, this.length);
return new SizeCachedKeyValue(copy, 0, copy.length);
}
}
private static class ShareableMemoryNoTagsKeyValue extends SizeCachedNoTagsKeyValue implements
ShareableMemory {
public ShareableMemoryNoTagsKeyValue(byte[] bytes, int offset, int length) {
super(bytes, offset, length);
}
@Override
public Cell cloneToCell() {
byte[] copy = Bytes.copy(this.bytes, this.offset, this.length);
return new SizeCachedNoTagsKeyValue(copy, 0, copy.length);
}
}
@Override @Override
public ByteBuffer getValue() { public ByteBuffer getValue() {
assertSeeked(); assertSeeked();
@ -849,7 +961,7 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
} }
protected void setNonSeekedState() { protected void setNonSeekedState() {
block = null; reset();
blockBuffer = null; blockBuffer = null;
currKeyLen = 0; currKeyLen = 0;
currValueLen = 0; currValueLen = 0;
@ -869,7 +981,7 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
+ "; currKeyLen = " + currKeyLen + "; currValLen = " + "; currKeyLen = " + currKeyLen + "; currValLen = "
+ currValueLen + "; block limit = " + blockBuffer.limit() + currValueLen + "; block limit = " + blockBuffer.limit()
+ "; HFile name = " + reader.getName() + "; HFile name = " + reader.getName()
+ "; currBlock currBlockOffset = " + block.getOffset()); + "; currBlock currBlockOffset = " + this.curBlock.getOffset());
throw e; throw e;
} }
} }
@ -882,7 +994,7 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
private boolean positionForNextBlock() throws IOException { private boolean positionForNextBlock() throws IOException {
// Methods are small so they get inlined because they are 'hot'. // Methods are small so they get inlined because they are 'hot'.
long lastDataBlockOffset = reader.getTrailer().getLastDataBlockOffset(); long lastDataBlockOffset = reader.getTrailer().getLastDataBlockOffset();
if (block.getOffset() >= lastDataBlockOffset) { if (this.curBlock.getOffset() >= lastDataBlockOffset) {
setNonSeekedState(); setNonSeekedState();
return false; return false;
} }
@ -897,7 +1009,7 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
setNonSeekedState(); setNonSeekedState();
return false; return false;
} }
updateCurrBlock(nextBlock); updateCurrentBlock(nextBlock);
return true; return true;
} }
@ -946,27 +1058,37 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
return false; return false;
} }
long firstDataBlockOffset = long firstDataBlockOffset = reader.getTrailer().getFirstDataBlockOffset();
reader.getTrailer().getFirstDataBlockOffset(); if (curBlock != null
if (block != null && block.getOffset() == firstDataBlockOffset) { && curBlock.getOffset() == firstDataBlockOffset) {
return processFirstDataBlock();
}
readAndUpdateNewBlock(firstDataBlockOffset);
return true;
}
protected boolean processFirstDataBlock() throws IOException{
blockBuffer.rewind(); blockBuffer.rewind();
readKeyValueLen(); readKeyValueLen();
return true; return true;
} }
block = reader.readBlock(firstDataBlockOffset, -1, cacheBlocks, pread, protected void readAndUpdateNewBlock(long firstDataBlockOffset) throws IOException,
CorruptHFileException {
HFileBlock newBlock = reader.readBlock(firstDataBlockOffset, -1, cacheBlocks, pread,
isCompaction, true, BlockType.DATA, getEffectiveDataBlockEncoding()); isCompaction, true, BlockType.DATA, getEffectiveDataBlockEncoding());
if (block.getOffset() < 0) { if (newBlock.getOffset() < 0) {
throw new IOException("Invalid block offset: " + block.getOffset()); throw new IOException("Invalid block offset: " + newBlock.getOffset());
} }
updateCurrBlock(block); updateCurrentBlock(newBlock);
return true;
} }
protected int loadBlockAndSeekToKey(HFileBlock seekToBlock, Cell nextIndexedKey, protected int loadBlockAndSeekToKey(HFileBlock seekToBlock, Cell nextIndexedKey,
boolean rewind, Cell key, boolean seekBefore) throws IOException { boolean rewind, Cell key, boolean seekBefore) throws IOException {
if (block == null || block.getOffset() != seekToBlock.getOffset()) { if (this.curBlock == null
updateCurrBlock(seekToBlock); || this.curBlock.getOffset() != seekToBlock.getOffset()) {
updateCurrentBlock(seekToBlock);
} else if (rewind) { } else if (rewind) {
blockBuffer.rewind(); blockBuffer.rewind();
} }
@ -989,10 +1111,11 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
*/ */
protected final void checkKeyValueLen() { protected final void checkKeyValueLen() {
if (checkLen(this.currKeyLen) || checkLen(this.currValueLen)) { if (checkLen(this.currKeyLen) || checkLen(this.currValueLen)) {
throw new IllegalStateException("Invalid currKeyLen " + this.currKeyLen + throw new IllegalStateException("Invalid currKeyLen " + this.currKeyLen
" or currValueLen " + this.currValueLen + ". Block offset: " + block.getOffset() + + " or currValueLen " + this.currValueLen + ". Block offset: "
", block length: " + this.blockBuffer.limit() + ", position: " + + this.curBlock.getOffset() + ", block length: "
this.blockBuffer.position() + " (without header)."); + this.blockBuffer.limit() + ", position: " + this.blockBuffer.position()
+ " (without header).");
} }
} }
@ -1002,19 +1125,18 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
* *
* @param newBlock the block to make current * @param newBlock the block to make current
*/ */
protected void updateCurrBlock(HFileBlock newBlock) { protected void updateCurrentBlock(HFileBlock newBlock) throws IOException {
block = newBlock; // Set the active block on the reader
// sanity check // sanity check
if (block.getBlockType() != BlockType.DATA) { if (newBlock.getBlockType() != BlockType.DATA) {
throw new IllegalStateException("Scanner works only on data " + throw new IllegalStateException("ScannerV2 works only on data " + "blocks, got "
"blocks, got " + block.getBlockType() + "; " + + newBlock.getBlockType() + "; " + "fileName=" + reader.getName()
"fileName=" + reader.getName() + ", " + + ", " + "dataBlockEncoder=" + reader.getDataBlockEncoding() + ", " + "isCompaction="
"dataBlockEncoder=" + reader.getDataBlockEncoding() + ", " + + isCompaction);
"isCompaction=" + isCompaction);
} }
blockBuffer = block.getBufferWithoutHeader(); updateCurrBlockRef(newBlock);
blockBuffer = newBlock.getBufferWithoutHeader();
readKeyValueLen(); readKeyValueLen();
blockFetches++; blockFetches++;
@ -1057,14 +1179,9 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
key, this.keyOnlyKv); key, this.keyOnlyKv);
} }
@Override
public void close() {
// HBASE-12295 will add code here.
}
@Override @Override
public void shipped() throws IOException { public void shipped() throws IOException {
// HBASE-12295 will add code here. this.returnBlocks(false);
} }
} }
@ -1127,7 +1244,12 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
updateCacheMetrics); updateCacheMetrics);
if (cachedBlock != null) { if (cachedBlock != null) {
if (cacheConf.shouldCacheCompressed(cachedBlock.getBlockType().getCategory())) { if (cacheConf.shouldCacheCompressed(cachedBlock.getBlockType().getCategory())) {
cachedBlock = cachedBlock.unpack(hfileContext, fsBlockReader); HFileBlock compressedBlock = cachedBlock;
cachedBlock = compressedBlock.unpack(hfileContext, fsBlockReader);
// In case of compressed block after unpacking we can return the compressed block
if (compressedBlock != cachedBlock) {
cache.returnBlock(cacheKey, compressedBlock);
}
} }
validateBlockType(cachedBlock, expectedBlockType); validateBlockType(cachedBlock, expectedBlockType);
@ -1163,6 +1285,9 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
" because of a data block encoding mismatch" + " because of a data block encoding mismatch" +
"; expected: " + expectedDataBlockEncoding + "; expected: " + expectedDataBlockEncoding +
", actual: " + actualDataBlockEncoding); ", actual: " + actualDataBlockEncoding);
// This is an error scenario. so here we need to decrement the
// count.
cache.returnBlock(cacheKey, cachedBlock);
cache.evictBlock(cacheKey); cache.evictBlock(cacheKey);
} }
return null; return null;
@ -1180,7 +1305,7 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
* @throws IOException * @throws IOException
*/ */
@Override @Override
public ByteBuff getMetaBlock(String metaBlockName, boolean cacheBlock) public HFileBlock getMetaBlock(String metaBlockName, boolean cacheBlock)
throws IOException { throws IOException {
if (trailer.getMetaIndexCount() == 0) { if (trailer.getMetaIndexCount() == 0) {
return null; // there are no meta blocks return null; // there are no meta blocks
@ -1213,7 +1338,7 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
assert cachedBlock.isUnpacked() : "Packed block leak."; assert cachedBlock.isUnpacked() : "Packed block leak.";
// Return a distinct 'shallow copy' of the block, // Return a distinct 'shallow copy' of the block,
// so pos does not get messed by the scanner // so pos does not get messed by the scanner
return cachedBlock.getBufferWithoutHeader(); return cachedBlock;
} }
// Cache Miss, please load. // Cache Miss, please load.
} }
@ -1227,7 +1352,7 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
cacheConf.isInMemory(), this.cacheConf.isCacheDataInL1()); cacheConf.isInMemory(), this.cacheConf.isCacheDataInL1());
} }
return metaBlock.getBufferWithoutHeader(); return metaBlock;
} }
} }
@ -1424,7 +1549,11 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
@Override @Override
public boolean isSeeked(){ public boolean isSeeked(){
return this.block != null; return curBlock != null;
}
public void setNonSeekedState() {
reset();
} }
/** /**
@ -1434,21 +1563,21 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
* @param newBlock the block to make current * @param newBlock the block to make current
* @throws CorruptHFileException * @throws CorruptHFileException
*/ */
private void updateCurrentBlock(HFileBlock newBlock) throws CorruptHFileException { @Override
block = newBlock; protected void updateCurrentBlock(HFileBlock newBlock) throws CorruptHFileException {
// sanity checks // sanity checks
if (block.getBlockType() != BlockType.ENCODED_DATA) { if (newBlock.getBlockType() != BlockType.ENCODED_DATA) {
throw new IllegalStateException( throw new IllegalStateException("EncodedScanner works only on encoded data blocks");
"EncodedScanner works only on encoded data blocks");
} }
short dataBlockEncoderId = block.getDataBlockEncodingId(); short dataBlockEncoderId = newBlock.getDataBlockEncodingId();
if (!DataBlockEncoding.isCorrectEncoder(dataBlockEncoder, dataBlockEncoderId)) { if (!DataBlockEncoding.isCorrectEncoder(dataBlockEncoder, dataBlockEncoderId)) {
String encoderCls = dataBlockEncoder.getClass().getName(); String encoderCls = dataBlockEncoder.getClass().getName();
throw new CorruptHFileException("Encoder " + encoderCls throw new CorruptHFileException("Encoder " + encoderCls
+ " doesn't support data block encoding " + " doesn't support data block encoding "
+ DataBlockEncoding.getNameFromId(dataBlockEncoderId)); + DataBlockEncoding.getNameFromId(dataBlockEncoderId));
} }
updateCurrBlockRef(newBlock);
ByteBuff encodedBuffer = getEncodedBuffer(newBlock); ByteBuff encodedBuffer = getEncodedBuffer(newBlock);
seeker.setCurrentBuffer(encodedBuffer); seeker.setCurrentBuffer(encodedBuffer);
blockFetches++; blockFetches++;
@ -1467,40 +1596,21 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
} }
@Override @Override
public boolean seekTo() throws IOException { protected boolean processFirstDataBlock() throws IOException {
if (reader == null) {
return false;
}
if (reader.getTrailer().getEntryCount() == 0) {
// No data blocks.
return false;
}
long firstDataBlockOffset =
reader.getTrailer().getFirstDataBlockOffset();
if (block != null && block.getOffset() == firstDataBlockOffset) {
seeker.rewind(); seeker.rewind();
return true; return true;
} }
block = reader.readBlock(firstDataBlockOffset, -1, cacheBlocks, pread,
isCompaction, true, BlockType.DATA, getEffectiveDataBlockEncoding());
if (block.getOffset() < 0) {
throw new IOException("Invalid block offset: " + block.getOffset());
}
updateCurrentBlock(block);
return true;
}
@Override @Override
public boolean next() throws IOException { public boolean next() throws IOException {
boolean isValid = seeker.next(); boolean isValid = seeker.next();
if (!isValid) { if (!isValid) {
block = readNextDataBlock(); HFileBlock newBlock = readNextDataBlock();
isValid = block != null; isValid = newBlock != null;
if (isValid) { if (isValid) {
updateCurrentBlock(block); updateCurrentBlock(newBlock);
} else {
setNonSeekedState();
} }
} }
return isValid; return isValid;
@ -1520,7 +1630,7 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
@Override @Override
public Cell getCell() { public Cell getCell() {
if (block == null) { if (this.curBlock == null) {
return null; return null;
} }
return seeker.getCell(); return seeker.getCell();
@ -1539,7 +1649,7 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
} }
private void assertValidSeek() { private void assertValidSeek() {
if (block == null) { if (this.curBlock == null) {
throw new NotSeekedException(); throw new NotSeekedException();
} }
} }
@ -1548,9 +1658,11 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
return dataBlockEncoder.getFirstKeyCellInBlock(getEncodedBuffer(curBlock)); return dataBlockEncoder.getFirstKeyCellInBlock(getEncodedBuffer(curBlock));
} }
@Override
protected int loadBlockAndSeekToKey(HFileBlock seekToBlock, Cell nextIndexedKey, protected int loadBlockAndSeekToKey(HFileBlock seekToBlock, Cell nextIndexedKey,
boolean rewind, Cell key, boolean seekBefore) throws IOException { boolean rewind, Cell key, boolean seekBefore) throws IOException {
if (block == null || block.getOffset() != seekToBlock.getOffset()) { if (this.curBlock == null
|| this.curBlock.getOffset() != seekToBlock.getOffset()) {
updateCurrentBlock(seekToBlock); updateCurrentBlock(seekToBlock);
} else if (rewind) { } else if (rewind) {
seeker.rewind(); seeker.rewind();
@ -1631,6 +1743,7 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
HFileContextBuilder builder = new HFileContextBuilder() HFileContextBuilder builder = new HFileContextBuilder()
.withIncludesMvcc(shouldIncludeMemstoreTS()) .withIncludesMvcc(shouldIncludeMemstoreTS())
.withHBaseCheckSum(true) .withHBaseCheckSum(true)
.withHFileName(this.getName())
.withCompression(this.compressAlgo); .withCompression(this.compressAlgo);
// Check for any key material available // Check for any key material available

View File

@ -34,11 +34,10 @@ import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.locks.ReentrantLock; import java.util.concurrent.locks.ReentrantLock;
import com.google.common.base.Objects;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.io.HeapSize; import org.apache.hadoop.hbase.io.HeapSize;
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding; import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
import org.apache.hadoop.hbase.io.hfile.bucket.BucketCache; import org.apache.hadoop.hbase.io.hfile.bucket.BucketCache;
@ -49,6 +48,7 @@ import org.apache.hadoop.util.StringUtils;
import org.codehaus.jackson.annotate.JsonIgnoreProperties; import org.codehaus.jackson.annotate.JsonIgnoreProperties;
import com.google.common.annotations.VisibleForTesting; import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Objects;
import com.google.common.util.concurrent.ThreadFactoryBuilder; import com.google.common.util.concurrent.ThreadFactoryBuilder;
/** /**
@ -1090,4 +1090,9 @@ public class LruBlockCache implements ResizableBlockCache, HeapSize {
public BlockCache[] getBlockCaches() { public BlockCache[] getBlockCaches() {
return null; return null;
} }
@Override
public void returnBlock(BlockCacheKey cacheKey, Cacheable block) {
// There is no SHARED type here. Just return
}
} }

View File

@ -29,6 +29,7 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.io.hfile.Cacheable.MemoryType;
import org.apache.hadoop.hbase.nio.ByteBuff; import org.apache.hadoop.hbase.nio.ByteBuff;
import org.apache.hadoop.hbase.nio.SingleByteBuff; import org.apache.hadoop.hbase.nio.SingleByteBuff;
import org.apache.hadoop.hbase.util.Addressing; import org.apache.hadoop.hbase.util.Addressing;
@ -259,7 +260,8 @@ public class MemcachedBlockCache implements BlockCache {
public HFileBlock decode(CachedData d) { public HFileBlock decode(CachedData d) {
try { try {
ByteBuff buf = new SingleByteBuff(ByteBuffer.wrap(d.getData())); ByteBuff buf = new SingleByteBuff(ByteBuffer.wrap(d.getData()));
return (HFileBlock) HFileBlock.blockDeserializer.deserialize(buf, true); return (HFileBlock) HFileBlock.blockDeserializer.deserialize(buf, true,
MemoryType.EXCLUSIVE);
} catch (IOException e) { } catch (IOException e) {
LOG.warn("Error deserializing data from memcached",e); LOG.warn("Error deserializing data from memcached",e);
} }
@ -272,4 +274,9 @@ public class MemcachedBlockCache implements BlockCache {
} }
} }
@Override
public void returnBlock(BlockCacheKey cacheKey, Cacheable block) {
// Not doing reference counting. All blocks here are EXCLUSIVE
}
} }

View File

@ -43,6 +43,7 @@ import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.Executors; import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantLock; import java.util.concurrent.locks.ReentrantLock;
@ -58,16 +59,17 @@ import org.apache.hadoop.hbase.io.hfile.BlockPriority;
import org.apache.hadoop.hbase.io.hfile.BlockType; import org.apache.hadoop.hbase.io.hfile.BlockType;
import org.apache.hadoop.hbase.io.hfile.CacheStats; import org.apache.hadoop.hbase.io.hfile.CacheStats;
import org.apache.hadoop.hbase.io.hfile.Cacheable; import org.apache.hadoop.hbase.io.hfile.Cacheable;
import org.apache.hadoop.hbase.io.hfile.Cacheable.MemoryType;
import org.apache.hadoop.hbase.io.hfile.CacheableDeserializer; import org.apache.hadoop.hbase.io.hfile.CacheableDeserializer;
import org.apache.hadoop.hbase.io.hfile.CacheableDeserializerIdManager; import org.apache.hadoop.hbase.io.hfile.CacheableDeserializerIdManager;
import org.apache.hadoop.hbase.io.hfile.CachedBlock; import org.apache.hadoop.hbase.io.hfile.CachedBlock;
import org.apache.hadoop.hbase.io.hfile.HFileBlock; import org.apache.hadoop.hbase.io.hfile.HFileBlock;
import org.apache.hadoop.hbase.nio.ByteBuff; import org.apache.hadoop.hbase.nio.ByteBuff;
import org.apache.hadoop.hbase.nio.SingleByteBuff;
import org.apache.hadoop.hbase.util.ConcurrentIndex; import org.apache.hadoop.hbase.util.ConcurrentIndex;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.HasThread; import org.apache.hadoop.hbase.util.HasThread;
import org.apache.hadoop.hbase.util.IdLock; import org.apache.hadoop.hbase.util.IdLock;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.util.StringUtils;
import com.google.common.annotations.VisibleForTesting; import com.google.common.annotations.VisibleForTesting;
@ -421,20 +423,19 @@ public class BucketCache implements BlockCache, HeapSize {
// TODO : change this area - should be removed after server cells and // TODO : change this area - should be removed after server cells and
// 12295 are available // 12295 are available
int len = bucketEntry.getLength(); int len = bucketEntry.getLength();
ByteBuffer buf = ByteBuffer.allocate(len); Pair<ByteBuff, MemoryType> pair = ioEngine.read(bucketEntry.offset(), len);
int lenRead = ioEngine.read(buf, bucketEntry.offset()); ByteBuff bb = pair.getFirst();
ByteBuff bb = new SingleByteBuff(buf);
if (lenRead != len) {
throw new RuntimeException("Only " + lenRead + " bytes read, " + len + " expected");
}
CacheableDeserializer<Cacheable> deserializer = CacheableDeserializer<Cacheable> deserializer =
bucketEntry.deserializerReference(this.deserialiserMap); bucketEntry.deserializerReference(this.deserialiserMap);
Cacheable cachedBlock = deserializer.deserialize(bb, true); Cacheable cachedBlock = deserializer.deserialize(bb, true, pair.getSecond());
long timeTaken = System.nanoTime() - start; long timeTaken = System.nanoTime() - start;
if (updateCacheMetrics) { if (updateCacheMetrics) {
cacheStats.hit(caching); cacheStats.hit(caching);
cacheStats.ioHit(timeTaken); cacheStats.ioHit(timeTaken);
} }
if (pair.getSecond() == MemoryType.SHARED) {
bucketEntry.refCount.incrementAndGet();
}
bucketEntry.access(accessCount.incrementAndGet()); bucketEntry.access(accessCount.incrementAndGet());
if (this.ioErrorStartTime > 0) { if (this.ioErrorStartTime > 0) {
ioErrorStartTime = -1; ioErrorStartTime = -1;
@ -468,14 +469,16 @@ public class BucketCache implements BlockCache, HeapSize {
@Override @Override
public boolean evictBlock(BlockCacheKey cacheKey) { public boolean evictBlock(BlockCacheKey cacheKey) {
return evictBlock(cacheKey, true);
}
// does not check for the ref count. Just tries to evict it if found in the
// bucket map
private boolean forceEvict(BlockCacheKey cacheKey) {
if (!cacheEnabled) { if (!cacheEnabled) {
return false; return false;
} }
RAMQueueEntry removedBlock = ramCache.remove(cacheKey); RAMQueueEntry removedBlock = checkRamCache(cacheKey);
if (removedBlock != null) {
this.blockNumber.decrementAndGet();
this.heapSize.addAndGet(-1 * removedBlock.getData().heapSize());
}
BucketEntry bucketEntry = backingMap.get(cacheKey); BucketEntry bucketEntry = backingMap.get(cacheKey);
if (bucketEntry == null) { if (bucketEntry == null) {
if (removedBlock != null) { if (removedBlock != null) {
@ -505,6 +508,67 @@ public class BucketCache implements BlockCache, HeapSize {
return true; return true;
} }
private RAMQueueEntry checkRamCache(BlockCacheKey cacheKey) {
RAMQueueEntry removedBlock = ramCache.remove(cacheKey);
if (removedBlock != null) {
this.blockNumber.decrementAndGet();
this.heapSize.addAndGet(-1 * removedBlock.getData().heapSize());
}
return removedBlock;
}
public boolean evictBlock(BlockCacheKey cacheKey, boolean deletedBlock) {
if (!cacheEnabled) {
return false;
}
RAMQueueEntry removedBlock = checkRamCache(cacheKey);
BucketEntry bucketEntry = backingMap.get(cacheKey);
if (bucketEntry == null) {
if (removedBlock != null) {
cacheStats.evicted(0);
return true;
} else {
return false;
}
}
IdLock.Entry lockEntry = null;
try {
lockEntry = offsetLock.getLockEntry(bucketEntry.offset());
int refCount = bucketEntry.refCount.get();
if(refCount == 0) {
if (backingMap.remove(cacheKey, bucketEntry)) {
blockEvicted(cacheKey, bucketEntry, removedBlock == null);
} else {
return false;
}
} else {
if(!deletedBlock) {
if (LOG.isDebugEnabled()) {
LOG.debug("This block " + cacheKey + " is still referred by " + refCount
+ " readers. Can not be freed now");
}
return false;
} else {
if (LOG.isDebugEnabled()) {
LOG.debug("This block " + cacheKey + " is still referred by " + refCount
+ " readers. Can not be freed now. Hence will mark this"
+ " for evicting at a later point");
}
bucketEntry.markedForEvict = true;
}
}
} catch (IOException ie) {
LOG.warn("Failed evicting block " + cacheKey);
return false;
} finally {
if (lockEntry != null) {
offsetLock.releaseLockEntry(lockEntry);
}
}
cacheStats.evicted(bucketEntry.getCachedTime());
return true;
}
/* /*
* Statistics thread. Periodically output cache statistics to the log. * Statistics thread. Periodically output cache statistics to the log.
*/ */
@ -1107,6 +1171,10 @@ public class BucketCache implements BlockCache, HeapSize {
byte deserialiserIndex; byte deserialiserIndex;
private volatile long accessCounter; private volatile long accessCounter;
private BlockPriority priority; private BlockPriority priority;
// Set this when we were not able to forcefully evict the block
private volatile boolean markedForEvict;
private AtomicInteger refCount = new AtomicInteger(0);
/** /**
* Time this block was cached. Presumes we are created just before we are added to the cache. * Time this block was cached. Presumes we are created just before we are added to the cache.
*/ */
@ -1198,9 +1266,12 @@ public class BucketCache implements BlockCache, HeapSize {
public long free(long toFree) { public long free(long toFree) {
Map.Entry<BlockCacheKey, BucketEntry> entry; Map.Entry<BlockCacheKey, BucketEntry> entry;
long freedBytes = 0; long freedBytes = 0;
// TODO avoid a cycling siutation. We find no block which is not in use and so no way to free
// What to do then? Caching attempt fail? Need some changes in cacheBlock API?
while ((entry = queue.pollLast()) != null) { while ((entry = queue.pollLast()) != null) {
evictBlock(entry.getKey()); if (evictBlock(entry.getKey(), false)) {
freedBytes += entry.getValue().getLength(); freedBytes += entry.getValue().getLength();
}
if (freedBytes >= toFree) { if (freedBytes >= toFree) {
return freedBytes; return freedBytes;
} }
@ -1404,4 +1475,26 @@ public class BucketCache implements BlockCache, HeapSize {
public BlockCache[] getBlockCaches() { public BlockCache[] getBlockCaches() {
return null; return null;
} }
@Override
public void returnBlock(BlockCacheKey cacheKey, Cacheable block) {
if (block.getMemoryType() == MemoryType.SHARED) {
BucketEntry bucketEntry = backingMap.get(cacheKey);
if (bucketEntry != null) {
int refCount = bucketEntry.refCount.decrementAndGet();
if (bucketEntry.markedForEvict && refCount == 0) {
forceEvict(cacheKey);
}
}
}
}
@VisibleForTesting
public int getRefCount(BlockCacheKey cacheKey) {
BucketEntry bucketEntry = backingMap.get(cacheKey);
if (bucketEntry != null) {
return bucketEntry.refCount.get();
}
return 0;
}
} }

View File

@ -22,8 +22,11 @@ import java.io.IOException;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.io.hfile.Cacheable.MemoryType;
import org.apache.hadoop.hbase.nio.ByteBuff; import org.apache.hadoop.hbase.nio.ByteBuff;
import org.apache.hadoop.hbase.nio.SingleByteBuff;
import org.apache.hadoop.hbase.util.ByteBufferArray; import org.apache.hadoop.hbase.util.ByteBufferArray;
import org.apache.hadoop.hbase.util.Pair;
/** /**
* IO engine that stores data in memory using an array of ByteBuffers * IO engine that stores data in memory using an array of ByteBuffers
@ -64,24 +67,24 @@ public class ByteBufferIOEngine implements IOEngine {
return false; return false;
} }
/**
* Transfers data from the buffer array to the given byte buffer
* @param dstBuffer the given byte buffer into which bytes are to be written
* @param offset The offset in the ByteBufferArray of the first byte to be
* read
* @return number of bytes read
* @throws IOException
*/
@Override @Override
public int read(ByteBuffer dstBuffer, long offset) throws IOException { public Pair<ByteBuff, MemoryType> read(long offset, int length) throws IOException {
assert dstBuffer.hasArray(); // TODO : this allocate and copy will go away once we create BB backed cells
return bufferArray.getMultiple(offset, dstBuffer.remaining(), dstBuffer.array(), ByteBuffer dstBuffer = ByteBuffer.allocate(length);
bufferArray.getMultiple(offset, dstBuffer.remaining(), dstBuffer.array(),
dstBuffer.arrayOffset()); dstBuffer.arrayOffset());
// Here the buffer that is created directly refers to the buffer in the actual buckets.
// When any cell is referring to the blocks created out of these buckets then it means that
// those cells are referring to a shared memory area which if evicted by the BucketCache would
// lead to corruption of results. Hence we set the type of the buffer as SHARED_MEMORY
// so that the readers using this block are aware of this fact and do the necessary action
// to prevent eviction till the results are either consumed or copied
if (dstBuffer.limit() != length) {
throw new RuntimeException("Only " + dstBuffer.limit() + " bytes read, " + length
+ " expected");
} }
// TODO : to be removed - make it conditional
@Override return new Pair<ByteBuff, MemoryType>(new SingleByteBuff(dstBuffer), MemoryType.SHARED);
public ByteBuff read(long offset, int len) throws IOException {
return bufferArray.asSubByteBuff(offset, len);
} }
/** /**

View File

@ -26,8 +26,10 @@ import java.nio.channels.FileChannel;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.io.hfile.Cacheable.MemoryType;
import org.apache.hadoop.hbase.nio.ByteBuff; import org.apache.hadoop.hbase.nio.ByteBuff;
import org.apache.hadoop.hbase.nio.SingleByteBuff; import org.apache.hadoop.hbase.nio.SingleByteBuff;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.util.StringUtils;
/** /**
@ -81,14 +83,25 @@ public class FileIOEngine implements IOEngine {
/** /**
* Transfers data from file to the given byte buffer * Transfers data from file to the given byte buffer
* @param dstBuffer the given byte buffer into which bytes are to be written
* @param offset The offset in the file where the first byte to be read * @param offset The offset in the file where the first byte to be read
* @param length The length of buffer that should be allocated for reading
* from the file channel
* @return number of bytes read * @return number of bytes read
* @throws IOException * @throws IOException
*/ */
@Override @Override
public int read(ByteBuffer dstBuffer, long offset) throws IOException { public Pair<ByteBuff, MemoryType> read(long offset, int length) throws IOException {
return fileChannel.read(dstBuffer, offset); ByteBuffer dstBuffer = ByteBuffer.allocate(length);
fileChannel.read(dstBuffer, offset);
// The buffer created out of the fileChannel is formed by copying the data from the file
// Hence in this case there is no shared memory that we point to. Even if the BucketCache evicts
// this buffer from the file the data is already copied and there is no need to ensure that
// the results are not corrupted before consuming them.
if (dstBuffer.limit() != length) {
throw new RuntimeException("Only " + dstBuffer.limit() + " bytes read, " + length
+ " expected");
}
return new Pair<ByteBuff, MemoryType>(new SingleByteBuff(dstBuffer), MemoryType.EXCLUSIVE);
} }
/** /**
@ -128,14 +141,6 @@ public class FileIOEngine implements IOEngine {
} }
} }
@Override
public ByteBuff read(long offset, int len) throws IOException {
ByteBuffer dstBuffer = ByteBuffer.allocate(len);
int read = read(dstBuffer, offset);
dstBuffer.limit(read);
return new SingleByteBuff(dstBuffer);
}
@Override @Override
public void write(ByteBuff srcBuffer, long offset) throws IOException { public void write(ByteBuff srcBuffer, long offset) throws IOException {
// When caching block into BucketCache there will be single buffer backing for this HFileBlock. // When caching block into BucketCache there will be single buffer backing for this HFileBlock.

View File

@ -22,7 +22,9 @@ import java.io.IOException;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.io.hfile.Cacheable.MemoryType;
import org.apache.hadoop.hbase.nio.ByteBuff; import org.apache.hadoop.hbase.nio.ByteBuff;
import org.apache.hadoop.hbase.util.Pair;
/** /**
* A class implementing IOEngine interface supports data services for * A class implementing IOEngine interface supports data services for
@ -36,25 +38,14 @@ public interface IOEngine {
boolean isPersistent(); boolean isPersistent();
/** /**
* Transfers data from IOEngine to the given byte buffer * Transfers data from IOEngine to a byte buffer
* @param dstBuffer the given byte buffer into which bytes are to be written * @param length How many bytes to be read from the offset
* @param offset The offset in the IO engine where the first byte to be read * @param offset The offset in the IO engine where the first byte to be read
* @return number of bytes read * @return Pair of ByteBuffer where data is read and its MemoryType ({@link MemoryType})
* @throws IOException * @throws IOException
* @throws RuntimeException when the length of the ByteBuff read is less than 'len' * @throws RuntimeException when the length of the ByteBuff read is less than 'len'
*/ */
int read(ByteBuffer dstBuffer, long offset) throws IOException; Pair<ByteBuff, MemoryType> read(long offset, int length) throws IOException;
/**
* Transfers data from IOEngine at the given offset to an MultiByteBuffer
* @param offset the offset from which the underlying buckets should be read
* @param len the length upto which the buckets should be read
* @return the MultiByteBuffer formed from the underlying ByteBuffers forming the
* buckets
* @throws IOException
* @throws RuntimeException when the length of the ByteBuff read is less than 'len'
*/
ByteBuff read(long offset, int len) throws IOException;
/** /**
* Transfers data from the given byte buffer to IOEngine * Transfers data from the given byte buffer to IOEngine

View File

@ -33,6 +33,7 @@ import java.util.HashMap;
import java.util.HashSet; import java.util.HashSet;
import java.util.Iterator; import java.util.Iterator;
import java.util.List; import java.util.List;
import java.util.ListIterator;
import java.util.Map; import java.util.Map;
import java.util.Map.Entry; import java.util.Map.Entry;
import java.util.NavigableMap; import java.util.NavigableMap;
@ -87,6 +88,7 @@ import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.NamespaceDescriptor; import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.NotServingRegionException; import org.apache.hadoop.hbase.NotServingRegionException;
import org.apache.hadoop.hbase.RegionTooBusyException; import org.apache.hadoop.hbase.RegionTooBusyException;
import org.apache.hadoop.hbase.ShareableMemory;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.Tag; import org.apache.hadoop.hbase.Tag;
import org.apache.hadoop.hbase.TagRewriteCell; import org.apache.hadoop.hbase.TagRewriteCell;
@ -2432,39 +2434,44 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
@Override @Override
public RegionScanner getScanner(Scan scan) throws IOException { public RegionScanner getScanner(Scan scan) throws IOException {
return getScanner(scan, null); return getScanner(scan, true);
} }
protected RegionScanner getScanner(Scan scan, public RegionScanner getScanner(Scan scan, boolean copyCellsFromSharedMem) throws IOException {
List<KeyValueScanner> additionalScanners) throws IOException { RegionScanner scanner = getScanner(scan, null, copyCellsFromSharedMem);
return scanner;
}
protected RegionScanner getScanner(Scan scan, List<KeyValueScanner> additionalScanners,
boolean copyCellsFromSharedMem) throws IOException {
startRegionOperation(Operation.SCAN); startRegionOperation(Operation.SCAN);
try { try {
// Verify families are all valid // Verify families are all valid
if (!scan.hasFamilies()) { if (!scan.hasFamilies()) {
// Adding all families to scanner // Adding all families to scanner
for (byte[] family: this.htableDescriptor.getFamiliesKeys()) { for (byte[] family : this.htableDescriptor.getFamiliesKeys()) {
scan.addFamily(family); scan.addFamily(family);
} }
} else { } else {
for (byte [] family : scan.getFamilyMap().keySet()) { for (byte[] family : scan.getFamilyMap().keySet()) {
checkFamily(family); checkFamily(family);
} }
} }
return instantiateRegionScanner(scan, additionalScanners); return instantiateRegionScanner(scan, additionalScanners, copyCellsFromSharedMem);
} finally { } finally {
closeRegionOperation(Operation.SCAN); closeRegionOperation(Operation.SCAN);
} }
} }
protected RegionScanner instantiateRegionScanner(Scan scan, protected RegionScanner instantiateRegionScanner(Scan scan,
List<KeyValueScanner> additionalScanners) throws IOException { List<KeyValueScanner> additionalScanners, boolean copyCellsFromSharedMem) throws IOException {
if (scan.isReversed()) { if (scan.isReversed()) {
if (scan.getFilter() != null) { if (scan.getFilter() != null) {
scan.getFilter().setReversed(true); scan.getFilter().setReversed(true);
} }
return new ReversedRegionScannerImpl(scan, additionalScanners, this); return new ReversedRegionScannerImpl(scan, additionalScanners, this, copyCellsFromSharedMem);
} }
return new RegionScannerImpl(scan, additionalScanners, this); return new RegionScannerImpl(scan, additionalScanners, this, copyCellsFromSharedMem);
} }
@Override @Override
@ -5210,6 +5217,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
protected final byte[] stopRow; protected final byte[] stopRow;
protected final HRegion region; protected final HRegion region;
protected final CellComparator comparator; protected final CellComparator comparator;
protected boolean copyCellsFromSharedMem = false;
private final long readPt; private final long readPt;
private final long maxResultSize; private final long maxResultSize;
@ -5221,7 +5229,12 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
return region.getRegionInfo(); return region.getRegionInfo();
} }
RegionScannerImpl(Scan scan, List<KeyValueScanner> additionalScanners, HRegion region) public void setCopyCellsFromSharedMem(boolean copyCells) {
this.copyCellsFromSharedMem = copyCells;
}
RegionScannerImpl(Scan scan, List<KeyValueScanner> additionalScanners, HRegion region,
boolean copyCellsFromSharedMem)
throws IOException { throws IOException {
this.region = region; this.region = region;
this.maxResultSize = scan.getMaxResultSize(); this.maxResultSize = scan.getMaxResultSize();
@ -5231,13 +5244,13 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
this.filter = null; this.filter = null;
} }
this.comparator = region.getCellCompartor(); this.comparator = region.getCellCompartor();
/** /**
* By default, calls to next/nextRaw must enforce the batch limit. Thus, construct a default * By default, calls to next/nextRaw must enforce the batch limit. Thus, construct a default
* scanner context that can be used to enforce the batch limit in the event that a * scanner context that can be used to enforce the batch limit in the event that a
* ScannerContext is not specified during an invocation of next/nextRaw * ScannerContext is not specified during an invocation of next/nextRaw
*/ */
defaultScannerContext = ScannerContext.newBuilder().setBatchLimit(scan.getBatch()).build(); defaultScannerContext = ScannerContext.newBuilder()
.setBatchLimit(scan.getBatch()).build();
if (Bytes.equals(scan.getStopRow(), HConstants.EMPTY_END_ROW) && !scan.isGetScan()) { if (Bytes.equals(scan.getStopRow(), HConstants.EMPTY_END_ROW) && !scan.isGetScan()) {
this.stopRow = null; this.stopRow = null;
@ -5279,6 +5292,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
joinedScanners.add(scanner); joinedScanners.add(scanner);
} }
} }
this.copyCellsFromSharedMem = copyCellsFromSharedMem;
initializeKVHeap(scanners, joinedScanners, region); initializeKVHeap(scanners, joinedScanners, region);
} }
@ -5353,7 +5367,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
// scanner is closed // scanner is closed
throw new UnknownScannerException("Scanner was closed"); throw new UnknownScannerException("Scanner was closed");
} }
boolean moreValues; boolean moreValues = false;
try {
if (outResults.isEmpty()) { if (outResults.isEmpty()) {
// Usually outResults is empty. This is true when next is called // Usually outResults is empty. This is true when next is called
// to handle scan or get operation. // to handle scan or get operation.
@ -5364,14 +5379,37 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
outResults.addAll(tmpList); outResults.addAll(tmpList);
} }
// If the size limit was reached it means a partial Result is being returned. Returning a // If the size limit was reached it means a partial Result is being
// partial Result means that we should not reset the filters; filters should only be reset in // returned. Returning a
// partial Result means that we should not reset the filters; filters
// should only be reset in
// between rows // between rows
if (!scannerContext.partialResultFormed()) resetFilters(); if (!scannerContext.partialResultFormed()) resetFilters();
if (isFilterDoneInternal()) { if (isFilterDoneInternal()) {
moreValues = false; moreValues = false;
} }
// If copyCellsFromSharedMem = true, then we need to copy the cells. Otherwise
// it is a call coming from the RsRpcServices.scan().
if (copyCellsFromSharedMem && !outResults.isEmpty()) {
// Do the copy of the results here.
ListIterator<Cell> listItr = outResults.listIterator();
Cell cell = null;
while (listItr.hasNext()) {
cell = listItr.next();
if (cell instanceof ShareableMemory) {
listItr.set(((ShareableMemory) cell).cloneToCell());
}
}
}
} finally {
if (copyCellsFromSharedMem) {
// In case of copyCellsFromSharedMem==true (where the CPs wrap a scanner) we return
// the blocks then and there (for wrapped CPs)
this.shipped();
}
}
return moreValues; return moreValues;
} }
@ -6365,6 +6403,13 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
@Override @Override
public Result get(final Get get) throws IOException { public Result get(final Get get) throws IOException {
prepareGet(get);
List<Cell> results = get(get, true);
boolean stale = this.getRegionInfo().getReplicaId() != 0;
return Result.create(results, get.isCheckExistenceOnly() ? !results.isEmpty() : null, stale);
}
void prepareGet(final Get get) throws IOException, NoSuchColumnFamilyException {
checkRow(get.getRow(), "Get"); checkRow(get.getRow(), "Get");
// Verify families are all valid // Verify families are all valid
if (get.hasFamilies()) { if (get.hasFamilies()) {
@ -6376,9 +6421,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
get.addFamily(family); get.addFamily(family);
} }
} }
List<Cell> results = get(get, true);
boolean stale = this.getRegionInfo().getReplicaId() != 0;
return Result.create(results, get.isCheckExistenceOnly() ? !results.isEmpty() : null, stale);
} }
@Override @Override
@ -6409,16 +6451,21 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
coprocessorHost.postGet(get, results); coprocessorHost.postGet(get, results);
} }
// do after lock metricsUpdateForGet(results);
return results;
}
void metricsUpdateForGet(List<Cell> results) {
if (this.metricsRegion != null) { if (this.metricsRegion != null) {
long totalSize = 0L; long totalSize = 0L;
for (Cell cell : results) { for (Cell cell : results) {
// This should give an estimate of the cell in the result. Why do we need
// to know the serialization of how the codec works with it??
totalSize += CellUtil.estimatedSerializedSizeOf(cell); totalSize += CellUtil.estimatedSerializedSizeOf(cell);
} }
this.metricsRegion.updateGet(totalSize); this.metricsRegion.updateGet(totalSize);
} }
return results;
} }
@Override @Override
@ -7179,7 +7226,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
// New HBASE-880 Helpers // New HBASE-880 Helpers
// //
private void checkFamily(final byte [] family) void checkFamily(final byte [] family)
throws NoSuchColumnFamilyException { throws NoSuchColumnFamilyException {
if (!this.htableDescriptor.hasFamily(family)) { if (!this.htableDescriptor.hasFamily(family)) {
throw new NoSuchColumnFamilyException("Column family " + throw new NoSuchColumnFamilyException("Column family " +

View File

@ -47,6 +47,10 @@ import org.apache.hadoop.hbase.regionserver.ScannerContext.NextState;
public class KeyValueHeap extends NonReversedNonLazyKeyValueScanner public class KeyValueHeap extends NonReversedNonLazyKeyValueScanner
implements KeyValueScanner, InternalScanner { implements KeyValueScanner, InternalScanner {
protected PriorityQueue<KeyValueScanner> heap = null; protected PriorityQueue<KeyValueScanner> heap = null;
// Holds the scanners when a ever a eager close() happens. All such eagerly closed
// scans are collected and when the final scanner.close() happens will perform the
// actual close.
protected Set<KeyValueScanner> scannersForDelayedClose = new HashSet<KeyValueScanner>();
/** /**
* The current sub-scanner, i.e. the one that contains the next key/value * The current sub-scanner, i.e. the one that contains the next key/value
@ -62,8 +66,6 @@ public class KeyValueHeap extends NonReversedNonLazyKeyValueScanner
protected KVScannerComparator comparator; protected KVScannerComparator comparator;
protected Set<KeyValueScanner> scannersForDelayedClose = new HashSet<KeyValueScanner>();
/** /**
* Constructor. This KeyValueHeap will handle closing of passed in * Constructor. This KeyValueHeap will handle closing of passed in
* KeyValueScanners. * KeyValueScanners.
@ -160,6 +162,7 @@ public class KeyValueHeap extends NonReversedNonLazyKeyValueScanner
*/ */
if (pee == null || !moreCells) { if (pee == null || !moreCells) {
// add the scanner that is to be closed
this.scannersForDelayedClose.add(this.current); this.scannersForDelayedClose.add(this.current);
} else { } else {
this.heap.add(this.current); this.heap.add(this.current);

View File

@ -155,6 +155,7 @@ import org.apache.hadoop.hbase.protobuf.generated.WALProtos.FlushDescriptor;
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor; import org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor;
import org.apache.hadoop.hbase.quotas.OperationQuota; import org.apache.hadoop.hbase.quotas.OperationQuota;
import org.apache.hadoop.hbase.quotas.RegionServerQuotaManager; import org.apache.hadoop.hbase.quotas.RegionServerQuotaManager;
import org.apache.hadoop.hbase.regionserver.HRegion.RegionScannerImpl;
import org.apache.hadoop.hbase.regionserver.Leases.Lease; import org.apache.hadoop.hbase.regionserver.Leases.Lease;
import org.apache.hadoop.hbase.regionserver.Leases.LeaseStillHeldException; import org.apache.hadoop.hbase.regionserver.Leases.LeaseStillHeldException;
import org.apache.hadoop.hbase.regionserver.Region.FlushResult; import org.apache.hadoop.hbase.regionserver.Region.FlushResult;
@ -243,7 +244,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
/** /**
* An Rpc callback for closing a RegionScanner. * An Rpc callback for closing a RegionScanner.
*/ */
private static class RegionScannerCloseCallBack implements RpcCallback { static class RegionScannerCloseCallBack implements RpcCallback {
private final RegionScanner scanner; private final RegionScanner scanner;
@ -283,6 +284,29 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
} }
} }
/**
* An RpcCallBack that creates a list of scanners that needs to perform callBack operation on
* completion of multiGets.
*/
static class RegionScannersCloseCallBack implements RpcCallback {
private final List<RegionScanner> scanners = new ArrayList<RegionScanner>();
public void addScanner(RegionScanner scanner) {
this.scanners.add(scanner);
}
@Override
public void run() {
for (RegionScanner scanner : scanners) {
try {
scanner.close();
} catch (IOException e) {
LOG.error("Exception while closing the scanner " + scanner, e);
}
}
}
}
/** /**
* Holder class which holds the RegionScanner, nextCallSeq and RpcCallbacks together. * Holder class which holds the RegionScanner, nextCallSeq and RpcCallbacks together.
*/ */
@ -337,7 +361,6 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
if (region != null && region.getCoprocessorHost() != null) { if (region != null && region.getCoprocessorHost() != null) {
region.getCoprocessorHost().preScannerClose(s); region.getCoprocessorHost().preScannerClose(s);
} }
s.close(); s.close();
if (region != null && region.getCoprocessorHost() != null) { if (region != null && region.getCoprocessorHost() != null) {
region.getCoprocessorHost().postScannerClose(s); region.getCoprocessorHost().postScannerClose(s);
@ -418,8 +441,8 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
return context != null && context.isClientCellBlockSupport(); return context != null && context.isClientCellBlockSupport();
} }
private void addResult(final MutateResponse.Builder builder, private void addResult(final MutateResponse.Builder builder, final Result result,
final Result result, final PayloadCarryingRpcController rpcc) { final PayloadCarryingRpcController rpcc) {
if (result == null) return; if (result == null) return;
if (isClientCellBlockSupport()) { if (isClientCellBlockSupport()) {
builder.setResult(ProtobufUtil.toResultNoData(result)); builder.setResult(ProtobufUtil.toResultNoData(result));
@ -626,13 +649,23 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
// ResultOrException instance that matches each Put or Delete is then added down in the // ResultOrException instance that matches each Put or Delete is then added down in the
// doBatchOp call. We should be staying aligned though the Put and Delete are deferred/batched // doBatchOp call. We should be staying aligned though the Put and Delete are deferred/batched
List<ClientProtos.Action> mutations = null; List<ClientProtos.Action> mutations = null;
for (ClientProtos.Action action: actions.getActionList()) { RpcCallContext context = RpcServer.getCurrentCall();
// An RpcCallBack that creates a list of scanners that needs to perform callBack
// operation on completion of multiGets.
RegionScannersCloseCallBack closeCallBack = null;
for (ClientProtos.Action action : actions.getActionList()) {
ClientProtos.ResultOrException.Builder resultOrExceptionBuilder = null; ClientProtos.ResultOrException.Builder resultOrExceptionBuilder = null;
try { try {
Result r = null; Result r = null;
if (action.hasGet()) { if (action.hasGet()) {
if (closeCallBack == null) {
// Initialize only once
closeCallBack = new RegionScannersCloseCallBack();
// Set the call back here itself.
context.setCallBack(closeCallBack);
}
Get get = ProtobufUtil.toGet(action.getGet()); Get get = ProtobufUtil.toGet(action.getGet());
r = region.get(get); r = get(get, ((HRegion) region), closeCallBack, context);
} else if (action.hasServiceCall()) { } else if (action.hasServiceCall()) {
resultOrExceptionBuilder = ResultOrException.newBuilder(); resultOrExceptionBuilder = ResultOrException.newBuilder();
try { try {
@ -679,7 +712,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
} }
if (r != null) { if (r != null) {
ClientProtos.Result pbResult = null; ClientProtos.Result pbResult = null;
if (isClientCellBlockSupport()) { if (isClientCellBlockSupport(context)) {
pbResult = ProtobufUtil.toResultNoData(r); pbResult = ProtobufUtil.toResultNoData(r);
// Hard to guess the size here. Just make a rough guess. // Hard to guess the size here. Just make a rough guess.
if (cellsToReturn == null) cellsToReturn = new ArrayList<CellScannable>(); if (cellsToReturn == null) cellsToReturn = new ArrayList<CellScannable>();
@ -1930,7 +1963,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
ClientProtos.Get get = request.getGet(); ClientProtos.Get get = request.getGet();
Boolean existence = null; Boolean existence = null;
Result r = null; Result r = null;
RpcCallContext context = RpcServer.getCurrentCall();
quota = getQuotaManager().checkQuota(region, OperationQuota.OperationType.GET); quota = getQuotaManager().checkQuota(region, OperationQuota.OperationType.GET);
Get clientGet = ProtobufUtil.toGet(get); Get clientGet = ProtobufUtil.toGet(get);
@ -1938,7 +1971,12 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
existence = region.getCoprocessorHost().preExists(clientGet); existence = region.getCoprocessorHost().preExists(clientGet);
} }
if (existence == null) { if (existence == null) {
if (context != null) {
r = get(clientGet, ((HRegion) region), null, context);
} else {
// for test purpose
r = region.get(clientGet); r = region.get(clientGet);
}
if (get.getExistenceOnly()) { if (get.getExistenceOnly()) {
boolean exists = r.getExists(); boolean exists = r.getExists();
if (region.getCoprocessorHost() != null) { if (region.getCoprocessorHost() != null) {
@ -1971,6 +2009,52 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
} }
} }
private Result get(Get get, HRegion region, RegionScannersCloseCallBack closeCallBack,
RpcCallContext context) throws IOException {
region.prepareGet(get);
List<Cell> results = new ArrayList<Cell>();
boolean stale = region.getRegionInfo().getReplicaId() != 0;
// pre-get CP hook
if (region.getCoprocessorHost() != null) {
if (region.getCoprocessorHost().preGet(get, results)) {
return Result
.create(results, get.isCheckExistenceOnly() ? !results.isEmpty() : null, stale);
}
}
Scan scan = new Scan(get);
RegionScanner scanner = null;
try {
scanner = region.getScanner(scan, false);
scanner.next(results);
} finally {
if (scanner != null) {
if (closeCallBack == null) {
// If there is a context then the scanner can be added to the current
// RpcCallContext. The rpc callback will take care of closing the
// scanner, for eg in case
// of get()
assert scanner instanceof org.apache.hadoop.hbase.ipc.RpcCallback;
context.setCallBack((RegionScannerImpl) scanner);
} else {
// The call is from multi() where the results from the get() are
// aggregated and then send out to the
// rpc. The rpccall back will close all such scanners created as part
// of multi().
closeCallBack.addScanner(scanner);
}
}
}
// post-get CP hook
if (region.getCoprocessorHost() != null) {
region.getCoprocessorHost().postGet(get, results);
}
region.metricsUpdateForGet(results);
return Result.create(results, get.isCheckExistenceOnly() ? !results.isEmpty() : null, stale);
}
/** /**
* Execute multiple actions on a table: get, mutate, and/or execCoprocessor * Execute multiple actions on a table: get, mutate, and/or execCoprocessor
* *
@ -2230,6 +2314,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
boolean moreResults = true; boolean moreResults = true;
boolean closeScanner = false; boolean closeScanner = false;
boolean isSmallScan = false; boolean isSmallScan = false;
RegionScanner actualRegionScanner = null;
ScanResponse.Builder builder = ScanResponse.newBuilder(); ScanResponse.Builder builder = ScanResponse.newBuilder();
if (request.hasCloseScanner()) { if (request.hasCloseScanner()) {
closeScanner = request.getCloseScanner(); closeScanner = request.getCloseScanner();
@ -2274,17 +2359,27 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
scanner = region.getCoprocessorHost().preScannerOpen(scan); scanner = region.getCoprocessorHost().preScannerOpen(scan);
} }
if (scanner == null) { if (scanner == null) {
scanner = region.getScanner(scan); scanner = ((HRegion)region).getScanner(scan, false);
} }
actualRegionScanner = scanner;
if (region.getCoprocessorHost() != null) { if (region.getCoprocessorHost() != null) {
scanner = region.getCoprocessorHost().postScannerOpen(scan, scanner); scanner = region.getCoprocessorHost().postScannerOpen(scan, scanner);
} }
if (actualRegionScanner != scanner) {
// It means the RegionScanner has been wrapped
if (actualRegionScanner instanceof RegionScannerImpl) {
// Copy the results when nextRaw is called from the CP so that
// CP can have a cloned version of the results without bothering
// about the eviction. Ugly, yes!!!
((RegionScannerImpl) actualRegionScanner).setCopyCellsFromSharedMem(true);
}
}
scannerId = this.scannerIdGen.incrementAndGet(); scannerId = this.scannerIdGen.incrementAndGet();
scannerName = String.valueOf(scannerId); scannerName = String.valueOf(scannerId);
rsh = addScanner(scannerName, scanner, region); rsh = addScanner(scannerName, scanner, region);
ttl = this.scannerLeaseTimeoutPeriod; ttl = this.scannerLeaseTimeoutPeriod;
} }
assert scanner != null;
RpcCallContext context = RpcServer.getCurrentCall(); RpcCallContext context = RpcServer.getCurrentCall();
quota = getQuotaManager().checkQuota(region, OperationQuota.OperationType.SCAN); quota = getQuotaManager().checkQuota(region, OperationQuota.OperationType.SCAN);
@ -2295,9 +2390,6 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
// performed even before checking of Lease. // performed even before checking of Lease.
// See HBASE-5974 // See HBASE-5974
if (request.hasNextCallSeq()) { if (request.hasNextCallSeq()) {
if (rsh == null) {
rsh = scanners.get(scannerName);
}
if (rsh != null) { if (rsh != null) {
if (request.getNextCallSeq() != rsh.getNextCallSeq()) { if (request.getNextCallSeq() != rsh.getNextCallSeq()) {
throw new OutOfOrderScannerNextException( throw new OutOfOrderScannerNextException(
@ -2411,7 +2503,6 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
contextBuilder.setTimeLimit(timeScope, timeLimit); contextBuilder.setTimeLimit(timeScope, timeLimit);
contextBuilder.setTrackMetrics(trackMetrics); contextBuilder.setTrackMetrics(trackMetrics);
ScannerContext scannerContext = contextBuilder.build(); ScannerContext scannerContext = contextBuilder.build();
boolean limitReached = false; boolean limitReached = false;
while (i < rows) { while (i < rows) {
// Reset the batch progress to 0 before every call to RegionScanner#nextRaw. The // Reset the batch progress to 0 before every call to RegionScanner#nextRaw. The
@ -2488,7 +2579,6 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
} finally { } finally {
region.closeRegionOperation(); region.closeRegionOperation();
} }
// coprocessor postNext hook // coprocessor postNext hook
if (region != null && region.getCoprocessorHost() != null) { if (region != null && region.getCoprocessorHost() != null) {
region.getCoprocessorHost().postScannerNext(scanner, results, rows, true); region.getCoprocessorHost().postScannerNext(scanner, results, rows, true);

View File

@ -42,9 +42,9 @@ class ReversedRegionScannerImpl extends RegionScannerImpl {
* @throws IOException * @throws IOException
*/ */
ReversedRegionScannerImpl(Scan scan, ReversedRegionScannerImpl(Scan scan,
List<KeyValueScanner> additionalScanners, HRegion region) List<KeyValueScanner> additionalScanners, HRegion region, boolean copyCellsFromSharedMem)
throws IOException { throws IOException {
region.super(scan, additionalScanners, region); region.super(scan, additionalScanners, region, copyCellsFromSharedMem);
} }
@Override @Override

View File

@ -58,6 +58,7 @@ import org.apache.hadoop.hbase.util.BloomFilterWriter;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Writables; import org.apache.hadoop.hbase.util.Writables;
import org.apache.hadoop.io.WritableUtils; import org.apache.hadoop.io.WritableUtils;
import org.apache.hadoop.hbase.io.hfile.HFileBlock;
import com.google.common.base.Function; import com.google.common.base.Function;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
@ -1282,7 +1283,7 @@ public class StoreFile {
// Empty file // Empty file
if (reader.getTrailer().getEntryCount() == 0) if (reader.getTrailer().getEntryCount() == 0)
return false; return false;
HFileBlock bloomBlock = null;
try { try {
boolean shouldCheckBloom; boolean shouldCheckBloom;
ByteBuff bloom; ByteBuff bloom;
@ -1290,8 +1291,8 @@ public class StoreFile {
bloom = null; bloom = null;
shouldCheckBloom = true; shouldCheckBloom = true;
} else { } else {
bloom = reader.getMetaBlock(HFile.BLOOM_FILTER_DATA_KEY, bloomBlock = reader.getMetaBlock(HFile.BLOOM_FILTER_DATA_KEY, true);
true); bloom = bloomBlock.getBufferWithoutHeader();
shouldCheckBloom = bloom != null; shouldCheckBloom = bloom != null;
} }
@ -1343,8 +1344,10 @@ public class StoreFile {
} catch (IllegalArgumentException e) { } catch (IllegalArgumentException e) {
LOG.error("Bad bloom filter data -- proceeding without", e); LOG.error("Bad bloom filter data -- proceeding without", e);
setGeneralBloomFilterFaulty(); setGeneralBloomFilterFaulty();
} finally {
// Return the bloom block so that its ref count can be decremented.
reader.returnBlock(bloomBlock);
} }
return true; return true;
} }

View File

@ -85,6 +85,8 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
protected final long maxRowSize; protected final long maxRowSize;
protected final long cellsPerHeartbeatCheck; protected final long cellsPerHeartbeatCheck;
// Collects all the KVHeap that are eagerly getting closed during the
// course of a scan
protected Set<KeyValueHeap> heapsForDelayedClose = new HashSet<KeyValueHeap>(); protected Set<KeyValueHeap> heapsForDelayedClose = new HashSet<KeyValueHeap>();
/** /**
@ -446,8 +448,10 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
private void close(boolean withHeapClose){ private void close(boolean withHeapClose){
lock.lock(); lock.lock();
try { try {
if (this.closing) return; if (this.closing) {
this.closing = true; return;
}
if (withHeapClose) this.closing = true;
// under test, we dont have a this.store // under test, we dont have a this.store
if (this.store != null) this.store.deleteChangedReaderObserver(this); if (this.store != null) this.store.deleteChangedReaderObserver(this);
if (withHeapClose) { if (withHeapClose) {
@ -509,6 +513,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
// if the heap was left null, then the scanners had previously run out anyways, close and // if the heap was left null, then the scanners had previously run out anyways, close and
// return. // return.
if (this.heap == null) { if (this.heap == null) {
// By this time partial close should happened because already heap is null
close(false);// Do all cleanup except heap.close() close(false);// Do all cleanup except heap.close()
return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues(); return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();
} }

View File

@ -1406,7 +1406,8 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
desc.addFamily(hcd); desc.addFamily(hcd);
} }
getHBaseAdmin().createTable(desc, startKey, endKey, numRegions); getHBaseAdmin().createTable(desc, startKey, endKey, numRegions);
// HBaseAdmin only waits for regions to appear in hbase:meta we should wait until they are assigned // HBaseAdmin only waits for regions to appear in hbase:meta we
// should wait until they are assigned
waitUntilAllRegionsAssigned(tableName); waitUntilAllRegionsAssigned(tableName);
return (HTable) getConnection().getTable(tableName); return (HTable) getConnection().getTable(tableName);
} }
@ -1444,8 +1445,8 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
htd.addFamily(hcd); htd.addFamily(hcd);
} }
getHBaseAdmin().createTable(htd, splitKeys); getHBaseAdmin().createTable(htd, splitKeys);
// HBaseAdmin only waits for regions to appear in hbase:meta we should wait until they are // HBaseAdmin only waits for regions to appear in hbase:meta
// assigned // we should wait until they are assigned
waitUntilAllRegionsAssigned(htd.getTableName()); waitUntilAllRegionsAssigned(htd.getTableName());
return (HTable) getConnection().getTable(htd.getTableName()); return (HTable) getConnection().getTable(htd.getTableName());
} }
@ -1460,7 +1461,8 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
public HTable createTable(HTableDescriptor htd, byte[][] splitRows) public HTable createTable(HTableDescriptor htd, byte[][] splitRows)
throws IOException { throws IOException {
getHBaseAdmin().createTable(htd, splitRows); getHBaseAdmin().createTable(htd, splitRows);
// HBaseAdmin only waits for regions to appear in hbase:meta we should wait until they are assigned // HBaseAdmin only waits for regions to appear in hbase:meta
// we should wait until they are assigned
waitUntilAllRegionsAssigned(htd.getTableName()); waitUntilAllRegionsAssigned(htd.getTableName());
return (HTable) getConnection().getTable(htd.getTableName()); return (HTable) getConnection().getTable(htd.getTableName());
} }
@ -1700,6 +1702,24 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
return (HTable) getConnection().getTable(tableName); return (HTable) getConnection().getTable(tableName);
} }
public HTable createTable(TableName tableName, byte[][] families,
int numVersions, int blockSize, String cpName) throws IOException {
HTableDescriptor desc = new HTableDescriptor(tableName);
for (byte[] family : families) {
HColumnDescriptor hcd = new HColumnDescriptor(family)
.setMaxVersions(numVersions)
.setBlocksize(blockSize);
desc.addFamily(hcd);
}
if(cpName != null) {
desc.addCoprocessor(cpName);
}
getHBaseAdmin().createTable(desc);
// HBaseAdmin only waits for regions to appear in hbase:meta we should wait until they are assigned
waitUntilAllRegionsAssigned(tableName);
return (HTable) getConnection().getTable(tableName);
}
/** /**
* Create a table. * Create a table.
* @param tableName * @param tableName

View File

@ -270,7 +270,7 @@ public class CacheTestUtils {
} }
@Override @Override
public Cacheable deserialize(ByteBuff b, boolean reuse) public Cacheable deserialize(ByteBuff b, boolean reuse, MemoryType memType)
throws IOException { throws IOException {
return deserialize(b); return deserialize(b);
} }
@ -315,6 +315,11 @@ public class CacheTestUtils {
public BlockType getBlockType() { public BlockType getBlockType() {
return BlockType.DATA; return BlockType.DATA;
} }
@Override
public MemoryType getMemoryType() {
return MemoryType.EXCLUSIVE;
}
} }

View File

@ -37,6 +37,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.testclassification.IOTests; import org.apache.hadoop.hbase.testclassification.IOTests;
import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.io.hfile.Cacheable.MemoryType;
import org.apache.hadoop.hbase.io.hfile.bucket.BucketCache; import org.apache.hadoop.hbase.io.hfile.bucket.BucketCache;
import org.apache.hadoop.hbase.nio.ByteBuff; import org.apache.hadoop.hbase.nio.ByteBuff;
import org.apache.hadoop.hbase.nio.MultiByteBuff; import org.apache.hadoop.hbase.nio.MultiByteBuff;
@ -73,7 +74,7 @@ public class TestCacheConfig {
} }
@Override @Override
public Cacheable deserialize(ByteBuff b, boolean reuse) throws IOException { public Cacheable deserialize(ByteBuff b, boolean reuse, MemoryType memType) throws IOException {
LOG.info("Deserialized " + b + ", reuse=" + reuse); LOG.info("Deserialized " + b + ", reuse=" + reuse);
return cacheable; return cacheable;
} }
@ -140,6 +141,11 @@ public class TestCacheConfig {
public BlockType getBlockType() { public BlockType getBlockType() {
return BlockType.DATA; return BlockType.DATA;
} }
@Override
public MemoryType getMemoryType() {
return MemoryType.EXCLUSIVE;
}
}; };
static class MetaCacheEntry extends DataCacheEntry { static class MetaCacheEntry extends DataCacheEntry {

View File

@ -28,7 +28,10 @@ import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collection; import java.util.Collection;
import java.util.EnumMap; import java.util.EnumMap;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List; import java.util.List;
import java.util.Map;
import java.util.Random; import java.util.Random;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
@ -290,6 +293,8 @@ public class TestCacheOnWrite {
DataBlockEncoding encodingInCache = DataBlockEncoding encodingInCache =
encoderType.getEncoder().getDataBlockEncoding(); encoderType.getEncoder().getDataBlockEncoding();
List<Long> cachedBlocksOffset = new ArrayList<Long>();
Map<Long, HFileBlock> cachedBlocks = new HashMap<Long, HFileBlock>();
while (offset < reader.getTrailer().getLoadOnOpenDataOffset()) { while (offset < reader.getTrailer().getLoadOnOpenDataOffset()) {
long onDiskSize = -1; long onDiskSize = -1;
if (prevBlock != null) { if (prevBlock != null) {
@ -303,6 +308,8 @@ public class TestCacheOnWrite {
offset); offset);
HFileBlock fromCache = (HFileBlock) blockCache.getBlock(blockCacheKey, true, false, true); HFileBlock fromCache = (HFileBlock) blockCache.getBlock(blockCacheKey, true, false, true);
boolean isCached = fromCache != null; boolean isCached = fromCache != null;
cachedBlocksOffset.add(offset);
cachedBlocks.put(offset, fromCache);
boolean shouldBeCached = cowType.shouldBeCached(block.getBlockType()); boolean shouldBeCached = cowType.shouldBeCached(block.getBlockType());
assertTrue("shouldBeCached: " + shouldBeCached+ "\n" + assertTrue("shouldBeCached: " + shouldBeCached+ "\n" +
"isCached: " + isCached + "\n" + "isCached: " + isCached + "\n" +
@ -355,6 +362,28 @@ public class TestCacheOnWrite {
while (scanner.next()) { while (scanner.next()) {
scanner.getCell(); scanner.getCell();
} }
Iterator<Long> iterator = cachedBlocksOffset.iterator();
while(iterator.hasNext()) {
Long entry = iterator.next();
BlockCacheKey blockCacheKey = new BlockCacheKey(reader.getName(),
entry);
HFileBlock hFileBlock = cachedBlocks.get(entry);
if (hFileBlock != null) {
// call return twice because for the isCache cased the counter would have got incremented
// twice
blockCache.returnBlock(blockCacheKey, hFileBlock);
if(cacheCompressedData) {
if (this.compress == Compression.Algorithm.NONE
|| cowType == CacheOnWriteType.INDEX_BLOCKS
|| cowType == CacheOnWriteType.BLOOM_BLOCKS) {
blockCache.returnBlock(blockCacheKey, hFileBlock);
}
} else {
blockCache.returnBlock(blockCacheKey, hFileBlock);
}
}
}
scanner.shipped();
reader.close(); reader.close();
} }

View File

@ -141,6 +141,11 @@ public class TestCachedBlockQueue extends TestCase {
return BlockType.DATA; return BlockType.DATA;
} }
@Override
public MemoryType getMemoryType() {
return MemoryType.EXCLUSIVE;
}
}, accessTime, false); }, accessTime, false);
} }
} }

View File

@ -328,7 +328,7 @@ public class TestHFile extends HBaseTestCase {
private void readNumMetablocks(Reader reader, int n) throws IOException { private void readNumMetablocks(Reader reader, int n) throws IOException {
for (int i = 0; i < n; i++) { for (int i = 0; i < n; i++) {
ByteBuff actual = reader.getMetaBlock("HFileMeta" + i, false); ByteBuff actual = reader.getMetaBlock("HFileMeta" + i, false).getBufferWithoutHeader();
ByteBuffer expected = ByteBuffer expected =
ByteBuffer.wrap(("something to test" + i).getBytes()); ByteBuffer.wrap(("something to test" + i).getBytes());
assertEquals( assertEquals(

View File

@ -56,6 +56,7 @@ import org.apache.hadoop.hbase.fs.HFileSystem;
import org.apache.hadoop.hbase.io.compress.Compression; import org.apache.hadoop.hbase.io.compress.Compression;
import org.apache.hadoop.hbase.io.compress.Compression.Algorithm; import org.apache.hadoop.hbase.io.compress.Compression.Algorithm;
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding; import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
import org.apache.hadoop.hbase.io.hfile.Cacheable.MemoryType;
import org.apache.hadoop.hbase.nio.ByteBuff; import org.apache.hadoop.hbase.nio.ByteBuff;
import org.apache.hadoop.hbase.nio.MultiByteBuff; import org.apache.hadoop.hbase.nio.MultiByteBuff;
import org.apache.hadoop.hbase.nio.SingleByteBuff; import org.apache.hadoop.hbase.nio.SingleByteBuff;
@ -463,8 +464,9 @@ public class TestHFileBlock {
for (boolean reuseBuffer : new boolean[] { false, true }) { for (boolean reuseBuffer : new boolean[] { false, true }) {
ByteBuffer serialized = ByteBuffer.allocate(blockFromHFile.getSerializedLength()); ByteBuffer serialized = ByteBuffer.allocate(blockFromHFile.getSerializedLength());
blockFromHFile.serialize(serialized); blockFromHFile.serialize(serialized);
HFileBlock deserialized = (HFileBlock) blockFromHFile.getDeserializer().deserialize( HFileBlock deserialized =
new SingleByteBuff(serialized), reuseBuffer); (HFileBlock) blockFromHFile.getDeserializer().deserialize(
new SingleByteBuff(serialized), reuseBuffer, MemoryType.EXCLUSIVE);
assertEquals( assertEquals(
"Serialization did not preserve block state. reuseBuffer=" + reuseBuffer, "Serialization did not preserve block state. reuseBuffer=" + reuseBuffer,
blockFromHFile, deserialized); blockFromHFile, deserialized);

View File

@ -168,6 +168,10 @@ public class TestHFileBlockIndex {
this.realReader = realReader; this.realReader = realReader;
} }
@Override
public void returnBlock(HFileBlock block) {
}
@Override @Override
public HFileBlock readBlock(long offset, long onDiskSize, public HFileBlock readBlock(long offset, long onDiskSize,
boolean cacheBlock, boolean pread, boolean isCompaction, boolean cacheBlock, boolean pread, boolean isCompaction,

View File

@ -779,6 +779,11 @@ public class TestLruBlockCache {
return BlockType.DATA; return BlockType.DATA;
} }
@Override
public MemoryType getMemoryType() {
return MemoryType.EXCLUSIVE;
}
} }
} }

View File

@ -22,10 +22,12 @@ import static org.junit.Assert.assertTrue;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import org.apache.hadoop.hbase.io.hfile.Cacheable.MemoryType;
import org.apache.hadoop.hbase.nio.ByteBuff; import org.apache.hadoop.hbase.nio.ByteBuff;
import org.apache.hadoop.hbase.nio.MultiByteBuff; import org.apache.hadoop.hbase.nio.MultiByteBuff;
import org.apache.hadoop.hbase.testclassification.IOTests; import org.apache.hadoop.hbase.testclassification.IOTests;
import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.apache.hadoop.hbase.util.Pair;
import org.junit.Test; import org.junit.Test;
import org.junit.experimental.categories.Category; import org.junit.experimental.categories.Category;
@ -65,11 +67,10 @@ public class TestByteBufferIOEngine {
offset = (int) (Math.random() * (capacity - maxBlockSize)); offset = (int) (Math.random() * (capacity - maxBlockSize));
} }
ioEngine.write(srcBuffer, offset); ioEngine.write(srcBuffer, offset);
ByteBuffer dstBuffer = ByteBuffer.allocate(blockSize); Pair<ByteBuff, MemoryType> pair = ioEngine.read(offset, blockSize);
ioEngine.read(dstBuffer, offset); ByteBuff dstBuffer = pair.getFirst();
byte[] byteArray2 = dstBuffer.array();
for (int j = 0; j < byteArray.length; ++j) { for (int j = 0; j < byteArray.length; ++j) {
assertTrue(byteArray[j] == byteArray2[j]); assertTrue(byteArray[j] == dstBuffer.get(j));
} }
} }
assert testOffsetAtStartNum == 0; assert testOffsetAtStartNum == 0;
@ -110,9 +111,9 @@ public class TestByteBufferIOEngine {
//ioEngine.read(dstBuffer, offset); //ioEngine.read(dstBuffer, offset);
//MultiByteBuffer read = new MultiByteBuffer(dstBuffer); //MultiByteBuffer read = new MultiByteBuffer(dstBuffer);
// TODO : this will get changed after HBASE-12295 goes in // TODO : this will get changed after HBASE-12295 goes in
ByteBuff read = ioEngine.read(offset, blockSize); Pair<ByteBuff, MemoryType> read = ioEngine.read(offset, blockSize);
for (int j = 0; j < byteArray.length; ++j) { for (int j = 0; j < byteArray.length; ++j) {
assertTrue(srcBuffer.get(j) == read.get(j)); assertTrue(srcBuffer.get(j) == read.getFirst().get(j));
} }
} }
assert testOffsetAtStartNum == 0; assert testOffsetAtStartNum == 0;

View File

@ -24,8 +24,11 @@ import java.io.File;
import java.io.IOException; import java.io.IOException;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import org.apache.hadoop.hbase.io.hfile.Cacheable.MemoryType;
import org.apache.hadoop.hbase.nio.ByteBuff;
import org.apache.hadoop.hbase.testclassification.IOTests; import org.apache.hadoop.hbase.testclassification.IOTests;
import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.apache.hadoop.hbase.util.Pair;
import org.junit.Test; import org.junit.Test;
import org.junit.experimental.categories.Category; import org.junit.experimental.categories.Category;
@ -47,9 +50,9 @@ public class TestFileIOEngine {
for (int j = 0; j < data1.length; ++j) { for (int j = 0; j < data1.length; ++j) {
data1[j] = (byte) (Math.random() * 255); data1[j] = (byte) (Math.random() * 255);
} }
byte[] data2 = new byte[len];
fileIOEngine.write(ByteBuffer.wrap(data1), offset); fileIOEngine.write(ByteBuffer.wrap(data1), offset);
fileIOEngine.read(ByteBuffer.wrap(data2), offset); Pair<ByteBuff, MemoryType> pair = fileIOEngine.read(offset, len);
byte[] data2 = pair.getFirst().array();
for (int j = 0; j < data1.length; ++j) { for (int j = 0; j < data1.length; ++j) {
assertTrue(data1[j] == data2[j]); assertTrue(data1[j] == data2[j]);
} }

View File

@ -39,6 +39,7 @@ import org.apache.hadoop.hbase.io.hfile.BlockCacheKey;
import org.apache.hadoop.hbase.io.hfile.CacheStats; import org.apache.hadoop.hbase.io.hfile.CacheStats;
import org.apache.hadoop.hbase.io.hfile.Cacheable; import org.apache.hadoop.hbase.io.hfile.Cacheable;
import org.apache.hadoop.hbase.io.hfile.CachedBlock; import org.apache.hadoop.hbase.io.hfile.CachedBlock;
import org.apache.hadoop.hbase.io.hfile.HFileBlock;
import org.apache.hadoop.hbase.io.hfile.ResizableBlockCache; import org.apache.hadoop.hbase.io.hfile.ResizableBlockCache;
import org.apache.hadoop.hbase.io.util.HeapMemorySizeUtil; import org.apache.hadoop.hbase.io.util.HeapMemorySizeUtil;
import org.apache.hadoop.hbase.regionserver.HeapMemoryManager.TunerContext; import org.apache.hadoop.hbase.regionserver.HeapMemoryManager.TunerContext;
@ -578,6 +579,10 @@ public class TestHeapMemoryManager {
return null; return null;
} }
@Override
public void returnBlock(BlockCacheKey cacheKey, Cacheable buf) {
}
public void setTestBlockSize(long testBlockSize) { public void setTestBlockSize(long testBlockSize) {
this.testBlockSize = testBlockSize; this.testBlockSize = testBlockSize;
} }

View File

@ -425,14 +425,14 @@ public class TestScannerHeartbeatMessages {
// Instantiate the custom heartbeat region scanners // Instantiate the custom heartbeat region scanners
@Override @Override
protected RegionScanner instantiateRegionScanner(Scan scan, protected RegionScanner instantiateRegionScanner(Scan scan,
List<KeyValueScanner> additionalScanners) throws IOException { List<KeyValueScanner> additionalScanners, boolean copyCells) throws IOException {
if (scan.isReversed()) { if (scan.isReversed()) {
if (scan.getFilter() != null) { if (scan.getFilter() != null) {
scan.getFilter().setReversed(true); scan.getFilter().setReversed(true);
} }
return new HeartbeatReversedRegionScanner(scan, additionalScanners, this); return new HeartbeatReversedRegionScanner(scan, additionalScanners, this, copyCells);
} }
return new HeartbeatRegionScanner(scan, additionalScanners, this); return new HeartbeatRegionScanner(scan, additionalScanners, this, copyCells);
} }
} }
@ -442,8 +442,8 @@ public class TestScannerHeartbeatMessages {
*/ */
private static class HeartbeatReversedRegionScanner extends ReversedRegionScannerImpl { private static class HeartbeatReversedRegionScanner extends ReversedRegionScannerImpl {
HeartbeatReversedRegionScanner(Scan scan, List<KeyValueScanner> additionalScanners, HeartbeatReversedRegionScanner(Scan scan, List<KeyValueScanner> additionalScanners,
HRegion region) throws IOException { HRegion region, boolean copyCells) throws IOException {
super(scan, additionalScanners, region); super(scan, additionalScanners, region, copyCells);
} }
@Override @Override
@ -469,9 +469,9 @@ public class TestScannerHeartbeatMessages {
* column family cells * column family cells
*/ */
private static class HeartbeatRegionScanner extends RegionScannerImpl { private static class HeartbeatRegionScanner extends RegionScannerImpl {
HeartbeatRegionScanner(Scan scan, List<KeyValueScanner> additionalScanners, HRegion region) HeartbeatRegionScanner(Scan scan, List<KeyValueScanner> additionalScanners, HRegion region,
throws IOException { boolean copyCells) throws IOException {
region.super(scan, additionalScanners, region); region.super(scan, additionalScanners, region, copyCells);
} }
@Override @Override