Package org.apache.hadoop.hbase.io.hfile
Class LruAdaptiveBlockCache
java.lang.Object
org.apache.hadoop.hbase.io.hfile.LruAdaptiveBlockCache
- All Implemented Interfaces:
- Iterable<CachedBlock>,- HeapSize,- BlockCache,- FirstLevelBlockCache,- ResizableBlockCache
This realisation improve performance of classical LRU cache up to 3 times via reduce GC
 job.
 
 The classical 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
 cacheBlock(org.apache.hadoop.hbase.io.hfile.BlockCacheKey, org.apache.hadoop.hbase.io.hfile.Cacheable, boolean) and getBlock(org.apache.hadoop.hbase.io.hfile.BlockCacheKey, boolean, boolean, boolean) operations.
 
 Contains three levels of block priority to allow for scan-resistance and in-memory families
 ColumnFamilyDescriptorBuilder.setInMemory(boolean) (An
 in-memory column family is a column family that should be served from memory if possible):
 single-access, multiple-accesses, and in-memory priority. A block is added with an in-memory
 priority flag if ColumnFamilyDescriptor.isInMemory(),
 otherwise a block becomes a single access priority the first time it is read into this block
 cache. If a block is accessed again while in cache, it is marked as a multiple access priority
 block. This delineation of blocks is used to prevent scans from thrashing the cache adding a
 least-frequently-used element to the eviction algorithm.
 
 Each priority is given its own chunk of the total cache to ensure fairness during eviction. Each
 priority will retain close to its maximum size, however, if any priority is not using its entire
 chunk the others are able to grow beyond their chunk size.
 
 Instantiated at a minimum with the total size and average block size. All sizes are in bytes. The
 block size is not especially important as this cache is fully dynamic in its sizing of blocks. It
 is only used for pre-allocating data structures and in initial heap estimation of the map.
 
 The detailed constructor defines the sizes for the three priorities (they should total to the
 maximum size defined). It also sets the levels that trigger and control the eviction
 thread.
 
 The acceptable size is the cache size level which triggers the eviction process to
 start. It evicts enough blocks to get the size below the minimum size specified.
 
 Eviction happens in a separate thread and involves a single full-scan of the map. It determines
 how many bytes must be freed to reach the minimum size, and then while scanning determines the
 fewest least-recently-used blocks necessary from each of the three priorities (would be 3 times
 bytes to free). It then uses the priority chunk sizes to evict fairly according to the relative
 sizes and usage.
 
 Adaptive LRU cache lets speed up performance while we are reading much more data than can fit
 into BlockCache and it is the cause of a high rate of evictions. This in turn leads to heavy
 Garbage Collector works. So a lot of blocks put into BlockCache but never read, but spending a
 lot of CPU resources for cleaning. We could avoid this situation via parameters:
 
 hbase.lru.cache.heavy.eviction.count.limit - set how many times we have to run the
 eviction process that starts to avoid putting data to BlockCache. By default it is 0 and it meats
 the feature will start at the beginning. But if we have some times short reading the same data
 and some times long-term reading - we can divide it by this parameter. For example we know that
 our short reading used to be about 1 minutes, then we have to set the parameter about 10 and it
 will enable the feature only for long time massive reading (after ~100 seconds). So when we use
 short-reading and want all of them in the cache we will have it (except for eviction of course).
 When we use long-term heavy reading the feature will be enabled after some time and bring better
 performance.
 
 hbase.lru.cache.heavy.eviction.mb.size.limit - set how many bytes in 10 seconds desirable
 putting into BlockCache (and evicted from it). The feature will try to reach this value and
 maintain it. Don't try to set it too small because it leads to premature exit from this mode. For
 powerful CPUs (about 20-40 physical cores) it could be about 400-500 MB. Average system (~10
 cores) 200-300 MB. Some weak systems (2-5 cores) may be good with 50-100 MB. How it works: we set
 the limit and after each ~10 second calculate how many bytes were freed. Overhead = Freed Bytes
 Sum (MB) * 100 / Limit (MB) - 100; For example we set the limit = 500 and were evicted 2000 MB.
 Overhead is: 2000 * 100 / 500 - 100 = 300% The feature is going to reduce a percent caching data
 blocks and fit evicted bytes closer to 100% (500 MB). Some kind of an auto-scaling. If freed
 bytes less then the limit we have got negative overhead. For example if were freed 200 MB: 200 *
 100 / 500 - 100 = -60% The feature will increase the percent of caching blocks. That leads to fit
 evicted bytes closer to 100% (500 MB). The current situation we can find out in the log of
 RegionServer: BlockCache evicted (MB): 0, overhead (%): -100, heavy eviction counter: 0, current
 caching DataBlock (%): 100 - means no eviction, 100% blocks is caching BlockCache evicted (MB):
 2000, overhead (%): 300, heavy eviction counter: 1, current caching DataBlock (%): 97 - means
 eviction begin, reduce of caching blocks by 3%. It help to tune your system and find out what
 value is better set. Don't try to reach 0% overhead, it is impossible. Quite good 50-100%
 overhead, it prevents premature exit from this mode.
 
 hbase.lru.cache.heavy.eviction.overhead.coefficient - set how fast we want to get the
 result. If we know that our reading is heavy for a long time, we don't want to wait and can
 increase the coefficient and get good performance sooner. But if we aren't sure we can do it
 slowly and it could prevent premature exit from this mode. So, when the coefficient is higher we
 can get better performance when heavy reading is stable. But when reading is changing we can
 adjust to it and set the coefficient to lower value. For example, we set the coefficient = 0.01.
 It means the overhead (see above) will be multiplied by 0.01 and the result is the value of
 reducing percent caching blocks. For example, if the overhead = 300% and the coefficient = 0.01,
 then percent of caching blocks will reduce by 3%. Similar logic when overhead has got negative
 value (overshooting). Maybe it is just short-term fluctuation and we will try to stay in this
 mode. It helps avoid premature exit during short-term fluctuation. Backpressure has simple logic:
 more overshooting - more caching blocks.
 
 Find more information about improvement: https://issues.apache.org/jira/browse/HBASE-23887- 
