View Javadoc

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