@InterfaceAudience.Private public 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 |
|---|---|
private int |
avgKeyLen
Average key length read from file info
|
private int |
avgValueLen
Average value length read from file info
|
private CacheConfig |
cacheConf
Block cache configuration.
|
private CellComparator |
comparator
Key comparator
|
private Compression.Algorithm |
compressAlgo
Filled when we read in the trailer.
|
private org.apache.hadoop.conf.Configuration |
conf |
private HFileDataBlockEncoder |
dataBlockEncoder
What kind of data block encoding should be used while reading, writing,
and handling cache.
|
private HFileBlockIndex.CellBasedKeyBlockIndexReader |
dataBlockIndexReader
Data block index reader keeping the root data index in memory
|
protected boolean |
decodeMemstoreTS |
private HFile.FileInfo |
fileInfo |
private long |
fileSize
Size of this file.
|
private HFileBlock.FSReader |
fsBlockReader
Filesystem-level block reader.
|
private HFileContext |
hfileContext |
private boolean |
includesMemstoreTS |
static int |
KEY_VALUE_LEN_SIZE
The size of a (key length, value length) tuple that prefixes each entry in
a data block.
|
private Cell |
lastKeyCell
Last key in the file.
|
private List<HFileBlock> |
loadOnOpenBlocks
Blocks read from the load-on-open section, excluding data root index, meta
index, and file info.
|
private static org.slf4j.Logger |
LOG |
(package private) static int |
MAX_MINOR_VERSION
Maximum minor version supported by this HFile format
|
private 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
|
private static int |
MIN_V2_MINOR_VERSION_WITH_PB
We can read files whose major version is v2 IFF their minor version is at least 3.
|
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
|
private 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.
|
private 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 |
private FixedFileTrailer |
trailer |
| Constructor and Description |
|---|
HFileReaderImpl(org.apache.hadoop.fs.Path path,
FixedFileTrailer trailer,
FSDataInputStreamWrapper fsdis,
long fileSize,
CacheConfig cacheConf,
HFileSystem hfs,
boolean primaryReplicaReader,
org.apache.hadoop.conf.Configuration conf)
Opens a HFile.
|
HFileReaderImpl(org.apache.hadoop.fs.Path path,
FixedFileTrailer trailer,
FSDataInputStreamWrapper fsdis,
long fileSize,
CacheConfig cacheConf,
HFileSystem hfs,
org.apache.hadoop.conf.Configuration conf)
Deprecated.
|
| Modifier and Type | Method and Description |
|---|---|
private void |
checkFileVersion()
File version check is a little sloppy.
|
void |
close() |
void |
close(boolean evictOnClose)
Close method with optional evictOnClose
|
protected HFileContext |
createHFileContext(FSDataInputStreamWrapper fsdis,
long fileSize,
HFileSystem hfs,
org.apache.hadoop.fs.Path path,
FixedFileTrailer trailer) |
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() |
DataBlockEncoding |
getDataBlockEncoding() |
HFileBlockIndex.BlockIndexReader |
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.
|
Optional<Cell> |
getLastKey() |
Optional<byte[]> |
getLastRowKey()
|
int |
getMajorVersion() |
HFileBlock |
getMetaBlock(String metaBlockName,
boolean cacheBlock) |
String |
getName()
Returns this reader's "name".
|
org.apache.hadoop.fs.Path |
getPath() |
private static String |
getPathOffsetEndStr(org.apache.hadoop.fs.Path path,
long offset,
long end) |
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 |
isDecodeMemStoreTS() |
boolean |
isFileInfoLoaded() |
boolean |
isPrimaryReplicaReader() |
long |
length() |
HFile.FileInfo |
loadFileInfo() |
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.
|
void |
returnBlock(HFileBlock block)
Return the given block back to the cache, if it was obtained from cache.
|
void |
setConf(org.apache.hadoop.conf.Configuration conf) |
boolean |
shouldIncludeMemStoreTS() |
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.
|
private static final org.slf4j.Logger LOG
private HFileBlockIndex.CellBasedKeyBlockIndexReader dataBlockIndexReader
private HFileBlockIndex.ByteArrayKeyBlockIndexReader metaBlockIndexReader
private final FixedFileTrailer trailer
private final Compression.Algorithm compressAlgo
private final boolean primaryReplicaReader
private HFileDataBlockEncoder dataBlockEncoder
private Cell lastKeyCell
private int avgKeyLen
private int avgValueLen
private CellComparator comparator
private final long fileSize
private final CacheConfig cacheConf
private final org.apache.hadoop.fs.Path path
private HFile.FileInfo fileInfo
private org.apache.hadoop.conf.Configuration conf
private HFileContext hfileContext
private HFileBlock.FSReader fsBlockReader
private IdLock offsetLock
private List<HFileBlock> loadOnOpenBlocks
static final int MIN_MINOR_VERSION
static final int MAX_MINOR_VERSION
private static final int MIN_V2_MINOR_VERSION_WITH_PB
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
private boolean includesMemstoreTS
protected boolean decodeMemstoreTS
@Deprecated public HFileReaderImpl(org.apache.hadoop.fs.Path path, FixedFileTrailer trailer, FSDataInputStreamWrapper fsdis, long fileSize, CacheConfig cacheConf, HFileSystem hfs, org.apache.hadoop.conf.Configuration conf) throws IOException
IOExceptionpublic HFileReaderImpl(org.apache.hadoop.fs.Path path, FixedFileTrailer trailer, FSDataInputStreamWrapper fsdis, long fileSize, CacheConfig cacheConf, HFileSystem hfs, boolean primaryReplicaReader, org.apache.hadoop.conf.Configuration conf) throws IOException
loadFileInfo().path - Path to HFile.trailer - File trailer.fsdis - input stream.fileSize - Length of the stream.cacheConf - Cache configuration.hfs - The file system.conf - ConfigurationIOExceptionprivate static String getPathOffsetEndStr(org.apache.hadoop.fs.Path path, long offset, long end)
private void checkFileVersion()
private Optional<String> toStringFirstKey()
private Optional<String> toStringLastKey()
public long length()
length in interface HFile.Readerpublic void returnBlock(HFileBlock block)
HFile.CachingBlockReaderreturnBlock in interface HFile.CachingBlockReaderblock - Block to be returned.public Optional<Cell> getFirstKey()
getFirstKey in interface HFile.Readerpublic Optional<byte[]> getFirstRowKey()
HFile version 1: move this to StoreFile after Ryan's
patch goes in to eliminate KeyValue here.getFirstRowKey in interface HFile.Readerpublic Optional<byte[]> getLastRowKey()
HFile version 1: move this to StoreFile after
Ryan's patch goes in to eliminate KeyValue here.getLastRowKey in interface HFile.Readerpublic long getEntries()
getEntries in interface HFile.Readerpublic CellComparator getComparator()
getComparator in interface HFile.Readerpublic Compression.Algorithm getCompressionAlgorithm()
getCompressionAlgorithm in interface HFile.Readerpublic long indexSize()
indexSize in interface HFile.Readerpublic String getName()
HFile.ReadergetName in interface HFile.Readerpublic HFileBlockIndex.BlockIndexReader getDataBlockIndexReader()
getDataBlockIndexReader in interface HFile.Readerpublic FixedFileTrailer getTrailer()
getTrailer in interface HFile.Readerpublic boolean isPrimaryReplicaReader()
isPrimaryReplicaReader in interface HFile.Readerpublic HFile.FileInfo loadFileInfo() throws IOException
loadFileInfo in interface HFile.ReaderIOExceptionpublic org.apache.hadoop.fs.Path getPath()
getPath in interface HFile.Readerpublic DataBlockEncoding getDataBlockEncoding()
getDataBlockEncoding in interface HFile.Readerpublic org.apache.hadoop.conf.Configuration getConf()
getConf in interface org.apache.hadoop.conf.Configurablepublic void setConf(org.apache.hadoop.conf.Configuration conf)
setConf in interface org.apache.hadoop.conf.Configurablepublic boolean isDecodeMemStoreTS()
isDecodeMemStoreTS in interface HFile.Readerpublic boolean shouldIncludeMemStoreTS()
shouldIncludeMemStoreTS in interface HFile.Readerprivate 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.IOExceptionpublic HFileBlock getMetaBlock(String metaBlockName, boolean cacheBlock) throws IOException
getMetaBlock in interface HFile.ReadermetaBlockName - cacheBlock - Add block to cache, if foundIOExceptionpublic HFileBlock readBlock(long dataBlockOffset, long onDiskBlockSize, boolean cacheBlock, boolean pread, boolean isCompaction, boolean updateCacheMetrics, BlockType expectedBlockType, DataBlockEncoding expectedDataBlockEncoding) throws IOException
HFile.CachingBlockReaderreadBlock in interface HFile.CachingBlockReaderdataBlockOffset - 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.IOExceptionpublic boolean hasMVCCInfo()
hasMVCCInfo in interface HFile.Readerprivate 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
checkIOExceptionpublic Optional<Cell> getLastKey()
getLastKey in interface HFile.Readerpublic Optional<Cell> midKey() throws IOException
midKey in interface HFile.ReaderIOExceptionpublic void close() throws IOException
close in interface Closeableclose in interface AutoCloseableIOExceptionpublic void close(boolean evictOnClose) throws IOException
HFile.Readerclose in interface HFile.ReaderIOExceptionpublic DataBlockEncoding getEffectiveEncodingInCache(boolean isCompaction)
getEffectiveEncodingInCache in interface HFile.Readerpublic HFileBlock.FSReader getUncachedBlockReader()
getUncachedBlockReader in interface HFile.Readerpublic DataInput getGeneralBloomFilterMetadata() throws IOException
getGeneralBloomFilterMetadata in interface HFile.ReaderIOExceptionpublic DataInput getDeleteBloomFilterMetadata() throws IOException
HFile.ReaderHFile version.
Knows nothing about how that metadata is structured.getDeleteBloomFilterMetadata in interface HFile.ReaderIOExceptionprivate DataInput getBloomFilterMetadata(BlockType blockType) throws IOException
IOExceptionpublic boolean isFileInfoLoaded()
public HFileContext getFileContext()
HFile.ReadergetFileContext in interface HFile.Readerpublic boolean prefetchComplete()
prefetchComplete in interface HFile.Readerprotected HFileContext createHFileContext(FSDataInputStreamWrapper fsdis, long fileSize, HFileSystem hfs, org.apache.hadoop.fs.Path path, FixedFileTrailer trailer) throws IOException
IOExceptionpublic 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.ReadercacheBlocks - 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.ReadercacheBlocks - 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.ReaderunbufferStream in interface HFile.ReaderCopyright © 2007–2019 The Apache Software Foundation. All rights reserved.