Nested Class SummaryNested ClassesModifier and TypeClassDescriptionprivate classUsed to group blocks into priority buckets.(package private) static class(package private) static class
- 
Field SummaryFieldsModifier and TypeFieldDescriptionprivate final floatAcceptable size of cache (no evictions if size < acceptable)private final longApproximate block sizestatic final longprivate intPercent of cached data blocksprivate final AtomicLongCache access count (sequential ID)private final LongAdderCurrent number of cached data block elementsprivate final LongAdderCurrent size of data blocks(package private) static final float(package private) static final intprivate static final floatprivate static final boolean(package private) static final floatprivate static final intprivate static final longprivate static final floatprivate static final longprivate static final floatprivate static final floatprivate static final floatprivate static final floatprivate final AtomicLongCurrent number of cached elementsprivate booleanVolatile boolean to track if we are in an eviction process or notprivate final ReentrantLockEviction lock (locked when eviction in process)private final LruAdaptiveBlockCache.EvictionThreadEviction threadprivate booleanWhether in-memory hfile's data block has higher priority when evictingprivate final floathard capacity limitprivate final intLimit of count eviction process when start to avoid to cache blocksprivate final longLimit of volume eviction process when start to avoid to cache blocksprivate final floatAdjust auto-scaling via overhead of evition rateprivate static final org.slf4j.Loggerprivate static final StringAcceptable size of cache (no evictions if size < acceptable)private static final Stringprivate static final Stringprivate static final String(package private) static final StringHard capacity limit of cache, will reject any put if size > this * acceptableprivate static final StringConfiguration key to force data-block always (except in-memory are too much) cached in memory for in-memory hfile, unlike inMemory, which is a column-family configuration, inMemoryForceMode is a cluster-wide configurationprivate static final Stringprivate static final Stringprivate static final StringPercentage of total size that eviction will evict until; e.g.private static final Stringprivate static final Stringprivate final ConcurrentHashMap<BlockCacheKey,LruCachedBlock> Defined the cache map asConcurrentHashMaphere, because ingetBlock(org.apache.hadoop.hbase.io.hfile.BlockCacheKey, boolean, boolean, boolean), we need to guarantee the atomicity of map#computeIfPresent (key, func).private final longprivate longMaximum allowable size of cache (block put if size > max, evict)private final floatIn-memory bucket sizeprivate final floatMinimum threshold of cache (when evicting, evict until size < min)private final floatMultiple access bucket sizeprivate final longOverhead of the structure itselfprivate final ScheduledExecutorServiceStatistics thread schedule pool (for heavy debugging, could remove)private final floatSingle access bucket sizeprivate final AtomicLongCurrent size of cacheprivate static final intprivate final CacheStatsCache statisticsprivate BlockCacheWhere to send victims (blocks evicted/missing from the cache).
- 
Constructor SummaryConstructorsConstructorDescriptionLruAdaptiveBlockCache(long maxSize, long blockSize) Default constructor.LruAdaptiveBlockCache(long maxSize, long blockSize, boolean evictionThread) Constructor used for testing.LruAdaptiveBlockCache(long maxSize, long blockSize, boolean evictionThread, int mapInitialSize, float mapLoadFactor, int mapConcurrencyLevel, float minFactor, float acceptableFactor, float singleFactor, float multiFactor, float memoryFactor, float hardLimitFactor, boolean forceInMemory, long maxBlockSize, int heavyEvictionCountLimit, long heavyEvictionMbSizeLimit, float heavyEvictionOverheadCoefficient) Configurable constructor.LruAdaptiveBlockCache(long maxSize, long blockSize, boolean evictionThread, org.apache.hadoop.conf.Configuration conf) LruAdaptiveBlockCache(long maxSize, long blockSize, org.apache.hadoop.conf.Configuration conf) 
- 
Method SummaryModifier and TypeMethodDescription(package private) longprivate CacheableThe block cached in LruAdaptiveBlockCache will always be an heap block: on the one side, the heap access will be more faster then off-heap, the small index block or meta block cached in CombinedBlockCache will benefit a lot.private static voidassertCounterSanity(long mapSize, long counterVal) Sanity-checking for parity between actual block cache content and metrics.voidcacheBlock(BlockCacheKey cacheKey, Cacheable buf) Cache the block with the specified name and buffer.voidcacheBlock(BlockCacheKey cacheKey, Cacheable buf, boolean inMemory) Cache the block with the specified name and buffer.private static longcalculateOverhead(long maxSize, long blockSize, int concurrency) voidClears the cache.booleancontainsBlock(BlockCacheKey cacheKey) Whether the cache contains block with specified cacheKey(package private) longevict()Eviction method.booleanevictBlock(BlockCacheKey cacheKey) Evict block from cache.protected longevictBlock(LruCachedBlock block, boolean evictedByEvictionProcess) Evict the block, and it will be cached by the victim handler if exists && block may be read again laterintevictBlocksByHfileName(String hfileName) Evicts all blocks for a specific HFile.getBlock(BlockCacheKey cacheKey, boolean caching, boolean repeat, boolean updateCacheMetrics) Get the buffer of the block with the specified name.Returns The list of sub blockcaches that make up this one; returns null if no sub caches.longReturns the number of blocks currently cached in the block cache.intlongReturns the occupied size of data blocks, in bytes.longReturns the occupied size of the block cache, in bytes.longReturns the number of data blocks currently cached in the block cache.(package private) LruAdaptiveBlockCache.EvictionThreadlongReturns the free size of the block cache, in bytes.(package private) Map<BlockCacheKey,LruCachedBlock> longGet the maximum size of this cache.(package private) longgetStats()Get counter statistics for this cache.longheapSize()Return the approximate 'exclusive deep size' of implementing object.(package private) booleaniterator()Returns Iterator over the blocks in the cache.voidlogStats()private longprivate longminSize()private longprivate voidMulti-threaded call to run the eviction process.voidsetMaxSize(long maxSize) Sets the max heap size that can be used by the BlockCache.voidsetVictimCache(BlockCache victimCache) Specifies the secondary cache.voidshutdown()Shutdown the cache.private longlongsize()Returns the total size of the block cache, in bytes.toString()private longupdateSizeMetrics(LruCachedBlock cb, boolean evict) Helper function that updates the local size counter and also updates any per-cf or per-blocktype metrics it can discern from givenLruCachedBlockMethods inherited from class java.lang.Objectclone, equals, finalize, getClass, hashCode, notify, notifyAll, wait, wait, waitMethods inherited from interface org.apache.hadoop.hbase.io.hfile.BlockCacheblockFitsIntoTheCache, cacheBlock, getBlockSize, getFullyCachedFiles, isAlreadyCached, isMetaBlock, notifyFileBlockEvicted, notifyFileCachingCompleted, shouldCacheFileMethods inherited from interface java.lang.IterableforEach, spliterator
- 
Field Details- 
LOG
- 
LRU_MIN_FACTOR_CONFIG_NAMEPercentage of total size that eviction will evict until; e.g. if set to .8, then we will keep evicting during an eviction run till the cache size is down to 80% of the total.- See Also:
 
