View Javadoc

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