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. | 
| 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
  FirstLevelBlockCacheandBucketCache. | 
| 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 ofHFilegeneration 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  HFileto theCompoundBloomFilterclass. | 
| CompoundBloomFilterWriter.ReadyChunk | A Bloom filter chunk enqueued for writing | 
| ExclusiveMemHFileBlock | The  ByteBuffAllocatorwon't allocate pooled heapByteBuffnow; at the same time,
 if allocate an off-heapByteBufffrom allocator, then it must be a pooled one. | 
| FixedFileTrailer | The  HFilehas 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  HFileversion 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  HFileblock 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.Readerto 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.SimpleReporterinstances. | 
| 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.Readerto deal with stream read
 do not perform any prefetch operations (HFilePreadReader will do this). | 
| HFileUtil | |
| HFileWriterImpl | Common functionality needed by all versions of  HFilewriters. | 
| InclusiveCombinedBlockCache | |
| LruBlockCache | A block cache implementation that is memory-aware using  HeapSize,
 memory-bound using an LRU eviction algorithm, and concurrent: backed by aConcurrentHashMapand with a non-blocking eviction thread giving
 constant-timeLruBlockCache.cacheBlock(org.apache.hadoop.hbase.io.hfile.BlockCacheKey, org.apache.hadoop.hbase.io.hfile.Cacheable, boolean)andLruBlockCache.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  ByteBuffAllocatorwon't allocate pooled heapByteBuffnow; at the same time,
 if allocate an off-heapByteBufffrom 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–2021 The Apache Software Foundation. All rights reserved.