- 
LRU_ACCEPTABLE_FACTOR_CONFIG_NAMEAcceptable size of cache (no evictions if size < acceptable)- See Also:
 
- 
LRU_HARD_CAPACITY_LIMIT_FACTOR_CONFIG_NAMEHard capacity limit of cache, will reject any put if size > this * acceptable- See Also:
 
- 
LRU_SINGLE_PERCENTAGE_CONFIG_NAME- See Also:
 
- 
LRU_MULTI_PERCENTAGE_CONFIG_NAME- See Also:
 
- 
LRU_MEMORY_PERCENTAGE_CONFIG_NAME- See Also:
 
- 
LRU_IN_MEMORY_FORCE_MODE_CONFIG_NAMEConfiguration key to force data-block always (except in-memory are too much) cached in memory for in-memory hfile, unlike inMemory, which is a column-family configuration, inMemoryForceMode is a cluster-wide configuration- See Also:
 
- 
DEFAULT_LOAD_FACTOR- See Also:
 
- 
DEFAULT_CONCURRENCY_LEVEL- See Also:
 
- 
DEFAULT_MIN_FACTOR- See Also:
 
- 
DEFAULT_ACCEPTABLE_FACTOR- See Also:
 
- 
DEFAULT_SINGLE_FACTOR- See Also:
 
