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.lang.ref.WeakReference;
23  import java.nio.ByteBuffer;
24  import java.util.LinkedList;
25  import java.util.PriorityQueue;
26  import java.util.concurrent.atomic.AtomicLong;
27  import java.util.concurrent.locks.ReentrantLock;
28  import java.util.concurrent.ConcurrentHashMap;
29  import java.util.concurrent.ScheduledExecutorService;
30  import java.util.concurrent.Executors;
31  import java.util.concurrent.TimeUnit;
32  
33  import org.apache.commons.logging.Log;
34  import org.apache.commons.logging.LogFactory;
35  import org.apache.hadoop.hbase.io.HeapSize;
36  import org.apache.hadoop.hbase.util.Bytes;
37  import org.apache.hadoop.hbase.util.ClassSize;
38  import org.apache.hadoop.util.StringUtils;
39  
40  /**
41   * A block cache implementation that is memory-aware using {@link HeapSize},
42   * memory-bound using an LRU eviction algorithm, and concurrent: backed by a
43   * {@link ConcurrentHashMap} and with a non-blocking eviction thread giving
44   * constant-time {@link #cacheBlock} and {@link #getBlock} operations.<p>
45   *
46   * Contains three levels of block priority to allow for
47   * scan-resistance and in-memory families.  A block is added with an inMemory
48   * flag if necessary, otherwise a block becomes a single access priority.  Once
49   * a blocked is accessed again, it changes to multiple access.  This is used
50   * to prevent scans from thrashing the cache, adding a least-frequently-used
51   * element to the eviction algorithm.<p>
52   *
53   * Each priority is given its own chunk of the total cache to ensure
54   * fairness during eviction.  Each priority will retain close to its maximum
55   * size, however, if any priority is not using its entire chunk the others
56   * are able to grow beyond their chunk size.<p>
57   *
58   * Instantiated at a minimum with the total size and average block size.
59   * All sizes are in bytes.  The block size is not especially important as this
60   * cache is fully dynamic in its sizing of blocks.  It is only used for
61   * pre-allocating data structures and in initial heap estimation of the map.<p>
62   *
63   * The detailed constructor defines the sizes for the three priorities (they
64   * should total to the maximum size defined).  It also sets the levels that
65   * trigger and control the eviction thread.<p>
66   *
67   * The acceptable size is the cache size level which triggers the eviction
68   * process to start.  It evicts enough blocks to get the size below the
69   * minimum size specified.<p>
70   *
71   * Eviction happens in a separate thread and involves a single full-scan
72   * of the map.  It determines how many bytes must be freed to reach the minimum
73   * size, and then while scanning determines the fewest least-recently-used
74   * blocks necessary from each of the three priorities (would be 3 times bytes
75   * to free).  It then uses the priority chunk sizes to evict fairly according
76   * to the relative sizes and usage.
77   */
78  public class LruBlockCache implements BlockCache, HeapSize {
79  
80    static final Log LOG = LogFactory.getLog(LruBlockCache.class);
81  
82    /** Default Configuration Parameters*/
83  
84    /** Backing Concurrent Map Configuration */
85    static final float DEFAULT_LOAD_FACTOR = 0.75f;
86    static final int DEFAULT_CONCURRENCY_LEVEL = 16;
87  
88    /** Eviction thresholds */
89    static final float DEFAULT_MIN_FACTOR = 0.75f;
90    static final float DEFAULT_ACCEPTABLE_FACTOR = 0.85f;
91  
92    /** Priority buckets */
93    static final float DEFAULT_SINGLE_FACTOR = 0.25f;
94    static final float DEFAULT_MULTI_FACTOR = 0.50f;
95    static final float DEFAULT_MEMORY_FACTOR = 0.25f;
96  
97    /** Statistics thread */
98    static final int statThreadPeriod = 60;
99  
100   /** Concurrent map (the cache) */
101   private final ConcurrentHashMap<String,CachedBlock> map;
102 
103   /** Eviction lock (locked when eviction in process) */
104   private final ReentrantLock evictionLock = new ReentrantLock(true);
105 
106   /** Volatile boolean to track if we are in an eviction process or not */
107   private volatile boolean evictionInProgress = false;
108 
109   /** Eviction thread */
110   private final EvictionThread evictionThread;
111 
112   /** Statistics thread schedule pool (for heavy debugging, could remove) */
113   private final ScheduledExecutorService scheduleThreadPool =
114     Executors.newScheduledThreadPool(1);
115 
116   /** Current size of cache */
117   private final AtomicLong size;
118 
119   /** Current number of cached elements */
120   private final AtomicLong elements;
121 
122   /** Cache access count (sequential ID) */
123   private final AtomicLong count;
124 
125   /** Cache statistics */
126   private final CacheStats stats;
127 
128   /** Maximum allowable size of cache (block put if size > max, evict) */
129   private long maxSize;
130 
131   /** Approximate block size */
132   private long blockSize;
133 
134   /** Acceptable size of cache (no evictions if size < acceptable) */
135   private float acceptableFactor;
136 
137   /** Minimum threshold of cache (when evicting, evict until size < min) */
138   private float minFactor;
139 
140   /** Single access bucket size */
141   private float singleFactor;
142 
143   /** Multiple access bucket size */
144   private float multiFactor;
145 
146   /** In-memory bucket size */
147   private float memoryFactor;
148 
149   /** Overhead of the structure itself */
150   private long overhead;
151 
152   /**
153    * Default constructor.  Specify maximum size and expected average block
154    * size (approximation is fine).
155    *
156    * <p>All other factors will be calculated based on defaults specified in
157    * this class.
158    * @param maxSize maximum size of cache, in bytes
159    * @param blockSize approximate size of each block, in bytes
160    */
161   public LruBlockCache(long maxSize, long blockSize) {
162     this(maxSize, blockSize, true);
163   }
164 
165   /**
166    * Constructor used for testing.  Allows disabling of the eviction thread.
167    */
168   public LruBlockCache(long maxSize, long blockSize, boolean evictionThread) {
169     this(maxSize, blockSize, evictionThread,
170         (int)Math.ceil(1.2*maxSize/blockSize),
171         DEFAULT_LOAD_FACTOR, DEFAULT_CONCURRENCY_LEVEL,
172         DEFAULT_MIN_FACTOR, DEFAULT_ACCEPTABLE_FACTOR,
173         DEFAULT_SINGLE_FACTOR, DEFAULT_MULTI_FACTOR,
174         DEFAULT_MEMORY_FACTOR);
175   }
176 
177   /**
178    * Configurable constructor.  Use this constructor if not using defaults.
179    * @param maxSize maximum size of this cache, in bytes
180    * @param blockSize expected average size of blocks, in bytes
181    * @param evictionThread whether to run evictions in a bg thread or not
182    * @param mapInitialSize initial size of backing ConcurrentHashMap
183    * @param mapLoadFactor initial load factor of backing ConcurrentHashMap
184    * @param mapConcurrencyLevel initial concurrency factor for backing CHM
185    * @param minFactor percentage of total size that eviction will evict until
186    * @param acceptableFactor percentage of total size that triggers eviction
187    * @param singleFactor percentage of total size for single-access blocks
188    * @param multiFactor percentage of total size for multiple-access blocks
189    * @param memoryFactor percentage of total size for in-memory blocks
190    */
191   public LruBlockCache(long maxSize, long blockSize, boolean evictionThread,
192       int mapInitialSize, float mapLoadFactor, int mapConcurrencyLevel,
193       float minFactor, float acceptableFactor,
194       float singleFactor, float multiFactor, float memoryFactor) {
195     if(singleFactor + multiFactor + memoryFactor != 1) {
196       throw new IllegalArgumentException("Single, multi, and memory factors " +
197           " should total 1.0");
198     }
199     if(minFactor >= acceptableFactor) {
200       throw new IllegalArgumentException("minFactor must be smaller than acceptableFactor");
201     }
202     if(minFactor >= 1.0f || acceptableFactor >= 1.0f) {
203       throw new IllegalArgumentException("all factors must be < 1");
204     }
205     this.maxSize = maxSize;
206     this.blockSize = blockSize;
207     map = new ConcurrentHashMap<String,CachedBlock>(mapInitialSize,
208         mapLoadFactor, mapConcurrencyLevel);
209     this.minFactor = minFactor;
210     this.acceptableFactor = acceptableFactor;
211     this.singleFactor = singleFactor;
212     this.multiFactor = multiFactor;
213     this.memoryFactor = memoryFactor;
214     this.stats = new CacheStats();
215     this.count = new AtomicLong(0);
216     this.elements = new AtomicLong(0);
217     this.overhead = calculateOverhead(maxSize, blockSize, mapConcurrencyLevel);
218     this.size = new AtomicLong(this.overhead);
219     if(evictionThread) {
220       this.evictionThread = new EvictionThread(this);
221       this.evictionThread.start(); // FindBugs SC_START_IN_CTOR
222     } else {
223       this.evictionThread = null;
224     }
225     this.scheduleThreadPool.scheduleAtFixedRate(new StatisticsThread(this),
226         statThreadPeriod, statThreadPeriod, TimeUnit.SECONDS);
227   }
228 
229   public void setMaxSize(long maxSize) {
230     this.maxSize = maxSize;
231     if(this.size.get() > acceptableSize() && !evictionInProgress) {
232       runEviction();
233     }
234   }
235 
236   // BlockCache implementation
237 
238   /**
239    * Cache the block with the specified name and buffer.
240    * <p>
241    * It is assumed this will NEVER be called on an already cached block.  If
242    * that is done, it is assumed that you are reinserting the same exact
243    * block due to a race condition and will update the buffer but not modify
244    * the size of the cache.
245    * @param blockName block name
246    * @param buf block buffer
247    * @param inMemory if block is in-memory
248    */
249   public void cacheBlock(String blockName, ByteBuffer buf, boolean inMemory) {
250     CachedBlock cb = map.get(blockName);
251     if(cb != null) {
252       throw new RuntimeException("Cached an already cached block");
253     }
254     cb = new CachedBlock(blockName, buf, count.incrementAndGet(), inMemory);
255     long newSize = size.addAndGet(cb.heapSize());
256     map.put(blockName, cb);
257     elements.incrementAndGet();
258     if(newSize > acceptableSize() && !evictionInProgress) {
259       runEviction();
260     }
261   }
262 
263   /**
264    * Cache the block with the specified name and buffer.
265    * <p>
266    * It is assumed this will NEVER be called on an already cached block.  If
267    * that is done, it is assumed that you are reinserting the same exact
268    * block due to a race condition and will update the buffer but not modify
269    * the size of the cache.
270    * @param blockName block name
271    * @param buf block buffer
272    */
273   public void cacheBlock(String blockName, ByteBuffer buf) {
274     cacheBlock(blockName, buf, false);
275   }
276 
277   /**
278    * Get the buffer of the block with the specified name.
279    * @param blockName block name
280    * @return buffer of specified block name, or null if not in cache
281    */
282   public ByteBuffer getBlock(String blockName) {
283     CachedBlock cb = map.get(blockName);
284     if(cb == null) {
285       stats.miss();
286       return null;
287     }
288     stats.hit();
289     cb.access(count.incrementAndGet());
290     return cb.getBuffer();
291   }
292 
293   protected long evictBlock(CachedBlock block) {
294     map.remove(block.getName());
295     size.addAndGet(-1 * block.heapSize());
296     elements.decrementAndGet();
297     stats.evicted();
298     return block.heapSize();
299   }
300 
301   /**
302    * Multi-threaded call to run the eviction process.
303    */
304   private void runEviction() {
305     if(evictionThread == null) {
306       evict();
307     } else {
308       evictionThread.evict();
309     }
310   }
311 
312   /**
313    * Eviction method.
314    */
315   void evict() {
316 
317     // Ensure only one eviction at a time
318     if(!evictionLock.tryLock()) return;
319 
320     try {
321       evictionInProgress = true;
322       long currentSize = this.size.get();
323       long bytesToFree = currentSize - minSize();
324 
325       if (LOG.isDebugEnabled()) {
326         LOG.debug("Block cache LRU eviction started; Attempting to free " +
327           StringUtils.byteDesc(bytesToFree) + " of total=" +
328           StringUtils.byteDesc(currentSize));
329       }
330 
331       if(bytesToFree <= 0) return;
332 
333       // Instantiate priority buckets
334       BlockBucket bucketSingle = new BlockBucket(bytesToFree, blockSize,
335           singleSize());
336       BlockBucket bucketMulti = new BlockBucket(bytesToFree, blockSize,
337           multiSize());
338       BlockBucket bucketMemory = new BlockBucket(bytesToFree, blockSize,
339           memorySize());
340 
341       // Scan entire map putting into appropriate buckets
342       for(CachedBlock cachedBlock : map.values()) {
343         switch(cachedBlock.getPriority()) {
344           case SINGLE: {
345             bucketSingle.add(cachedBlock);
346             break;
347           }
348           case MULTI: {
349             bucketMulti.add(cachedBlock);
350             break;
351           }
352           case MEMORY: {
353             bucketMemory.add(cachedBlock);
354             break;
355           }
356         }
357       }
358 
359       PriorityQueue<BlockBucket> bucketQueue =
360         new PriorityQueue<BlockBucket>(3);
361 
362       bucketQueue.add(bucketSingle);
363       bucketQueue.add(bucketMulti);
364       bucketQueue.add(bucketMemory);
365 
366       int remainingBuckets = 3;
367       long bytesFreed = 0;
368 
369       BlockBucket bucket;
370       while((bucket = bucketQueue.poll()) != null) {
371         long overflow = bucket.overflow();
372         if(overflow > 0) {
373           long bucketBytesToFree = Math.min(overflow,
374             (bytesToFree - bytesFreed) / remainingBuckets);
375           bytesFreed += bucket.free(bucketBytesToFree);
376         }
377         remainingBuckets--;
378       }
379 
380       if (LOG.isDebugEnabled()) {
381         long single = bucketSingle.totalSize();
382         long multi = bucketMulti.totalSize();
383         long memory = bucketMemory.totalSize();
384         LOG.debug("Block cache LRU eviction completed; " +
385           "freed=" + StringUtils.byteDesc(bytesFreed) + ", " +
386           "total=" + StringUtils.byteDesc(this.size.get()) + ", " +
387           "single=" + StringUtils.byteDesc(single) + ", " +
388           "multi=" + StringUtils.byteDesc(multi) + ", " +
389           "memory=" + StringUtils.byteDesc(memory));
390       }
391     } finally {
392       stats.evict();
393       evictionInProgress = false;
394       evictionLock.unlock();
395     }
396   }
397 
398   /**
399    * Used to group blocks into priority buckets.  There will be a BlockBucket
400    * for each priority (single, multi, memory).  Once bucketed, the eviction
401    * algorithm takes the appropriate number of elements out of each according
402    * to configuration parameters and their relatives sizes.
403    */
404   private class BlockBucket implements Comparable<BlockBucket> {
405 
406     private CachedBlockQueue queue;
407     private long totalSize = 0;
408     private long bucketSize;
409 
410     public BlockBucket(long bytesToFree, long blockSize, long bucketSize) {
411       this.bucketSize = bucketSize;
412       queue = new CachedBlockQueue(bytesToFree, blockSize);
413       totalSize = 0;
414     }
415 
416     public void add(CachedBlock block) {
417       totalSize += block.heapSize();
418       queue.add(block);
419     }
420 
421     public long free(long toFree) {
422       LinkedList<CachedBlock> blocks = queue.get();
423       long freedBytes = 0;
424       for(CachedBlock cb: blocks) {
425         freedBytes += evictBlock(cb);
426         if(freedBytes >= toFree) {
427           return freedBytes;
428         }
429       }
430       return freedBytes;
431     }
432 
433     public long overflow() {
434       return totalSize - bucketSize;
435     }
436 
437     public long totalSize() {
438       return totalSize;
439     }
440 
441     public int compareTo(BlockBucket that) {
442       if(this.overflow() == that.overflow()) return 0;
443       return this.overflow() > that.overflow() ? 1 : -1;
444     }
445   }
446 
447   /**
448    * Get the maximum size of this cache.
449    * @return max size in bytes
450    */
451   public long getMaxSize() {
452     return this.maxSize;
453   }
454 
455   /**
456    * Get the current size of this cache.
457    * @return current size in bytes
458    */
459   public long getCurrentSize() {
460     return this.size.get();
461   }
462 
463   /**
464    * Get the current size of this cache.
465    * @return current size in bytes
466    */
467   public long getFreeSize() {
468     return getMaxSize() - getCurrentSize();
469   }
470 
471   /**
472    * Get the size of this cache (number of cached blocks)
473    * @return number of cached blocks
474    */
475   public long size() {
476     return this.elements.get();
477   }
478 
479   /**
480    * Get the number of eviction runs that have occurred
481    */
482   public long getEvictionCount() {
483     return this.stats.getEvictionCount();
484   }
485 
486   /**
487    * Get the number of blocks that have been evicted during the lifetime
488    * of this cache.
489    */
490   public long getEvictedCount() {
491     return this.stats.getEvictedCount();
492   }
493 
494   /*
495    * Eviction thread.  Sits in waiting state until an eviction is triggered
496    * when the cache size grows above the acceptable level.<p>
497    *
498    * Thread is triggered into action by {@link LruBlockCache#runEviction()}
499    */
500   private static class EvictionThread extends Thread {
501     private WeakReference<LruBlockCache> cache;
502 
503     public EvictionThread(LruBlockCache cache) {
504       super("LruBlockCache.EvictionThread");
505       setDaemon(true);
506       this.cache = new WeakReference<LruBlockCache>(cache);
507     }
508 
509     @Override
510     public void run() {
511       while(true) {
512         synchronized(this) {
513           try {
514             this.wait();
515           } catch(InterruptedException e) {}
516         }
517         LruBlockCache cache = this.cache.get();
518         if(cache == null) break;
519         cache.evict();
520       }
521     }
522     public void evict() {
523       synchronized(this) {
524         this.notify(); // FindBugs NN_NAKED_NOTIFY
525       }
526     }
527   }
528 
529   /*
530    * Statistics thread.  Periodically prints the cache statistics to the log.
531    */
532   static class StatisticsThread extends Thread {
533     LruBlockCache lru;
534 
535     public StatisticsThread(LruBlockCache lru) {
536       super("LruBlockCache.StatisticsThread");
537       setDaemon(true);
538       this.lru = lru;
539     }
540     @Override
541     public void run() {
542       lru.logStats();
543     }
544   }
545 
546   public void logStats() {
547     if (!LOG.isDebugEnabled()) return;
548     // Log size
549     long totalSize = heapSize();
550     long freeSize = maxSize - totalSize;
551     LruBlockCache.LOG.debug("LRU Stats: " +
552         "total=" + StringUtils.byteDesc(totalSize) + ", " +
553         "free=" + StringUtils.byteDesc(freeSize) + ", " +
554         "max=" + StringUtils.byteDesc(this.maxSize) + ", " +
555         "blocks=" + size() +", " +
556         "accesses=" + stats.getRequestCount() + ", " +
557         "hits=" + stats.getHitCount() + ", " +
558         "hitRatio=" + StringUtils.formatPercent(stats.getHitRatio(), 2) + "%, " +
559         "evictions=" + stats.getEvictionCount() + ", " +
560         "evicted=" + stats.getEvictedCount() + ", " +
561         "evictedPerRun=" + stats.evictedPerEviction());
562   }
563 
564   /**
565    * Get counter statistics for this cache.
566    *
567    * <p>Includes: total accesses, hits, misses, evicted blocks, and runs
568    * of the eviction processes.
569    */
570   public CacheStats getStats() {
571     return this.stats;
572   }
573 
574   public static class CacheStats {
575     private final AtomicLong accessCount = new AtomicLong(0);
576     private final AtomicLong hitCount = new AtomicLong(0);
577     private final AtomicLong missCount = new AtomicLong(0);
578     private final AtomicLong evictionCount = new AtomicLong(0);
579     private final AtomicLong evictedCount = new AtomicLong(0);
580 
581     public void miss() {
582       missCount.incrementAndGet();
583       accessCount.incrementAndGet();
584     }
585 
586     public void hit() {
587       hitCount.incrementAndGet();
588       accessCount.incrementAndGet();
589     }
590 
591     public void evict() {
592       evictionCount.incrementAndGet();
593     }
594 
595     public void evicted() {
596       evictedCount.incrementAndGet();
597     }
598 
599     public long getRequestCount() {
600       return accessCount.get();
601     }
602 
603     public long getMissCount() {
604       return missCount.get();
605     }
606 
607     public long getHitCount() {
608       return hitCount.get();
609     }
610 
611     public long getEvictionCount() {
612       return evictionCount.get();
613     }
614 
615     public long getEvictedCount() {
616       return evictedCount.get();
617     }
618 
619     public double getHitRatio() {
620       return ((float)getHitCount()/(float)getRequestCount());
621     }
622 
623     public double getMissRatio() {
624       return ((float)getMissCount()/(float)getRequestCount());
625     }
626 
627     public double evictedPerEviction() {
628       return (float)((float)getEvictedCount()/(float)getEvictionCount());
629     }
630   }
631 
632   public final static long CACHE_FIXED_OVERHEAD = ClassSize.align(
633       (3 * Bytes.SIZEOF_LONG) + (8 * ClassSize.REFERENCE) +
634       (5 * Bytes.SIZEOF_FLOAT) + Bytes.SIZEOF_BOOLEAN
635       + ClassSize.OBJECT);
636 
637   // HeapSize implementation
638   public long heapSize() {
639     return getCurrentSize();
640   }
641 
642   public static long calculateOverhead(long maxSize, long blockSize, int concurrency){
643     // FindBugs ICAST_INTEGER_MULTIPLY_CAST_TO_LONG
644     return CACHE_FIXED_OVERHEAD + ClassSize.CONCURRENT_HASHMAP +
645         ((long)Math.ceil(maxSize*1.2/blockSize)
646             * ClassSize.CONCURRENT_HASHMAP_ENTRY) +
647         (concurrency * ClassSize.CONCURRENT_HASHMAP_SEGMENT);
648   }
649 
650   // Simple calculators of sizes given factors and maxSize
651 
652   private long acceptableSize() {
653     return (long)Math.floor(this.maxSize * this.acceptableFactor);
654   }
655   private long minSize() {
656     return (long)Math.floor(this.maxSize * this.minFactor);
657   }
658   private long singleSize() {
659     return (long)Math.floor(this.maxSize * this.singleFactor * this.minFactor);
660   }
661   private long multiSize() {
662     return (long)Math.floor(this.maxSize * this.multiFactor * this.minFactor);
663   }
664   private long memorySize() {
665     return (long)Math.floor(this.maxSize * this.memoryFactor * this.minFactor);
666   }
667 
668   public void shutdown() {
669     this.scheduleThreadPool.shutdown();
670   }
671 }