View Javadoc

1   /**
2    * Copyright 2009 The Apache Software Foundation
3    *
4    * Licensed to the Apache Software Foundation (ASF) under one
5    * or more contributor license agreements.  See the NOTICE file
6    * distributed with this work for additional information
7    * regarding copyright ownership.  The ASF licenses this file
8    * to you under the Apache License, Version 2.0 (the
9    * "License"); you may not use this file except in compliance
10   * with the License.  You may obtain a copy of the License at
11   *
12   *     http://www.apache.org/licenses/LICENSE-2.0
13   *
14   * Unless required by applicable law or agreed to in writing, software
15   * distributed under the License is distributed on an "AS IS" BASIS,
16   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
17   * See the License for the specific language governing permissions and
18   * limitations under the License.
19   */
20  package org.apache.hadoop.hbase.io.hfile;
21  
22  import java.io.IOException;
23  import java.lang.ref.WeakReference;
24  import java.util.ArrayList;
25  import java.util.Collections;
26  import java.util.EnumMap;
27  import java.util.HashMap;
28  import java.util.List;
29  import java.util.Map;
30  import java.util.PriorityQueue;
31  import java.util.SortedSet;
32  import java.util.TreeSet;
33  import java.util.concurrent.ConcurrentHashMap;
34  import java.util.concurrent.Executors;
35  import java.util.concurrent.ScheduledExecutorService;
36  import java.util.concurrent.TimeUnit;
37  import java.util.concurrent.atomic.AtomicLong;
38  import java.util.concurrent.locks.ReentrantLock;
39  
40  import org.apache.commons.logging.Log;
41  import org.apache.commons.logging.LogFactory;
42  import org.apache.hadoop.conf.Configuration;
43  import org.apache.hadoop.fs.FileSystem;
44  import org.apache.hadoop.fs.Path;
45  import org.apache.hadoop.hbase.io.HeapSize;
46  import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
47  import org.apache.hadoop.hbase.regionserver.metrics.SchemaMetrics;
48  import org.apache.hadoop.hbase.util.Bytes;
49  import org.apache.hadoop.hbase.util.ClassSize;
50  import org.apache.hadoop.hbase.util.FSUtils;
51  import org.apache.hadoop.hbase.util.HasThread;
52  import org.apache.hadoop.hbase.util.Threads;
53  import org.apache.hadoop.util.StringUtils;
54  
55  import com.google.common.util.concurrent.ThreadFactoryBuilder;
56  
57  /**
58   * A block cache implementation that is memory-aware using {@link HeapSize},
59   * memory-bound using an LRU eviction algorithm, and concurrent: backed by a
60   * {@link ConcurrentHashMap} and with a non-blocking eviction thread giving
61   * constant-time {@link #cacheBlock} and {@link #getBlock} operations.<p>
62   *
63   * Contains three levels of block priority to allow for
64   * scan-resistance and in-memory families.  A block is added with an inMemory
65   * flag if necessary, otherwise a block becomes a single access priority.  Once
66   * a blocked is accessed again, it changes to multiple access.  This is used
67   * to prevent scans from thrashing the cache, adding a least-frequently-used
68   * element to the eviction algorithm.<p>
69   *
70   * Each priority is given its own chunk of the total cache to ensure
71   * fairness during eviction.  Each priority will retain close to its maximum
72   * size, however, if any priority is not using its entire chunk the others
73   * are able to grow beyond their chunk size.<p>
74   *
75   * Instantiated at a minimum with the total size and average block size.
76   * All sizes are in bytes.  The block size is not especially important as this
77   * cache is fully dynamic in its sizing of blocks.  It is only used for
78   * pre-allocating data structures and in initial heap estimation of the map.<p>
79   *
80   * The detailed constructor defines the sizes for the three priorities (they
81   * should total to the maximum size defined).  It also sets the levels that
82   * trigger and control the eviction thread.<p>
83   *
84   * The acceptable size is the cache size level which triggers the eviction
85   * process to start.  It evicts enough blocks to get the size below the
86   * minimum size specified.<p>
87   *
88   * Eviction happens in a separate thread and involves a single full-scan
89   * of the map.  It determines how many bytes must be freed to reach the minimum
90   * size, and then while scanning determines the fewest least-recently-used
91   * blocks necessary from each of the three priorities (would be 3 times bytes
92   * to free).  It then uses the priority chunk sizes to evict fairly according
93   * to the relative sizes and usage.
94   */
95  public class LruBlockCache implements BlockCache, HeapSize {
96  
97    static final Log LOG = LogFactory.getLog(LruBlockCache.class);
98  
99    static final String LRU_MIN_FACTOR_CONFIG_NAME = "hbase.lru.blockcache.min.factor";
100   static final String LRU_ACCEPTABLE_FACTOR_CONFIG_NAME = "hbase.lru.blockcache.acceptable.factor";
101 
102   /** Default Configuration Parameters*/
103 
104   /** Backing Concurrent Map Configuration */
105   static final float DEFAULT_LOAD_FACTOR = 0.75f;
106   static final int DEFAULT_CONCURRENCY_LEVEL = 16;
107 
108   /** Eviction thresholds */
109   static final float DEFAULT_MIN_FACTOR = 0.75f;
110   static final float DEFAULT_ACCEPTABLE_FACTOR = 0.85f;
111 
112   /** Priority buckets */
113   static final float DEFAULT_SINGLE_FACTOR = 0.25f;
114   static final float DEFAULT_MULTI_FACTOR = 0.50f;
115   static final float DEFAULT_MEMORY_FACTOR = 0.25f;
116 
117   /** Statistics thread */
118   static final int statThreadPeriod = 60 * 5;
119 
120   /** Concurrent map (the cache) */
121   private final ConcurrentHashMap<BlockCacheKey,CachedBlock> map;
122 
123   /** Eviction lock (locked when eviction in process) */
124   private final ReentrantLock evictionLock = new ReentrantLock(true);
125 
126   /** Volatile boolean to track if we are in an eviction process or not */
127   private volatile boolean evictionInProgress = false;
128 
129   /** Eviction thread */
130   private final EvictionThread evictionThread;
131 
132   /** Statistics thread schedule pool (for heavy debugging, could remove) */
133   private final ScheduledExecutorService scheduleThreadPool =
134     Executors.newScheduledThreadPool(1,
135       new ThreadFactoryBuilder()
136         .setNameFormat("LRU Statistics #%d")
137         .setDaemon(true)
138         .build());
139 
140   /** Current size of cache */
141   private final AtomicLong size;
142 
143   /** Current number of cached elements */
144   private final AtomicLong elements;
145 
146   /** Cache access count (sequential ID) */
147   private final AtomicLong count;
148 
149   /** Cache statistics */
150   private final CacheStats stats;
151 
152   /** Maximum allowable size of cache (block put if size > max, evict) */
153   private long maxSize;
154 
155   /** Approximate block size */
156   private long blockSize;
157 
158   /** Acceptable size of cache (no evictions if size < acceptable) */
159   private float acceptableFactor;
160 
161   /** Minimum threshold of cache (when evicting, evict until size < min) */
162   private float minFactor;
163 
164   /** Single access bucket size */
165   private float singleFactor;
166 
167   /** Multiple access bucket size */
168   private float multiFactor;
169 
170   /** In-memory bucket size */
171   private float memoryFactor;
172 
173   /** Overhead of the structure itself */
174   private long overhead;
175 
176   /**
177    * Default constructor.  Specify maximum size and expected average block
178    * size (approximation is fine).
179    *
180    * <p>All other factors will be calculated based on defaults specified in
181    * this class.
182    * @param maxSize maximum size of cache, in bytes
183    * @param blockSize approximate size of each block, in bytes
184    * @param conf configuration
185    */
186   public LruBlockCache(long maxSize, long blockSize, Configuration conf) {
187     this(maxSize, blockSize, true, conf);
188   }
189 
190   /**
191    * Constructor used for testing.  Allows disabling of the eviction thread.
192    */
193   public LruBlockCache(long maxSize, long blockSize, boolean evictionThread, Configuration conf) {
194     this(maxSize, blockSize, evictionThread,
195         (int)Math.ceil(1.2*maxSize/blockSize),
196         DEFAULT_LOAD_FACTOR, 
197         DEFAULT_CONCURRENCY_LEVEL,
198         conf.getFloat(LRU_MIN_FACTOR_CONFIG_NAME, DEFAULT_MIN_FACTOR), 
199         conf.getFloat(LRU_ACCEPTABLE_FACTOR_CONFIG_NAME, DEFAULT_ACCEPTABLE_FACTOR), 
200         DEFAULT_SINGLE_FACTOR, 
201         DEFAULT_MULTI_FACTOR,
202         DEFAULT_MEMORY_FACTOR);
203   }
204 
205 
206   /**
207    * Configurable constructor.  Use this constructor if not using defaults.
208    * @param maxSize maximum size of this cache, in bytes
209    * @param blockSize expected average size of blocks, in bytes
210    * @param evictionThread whether to run evictions in a bg thread or not
211    * @param mapInitialSize initial size of backing ConcurrentHashMap
212    * @param mapLoadFactor initial load factor of backing ConcurrentHashMap
213    * @param mapConcurrencyLevel initial concurrency factor for backing CHM
214    * @param minFactor percentage of total size that eviction will evict until
215    * @param acceptableFactor percentage of total size that triggers eviction
216    * @param singleFactor percentage of total size for single-access blocks
217    * @param multiFactor percentage of total size for multiple-access blocks
218    * @param memoryFactor percentage of total size for in-memory blocks
219    */
220   public LruBlockCache(long maxSize, long blockSize, boolean evictionThread,
221       int mapInitialSize, float mapLoadFactor, int mapConcurrencyLevel,
222       float minFactor, float acceptableFactor,
223       float singleFactor, float multiFactor, float memoryFactor) {
224     if(singleFactor + multiFactor + memoryFactor != 1) {
225       throw new IllegalArgumentException("Single, multi, and memory factors " +
226           " should total 1.0");
227     }
228     if(minFactor >= acceptableFactor) {
229       throw new IllegalArgumentException("minFactor must be smaller than acceptableFactor");
230     }
231     if(minFactor >= 1.0f || acceptableFactor >= 1.0f) {
232       throw new IllegalArgumentException("all factors must be < 1");
233     }
234     this.maxSize = maxSize;
235     this.blockSize = blockSize;
236     map = new ConcurrentHashMap<BlockCacheKey,CachedBlock>(mapInitialSize,
237         mapLoadFactor, mapConcurrencyLevel);
238     this.minFactor = minFactor;
239     this.acceptableFactor = acceptableFactor;
240     this.singleFactor = singleFactor;
241     this.multiFactor = multiFactor;
242     this.memoryFactor = memoryFactor;
243     this.stats = new CacheStats();
244     this.count = new AtomicLong(0);
245     this.elements = new AtomicLong(0);
246     this.overhead = calculateOverhead(maxSize, blockSize, mapConcurrencyLevel);
247     this.size = new AtomicLong(this.overhead);
248     if(evictionThread) {
249       this.evictionThread = new EvictionThread(this);
250       this.evictionThread.start(); // FindBugs SC_START_IN_CTOR
251     } else {
252       this.evictionThread = null;
253     }
254     this.scheduleThreadPool.scheduleAtFixedRate(new StatisticsThread(this),
255         statThreadPeriod, statThreadPeriod, TimeUnit.SECONDS);
256   }
257 
258   public void setMaxSize(long maxSize) {
259     this.maxSize = maxSize;
260     if(this.size.get() > acceptableSize() && !evictionInProgress) {
261       runEviction();
262     }
263   }
264 
265   // BlockCache implementation
266 
267   /**
268    * Cache the block with the specified name and buffer.
269    * <p>
270    * It is assumed this will NEVER be called on an already cached block.  If
271    * that is done, an exception will be thrown.
272    * @param cacheKey block's cache key
273    * @param buf block buffer
274    * @param inMemory if block is in-memory
275    */
276   public void cacheBlock(BlockCacheKey cacheKey, Cacheable buf, boolean inMemory) {
277     CachedBlock cb = map.get(cacheKey);
278     if(cb != null) {
279       throw new RuntimeException("Cached an already cached block");
280     }
281     cb = new CachedBlock(cacheKey, buf, count.incrementAndGet(), inMemory);
282     long newSize = updateSizeMetrics(cb, false);
283     map.put(cacheKey, cb);
284     elements.incrementAndGet();
285     if(newSize > acceptableSize() && !evictionInProgress) {
286       runEviction();
287     }
288   }
289 
290   /**
291    * Cache the block with the specified name and buffer.
292    * <p>
293    * It is assumed this will NEVER be called on an already cached block.  If
294    * that is done, it is assumed that you are reinserting the same exact
295    * block due to a race condition and will update the buffer but not modify
296    * the size of the cache.
297    * @param cacheKey block's cache key
298    * @param buf block buffer
299    */
300   public void cacheBlock(BlockCacheKey cacheKey, Cacheable buf) {
301     cacheBlock(cacheKey, buf, false);
302   }
303 
304   /**
305    * Helper function that updates the local size counter and also updates any
306    * per-cf or per-blocktype metrics it can discern from given
307    * {@link CachedBlock}
308    *
309    * @param cb
310    * @param evict
311    */
312   protected long updateSizeMetrics(CachedBlock cb, boolean evict) {
313     long heapsize = cb.heapSize();
314     if (evict) {
315       heapsize *= -1;
316     }
317     Cacheable cachedBlock = cb.getBuffer();
318     SchemaMetrics schemaMetrics = cachedBlock.getSchemaMetrics();
319     if (schemaMetrics != null) {
320       schemaMetrics.updateOnCachePutOrEvict(
321           cachedBlock.getBlockType().getCategory(), heapsize, evict);
322     }
323     return size.addAndGet(heapsize);
324   }
325 
326   /**
327    * Get the buffer of the block with the specified name.
328    * @param cacheKey block's cache key
329    * @param caching true if the caller caches blocks on cache misses
330    * @param repeat Whether this is a repeat lookup for the same block
331    *        (used to avoid double counting cache misses when doing double-check locking)
332    *        {@see HFileReaderV2#readBlock(long, long, boolean, boolean, boolean, BlockType)}
333    * @return buffer of specified cache key, or null if not in cache
334    */
335   @Override
336   public Cacheable getBlock(BlockCacheKey cacheKey, boolean caching, boolean repeat) {
337     CachedBlock cb = map.get(cacheKey);
338     if(cb == null) {
339       if (!repeat) stats.miss(caching);
340       return null;
341     }
342     stats.hit(caching);
343     cb.access(count.incrementAndGet());
344     return cb.getBuffer();
345   }
346 
347 
348   @Override
349   public boolean evictBlock(BlockCacheKey cacheKey) {
350     CachedBlock cb = map.get(cacheKey);
351     if (cb == null) return false;
352     evictBlock(cb);
353     return true;
354   }
355 
356   /**
357    * Evicts all blocks for a specific HFile. This is an
358    * expensive operation implemented as a linear-time search through all blocks
359    * in the cache. Ideally this should be a search in a log-access-time map.
360    *
361    * <p>
362    * This is used for evict-on-close to remove all blocks of a specific HFile.
363    *
364    * @return the number of blocks evicted
365    */
366   @Override
367   public int evictBlocksByHfileName(String hfileName) {
368     int numEvicted = 0;
369     for (BlockCacheKey key : map.keySet()) {
370       if (key.getHfileName().equals(hfileName)) {
371         if (evictBlock(key))
372           ++numEvicted;
373       }
374     }
375     return numEvicted;
376   }
377 
378   protected long evictBlock(CachedBlock block) {
379     map.remove(block.getCacheKey());
380     updateSizeMetrics(block, true);
381     elements.decrementAndGet();
382     stats.evicted();
383     return block.heapSize();
384   }
385 
386   /**
387    * Multi-threaded call to run the eviction process.
388    */
389   private void runEviction() {
390     if(evictionThread == null) {
391       evict();
392     } else {
393       evictionThread.evict();
394     }
395   }
396 
397   /**
398    * Eviction method.
399    */
400   void evict() {
401 
402     // Ensure only one eviction at a time
403     if(!evictionLock.tryLock()) return;
404 
405     try {
406       evictionInProgress = true;
407       long currentSize = this.size.get();
408       long bytesToFree = currentSize - minSize();
409 
410       if (LOG.isDebugEnabled()) {
411         LOG.debug("Block cache LRU eviction started; Attempting to free " +
412           StringUtils.byteDesc(bytesToFree) + " of total=" +
413           StringUtils.byteDesc(currentSize));
414       }
415 
416       if(bytesToFree <= 0) return;
417 
418       // Instantiate priority buckets
419       BlockBucket bucketSingle = new BlockBucket(bytesToFree, blockSize,
420           singleSize());
421       BlockBucket bucketMulti = new BlockBucket(bytesToFree, blockSize,
422           multiSize());
423       BlockBucket bucketMemory = new BlockBucket(bytesToFree, blockSize,
424           memorySize());
425 
426       // Scan entire map putting into appropriate buckets
427       for(CachedBlock cachedBlock : map.values()) {
428         switch(cachedBlock.getPriority()) {
429           case SINGLE: {
430             bucketSingle.add(cachedBlock);
431             break;
432           }
433           case MULTI: {
434             bucketMulti.add(cachedBlock);
435             break;
436           }
437           case MEMORY: {
438             bucketMemory.add(cachedBlock);
439             break;
440           }
441         }
442       }
443 
444       PriorityQueue<BlockBucket> bucketQueue =
445         new PriorityQueue<BlockBucket>(3);
446 
447       bucketQueue.add(bucketSingle);
448       bucketQueue.add(bucketMulti);
449       bucketQueue.add(bucketMemory);
450 
451       int remainingBuckets = 3;
452       long bytesFreed = 0;
453 
454       BlockBucket bucket;
455       while((bucket = bucketQueue.poll()) != null) {
456         long overflow = bucket.overflow();
457         if(overflow > 0) {
458           long bucketBytesToFree = Math.min(overflow,
459             (bytesToFree - bytesFreed) / remainingBuckets);
460           bytesFreed += bucket.free(bucketBytesToFree);
461         }
462         remainingBuckets--;
463       }
464 
465       if (LOG.isDebugEnabled()) {
466         long single = bucketSingle.totalSize();
467         long multi = bucketMulti.totalSize();
468         long memory = bucketMemory.totalSize();
469         LOG.debug("Block cache LRU eviction completed; " +
470           "freed=" + StringUtils.byteDesc(bytesFreed) + ", " +
471           "total=" + StringUtils.byteDesc(this.size.get()) + ", " +
472           "single=" + StringUtils.byteDesc(single) + ", " +
473           "multi=" + StringUtils.byteDesc(multi) + ", " +
474           "memory=" + StringUtils.byteDesc(memory));
475       }
476     } finally {
477       stats.evict();
478       evictionInProgress = false;
479       evictionLock.unlock();
480     }
481   }
482 
483   /**
484    * Used to group blocks into priority buckets.  There will be a BlockBucket
485    * for each priority (single, multi, memory).  Once bucketed, the eviction
486    * algorithm takes the appropriate number of elements out of each according
487    * to configuration parameters and their relatives sizes.
488    */
489   private class BlockBucket implements Comparable<BlockBucket> {
490 
491     private CachedBlockQueue queue;
492     private long totalSize = 0;
493     private long bucketSize;
494 
495     public BlockBucket(long bytesToFree, long blockSize, long bucketSize) {
496       this.bucketSize = bucketSize;
497       queue = new CachedBlockQueue(bytesToFree, blockSize);
498       totalSize = 0;
499     }
500 
501     public void add(CachedBlock block) {
502       totalSize += block.heapSize();
503       queue.add(block);
504     }
505 
506     public long free(long toFree) {
507       CachedBlock cb;
508       long freedBytes = 0;
509       while ((cb = queue.pollLast()) != null) {
510         freedBytes += evictBlock(cb);
511         if (freedBytes >= toFree) {
512           return freedBytes;
513         }
514       }
515       return freedBytes;
516     }
517 
518     public long overflow() {
519       return totalSize - bucketSize;
520     }
521 
522     public long totalSize() {
523       return totalSize;
524     }
525 
526     public int compareTo(BlockBucket that) {
527       if(this.overflow() == that.overflow()) return 0;
528       return this.overflow() > that.overflow() ? 1 : -1;
529     }
530   }
531 
532   /**
533    * Get the maximum size of this cache.
534    * @return max size in bytes
535    */
536   public long getMaxSize() {
537     return this.maxSize;
538   }
539 
540   /**
541    * Get the current size of this cache.
542    * @return current size in bytes
543    */
544   public long getCurrentSize() {
545     return this.size.get();
546   }
547 
548   /**
549    * Get the current size of this cache.
550    * @return current size in bytes
551    */
552   public long getFreeSize() {
553     return getMaxSize() - getCurrentSize();
554   }
555 
556   /**
557    * Get the size of this cache (number of cached blocks)
558    * @return number of cached blocks
559    */
560   public long size() {
561     return this.elements.get();
562   }
563 
564   @Override
565   public long getBlockCount() {
566     return this.elements.get();
567   }
568 
569   /**
570    * Get the number of eviction runs that have occurred
571    */
572   public long getEvictionCount() {
573     return this.stats.getEvictionCount();
574   }
575 
576   /**
577    * Get the number of blocks that have been evicted during the lifetime
578    * of this cache.
579    */
580   public long getEvictedCount() {
581     return this.stats.getEvictedCount();
582   }
583 
584   EvictionThread getEvictionThread() {
585     return this.evictionThread;
586   }
587 
588   /*
589    * Eviction thread.  Sits in waiting state until an eviction is triggered
590    * when the cache size grows above the acceptable level.<p>
591    *
592    * Thread is triggered into action by {@link LruBlockCache#runEviction()}
593    */
594   static class EvictionThread extends HasThread {
595     private WeakReference<LruBlockCache> cache;
596     private boolean go = true;
597     // flag set after enter the run method, used for test
598     private boolean enteringRun = false;
599 
600     public EvictionThread(LruBlockCache cache) {
601       super(Thread.currentThread().getName() + ".LruBlockCache.EvictionThread");
602       setDaemon(true);
603       this.cache = new WeakReference<LruBlockCache>(cache);
604     }
605 
606     @Override
607     public void run() {
608       enteringRun = true;
609       while (this.go) {
610         synchronized(this) {
611           try {
612             this.wait();
613           } catch(InterruptedException e) {}
614         }
615         LruBlockCache cache = this.cache.get();
616         if(cache == null) break;
617         cache.evict();
618       }
619     }
620 
621     public void evict() {
622       synchronized(this) {
623         this.notify(); // FindBugs NN_NAKED_NOTIFY
624       }
625     }
626 
627     void shutdown() {
628       this.go = false;
629       interrupt();
630     }
631 
632     /**
633      * Used for the test.
634      */
635     boolean isEnteringRun() {
636       return this.enteringRun;
637     }
638   }
639 
640   /*
641    * Statistics thread.  Periodically prints the cache statistics to the log.
642    */
643   static class StatisticsThread extends Thread {
644     LruBlockCache lru;
645 
646     public StatisticsThread(LruBlockCache lru) {
647       super("LruBlockCache.StatisticsThread");
648       setDaemon(true);
649       this.lru = lru;
650     }
651     @Override
652     public void run() {
653       lru.logStats();
654     }
655   }
656 
657   public void logStats() {
658     if (!LOG.isDebugEnabled()) return;
659     // Log size
660     long totalSize = heapSize();
661     long freeSize = maxSize - totalSize;
662     LruBlockCache.LOG.debug("Stats: " +
663         "total=" + StringUtils.byteDesc(totalSize) + ", " +
664         "free=" + StringUtils.byteDesc(freeSize) + ", " +
665         "max=" + StringUtils.byteDesc(this.maxSize) + ", " +
666         "blocks=" + size() +", " +
667         "accesses=" + stats.getRequestCount() + ", " +
668         "hits=" + stats.getHitCount() + ", " +
669         "hitRatio=" +
670           (stats.getHitCount() == 0 ? "0" : (StringUtils.formatPercent(stats.getHitRatio(), 2)+ ", ")) + ", " +
671         "cachingAccesses=" + stats.getRequestCachingCount() + ", " +
672         "cachingHits=" + stats.getHitCachingCount() + ", " +
673         "cachingHitsRatio=" +
674           (stats.getHitCachingCount() == 0 ? "0" : (StringUtils.formatPercent(stats.getHitCachingRatio(), 2)+ ", ")) + ", " +
675         "evictions=" + stats.getEvictionCount() + ", " +
676         "evicted=" + stats.getEvictedCount() + ", " +
677         "evictedPerRun=" + stats.evictedPerEviction());
678   }
679 
680   /**
681    * Get counter statistics for this cache.
682    *
683    * <p>Includes: total accesses, hits, misses, evicted blocks, and runs
684    * of the eviction processes.
685    */
686   public CacheStats getStats() {
687     return this.stats;
688   }
689 
690   public final static long CACHE_FIXED_OVERHEAD = ClassSize.align(
691       (3 * Bytes.SIZEOF_LONG) + (8 * ClassSize.REFERENCE) +
692       (5 * Bytes.SIZEOF_FLOAT) + Bytes.SIZEOF_BOOLEAN
693       + ClassSize.OBJECT);
694 
695   // HeapSize implementation
696   public long heapSize() {
697     return getCurrentSize();
698   }
699 
700   public static long calculateOverhead(long maxSize, long blockSize, int concurrency){
701     // FindBugs ICAST_INTEGER_MULTIPLY_CAST_TO_LONG
702     return CACHE_FIXED_OVERHEAD + ClassSize.CONCURRENT_HASHMAP +
703         ((long)Math.ceil(maxSize*1.2/blockSize)
704             * ClassSize.CONCURRENT_HASHMAP_ENTRY) +
705         (concurrency * ClassSize.CONCURRENT_HASHMAP_SEGMENT);
706   }
707 
708   @Override
709   public List<BlockCacheColumnFamilySummary> getBlockCacheColumnFamilySummaries(Configuration conf) throws IOException {
710 
711     Map<String, Path> sfMap = FSUtils.getTableStoreFilePathMap(
712         FileSystem.get(conf),
713         FSUtils.getRootDir(conf));
714 
715     // quirky, but it's a compound key and this is a shortcut taken instead of
716     // creating a class that would represent only a key.
717     Map<BlockCacheColumnFamilySummary, BlockCacheColumnFamilySummary> bcs =
718       new HashMap<BlockCacheColumnFamilySummary, BlockCacheColumnFamilySummary>();
719 
720     for (CachedBlock cb : map.values()) {
721       String sf = cb.getCacheKey().getHfileName();
722       Path path = sfMap.get(sf);
723       if ( path != null) {
724         BlockCacheColumnFamilySummary lookup =
725           BlockCacheColumnFamilySummary.createFromStoreFilePath(path);
726         BlockCacheColumnFamilySummary bcse = bcs.get(lookup);
727         if (bcse == null) {
728           bcse = BlockCacheColumnFamilySummary.create(lookup);
729           bcs.put(lookup,bcse);
730         }
731         bcse.incrementBlocks();
732         bcse.incrementHeapSize(cb.heapSize());
733       }
734     }
735     List<BlockCacheColumnFamilySummary> list =
736         new ArrayList<BlockCacheColumnFamilySummary>(bcs.values());
737     Collections.sort( list );
738     return list;
739   }
740 
741   // Simple calculators of sizes given factors and maxSize
742 
743   private long acceptableSize() {
744     return (long)Math.floor(this.maxSize * this.acceptableFactor);
745   }
746   private long minSize() {
747     return (long)Math.floor(this.maxSize * this.minFactor);
748   }
749   private long singleSize() {
750     return (long)Math.floor(this.maxSize * this.singleFactor * this.minFactor);
751   }
752   private long multiSize() {
753     return (long)Math.floor(this.maxSize * this.multiFactor * this.minFactor);
754   }
755   private long memorySize() {
756     return (long)Math.floor(this.maxSize * this.memoryFactor * this.minFactor);
757   }
758 
759   public void shutdown() {
760     this.scheduleThreadPool.shutdown();
761     for (int i = 0; i < 10; i++) {
762       if (!this.scheduleThreadPool.isShutdown()) Threads.sleep(10);
763     }
764     if (!this.scheduleThreadPool.isShutdown()) {
765       List<Runnable> runnables = this.scheduleThreadPool.shutdownNow();
766       LOG.debug("Still running " + runnables);
767     }
768     this.evictionThread.shutdown();
769   }
770 
771   /** Clears the cache. Used in tests. */
772   public void clearCache() {
773     map.clear();
774   }
775 
776   /**
777    * Used in testing. May be very inefficient.
778    * @return the set of cached file names
779    */
780   SortedSet<String> getCachedFileNamesForTest() {
781     SortedSet<String> fileNames = new TreeSet<String>();
782     for (BlockCacheKey cacheKey : map.keySet()) {
783       fileNames.add(cacheKey.getHfileName());
784     }
785     return fileNames;
786   }
787 
788   Map<BlockType, Integer> getBlockTypeCountsForTest() {
789     Map<BlockType, Integer> counts =
790         new EnumMap<BlockType, Integer>(BlockType.class);
791     for (CachedBlock cb : map.values()) {
792       BlockType blockType = ((HFileBlock) cb.getBuffer()).getBlockType();
793       Integer count = counts.get(blockType);
794       counts.put(blockType, (count == null ? 0 : count) + 1);
795     }
796     return counts;
797   }
798 
799   public Map<DataBlockEncoding, Integer> getEncodingCountsForTest() {
800     Map<DataBlockEncoding, Integer> counts =
801         new EnumMap<DataBlockEncoding, Integer>(DataBlockEncoding.class);
802     for (BlockCacheKey cacheKey : map.keySet()) {
803       DataBlockEncoding encoding = cacheKey.getDataBlockEncoding();
804       Integer count = counts.get(encoding);
805       counts.put(encoding, (count == null ? 0 : count) + 1);
806     }
807     return counts;
808   }
809 
810 }