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.lang.ref.WeakReference;
22  import java.nio.ByteBuffer;
23  import java.util.EnumMap;
24  import java.util.Iterator;
25  import java.util.List;
26  import java.util.Map;
27  import java.util.PriorityQueue;
28  import java.util.SortedSet;
29  import java.util.TreeSet;
30  import java.util.concurrent.ConcurrentHashMap;
31  import java.util.concurrent.Executors;
32  import java.util.concurrent.ScheduledExecutorService;
33  import java.util.concurrent.TimeUnit;
34  import java.util.concurrent.atomic.AtomicLong;
35  import java.util.concurrent.locks.ReentrantLock;
36  
37  import com.google.common.base.Objects;
38  import org.apache.commons.logging.Log;
39  import org.apache.commons.logging.LogFactory;
40  import org.apache.hadoop.hbase.classification.InterfaceAudience;
41  import org.apache.hadoop.conf.Configuration;
42  import org.apache.hadoop.hbase.io.HeapSize;
43  import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
44  import org.apache.hadoop.hbase.io.hfile.bucket.BucketCache;
45  import org.apache.hadoop.hbase.util.Bytes;
46  import org.apache.hadoop.hbase.util.ClassSize;
47  import org.apache.hadoop.hbase.util.HasThread;
48  import org.apache.hadoop.util.StringUtils;
49  import org.codehaus.jackson.annotate.JsonIgnoreProperties;
50  
51  import com.google.common.annotations.VisibleForTesting;
52  import com.google.common.util.concurrent.ThreadFactoryBuilder;
53  
54  /**
55   * A block cache implementation that is memory-aware using {@link HeapSize},
56   * memory-bound using an LRU eviction algorithm, and concurrent: backed by a
57   * {@link ConcurrentHashMap} and with a non-blocking eviction thread giving
58   * constant-time {@link #cacheBlock} and {@link #getBlock} operations.<p>
59   *
60   * Contains three levels of block priority to allow for
61   * scan-resistance and in-memory families 
62   * {@link org.apache.hadoop.hbase.HColumnDescriptor#setInMemory(boolean)} (An
63   * in-memory column family is a column family that should be served from memory if possible):
64   * single-access, multiple-accesses, and in-memory priority.
65   * A block is added with an in-memory priority flag if
66   * {@link org.apache.hadoop.hbase.HColumnDescriptor#isInMemory()}, 
67   * otherwise a block becomes a single access
68   * priority the first time it is read into this block cache.  If a block is accessed again while
69   * in cache, it is marked as a multiple access priority block.  This delineation of blocks is used
70   * to prevent scans from thrashing the cache adding a least-frequently-used
71   * element to the eviction algorithm.<p>
72   *
73   * Each priority is given its own chunk of the total cache to ensure
74   * fairness during eviction.  Each priority will retain close to its maximum
75   * size, however, if any priority is not using its entire chunk the others
76   * are able to grow beyond their chunk size.<p>
77   *
78   * Instantiated at a minimum with the total size and average block size.
79   * All sizes are in bytes.  The block size is not especially important as this
80   * cache is fully dynamic in its sizing of blocks.  It is only used for
81   * pre-allocating data structures and in initial heap estimation of the map.<p>
82   *
83   * The detailed constructor defines the sizes for the three priorities (they
84   * should total to the <code>maximum size</code> defined).  It also sets the levels that
85   * trigger and control the eviction thread.<p>
86   *
87   * The <code>acceptable size</code> is the cache size level which triggers the eviction
88   * process to start.  It evicts enough blocks to get the size below the
89   * minimum size specified.<p>
90   *
91   * Eviction happens in a separate thread and involves a single full-scan
92   * of the map.  It determines how many bytes must be freed to reach the minimum
93   * size, and then while scanning determines the fewest least-recently-used
94   * blocks necessary from each of the three priorities (would be 3 times bytes
95   * to free).  It then uses the priority chunk sizes to evict fairly according
96   * to the relative sizes and usage.
97   */
98  @InterfaceAudience.Private
99  @JsonIgnoreProperties({"encodingCountsForTest"})
100 public class LruBlockCache implements ResizableBlockCache, HeapSize {
101 
102   static final Log LOG = LogFactory.getLog(LruBlockCache.class);
103 
104   /**
105    * Percentage of total size that eviction will evict until; e.g. if set to .8, then we will keep
106    * evicting during an eviction run till the cache size is down to 80% of the total.
107    */
108   static final String LRU_MIN_FACTOR_CONFIG_NAME = "hbase.lru.blockcache.min.factor";
109 
110   /**
111    * Acceptable size of cache (no evictions if size < acceptable)
112    */
113   static final String LRU_ACCEPTABLE_FACTOR_CONFIG_NAME = "hbase.lru.blockcache.acceptable.factor";
114 
115   static final String LRU_SINGLE_PERCENTAGE_CONFIG_NAME = "hbase.lru.blockcache.single.percentage";
116   static final String LRU_MULTI_PERCENTAGE_CONFIG_NAME = "hbase.lru.blockcache.multi.percentage";
117   static final String LRU_MEMORY_PERCENTAGE_CONFIG_NAME = "hbase.lru.blockcache.memory.percentage";
118 
119   /**
120    * Configuration key to force data-block always (except in-memory are too much)
121    * cached in memory for in-memory hfile, unlike inMemory, which is a column-family
122    * configuration, inMemoryForceMode is a cluster-wide configuration
123    */
124   static final String LRU_IN_MEMORY_FORCE_MODE_CONFIG_NAME = "hbase.lru.rs.inmemoryforcemode";
125 
126   /** Default Configuration Parameters*/
127 
128   /** Backing Concurrent Map Configuration */
129   static final float DEFAULT_LOAD_FACTOR = 0.75f;
130   static final int DEFAULT_CONCURRENCY_LEVEL = 16;
131 
132   /** Eviction thresholds */
133   static final float DEFAULT_MIN_FACTOR = 0.95f;
134   static final float DEFAULT_ACCEPTABLE_FACTOR = 0.99f;
135 
136   /** Priority buckets */
137   static final float DEFAULT_SINGLE_FACTOR = 0.25f;
138   static final float DEFAULT_MULTI_FACTOR = 0.50f;
139   static final float DEFAULT_MEMORY_FACTOR = 0.25f;
140 
141   static final boolean DEFAULT_IN_MEMORY_FORCE_MODE = false;
142 
143   /** Statistics thread */
144   static final int statThreadPeriod = 60 * 5;
145 
146   /** Concurrent map (the cache) */
147   private final Map<BlockCacheKey,LruCachedBlock> map;
148 
149   /** Eviction lock (locked when eviction in process) */
150   private final ReentrantLock evictionLock = new ReentrantLock(true);
151 
152   /** Volatile boolean to track if we are in an eviction process or not */
153   private volatile boolean evictionInProgress = false;
154 
155   /** Eviction thread */
156   private final EvictionThread evictionThread;
157 
158   /** Statistics thread schedule pool (for heavy debugging, could remove) */
159   private final ScheduledExecutorService scheduleThreadPool = Executors.newScheduledThreadPool(1,
160     new ThreadFactoryBuilder().setNameFormat("LruBlockCacheStatsExecutor").setDaemon(true).build());
161 
162   /** Current size of cache */
163   private final AtomicLong size;
164 
165   /** Current number of cached elements */
166   private final AtomicLong elements;
167 
168   /** Cache access count (sequential ID) */
169   private final AtomicLong count;
170 
171   /** Cache statistics */
172   private final CacheStats stats;
173 
174   /** Maximum allowable size of cache (block put if size > max, evict) */
175   private long maxSize;
176 
177   /** Approximate block size */
178   private long blockSize;
179 
180   /** Acceptable size of cache (no evictions if size < acceptable) */
181   private float acceptableFactor;
182 
183   /** Minimum threshold of cache (when evicting, evict until size < min) */
184   private float minFactor;
185 
186   /** Single access bucket size */
187   private float singleFactor;
188 
189   /** Multiple access bucket size */
190   private float multiFactor;
191 
192   /** In-memory bucket size */
193   private float memoryFactor;
194 
195   /** Overhead of the structure itself */
196   private long overhead;
197 
198   /** Whether in-memory hfile's data block has higher priority when evicting */
199   private boolean forceInMemory;
200 
201   /** Where to send victims (blocks evicted/missing from the cache) */
202   private BlockCache victimHandler = null;
203 
204   /**
205    * Default constructor.  Specify maximum size and expected average block
206    * size (approximation is fine).
207    *
208    * <p>All other factors will be calculated based on defaults specified in
209    * this class.
210    * @param maxSize maximum size of cache, in bytes
211    * @param blockSize approximate size of each block, in bytes
212    */
213   public LruBlockCache(long maxSize, long blockSize) {
214     this(maxSize, blockSize, true);
215   }
216 
217   /**
218    * Constructor used for testing.  Allows disabling of the eviction thread.
219    */
220   public LruBlockCache(long maxSize, long blockSize, boolean evictionThread) {
221     this(maxSize, blockSize, evictionThread,
222         (int)Math.ceil(1.2*maxSize/blockSize),
223         DEFAULT_LOAD_FACTOR, DEFAULT_CONCURRENCY_LEVEL,
224         DEFAULT_MIN_FACTOR, DEFAULT_ACCEPTABLE_FACTOR,
225         DEFAULT_SINGLE_FACTOR,
226         DEFAULT_MULTI_FACTOR,
227         DEFAULT_MEMORY_FACTOR,
228         false
229         );
230   }
231 
232   public LruBlockCache(long maxSize, long blockSize, boolean evictionThread, Configuration conf) {
233     this(maxSize, blockSize, evictionThread,
234         (int)Math.ceil(1.2*maxSize/blockSize),
235         DEFAULT_LOAD_FACTOR,
236         DEFAULT_CONCURRENCY_LEVEL,
237         conf.getFloat(LRU_MIN_FACTOR_CONFIG_NAME, DEFAULT_MIN_FACTOR),
238         conf.getFloat(LRU_ACCEPTABLE_FACTOR_CONFIG_NAME, DEFAULT_ACCEPTABLE_FACTOR),
239         conf.getFloat(LRU_SINGLE_PERCENTAGE_CONFIG_NAME, DEFAULT_SINGLE_FACTOR),
240         conf.getFloat(LRU_MULTI_PERCENTAGE_CONFIG_NAME, DEFAULT_MULTI_FACTOR),
241         conf.getFloat(LRU_MEMORY_PERCENTAGE_CONFIG_NAME, DEFAULT_MEMORY_FACTOR),
242         conf.getBoolean(LRU_IN_MEMORY_FORCE_MODE_CONFIG_NAME, DEFAULT_IN_MEMORY_FORCE_MODE)
243         );
244   }
245 
246   public LruBlockCache(long maxSize, long blockSize, Configuration conf) {
247     this(maxSize, blockSize, true, conf);
248   }
249 
250   /**
251    * Configurable constructor.  Use this constructor if not using defaults.
252    * @param maxSize maximum size of this cache, in bytes
253    * @param blockSize expected average size of blocks, in bytes
254    * @param evictionThread whether to run evictions in a bg thread or not
255    * @param mapInitialSize initial size of backing ConcurrentHashMap
256    * @param mapLoadFactor initial load factor of backing ConcurrentHashMap
257    * @param mapConcurrencyLevel initial concurrency factor for backing CHM
258    * @param minFactor percentage of total size that eviction will evict until
259    * @param acceptableFactor percentage of total size that triggers eviction
260    * @param singleFactor percentage of total size for single-access blocks
261    * @param multiFactor percentage of total size for multiple-access blocks
262    * @param memoryFactor percentage of total size for in-memory blocks
263    */
264   public LruBlockCache(long maxSize, long blockSize, boolean evictionThread,
265       int mapInitialSize, float mapLoadFactor, int mapConcurrencyLevel,
266       float minFactor, float acceptableFactor, float singleFactor,
267       float multiFactor, float memoryFactor, boolean forceInMemory) {
268     if(singleFactor + multiFactor + memoryFactor != 1 ||
269         singleFactor < 0 || multiFactor < 0 || memoryFactor < 0) {
270       throw new IllegalArgumentException("Single, multi, and memory factors " +
271           " should be non-negative and total 1.0");
272     }
273     if(minFactor >= acceptableFactor) {
274       throw new IllegalArgumentException("minFactor must be smaller than acceptableFactor");
275     }
276     if(minFactor >= 1.0f || acceptableFactor >= 1.0f) {
277       throw new IllegalArgumentException("all factors must be < 1");
278     }
279     this.maxSize = maxSize;
280     this.blockSize = blockSize;
281     this.forceInMemory = forceInMemory;
282     map = new ConcurrentHashMap<BlockCacheKey,LruCachedBlock>(mapInitialSize,
283         mapLoadFactor, mapConcurrencyLevel);
284     this.minFactor = minFactor;
285     this.acceptableFactor = acceptableFactor;
286     this.singleFactor = singleFactor;
287     this.multiFactor = multiFactor;
288     this.memoryFactor = memoryFactor;
289     this.stats = new CacheStats(this.getClass().getSimpleName());
290     this.count = new AtomicLong(0);
291     this.elements = new AtomicLong(0);
292     this.overhead = calculateOverhead(maxSize, blockSize, mapConcurrencyLevel);
293     this.size = new AtomicLong(this.overhead);
294     if(evictionThread) {
295       this.evictionThread = new EvictionThread(this);
296       this.evictionThread.start(); // FindBugs SC_START_IN_CTOR
297     } else {
298       this.evictionThread = null;
299     }
300     // TODO: Add means of turning this off.  Bit obnoxious running thread just to make a log
301     // every five minutes.
302     this.scheduleThreadPool.scheduleAtFixedRate(new StatisticsThread(this),
303         statThreadPeriod, statThreadPeriod, TimeUnit.SECONDS);
304   }
305 
306   @Override
307   public void setMaxSize(long maxSize) {
308     this.maxSize = maxSize;
309     if(this.size.get() > acceptableSize() && !evictionInProgress) {
310       runEviction();
311     }
312   }
313 
314   // BlockCache implementation
315 
316   /**
317    * Cache the block with the specified name and buffer.
318    * <p>
319    * It is assumed this will NOT be called on an already cached block. In rare cases (HBASE-8547)
320    * this can happen, for which we compare the buffer contents.
321    * @param cacheKey block's cache key
322    * @param buf block buffer
323    * @param inMemory if block is in-memory
324    * @param cacheDataInL1
325    */
326   @Override
327   public void cacheBlock(BlockCacheKey cacheKey, Cacheable buf, boolean inMemory,
328       final boolean cacheDataInL1) {
329     LruCachedBlock cb = map.get(cacheKey);
330     if (cb != null) {
331       // compare the contents, if they are not equal, we are in big trouble
332       if (compare(buf, cb.getBuffer()) != 0) {
333         throw new RuntimeException("Cached block contents differ, which should not have happened."
334           + "cacheKey:" + cacheKey);
335       }
336       String msg = "Cached an already cached block: " + cacheKey + " cb:" + cb.getCacheKey();
337       msg += ". This is harmless and can happen in rare cases (see HBASE-8547)";
338       LOG.warn(msg);
339       return;
340     }
341     cb = new LruCachedBlock(cacheKey, buf, count.incrementAndGet(), inMemory);
342     long newSize = updateSizeMetrics(cb, false);
343     map.put(cacheKey, cb);
344     long val = elements.incrementAndGet();
345     if (LOG.isTraceEnabled()) {
346       long size = map.size();
347       assertCounterSanity(size, val);
348     }
349     if (newSize > acceptableSize() && !evictionInProgress) {
350       runEviction();
351     }
352   }
353 
354   /**
355    * Sanity-checking for parity between actual block cache content and metrics.
356    * Intended only for use with TRACE level logging and -ea JVM.
357    */
358   private static void assertCounterSanity(long mapSize, long counterVal) {
359     if (counterVal < 0) {
360       LOG.trace("counterVal overflow. Assertions unreliable. counterVal=" + counterVal +
361         ", mapSize=" + mapSize);
362       return;
363     }
364     if (mapSize < Integer.MAX_VALUE) {
365       double pct_diff = Math.abs((((double) counterVal) / ((double) mapSize)) - 1.);
366       if (pct_diff > 0.05) {
367         LOG.trace("delta between reported and actual size > 5%. counterVal=" + counterVal +
368           ", mapSize=" + mapSize);
369       }
370     }
371   }
372 
373   private int compare(Cacheable left, Cacheable right) {
374     ByteBuffer l = ByteBuffer.allocate(left.getSerializedLength());
375     left.serialize(l);
376     ByteBuffer r = ByteBuffer.allocate(right.getSerializedLength());
377     right.serialize(r);
378     return Bytes.compareTo(l.array(), l.arrayOffset(), l.limit(),
379       r.array(), r.arrayOffset(), r.limit());
380   }
381 
382   /**
383    * Cache the block with the specified name and buffer.
384    * <p>
385    * @param cacheKey block's cache key
386    * @param buf block buffer
387    */
388   public void cacheBlock(BlockCacheKey cacheKey, Cacheable buf) {
389     cacheBlock(cacheKey, buf, false, false);
390   }
391 
392   /**
393    * Helper function that updates the local size counter and also updates any
394    * per-cf or per-blocktype metrics it can discern from given
395    * {@link LruCachedBlock}
396    *
397    * @param cb
398    * @param evict
399    */
400   protected long updateSizeMetrics(LruCachedBlock cb, boolean evict) {
401     long heapsize = cb.heapSize();
402     if (evict) {
403       heapsize *= -1;
404     }
405     return size.addAndGet(heapsize);
406   }
407 
408   /**
409    * Get the buffer of the block with the specified name.
410    * @param cacheKey block's cache key
411    * @param caching true if the caller caches blocks on cache misses
412    * @param repeat Whether this is a repeat lookup for the same block
413    *        (used to avoid double counting cache misses when doing double-check locking)
414    * @param updateCacheMetrics Whether to update cache metrics or not
415    * @return buffer of specified cache key, or null if not in cache
416    */
417   @Override
418   public Cacheable getBlock(BlockCacheKey cacheKey, boolean caching, boolean repeat,
419       boolean updateCacheMetrics) {
420     LruCachedBlock cb = map.get(cacheKey);
421     if (cb == null) {
422       if (!repeat && updateCacheMetrics) {
423         stats.miss(caching, cacheKey.isPrimary(), cacheKey.getBlockType());
424       }
425       // If there is another block cache then try and read there.
426       // However if this is a retry ( second time in double checked locking )
427       // And it's already a miss then the l2 will also be a miss.
428       if (victimHandler != null && !repeat) {
429         Cacheable result = victimHandler.getBlock(cacheKey, caching, repeat, updateCacheMetrics);
430 
431         // Promote this to L1.
432         if (result != null && caching) {
433           cacheBlock(cacheKey, result, /* inMemory = */ false, /* cacheData = */ true);
434         }
435         return result;
436       }
437       return null;
438     }
439     if (updateCacheMetrics) stats.hit(caching, cacheKey.isPrimary(), cacheKey.getBlockType());
440     cb.access(count.incrementAndGet());
441     return cb.getBuffer();
442   }
443 
444   /**
445    * Whether the cache contains block with specified cacheKey
446    * @param cacheKey
447    * @return true if contains the block
448    */
449   public boolean containsBlock(BlockCacheKey cacheKey) {
450     return map.containsKey(cacheKey);
451   }
452 
453   @Override
454   public boolean evictBlock(BlockCacheKey cacheKey) {
455     LruCachedBlock cb = map.get(cacheKey);
456     if (cb == null) return false;
457     evictBlock(cb, false);
458     return true;
459   }
460 
461   /**
462    * Evicts all blocks for a specific HFile. This is an
463    * expensive operation implemented as a linear-time search through all blocks
464    * in the cache. Ideally this should be a search in a log-access-time map.
465    *
466    * <p>
467    * This is used for evict-on-close to remove all blocks of a specific HFile.
468    *
469    * @return the number of blocks evicted
470    */
471   @Override
472   public int evictBlocksByHfileName(String hfileName) {
473     int numEvicted = 0;
474     for (BlockCacheKey key : map.keySet()) {
475       if (key.getHfileName().equals(hfileName)) {
476         if (evictBlock(key))
477           ++numEvicted;
478       }
479     }
480     if (victimHandler != null) {
481       numEvicted += victimHandler.evictBlocksByHfileName(hfileName);
482     }
483     return numEvicted;
484   }
485 
486   /**
487    * Evict the block, and it will be cached by the victim handler if exists &&
488    * block may be read again later
489    * @param block
490    * @param evictedByEvictionProcess true if the given block is evicted by
491    *          EvictionThread
492    * @return the heap size of evicted block
493    */
494   protected long evictBlock(LruCachedBlock block, boolean evictedByEvictionProcess) {
495     map.remove(block.getCacheKey());
496     updateSizeMetrics(block, true);
497     long val = elements.decrementAndGet();
498     if (LOG.isTraceEnabled()) {
499       long size = map.size();
500       assertCounterSanity(size, val);
501     }
502     stats.evicted(block.getCachedTime(), block.getCacheKey().isPrimary());
503     if (evictedByEvictionProcess && victimHandler != null) {
504       if (victimHandler instanceof BucketCache) {
505         boolean wait = getCurrentSize() < acceptableSize();
506         boolean inMemory = block.getPriority() == BlockPriority.MEMORY;
507         ((BucketCache)victimHandler).cacheBlockWithWait(block.getCacheKey(), block.getBuffer(),
508             inMemory, wait);
509       } else {
510         victimHandler.cacheBlock(block.getCacheKey(), block.getBuffer());
511       }
512     }
513     return block.heapSize();
514   }
515 
516   /**
517    * Multi-threaded call to run the eviction process.
518    */
519   private void runEviction() {
520     if(evictionThread == null) {
521       evict();
522     } else {
523       evictionThread.evict();
524     }
525   }
526 
527   /**
528    * Eviction method.
529    */
530   void evict() {
531 
532     // Ensure only one eviction at a time
533     if(!evictionLock.tryLock()) return;
534 
535     try {
536       evictionInProgress = true;
537       long currentSize = this.size.get();
538       long bytesToFree = currentSize - minSize();
539 
540       if (LOG.isTraceEnabled()) {
541         LOG.trace("Block cache LRU eviction started; Attempting to free " +
542           StringUtils.byteDesc(bytesToFree) + " of total=" +
543           StringUtils.byteDesc(currentSize));
544       }
545 
546       if(bytesToFree <= 0) return;
547 
548       // Instantiate priority buckets
549       BlockBucket bucketSingle = new BlockBucket("single", bytesToFree, blockSize,
550           singleSize());
551       BlockBucket bucketMulti = new BlockBucket("multi", bytesToFree, blockSize,
552           multiSize());
553       BlockBucket bucketMemory = new BlockBucket("memory", bytesToFree, blockSize,
554           memorySize());
555 
556       // Scan entire map putting into appropriate buckets
557       for(LruCachedBlock cachedBlock : map.values()) {
558         switch(cachedBlock.getPriority()) {
559           case SINGLE: {
560             bucketSingle.add(cachedBlock);
561             break;
562           }
563           case MULTI: {
564             bucketMulti.add(cachedBlock);
565             break;
566           }
567           case MEMORY: {
568             bucketMemory.add(cachedBlock);
569             break;
570           }
571         }
572       }
573 
574       long bytesFreed = 0;
575       if (forceInMemory || memoryFactor > 0.999f) {
576         long s = bucketSingle.totalSize();
577         long m = bucketMulti.totalSize();
578         if (bytesToFree > (s + m)) {
579           // this means we need to evict blocks in memory bucket to make room,
580           // so the single and multi buckets will be emptied
581           bytesFreed = bucketSingle.free(s);
582           bytesFreed += bucketMulti.free(m);
583           if (LOG.isTraceEnabled()) {
584             LOG.trace("freed " + StringUtils.byteDesc(bytesFreed) +
585               " from single and multi buckets");
586           }
587           bytesFreed += bucketMemory.free(bytesToFree - bytesFreed);
588           if (LOG.isTraceEnabled()) {
589             LOG.trace("freed " + StringUtils.byteDesc(bytesFreed) +
590               " total from all three buckets ");
591           }
592         } else {
593           // this means no need to evict block in memory bucket,
594           // and we try best to make the ratio between single-bucket and
595           // multi-bucket is 1:2
596           long bytesRemain = s + m - bytesToFree;
597           if (3 * s <= bytesRemain) {
598             // single-bucket is small enough that no eviction happens for it
599             // hence all eviction goes from multi-bucket
600             bytesFreed = bucketMulti.free(bytesToFree);
601           } else if (3 * m <= 2 * bytesRemain) {
602             // multi-bucket is small enough that no eviction happens for it
603             // hence all eviction goes from single-bucket
604             bytesFreed = bucketSingle.free(bytesToFree);
605           } else {
606             // both buckets need to evict some blocks
607             bytesFreed = bucketSingle.free(s - bytesRemain / 3);
608             if (bytesFreed < bytesToFree) {
609               bytesFreed += bucketMulti.free(bytesToFree - bytesFreed);
610             }
611           }
612         }
613       } else {
614         PriorityQueue<BlockBucket> bucketQueue =
615           new PriorityQueue<BlockBucket>(3);
616 
617         bucketQueue.add(bucketSingle);
618         bucketQueue.add(bucketMulti);
619         bucketQueue.add(bucketMemory);
620 
621         int remainingBuckets = 3;
622 
623         BlockBucket bucket;
624         while((bucket = bucketQueue.poll()) != null) {
625           long overflow = bucket.overflow();
626           if(overflow > 0) {
627             long bucketBytesToFree = Math.min(overflow,
628                 (bytesToFree - bytesFreed) / remainingBuckets);
629             bytesFreed += bucket.free(bucketBytesToFree);
630           }
631           remainingBuckets--;
632         }
633       }
634 
635       if (LOG.isTraceEnabled()) {
636         long single = bucketSingle.totalSize();
637         long multi = bucketMulti.totalSize();
638         long memory = bucketMemory.totalSize();
639         LOG.trace("Block cache LRU eviction completed; " +
640           "freed=" + StringUtils.byteDesc(bytesFreed) + ", " +
641           "total=" + StringUtils.byteDesc(this.size.get()) + ", " +
642           "single=" + StringUtils.byteDesc(single) + ", " +
643           "multi=" + StringUtils.byteDesc(multi) + ", " +
644           "memory=" + StringUtils.byteDesc(memory));
645       }
646     } finally {
647       stats.evict();
648       evictionInProgress = false;
649       evictionLock.unlock();
650     }
651   }
652 
653   @Override
654   public String toString() {
655     return Objects.toStringHelper(this)
656       .add("blockCount", getBlockCount())
657       .add("currentSize", getCurrentSize())
658       .add("freeSize", getFreeSize())
659       .add("maxSize", getMaxSize())
660       .add("heapSize", heapSize())
661       .add("minSize", minSize())
662       .add("minFactor", minFactor)
663       .add("multiSize", multiSize())
664       .add("multiFactor", multiFactor)
665       .add("singleSize", singleSize())
666       .add("singleFactor", singleFactor)
667       .toString();
668   }
669 
670   /**
671    * Used to group blocks into priority buckets.  There will be a BlockBucket
672    * for each priority (single, multi, memory).  Once bucketed, the eviction
673    * algorithm takes the appropriate number of elements out of each according
674    * to configuration parameters and their relatives sizes.
675    */
676   private class BlockBucket implements Comparable<BlockBucket> {
677 
678     private final String name;
679     private LruCachedBlockQueue queue;
680     private long totalSize = 0;
681     private long bucketSize;
682 
683     public BlockBucket(String name, long bytesToFree, long blockSize, long bucketSize) {
684       this.name = name;
685       this.bucketSize = bucketSize;
686       queue = new LruCachedBlockQueue(bytesToFree, blockSize);
687       totalSize = 0;
688     }
689 
690     public void add(LruCachedBlock block) {
691       totalSize += block.heapSize();
692       queue.add(block);
693     }
694 
695     public long free(long toFree) {
696       if (LOG.isTraceEnabled()) {
697         LOG.trace("freeing " + StringUtils.byteDesc(toFree) + " from " + this);
698       }
699       LruCachedBlock cb;
700       long freedBytes = 0;
701       while ((cb = queue.pollLast()) != null) {
702         freedBytes += evictBlock(cb, true);
703         if (freedBytes >= toFree) {
704           return freedBytes;
705         }
706       }
707       if (LOG.isTraceEnabled()) {
708         LOG.trace("freed " + StringUtils.byteDesc(freedBytes) + " from " + this);
709       }
710       return freedBytes;
711     }
712 
713     public long overflow() {
714       return totalSize - bucketSize;
715     }
716 
717     public long totalSize() {
718       return totalSize;
719     }
720 
721     public int compareTo(BlockBucket that) {
722       if(this.overflow() == that.overflow()) return 0;
723       return this.overflow() > that.overflow() ? 1 : -1;
724     }
725 
726     @Override
727     public boolean equals(Object that) {
728       if (that == null || !(that instanceof BlockBucket)){
729         return false;
730       }
731       return compareTo((BlockBucket)that) == 0;
732     }
733 
734     @Override
735     public int hashCode() {
736       return Objects.hashCode(name, bucketSize, queue, totalSize);
737     }
738 
739     @Override
740     public String toString() {
741       return Objects.toStringHelper(this)
742         .add("name", name)
743         .add("totalSize", StringUtils.byteDesc(totalSize))
744         .add("bucketSize", StringUtils.byteDesc(bucketSize))
745         .toString();
746     }
747   }
748 
749   /**
750    * Get the maximum size of this cache.
751    * @return max size in bytes
752    */
753   public long getMaxSize() {
754     return this.maxSize;
755   }
756 
757   @Override
758   public long getCurrentSize() {
759     return this.size.get();
760   }
761 
762   @Override
763   public long getFreeSize() {
764     return getMaxSize() - getCurrentSize();
765   }
766 
767   @Override
768   public long size() {
769     return getMaxSize();
770   }
771 
772   @Override
773   public long getBlockCount() {
774     return this.elements.get();
775   }
776 
777   EvictionThread getEvictionThread() {
778     return this.evictionThread;
779   }
780 
781   /*
782    * Eviction thread.  Sits in waiting state until an eviction is triggered
783    * when the cache size grows above the acceptable level.<p>
784    *
785    * Thread is triggered into action by {@link LruBlockCache#runEviction()}
786    */
787   static class EvictionThread extends HasThread {
788     private WeakReference<LruBlockCache> cache;
789     private volatile boolean go = true;
790     // flag set after enter the run method, used for test
791     private boolean enteringRun = false;
792 
793     public EvictionThread(LruBlockCache cache) {
794       super(Thread.currentThread().getName() + ".LruBlockCache.EvictionThread");
795       setDaemon(true);
796       this.cache = new WeakReference<LruBlockCache>(cache);
797     }
798 
799     @Override
800     public void run() {
801       enteringRun = true;
802       while (this.go) {
803         synchronized(this) {
804           try {
805             this.wait(1000 * 10/*Don't wait for ever*/);
806           } catch(InterruptedException e) {
807             LOG.warn("Interrupted eviction thread ", e);
808             Thread.currentThread().interrupt();
809           }
810         }
811         LruBlockCache cache = this.cache.get();
812         if (cache == null) break;
813         cache.evict();
814       }
815     }
816 
817     @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="NN_NAKED_NOTIFY",
818         justification="This is what we want")
819     public void evict() {
820       synchronized(this) {
821         this.notifyAll();
822       }
823     }
824 
825     synchronized void shutdown() {
826       this.go = false;
827       this.notifyAll();
828     }
829 
830     /**
831      * Used for the test.
832      */
833     boolean isEnteringRun() {
834       return this.enteringRun;
835     }
836   }
837 
838   /*
839    * Statistics thread.  Periodically prints the cache statistics to the log.
840    */
841   static class StatisticsThread extends Thread {
842     private final LruBlockCache lru;
843 
844     public StatisticsThread(LruBlockCache lru) {
845       super("LruBlockCacheStats");
846       setDaemon(true);
847       this.lru = lru;
848     }
849 
850     @Override
851     public void run() {
852       lru.logStats();
853     }
854   }
855 
856   public void logStats() {
857     // Log size
858     long totalSize = heapSize();
859     long freeSize = maxSize - totalSize;
860     LruBlockCache.LOG.info("totalSize=" + StringUtils.byteDesc(totalSize) + ", " +
861         "freeSize=" + StringUtils.byteDesc(freeSize) + ", " +
862         "max=" + StringUtils.byteDesc(this.maxSize) + ", " +
863         "blockCount=" + getBlockCount() + ", " +
864         "accesses=" + stats.getRequestCount() + ", " +
865         "hits=" + stats.getHitCount() + ", " +
866         "hitRatio=" + (stats.getHitCount() == 0 ?
867           "0" : (StringUtils.formatPercent(stats.getHitRatio(), 2)+ ", ")) + ", " +
868         "cachingAccesses=" + stats.getRequestCachingCount() + ", " +
869         "cachingHits=" + stats.getHitCachingCount() + ", " +
870         "cachingHitsRatio=" + (stats.getHitCachingCount() == 0 ?
871           "0,": (StringUtils.formatPercent(stats.getHitCachingRatio(), 2) + ", ")) +
872         "evictions=" + stats.getEvictionCount() + ", " +
873         "evicted=" + stats.getEvictedCount() + ", " +
874         "evictedPerRun=" + stats.evictedPerEviction());
875   }
876 
877   /**
878    * Get counter statistics for this cache.
879    *
880    * <p>Includes: total accesses, hits, misses, evicted blocks, and runs
881    * of the eviction processes.
882    */
883   public CacheStats getStats() {
884     return this.stats;
885   }
886 
887   public final static long CACHE_FIXED_OVERHEAD = ClassSize.align(
888       (3 * Bytes.SIZEOF_LONG) + (9 * ClassSize.REFERENCE) +
889       (5 * Bytes.SIZEOF_FLOAT) + Bytes.SIZEOF_BOOLEAN
890       + ClassSize.OBJECT);
891 
892   @Override
893   public long heapSize() {
894     return getCurrentSize();
895   }
896 
897   public static long calculateOverhead(long maxSize, long blockSize, int concurrency){
898     // FindBugs ICAST_INTEGER_MULTIPLY_CAST_TO_LONG
899     return CACHE_FIXED_OVERHEAD + ClassSize.CONCURRENT_HASHMAP +
900         ((long)Math.ceil(maxSize*1.2/blockSize)
901             * ClassSize.CONCURRENT_HASHMAP_ENTRY) +
902         ((long)concurrency * ClassSize.CONCURRENT_HASHMAP_SEGMENT);
903   }
904 
905   @Override
906   public Iterator<CachedBlock> iterator() {
907     final Iterator<LruCachedBlock> iterator = map.values().iterator();
908 
909     return new Iterator<CachedBlock>() {
910       private final long now = System.nanoTime();
911 
912       @Override
913       public boolean hasNext() {
914         return iterator.hasNext();
915       }
916 
917       @Override
918       public CachedBlock next() {
919         final LruCachedBlock b = iterator.next();
920         return new CachedBlock() {
921           @Override
922           public String toString() {
923             return BlockCacheUtil.toString(this, now);
924           }
925 
926           @Override
927           public BlockPriority getBlockPriority() {
928             return b.getPriority();
929           }
930 
931           @Override
932           public BlockType getBlockType() {
933             return b.getBuffer().getBlockType();
934           }
935 
936           @Override
937           public long getOffset() {
938             return b.getCacheKey().getOffset();
939           }
940 
941           @Override
942           public long getSize() {
943             return b.getBuffer().heapSize();
944           }
945 
946           @Override
947           public long getCachedTime() {
948             return b.getCachedTime();
949           }
950 
951           @Override
952           public String getFilename() {
953             return b.getCacheKey().getHfileName();
954           }
955 
956           @Override
957           public int compareTo(CachedBlock other) {
958             int diff = this.getFilename().compareTo(other.getFilename());
959             if (diff != 0) return diff;
960             diff = (int)(this.getOffset() - other.getOffset());
961             if (diff != 0) return diff;
962             if (other.getCachedTime() < 0 || this.getCachedTime() < 0) {
963               throw new IllegalStateException("" + this.getCachedTime() + ", " +
964                 other.getCachedTime());
965             }
966             return (int)(other.getCachedTime() - this.getCachedTime());
967           }
968 
969           @Override
970           public int hashCode() {
971             return b.hashCode();
972           }
973 
974           @Override
975           public boolean equals(Object obj) {
976             if (obj instanceof CachedBlock) {
977               CachedBlock cb = (CachedBlock)obj;
978               return compareTo(cb) == 0;
979             } else {
980               return false;
981             }
982           }
983         };
984       }
985 
986       @Override
987       public void remove() {
988         throw new UnsupportedOperationException();
989       }
990     };
991   }
992 
993   // Simple calculators of sizes given factors and maxSize
994 
995   long acceptableSize() {
996     return (long)Math.floor(this.maxSize * this.acceptableFactor);
997   }
998   private long minSize() {
999     return (long)Math.floor(this.maxSize * this.minFactor);
1000   }
1001   private long singleSize() {
1002     return (long)Math.floor(this.maxSize * this.singleFactor * this.minFactor);
1003   }
1004   private long multiSize() {
1005     return (long)Math.floor(this.maxSize * this.multiFactor * this.minFactor);
1006   }
1007   private long memorySize() {
1008     return (long)Math.floor(this.maxSize * this.memoryFactor * this.minFactor);
1009   }
1010 
1011   public void shutdown() {
1012     if (victimHandler != null)
1013       victimHandler.shutdown();
1014     this.scheduleThreadPool.shutdown();
1015     for (int i = 0; i < 10; i++) {
1016       if (!this.scheduleThreadPool.isShutdown()) {
1017         try {
1018           Thread.sleep(10);
1019         } catch (InterruptedException e) {
1020           LOG.warn("Interrupted while sleeping");
1021           Thread.currentThread().interrupt();
1022           break;
1023         }
1024       }
1025     }
1026 
1027     if (!this.scheduleThreadPool.isShutdown()) {
1028       List<Runnable> runnables = this.scheduleThreadPool.shutdownNow();
1029       LOG.debug("Still running " + runnables);
1030     }
1031     this.evictionThread.shutdown();
1032   }
1033 
1034   /** Clears the cache. Used in tests. */
1035   @VisibleForTesting
1036   public void clearCache() {
1037     this.map.clear();
1038     this.elements.set(0);
1039   }
1040 
1041   /**
1042    * Used in testing. May be very inefficient.
1043    * @return the set of cached file names
1044    */
1045   @VisibleForTesting
1046   SortedSet<String> getCachedFileNamesForTest() {
1047     SortedSet<String> fileNames = new TreeSet<String>();
1048     for (BlockCacheKey cacheKey : map.keySet()) {
1049       fileNames.add(cacheKey.getHfileName());
1050     }
1051     return fileNames;
1052   }
1053 
1054   @VisibleForTesting
1055   Map<BlockType, Integer> getBlockTypeCountsForTest() {
1056     Map<BlockType, Integer> counts =
1057         new EnumMap<BlockType, Integer>(BlockType.class);
1058     for (LruCachedBlock cb : map.values()) {
1059       BlockType blockType = ((Cacheable)cb.getBuffer()).getBlockType();
1060       Integer count = counts.get(blockType);
1061       counts.put(blockType, (count == null ? 0 : count) + 1);
1062     }
1063     return counts;
1064   }
1065 
1066   @VisibleForTesting
1067   public Map<DataBlockEncoding, Integer> getEncodingCountsForTest() {
1068     Map<DataBlockEncoding, Integer> counts =
1069         new EnumMap<DataBlockEncoding, Integer>(DataBlockEncoding.class);
1070     for (LruCachedBlock block : map.values()) {
1071       DataBlockEncoding encoding =
1072               ((HFileBlock) block.getBuffer()).getDataBlockEncoding();
1073       Integer count = counts.get(encoding);
1074       counts.put(encoding, (count == null ? 0 : count) + 1);
1075     }
1076     return counts;
1077   }
1078 
1079   public void setVictimCache(BlockCache handler) {
1080     assert victimHandler == null;
1081     victimHandler = handler;
1082   }
1083 
1084   @VisibleForTesting
1085   Map<BlockCacheKey, LruCachedBlock> getMapForTests() {
1086     return map;
1087   }
1088 
1089   BlockCache getVictimHandler() {
1090     return this.victimHandler;
1091   }
1092 
1093   @Override
1094   public BlockCache[] getBlockCaches() {
1095     return null;
1096   }
1097 }