HFile and HFile
BlockCache.See: Description
| Interface | Description |
|---|---|
| BlockCache |
Block cache interface.
|
| Cacheable |
Cacheable is an interface that allows for an object to be cached.
|
| CacheableDeserializer<T extends Cacheable> |
Interface for a deserializer.
|
| CachedBlock | |
| FirstLevelBlockCache |
In-memory BlockCache that may be backed by secondary layer(s).
|
| HFile.CachingBlockReader |
An abstraction used by the block index.
|
| HFile.Reader |
An interface used by clients to open and iterate an
HFile. |
| HFile.Writer |
API required to write an
HFile |
| HFileBlock.BlockIterator |
Iterator for reading
HFileBlocks in load-on-open-section, such as root data index
block, meta index block, file info block etc. |
| HFileBlock.BlockWritable |
Something that can be written into a block.
|
| HFileBlock.FSReader |
An HFile block reader with iteration ability.
|
| HFileDataBlockEncoder |
Controls what kind of data block encoding is used.
|
| HFileScanner |
A scanner allows you to position yourself within a HFile and scan through it.
|
| InlineBlockWriter |
A way to write "inline" blocks into an
HFile. |
| ResizableBlockCache |
BlockCache which is resizable.
|
| Class | Description |
|---|---|
| AgeSnapshot |
Snapshot of block cache age in cache.
|
| BlockCacheFactory | |
| BlockCacheKey |
Cache Key for use with implementations of
BlockCache |
| BlockCachesIterator |
Iterator over an array of BlockCache CachedBlocks.
|
| BlockCacheUtil |
Utilty for aggregating counts in CachedBlocks and toString/toJSON CachedBlocks and BlockCaches.
|
| BlockCacheUtil.CachedBlockCountsPerFile |
Little data structure to hold counts for a file.
|
| BlockCacheUtil.CachedBlocksByFile |
Use one of these to keep a running account of cached blocks by file.
|
| BlockWithScanInfo |
BlockWithScanInfo is wrapper class for HFileBlock with other attributes.
|
| BloomFilterMetrics | |
| CacheableDeserializerIdManager |
This class is used to manage the identifiers for
CacheableDeserializer. |
| CacheConfig |
Stores all of the cache objects and configuration for a single HFile.
|
| CacheStats |
Class that implements cache metrics.
|
| ChecksumUtil |
Utility methods to compute and validate checksums.
|
| CombinedBlockCache |
CombinedBlockCache is an abstraction layer that combines
FirstLevelBlockCache and
BucketCache. |
| CombinedBlockCache.CombinedCacheStats | |
| CompoundBloomFilter |
A Bloom filter implementation built on top of
BloomFilterChunk, encapsulating a set of fixed-size Bloom
filters written out at the time of HFile generation into
the data block stream, and loaded on demand at query time. |
| CompoundBloomFilterBase | |
| CompoundBloomFilterWriter |
Adds methods required for writing a compound Bloom filter to the data section of an
HFile to the CompoundBloomFilter class. |
| CompoundBloomFilterWriter.ReadyChunk |
A Bloom filter chunk enqueued for writing
|
| ExclusiveMemHFileBlock |
The
ByteBuffAllocator won't allocate pooled heap ByteBuff now; at the same time,
if allocate an off-heap ByteBuff from allocator, then it must be a pooled one. |
| FixedFileTrailer |
The
HFile has a fixed trailer which contains offsets to other variable parts of the file. |
| HFile |
File format for hbase.
|
| HFile.WriterFactory |
This variety of ways to construct writers is used throughout the code, and we want to be able
to swap writer implementations.
|
| HFileBlock |
Cacheable Blocks of an
HFile version 2 file. |
| HFileBlock.BlockDeserializer | |
| HFileBlock.FSReaderImpl |
Reads version 2 HFile blocks from the filesystem.
|
| HFileBlock.Header | |
| HFileBlock.PrefetchedHeader |
Data-structure to use caching the header of the NEXT block.
|
| HFileBlock.Writer |
Unified version 2
HFile block writer. |
| HFileBlockBuilder | |
| HFileBlockIndex |
Provides functionality to write (
HFileBlockIndex.BlockIndexWriter) and read BlockIndexReader single-level
and multi-level block indexes. |
| HFileBlockIndex.BlockIndexChunk |
A single chunk of the block index in the process of writing.
|
| HFileBlockIndex.BlockIndexReader |
The reader will always hold the root level index in the memory.
|
| HFileBlockIndex.BlockIndexWriter |
Writes the block index into the output stream.
|
| HFileBlockIndex.ByteArrayKeyBlockIndexReader |
An implementation of the BlockIndexReader that deals with block keys which are plain byte[]
like MetaBlock or the Bloom Block for ROW bloom.
|
| HFileBlockIndex.CellBasedKeyBlockIndexReader |
An implementation of the BlockIndexReader that deals with block keys which are the key part of
a cell like the Data block index or the ROW_COL bloom blocks This needs a comparator to work
with the Cells
|
| HFileContext |
Read-only HFile Context Information.
|
| HFileContextBuilder |
A builder that helps in building up the HFileContext
|
| HFileDataBlockEncoderImpl |
Do different kinds of data block encoding according to column family options.
|
| HFileInfo |
Metadata Map of attributes for HFile written out as HFile Trailer.
|
| HFilePreadReader |
Implementation of
HFile.Reader to deal with pread. |
| HFilePrettyPrinter |
Implements pretty-printing functionality for
HFiles. |
| HFilePrettyPrinter.KeyValueStatsCollector | |
| HFilePrettyPrinter.SimpleReporter |
Almost identical to ConsoleReporter, but extending ScheduledReporter, as extending
ConsoleReporter in this version of dropwizard is now too much trouble.
|
| HFilePrettyPrinter.SimpleReporter.Builder |
A builder for
HFilePrettyPrinter.SimpleReporter instances. |
| HFileReaderImpl |
Implementation that can handle all hfile versions of
HFile.Reader. |
| HFileReaderImpl.EncodedScanner |
Scanner that operates on encoded data blocks.
|
| HFileReaderImpl.HFileScannerImpl | |
| HFileStreamReader |
Implementation of
HFile.Reader to deal with stream read do not perform any prefetch
operations (HFilePreadReader will do this). |
| HFileUtil | |
| HFileWriterImpl |
Common functionality needed by all versions of
HFile writers. |
| InclusiveCombinedBlockCache | |
| IndexOnlyLruBlockCache |
An on heap block cache implementation extended LruBlockCache and only cache index block.
|
| LruAdaptiveBlockCache |
This realisation improve performance of classical LRU cache up to 3 times via reduce GC
job.
|
| LruAdaptiveBlockCache.EvictionThread | |
| LruAdaptiveBlockCache.StatisticsThread | |
| LruBlockCache |
A block cache implementation that is memory-aware using
HeapSize, memory-bound using an
LRU eviction algorithm, and concurrent: backed by a ConcurrentHashMap and with a
non-blocking eviction thread giving constant-time LruBlockCache.cacheBlock(org.apache.hadoop.hbase.io.hfile.BlockCacheKey, org.apache.hadoop.hbase.io.hfile.Cacheable, boolean) and LruBlockCache.getBlock(org.apache.hadoop.hbase.io.hfile.BlockCacheKey, boolean, boolean, boolean)
operations. |
| LruBlockCache.EvictionThread | |
| LruBlockCache.StatisticsThread | |
| LruCachedBlock |
Represents an entry in the
LruBlockCache. |
| LruCachedBlockQueue |
A memory-bound queue that will grow until an element brings total size >= maxSize.
|
| MemcachedBlockCache |
Class to store blocks into memcached.
|
| MemcachedBlockCache.HFileBlockTranscoder |
Class to encode and decode an HFileBlock to and from memecached's resulting byte arrays.
|
| NoOpDataBlockEncoder |
Does not perform any kind of encoding/decoding.
|
| NoOpDataBlockEncoder.NoneEncodingState | |
| PrefetchExecutor | |
| ReaderContext |
Carries the information on some of the meta data about the HFile Reader
|
| ReaderContextBuilder |
A builder that helps in building up the ReaderContext
|
| SharedMemHFileBlock |
The
ByteBuffAllocator won't allocate pooled heap ByteBuff now; at the same time,
if allocate an off-heap ByteBuff from allocator, then it must be a pooled one. |
| TinyLfuBlockCache |
A block cache that is memory-aware using
HeapSize, memory bounded using the W-TinyLFU
eviction algorithm, and concurrent. |
| TinyLfuBlockCache.CachedBlockView |
| Enum | Description |
|---|---|
| BlockCacheFactory.ExternalBlockCaches |
Enum of all built in external block caches.
|
| BlockPriority | |
| BlockType |
Various types of HFile blocks.
|
| BlockType.BlockCategory | |
| HFileBlock.Writer.State | |
| ReaderContext.ReaderType |
| Exception | Description |
|---|---|
| CorruptHFileException |
This exception is thrown when attempts to read an HFile fail due to corruption or truncation
issues.
|
| HFileReaderImpl.BlockIndexNotLoadedException | |
| HFileReaderImpl.NotSeekedException |
An exception thrown when an operation requiring a scanner to be seeked is invoked on a scanner
that is not seeked.
|
| InvalidHFileException |
Thrown when an invalid HFile format is detected
|
HFile and HFile
BlockCache. Caches are configured (and instantiated)
by CacheConfig. See head of the
CacheConfig class for constants that define
cache options and configuration keys to use setting cache options. Cache implementations
include the default, native on-heap LruBlockCache and a
BucketCache that has a bunch of deploy formats
including acting as a L2 for LruBlockCache -- when a block is evicted from LruBlockCache, it
goes to the BucketCache and when we search a block, we look in both places -- or, the
most common deploy type,
using CombinedBlockCache, BucketCache is used as
a host for data blocks with meta blocks in an instance of LruBlockCache. BucketCache
can also be onheap, offheap, and file-backed.
BucketCacheCopyright © 2007–2020 The Apache Software Foundation. All rights reserved.