- 
DEFAULT_MULTI_FACTOR- See Also:
 
- 
DEFAULT_MEMORY_FACTOR- See Also:
 
- 
DEFAULT_HARD_CAPACITY_LIMIT_FACTOR- See Also:
 
- 
DEFAULT_IN_MEMORY_FORCE_MODE- See Also:
 
- 
STAT_THREAD_PERIOD- See Also:
 
- 
LRU_MAX_BLOCK_SIZE- See Also:
 
- 
DEFAULT_MAX_BLOCK_SIZE- See Also:
 
- 
LRU_CACHE_HEAVY_EVICTION_COUNT_LIMIT- See Also:
 
- 
DEFAULT_LRU_CACHE_HEAVY_EVICTION_COUNT_LIMIT- See Also:
 
- 
LRU_CACHE_HEAVY_EVICTION_MB_SIZE_LIMIT- See Also:
 
- 
DEFAULT_LRU_CACHE_HEAVY_EVICTION_MB_SIZE_LIMIT- See Also:
 
- 
LRU_CACHE_HEAVY_EVICTION_OVERHEAD_COEFFICIENT- See Also:
 
- 
DEFAULT_LRU_CACHE_HEAVY_EVICTION_OVERHEAD_COEFFICIENT- See Also:
 
- 
mapDefined the cache map asConcurrentHashMaphere, because ingetBlock(org.apache.hadoop.hbase.io.hfile.BlockCacheKey, boolean, boolean, boolean), we need to guarantee the atomicity of map#computeIfPresent (key, func). Besides, the func method must execute exactly once only when the key is present and under the lock context, otherwise the reference count will be messed up. Notice that theConcurrentSkipListMapcan not guarantee that.
- 
evictionLockEviction lock (locked when eviction in process)
- 
maxBlockSize
- 
evictionInProgressVolatile boolean to track if we are in an eviction process or not
- 
evictionThreadEviction thread
- 
scheduleThreadPoolStatistics thread schedule pool (for heavy debugging, could remove)
- 
sizeCurrent size of cache
- 
dataBlockSizeCurrent size of data blocks
- 
elementsCurrent number of cached elements
- 
dataBlockElementsCurrent number of cached data block elements
- 
countCache access count (sequential ID)
- 
hardCapacityLimitFactorhard capacity limit
- 
statsCache statistics
- 
maxSizeMaximum allowable size of cache (block put if size > max, evict)
- 
blockSizeApproximate block size
- 
acceptableFactorAcceptable size of cache (no evictions if size < acceptable)
- 
minFactorMinimum threshold of cache (when evicting, evict until size < min)
- 
singleFactorSingle access bucket size
- 
multiFactorMultiple access bucket size
- 
memoryFactorIn-memory bucket size
- 
overheadOverhead of the structure itself
- 
forceInMemoryWhether in-memory hfile's data block has higher priority when evicting
- 
victimHandlerWhere to send victims (blocks evicted/missing from the cache). This is used only when we use an external cache as L2. Note: See org.apache.hadoop.hbase.io.hfile.MemcachedBlockCache
- 
cacheDataBlockPercentPercent of cached data blocks
- 
heavyEvictionCountLimitLimit of count eviction process when start to avoid to cache blocks
- 
heavyEvictionMbSizeLimitLimit of volume eviction process when start to avoid to cache blocks
- 
heavyEvictionOverheadCoefficientAdjust auto-scaling via overhead of evition rate
- 
CACHE_FIXED_OVERHEAD
 
