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 | |
| 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 | An interface allowing to iterate  HFileBlocks. | 
| HFileBlock.BlockWritable | Something that can be written into a block. | 
| HFileBlock.FSReader | A full-fledged 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 | 
|---|---|
| AbstractHFileReader | Common functionality needed by all versions of  HFilereaders. | 
| AbstractHFileReader.Scanner | |
| AbstractHFileWriter | Common functionality needed by all versions of  HFilewriters. | 
| AgeSnapshot | Snapshot of block cache age in cache. | 
| 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
  LruBlockCacheandBucketCache. | 
| CombinedBlockCache.CombinedCacheStats | |
| FixedFileTrailer | The  HFilehas a fixed trailer which contains offsets to other
 variable parts of the file. | 
| HFile | File format for hbase. | 
| HFile.FileInfo | Metadata for this file. | 
| 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 | Reading  HFileversion 1 and 2 blocks, and writing version 2 blocks. | 
| HFileBlock.AbstractFSReader | A common implementation of some methods of  HFileBlock.FSReaderand some
 tools for implementing HFile format version-specific block readers. | 
| HFileBlock.FSReaderImpl | Reads version 2 blocks from the filesystem. | 
| HFileBlock.Header | |
| HFileBlock.PrefetchedHeader | We always prefetch the header of the next block, so that we know its
 on-disk size in advance and can read it in one operation. | 
| HFileBlock.Writer | Unified version 2  HFileblock writer. | 
| HFileBlock.Writer.BufferGrabbingByteArrayOutputStream | |
| 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. | 
| HFileContext | This carries the information on some of the meta data about the HFile. | 
| HFileContextBuilder | A builder that helps in building up the HFileContext | 
| HFileDataBlockEncoderImpl | Do different kinds of data block encoding according to column family
 options. | 
| HFilePrettyPrinter | Implements pretty-printing functionality for  HFiles. | 
| HFilePrettyPrinter.KeyValueStatsCollector | |
| HFilePrettyPrinter.SimpleReporter | |
| HFileReaderV2 | HFilereader for version 2. | 
| HFileReaderV2.AbstractScannerV2 | |
| HFileReaderV2.EncodedScannerV2 | ScannerV2 that operates on encoded data blocks. | 
| HFileReaderV2.ScannerV2 | Implementation of  HFileScannerinterface. | 
| HFileReaderV3 | HFilereader for version 3. | 
| HFileReaderV3.EncodedScannerV3 | ScannerV3 that operates on encoded data blocks. | 
| HFileReaderV3.ScannerV3 | Implementation of  HFileScannerinterface. | 
| HFileUtil | |
| HFileWriterV2 | Writes HFile format version 2. | 
| HFileWriterV2.WriterFactoryV2 | |
| HFileWriterV3 | HFilewriter for version 3. | 
| HFileWriterV3.WriterFactoryV3 | |
| 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, 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. | 
| PrefetchExecutor | 
| Enum | Description | 
|---|---|
| BlockPriority | |
| BlockType | Various types of HFile blocks. | 
| BlockType.BlockCategory | |
| CacheConfig.ExternalBlockCaches | Enum of all built in external block caches. | 
| HFileBlock.Writer.State | 
| Exception | Description | 
|---|---|
| AbstractHFileReader.BlockIndexNotLoadedException | |
| AbstractHFileReader.NotSeekedException | An exception thrown when an operation requiring a scanner to be seeked
 is invoked on a scanner that is not seeked. | 
| CorruptHFileException | This exception is thrown when attempts to read an HFile fail due to corruption or truncation
 issues. | 
| 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–2019 The Apache Software Foundation. All rights reserved.