@InterfaceAudience.Private public abstract class HFileReaderImpl extends Object implements HFile.Reader, org.apache.hadoop.conf.Configurable
HFile.Reader
.Modifier and Type | Class and Description |
---|---|
static class |
HFileReaderImpl.BlockIndexNotLoadedException |
protected static class |
HFileReaderImpl.EncodedScanner
Scanner that operates on encoded data blocks.
|
protected static class |
HFileReaderImpl.HFileScannerImpl |
static class |
HFileReaderImpl.NotSeekedException
An exception thrown when an operation requiring a scanner to be seeked
is invoked on a scanner that is not seeked.
|
Modifier and Type | Field and Description |
---|---|
protected CacheConfig |
cacheConf
Block cache configuration.
|
private org.apache.hadoop.conf.Configuration |
conf |
protected ReaderContext |
context |
protected HFileDataBlockEncoder |
dataBlockEncoder
What kind of data block encoding should be used while reading, writing,
and handling cache.
|
protected HFileBlockIndex.CellBasedKeyBlockIndexReader |
dataBlockIndexReader
Data block index reader keeping the root data index in memory
|
protected HFileInfo |
fileInfo |
protected HFileBlock.FSReader |
fsBlockReader
Filesystem-level block reader.
|
protected HFileContext |
hfileContext |
static int |
KEY_VALUE_LEN_SIZE
The size of a (key length, value length) tuple that prefixes each entry in
a data block.
|
private static org.slf4j.Logger |
LOG |
(package private) static int |
MAX_MINOR_VERSION
Maximum minor version supported by this HFile format
|
protected HFileBlockIndex.ByteArrayKeyBlockIndexReader |
metaBlockIndexReader
Meta block index reader -- always single level
|
(package private) static int |
MIN_MINOR_VERSION
Minimum minor version supported by this HFile format
|
static int |
MINOR_VERSION_NO_CHECKSUM
In HFile minor version that does not support checksums
|
static int |
MINOR_VERSION_WITH_CHECKSUM
Minor versions in HFile starting with this number have hbase checksums
|
(package private) static int |
MINOR_VERSION_WITH_FAKED_KEY
Minor versions starting with this number have faked index key
|
protected String |
name
File name to be used for block names
|
private IdLock |
offsetLock
A "sparse lock" implementation allowing to lock on a particular block
identified by offset.
|
protected org.apache.hadoop.fs.Path |
path
Path of file
|
static int |
PBUF_TRAILER_MINOR_VERSION
HFile minor version that introduced pbuf filetrailer
|
private boolean |
primaryReplicaReader |
protected FixedFileTrailer |
trailer |
Constructor and Description |
---|
HFileReaderImpl(ReaderContext context,
HFileInfo fileInfo,
CacheConfig cacheConf,
org.apache.hadoop.conf.Configuration conf)
Opens a HFile.
|
Modifier and Type | Method and Description |
---|---|
void |
close() |
private DataInput |
getBloomFilterMetadata(BlockType blockType) |
private HFileBlock |
getCachedBlock(BlockCacheKey cacheKey,
boolean cacheBlock,
boolean useLock,
boolean isCompaction,
boolean updateCacheMetrics,
BlockType expectedBlockType,
DataBlockEncoding expectedDataBlockEncoding)
Retrieve block from cache.
|
CellComparator |
getComparator() |
Compression.Algorithm |
getCompressionAlgorithm() |
org.apache.hadoop.conf.Configuration |
getConf() |
ReaderContext |
getContext() |
DataBlockEncoding |
getDataBlockEncoding() |
HFileBlockIndex.CellBasedKeyBlockIndexReader |
getDataBlockIndexReader() |
DataInput |
getDeleteBloomFilterMetadata()
Retrieves delete family Bloom filter metadata as appropriate for each
HFile version. |
DataBlockEncoding |
getEffectiveEncodingInCache(boolean isCompaction) |
long |
getEntries() |
HFileContext |
getFileContext()
Return the file context of the HFile this reader belongs to
|
Optional<Cell> |
getFirstKey() |
Optional<byte[]> |
getFirstRowKey()
|
DataInput |
getGeneralBloomFilterMetadata()
Returns a buffer with the Bloom filter metadata.
|
HFileInfo |
getHFileInfo() |
Optional<Cell> |
getLastKey() |
Optional<byte[]> |
getLastRowKey()
|
int |
getMajorVersion() |
HFileBlock |
getMetaBlock(String metaBlockName,
boolean cacheBlock) |
HFileBlockIndex.ByteArrayKeyBlockIndexReader |
getMetaBlockIndexReader() |
String |
getName()
Returns this reader's "name".
|
org.apache.hadoop.fs.Path |
getPath() |
HFileScanner |
getScanner(boolean cacheBlocks,
boolean pread)
Create a Scanner on this file.
|
HFileScanner |
getScanner(boolean cacheBlocks,
boolean pread,
boolean isCompaction)
Create a Scanner on this file.
|
FixedFileTrailer |
getTrailer() |
HFileBlock.FSReader |
getUncachedBlockReader()
For testing
|
boolean |
hasMVCCInfo() |
long |
indexSize() |
boolean |
isFileInfoLoaded() |
boolean |
isPrimaryReplicaReader() |
long |
length() |
Optional<Cell> |
midKey() |
boolean |
prefetchComplete()
Returns false if block prefetching was requested for this file and has
not completed, true otherwise
|
HFileBlock |
readBlock(long dataBlockOffset,
long onDiskBlockSize,
boolean cacheBlock,
boolean pread,
boolean isCompaction,
boolean updateCacheMetrics,
BlockType expectedBlockType,
DataBlockEncoding expectedDataBlockEncoding)
Read in a file block.
|
private void |
returnAndEvictBlock(BlockCache cache,
BlockCacheKey cacheKey,
Cacheable block) |
void |
setConf(org.apache.hadoop.conf.Configuration conf) |
void |
setDataBlockEncoder(HFileDataBlockEncoder dataBlockEncoder) |
void |
setDataBlockIndexReader(HFileBlockIndex.CellBasedKeyBlockIndexReader reader) |
void |
setMetaBlockIndexReader(HFileBlockIndex.ByteArrayKeyBlockIndexReader reader) |
private boolean |
shouldUseHeap(BlockType expectedBlockType)
If expected block is data block, we'll allocate the ByteBuff of block from
ByteBuffAllocator and it's usually an off-heap one,
otherwise it will allocate from heap. |
String |
toString() |
private Optional<String> |
toStringFirstKey() |
private Optional<String> |
toStringLastKey() |
void |
unbufferStream()
To close the stream's socket.
|
private void |
validateBlockType(HFileBlock block,
BlockType expectedBlockType)
Compares the actual type of a block retrieved from cache or disk with its
expected type and throws an exception in case of a mismatch.
|
clone, equals, finalize, getClass, hashCode, notify, notifyAll, wait, wait, wait
close
private static final org.slf4j.Logger LOG
protected HFileBlockIndex.CellBasedKeyBlockIndexReader dataBlockIndexReader
protected HFileBlockIndex.ByteArrayKeyBlockIndexReader metaBlockIndexReader
protected FixedFileTrailer trailer
private final boolean primaryReplicaReader
protected HFileDataBlockEncoder dataBlockEncoder
protected final CacheConfig cacheConf
protected ReaderContext context
protected final org.apache.hadoop.fs.Path path
private org.apache.hadoop.conf.Configuration conf
protected HFileContext hfileContext
protected HFileBlock.FSReader fsBlockReader
private IdLock offsetLock
static final int MIN_MINOR_VERSION
static final int MAX_MINOR_VERSION
static final int MINOR_VERSION_WITH_FAKED_KEY
public static final int MINOR_VERSION_WITH_CHECKSUM
public static final int MINOR_VERSION_NO_CHECKSUM
public static final int PBUF_TRAILER_MINOR_VERSION
public static final int KEY_VALUE_LEN_SIZE
public HFileReaderImpl(ReaderContext context, HFileInfo fileInfo, CacheConfig cacheConf, org.apache.hadoop.conf.Configuration conf) throws IOException
context
- Reader context infofileInfo
- HFile infocacheConf
- Cache configuration.conf
- ConfigurationIOException
private Optional<String> toStringFirstKey()
private Optional<String> toStringLastKey()
public long length()
length
in interface HFile.Reader
public Optional<Cell> getFirstKey()
getFirstKey
in interface HFile.Reader
public Optional<byte[]> getFirstRowKey()
HFile
version 1: move this to StoreFile after Ryan's
patch goes in to eliminate KeyValue
here.getFirstRowKey
in interface HFile.Reader
public Optional<byte[]> getLastRowKey()
HFile
version 1: move this to StoreFile after
Ryan's patch goes in to eliminate KeyValue
here.getLastRowKey
in interface HFile.Reader
public long getEntries()
getEntries
in interface HFile.Reader
public CellComparator getComparator()
getComparator
in interface HFile.Reader
public Compression.Algorithm getCompressionAlgorithm()
public long indexSize()
indexSize
in interface HFile.Reader
public String getName()
HFile.Reader
getName
in interface HFile.Reader
public void setDataBlockEncoder(HFileDataBlockEncoder dataBlockEncoder)
setDataBlockEncoder
in interface HFile.Reader
public void setDataBlockIndexReader(HFileBlockIndex.CellBasedKeyBlockIndexReader reader)
setDataBlockIndexReader
in interface HFile.Reader
public HFileBlockIndex.CellBasedKeyBlockIndexReader getDataBlockIndexReader()
getDataBlockIndexReader
in interface HFile.Reader
public void setMetaBlockIndexReader(HFileBlockIndex.ByteArrayKeyBlockIndexReader reader)
setMetaBlockIndexReader
in interface HFile.Reader
public HFileBlockIndex.ByteArrayKeyBlockIndexReader getMetaBlockIndexReader()
getMetaBlockIndexReader
in interface HFile.Reader
public FixedFileTrailer getTrailer()
getTrailer
in interface HFile.Reader
public ReaderContext getContext()
getContext
in interface HFile.Reader
public HFileInfo getHFileInfo()
getHFileInfo
in interface HFile.Reader
public boolean isPrimaryReplicaReader()
isPrimaryReplicaReader
in interface HFile.Reader
public org.apache.hadoop.fs.Path getPath()
getPath
in interface HFile.Reader
public DataBlockEncoding getDataBlockEncoding()
getDataBlockEncoding
in interface HFile.Reader
public org.apache.hadoop.conf.Configuration getConf()
getConf
in interface org.apache.hadoop.conf.Configurable
public void setConf(org.apache.hadoop.conf.Configuration conf)
setConf
in interface org.apache.hadoop.conf.Configurable
private HFileBlock getCachedBlock(BlockCacheKey cacheKey, boolean cacheBlock, boolean useLock, boolean isCompaction, boolean updateCacheMetrics, BlockType expectedBlockType, DataBlockEncoding expectedDataBlockEncoding) throws IOException
expectedBlockType
and its encoding vs. expectedDataBlockEncoding
. Unpacks the block as necessary.IOException
private void returnAndEvictBlock(BlockCache cache, BlockCacheKey cacheKey, Cacheable block)
public HFileBlock getMetaBlock(String metaBlockName, boolean cacheBlock) throws IOException
getMetaBlock
in interface HFile.Reader
cacheBlock
- Add block to cache, if foundIOException
private boolean shouldUseHeap(BlockType expectedBlockType)
ByteBuffAllocator
and it's usually an off-heap one,
otherwise it will allocate from heap.public HFileBlock readBlock(long dataBlockOffset, long onDiskBlockSize, boolean cacheBlock, boolean pread, boolean isCompaction, boolean updateCacheMetrics, BlockType expectedBlockType, DataBlockEncoding expectedDataBlockEncoding) throws IOException
HFile.CachingBlockReader
readBlock
in interface HFile.CachingBlockReader
dataBlockOffset
- offset to read.onDiskBlockSize
- size of the blockisCompaction
- is this block being read as part of a compactionexpectedBlockType
- the block type we are expecting to read with this read operation,
or null to read whatever block type is available and avoid checking (that might reduce
caching efficiency of encoded data blocks)expectedDataBlockEncoding
- the data block encoding the caller is expecting data blocks
to be in, or null to not perform this check and return the block irrespective of the
encoding. This check only applies to data blocks and can be set to null when the caller is
expecting to read a non-data block and has set expectedBlockType accordingly.IOException
public boolean hasMVCCInfo()
hasMVCCInfo
in interface HFile.Reader
private void validateBlockType(HFileBlock block, BlockType expectedBlockType) throws IOException
BlockType.DATA
is considered to match the actual
block type [@link BlockType.ENCODED_DATA
as well.block
- a block retrieved from cache or diskexpectedBlockType
- the expected block type, or null to skip the
checkIOException
public Optional<Cell> getLastKey()
getLastKey
in interface HFile.Reader
public Optional<Cell> midKey() throws IOException
midKey
in interface HFile.Reader
IOException
public void close() throws IOException
close
in interface Closeable
close
in interface AutoCloseable
IOException
public DataBlockEncoding getEffectiveEncodingInCache(boolean isCompaction)
getEffectiveEncodingInCache
in interface HFile.Reader
public HFileBlock.FSReader getUncachedBlockReader()
getUncachedBlockReader
in interface HFile.Reader
public DataInput getGeneralBloomFilterMetadata() throws IOException
getGeneralBloomFilterMetadata
in interface HFile.Reader
IOException
public DataInput getDeleteBloomFilterMetadata() throws IOException
HFile.Reader
HFile
version.
Knows nothing about how that metadata is structured.getDeleteBloomFilterMetadata
in interface HFile.Reader
IOException
private DataInput getBloomFilterMetadata(BlockType blockType) throws IOException
IOException
public boolean isFileInfoLoaded()
public HFileContext getFileContext()
HFile.Reader
getFileContext
in interface HFile.Reader
public boolean prefetchComplete()
prefetchComplete
in interface HFile.Reader
public HFileScanner getScanner(boolean cacheBlocks, boolean pread)
HFileScanner.seekTo(Cell)
to position an start the read. There is
nothing to clean up in a Scanner. Letting go of your references to the
scanner is sufficient. NOTE: Do not use this overload of getScanner for
compactions. See getScanner(boolean, boolean, boolean)
getScanner
in interface HFile.Reader
cacheBlocks
- True if we should cache blocks read in by this scanner.pread
- Use positional read rather than seek+read if true (pread is
better for random reads, seek+read is better scanning).public HFileScanner getScanner(boolean cacheBlocks, boolean pread, boolean isCompaction)
HFileScanner.seekTo(Cell)
to position an start the read. There is
nothing to clean up in a Scanner. Letting go of your references to the
scanner is sufficient.getScanner
in interface HFile.Reader
cacheBlocks
- True if we should cache blocks read in by this scanner.pread
- Use positional read rather than seek+read if true (pread is better
for random reads, seek+read is better scanning).isCompaction
- is scanner being used for a compaction?public int getMajorVersion()
public void unbufferStream()
HFile.Reader
unbufferStream
in interface HFile.Reader
Copyright © 2007–2020 The Apache Software Foundation. All rights reserved.