- 
- 
Constructor Details- 
LruAdaptiveBlockCacheDefault constructor. Specify maximum size and expected average block size (approximation is fine).All other factors will be calculated based on defaults specified in this class. - Parameters:
- maxSize- maximum size of cache, in bytes
- blockSize- approximate size of each block, in bytes
 
- 
LruAdaptiveBlockCacheConstructor used for testing. Allows disabling of the eviction thread.
- 
LruAdaptiveBlockCachepublic LruAdaptiveBlockCache(long maxSize, long blockSize, boolean evictionThread, org.apache.hadoop.conf.Configuration conf) 
- 
LruAdaptiveBlockCachepublic LruAdaptiveBlockCache(long maxSize, long blockSize, org.apache.hadoop.conf.Configuration conf) 
- 
LruAdaptiveBlockCachepublic LruAdaptiveBlockCache(long maxSize, long blockSize, boolean evictionThread, int mapInitialSize, float mapLoadFactor, int mapConcurrencyLevel, float minFactor, float acceptableFactor, float singleFactor, float multiFactor, float memoryFactor, float hardLimitFactor, boolean forceInMemory, long maxBlockSize, int heavyEvictionCountLimit, long heavyEvictionMbSizeLimit, float heavyEvictionOverheadCoefficient) Configurable constructor. Use this constructor if not using defaults.- Parameters:
- maxSize- maximum size of this cache, in bytes
- blockSize- expected average size of blocks, in bytes
- evictionThread- whether to run evictions in a bg thread or not
- mapInitialSize- initial size of backing ConcurrentHashMap
- mapLoadFactor- initial load factor of backing ConcurrentHashMap
- mapConcurrencyLevel- initial concurrency factor for backing CHM
- minFactor- percentage of total size that eviction will evict until
- acceptableFactor- percentage of total size that triggers eviction
- singleFactor- percentage of total size for single-access blocks
- multiFactor- percentage of total size for multiple-access blocks
- memoryFactor- percentage of total size for in-memory blocks
- hardLimitFactor- hard capacity limit
- forceInMemory- in-memory hfile's data block has higher priority when evicting
- maxBlockSize- maximum block size for caching
- heavyEvictionCountLimit- when starts AdaptiveLRU algoritm work
- heavyEvictionMbSizeLimit- how many bytes desirable putting into BlockCache
- heavyEvictionOverheadCoefficient- how aggressive AdaptiveLRU will reduce GC
 
 
- 
- 
Method Details- 
setVictimCacheDescription copied from interface:FirstLevelBlockCacheSpecifies the secondary cache. An entry that is evicted from this cache due to a size constraint will be inserted into the victim cache.- Specified by:
- setVictimCachein interface- FirstLevelBlockCache
- Parameters:
- victimCache- the second level cache
 
- 
setMaxSizeDescription copied from interface:ResizableBlockCacheSets the max heap size that can be used by the BlockCache.- Specified by:
- setMaxSizein interface- ResizableBlockCache
- Parameters:
- maxSize- The max heap size.
 
- 
getCacheDataBlockPercent
- 
asReferencedHeapBlockThe block cached in LruAdaptiveBlockCache will always be an heap block: on the one side, the heap access will be more faster then off-heap, the small index block or meta block cached in CombinedBlockCache will benefit a lot. on other side, the LruAdaptiveBlockCache size is always calculated based on the total heap size, if caching an off-heap block in LruAdaptiveBlockCache, the heap size will be messed up. Here we will clone the block into an heap block if it's an off-heap block, otherwise just use the original block. The key point is maintain the refCnt of the block (HBASE-22127):
 1. if cache the cloned heap block, its refCnt is an totally new one, it's easy to handle;
 2. if cache the original heap block, we're sure that it won't be tracked in ByteBuffAllocator's reservoir, if both RPC and LruAdaptiveBlockCache release the block, then it can be garbage collected by JVM, so need a retain here.- Parameters:
- buf- the original block
- Returns:
- an block with an heap memory backend.
 
- 
cacheBlockCache the block with the specified name and buffer.It is assumed this will NOT be called on an already cached block. In rare cases (HBASE-8547) this can happen, for which we compare the buffer contents. - Specified by:
- cacheBlockin interface- BlockCache
- Parameters:
- cacheKey- block's cache key
- buf- block buffer
- inMemory- if block is in-memory
 
- 
assertCounterSanitySanity-checking for parity between actual block cache content and metrics. Intended only for use with TRACE level logging and -ea JVM.
- 
cacheBlockCache the block with the specified name and buffer.TODO after HBASE-22005, we may cache an block which allocated from off-heap, but our LRU cache sizing is based on heap size, so we should handle this in HBASE-22127. It will introduce an switch whether make the LRU on-heap or not, if so we may need copy the memory to on-heap, otherwise the caching size is based on off-heap. - Specified by:
- cacheBlockin interface- BlockCache
- Parameters:
- cacheKey- block's cache key
- buf- block buffer
 
- 
updateSizeMetricsHelper function that updates the local size counter and also updates any per-cf or per-blocktype metrics it can discern from givenLruCachedBlock
- 
getBlockpublic Cacheable getBlock(BlockCacheKey cacheKey, boolean caching, boolean repeat, boolean updateCacheMetrics) Get the buffer of the block with the specified name.- Specified by:
- getBlockin interface- BlockCache
- Parameters:
- cacheKey- block's cache key
- caching- true if the caller caches blocks on cache misses
- repeat- Whether this is a repeat lookup for the same block (used to avoid double counting cache misses when doing double-check locking)
- updateCacheMetrics- Whether to update cache metrics or not
- Returns:
- buffer of specified cache key, or null if not in cache
 
- 
containsBlockWhether the cache contains block with specified cacheKey- Specified by:
- containsBlockin interface- FirstLevelBlockCache
- Parameters:
- cacheKey- cache key for the block
- Returns:
- true if contains the block
 
- 
evictBlockDescription copied from interface:BlockCacheEvict block from cache.- Specified by:
- evictBlockin interface- BlockCache
- Parameters:
- cacheKey- Block to evict
- Returns:
- true if block existed and was evicted, false if not
 
- 
evictBlocksByHfileNameEvicts all blocks for a specific HFile. This is an expensive operation implemented as a linear-time search through all blocks in the cache. Ideally this should be a search in a log-access-time map.This is used for evict-on-close to remove all blocks of a specific HFile. - Specified by:
- evictBlocksByHfileNamein interface- BlockCache
- Returns:
- the number of blocks evicted
 
- 
evictBlockEvict the block, and it will be cached by the victim handler if exists && block may be read again later- Parameters:
- evictedByEvictionProcess- true if the given block is evicted by EvictionThread
- Returns:
- the heap size of evicted block
 
- 
runEvictionMulti-threaded call to run the eviction process.
- 
isEvictionInProgressboolean isEvictionInProgress()
- 
getOverheadlong getOverhead()
- 
evictlong evict()Eviction method. Evict items in order of use, allowing delete items which haven't been used for the longest amount of time.- Returns:
- how many bytes were freed
 
- 
toString
- 
getMaxSizeGet the maximum size of this cache.- Specified by:
- getMaxSizein interface- BlockCache
- Returns:
- max size in bytes
 
- 
getCurrentSizeDescription copied from interface:BlockCacheReturns the occupied size of the block cache, in bytes.- Specified by:
- getCurrentSizein interface- BlockCache
- Returns:
- occupied space in cache, in bytes
 
- 
getCurrentDataSizeDescription copied from interface:BlockCacheReturns the occupied size of data blocks, in bytes.- Specified by:
- getCurrentDataSizein interface- BlockCache
- Returns:
- occupied space in cache, in bytes
 
- 
getFreeSizeDescription copied from interface:BlockCacheReturns the free size of the block cache, in bytes.- Specified by:
- getFreeSizein interface- BlockCache
- Returns:
- free space in cache, in bytes
 
- 
sizeDescription copied from interface:BlockCacheReturns the total size of the block cache, in bytes.- Specified by:
- sizein interface- BlockCache
- Returns:
- size of cache, in bytes
 
- 
getBlockCountDescription copied from interface:BlockCacheReturns the number of blocks currently cached in the block cache.- Specified by:
- getBlockCountin interface- BlockCache
- Returns:
- number of blocks in the cache
 
- 
getDataBlockCountDescription copied from interface:BlockCacheReturns the number of data blocks currently cached in the block cache.- Specified by:
- getDataBlockCountin interface- BlockCache
- Returns:
- number of blocks in the cache
 
- 
getEvictionThread
- 
logStats
- 
getStatsGet counter statistics for this cache.Includes: total accesses, hits, misses, evicted blocks, and runs of the eviction processes. - Specified by:
- getStatsin interface- BlockCache
 
- 
heapSizeDescription copied from interface:HeapSizeReturn the approximate 'exclusive deep size' of implementing object. Includes count of payload and hosting object sizings.
- 
calculateOverhead
- 
iteratorDescription copied from interface:BlockCacheReturns Iterator over the blocks in the cache.- Specified by:
- iteratorin interface- BlockCache
- Specified by:
- iteratorin interface- Iterable<CachedBlock>
 
- 
acceptableSizelong acceptableSize()
- 
minSize
- 
singleSize
- 
multiSize
- 
memorySize
- 
shutdownDescription copied from interface:BlockCacheShutdown the cache.- Specified by:
- shutdownin interface- BlockCache
 
- 
clearCacheClears the cache. Used in tests.
- 
getEncodingCountsForTest
- 
getMapForTests
- 
getBlockCachesDescription copied from interface:BlockCacheReturns The list of sub blockcaches that make up this one; returns null if no sub caches.- Specified by:
- getBlockCachesin interface- BlockCache
 
 
-