View Javadoc

1   /**
2    * Copyright 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  
16   * distributed under the License is distributed on an "AS IS" BASIS,
17   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
18   * See the License for the specific language governing permissions and
19   * limitations under the License.
20   */
21  package org.apache.hadoop.hbase.io.hfile.bucket;
22  
23  import java.io.File;
24  import java.io.FileInputStream;
25  import java.io.FileNotFoundException;
26  import java.io.FileOutputStream;
27  import java.io.IOException;
28  import java.io.ObjectInputStream;
29  import java.io.ObjectOutputStream;
30  import java.io.Serializable;
31  import java.nio.ByteBuffer;
32  import java.util.ArrayList;
33  import java.util.Comparator;
34  import java.util.Iterator;
35  import java.util.List;
36  import java.util.Map;
37  import java.util.PriorityQueue;
38  import java.util.Set;
39  import java.util.concurrent.ArrayBlockingQueue;
40  import java.util.concurrent.BlockingQueue;
41  import java.util.concurrent.ConcurrentHashMap;
42  import java.util.concurrent.ConcurrentMap;
43  import java.util.concurrent.Executors;
44  import java.util.concurrent.ScheduledExecutorService;
45  import java.util.concurrent.TimeUnit;
46  import java.util.concurrent.atomic.AtomicInteger;
47  import java.util.concurrent.atomic.AtomicLong;
48  import java.util.concurrent.locks.Lock;
49  import java.util.concurrent.locks.ReentrantLock;
50  import java.util.concurrent.locks.ReentrantReadWriteLock;
51  
52  import org.apache.commons.logging.Log;
53  import org.apache.commons.logging.LogFactory;
54  import org.apache.hadoop.hbase.classification.InterfaceAudience;
55  import org.apache.hadoop.hbase.io.HeapSize;
56  import org.apache.hadoop.hbase.io.hfile.BlockCache;
57  import org.apache.hadoop.hbase.io.hfile.BlockCacheKey;
58  import org.apache.hadoop.hbase.io.hfile.BlockCacheUtil;
59  import org.apache.hadoop.hbase.io.hfile.BlockPriority;
60  import org.apache.hadoop.hbase.io.hfile.BlockType;
61  import org.apache.hadoop.hbase.io.hfile.CacheStats;
62  import org.apache.hadoop.hbase.io.hfile.Cacheable;
63  import org.apache.hadoop.hbase.io.hfile.Cacheable.MemoryType;
64  import org.apache.hadoop.hbase.io.hfile.CacheableDeserializer;
65  import org.apache.hadoop.hbase.io.hfile.CacheableDeserializerIdManager;
66  import org.apache.hadoop.hbase.io.hfile.CachedBlock;
67  import org.apache.hadoop.hbase.io.hfile.HFileBlock;
68  import org.apache.hadoop.hbase.nio.ByteBuff;
69  import org.apache.hadoop.hbase.util.ConcurrentIndex;
70  import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
71  import org.apache.hadoop.hbase.util.HasThread;
72  import org.apache.hadoop.hbase.util.IdReadWriteLock;
73  import org.apache.hadoop.util.StringUtils;
74  
75  import com.google.common.annotations.VisibleForTesting;
76  import com.google.common.collect.ImmutableList;
77  import com.google.common.util.concurrent.ThreadFactoryBuilder;
78  
79  /**
80   * BucketCache uses {@link BucketAllocator} to allocate/free blocks, and uses
81   * {@link BucketCache#ramCache} and {@link BucketCache#backingMap} in order to
82   * determine if a given element is in the cache. The bucket cache can use on-heap or
83   * off-heap memory {@link ByteBufferIOEngine} or in a file {@link FileIOEngine} to
84   * store/read the block data.
85   *
86   * <p>Eviction is via a similar algorithm as used in
87   * {@link org.apache.hadoop.hbase.io.hfile.LruBlockCache}
88   *
89   * <p>BucketCache can be used as mainly a block cache (see
90   * {@link org.apache.hadoop.hbase.io.hfile.CombinedBlockCache}), combined with 
91   * LruBlockCache to decrease CMS GC and heap fragmentation.
92   *
93   * <p>It also can be used as a secondary cache (e.g. using a file on ssd/fusionio to store
94   * blocks) to enlarge cache space via
95   * {@link org.apache.hadoop.hbase.io.hfile.LruBlockCache#setVictimCache}
96   */
97  @InterfaceAudience.Private
98  public class BucketCache implements BlockCache, HeapSize {
99    private static final Log LOG = LogFactory.getLog(BucketCache.class);
100 
101   /** Priority buckets */
102   private static final float DEFAULT_SINGLE_FACTOR = 0.25f;
103   private static final float DEFAULT_MULTI_FACTOR = 0.50f;
104   private static final float DEFAULT_MEMORY_FACTOR = 0.25f;
105   private static final float DEFAULT_EXTRA_FREE_FACTOR = 0.10f;
106 
107   private static final float DEFAULT_ACCEPT_FACTOR = 0.95f;
108   private static final float DEFAULT_MIN_FACTOR = 0.85f;
109 
110   /** Statistics thread */
111   private static final int statThreadPeriod = 5 * 60;
112 
113   final static int DEFAULT_WRITER_THREADS = 3;
114   final static int DEFAULT_WRITER_QUEUE_ITEMS = 64;
115 
116   // Store/read block data
117   final IOEngine ioEngine;
118 
119   // Store the block in this map before writing it to cache
120   @VisibleForTesting
121   final ConcurrentMap<BlockCacheKey, RAMQueueEntry> ramCache;
122   // In this map, store the block's meta data like offset, length
123   @VisibleForTesting
124   ConcurrentMap<BlockCacheKey, BucketEntry> backingMap;
125 
126   /**
127    * Flag if the cache is enabled or not... We shut it off if there are IO
128    * errors for some time, so that Bucket IO exceptions/errors don't bring down
129    * the HBase server.
130    */
131   private volatile boolean cacheEnabled;
132 
133   /**
134    * A list of writer queues.  We have a queue per {@link WriterThread} we have running.
135    * In other words, the work adding blocks to the BucketCache is divided up amongst the
136    * running WriterThreads.  Its done by taking hash of the cache key modulo queue count.
137    * WriterThread when it runs takes whatever has been recently added and 'drains' the entries
138    * to the BucketCache.  It then updates the ramCache and backingMap accordingly.
139    */
140   @VisibleForTesting
141   final ArrayList<BlockingQueue<RAMQueueEntry>> writerQueues =
142       new ArrayList<BlockingQueue<RAMQueueEntry>>();
143   @VisibleForTesting
144   final WriterThread[] writerThreads;
145 
146   /** Volatile boolean to track if free space is in process or not */
147   private volatile boolean freeInProgress = false;
148   private final Lock freeSpaceLock = new ReentrantLock();
149 
150   private UniqueIndexMap<Integer> deserialiserMap = new UniqueIndexMap<Integer>();
151 
152   private final AtomicLong realCacheSize = new AtomicLong(0);
153   private final AtomicLong heapSize = new AtomicLong(0);
154   /** Current number of cached elements */
155   private final AtomicLong blockNumber = new AtomicLong(0);
156 
157   /** Cache access count (sequential ID) */
158   private final AtomicLong accessCount = new AtomicLong(0);
159 
160   private static final int DEFAULT_CACHE_WAIT_TIME = 50;
161   // Used in test now. If the flag is false and the cache speed is very fast,
162   // bucket cache will skip some blocks when caching. If the flag is true, we
163   // will wait blocks flushed to IOEngine for some time when caching
164   boolean wait_when_cache = false;
165 
166   private final BucketCacheStats cacheStats = new BucketCacheStats();
167 
168   private final String persistencePath;
169   private final long cacheCapacity;
170   /** Approximate block size */
171   private final long blockSize;
172 
173   /** Duration of IO errors tolerated before we disable cache, 1 min as default */
174   private final int ioErrorsTolerationDuration;
175   // 1 min
176   public static final int DEFAULT_ERROR_TOLERATION_DURATION = 60 * 1000;
177 
178   // Start time of first IO error when reading or writing IO Engine, it will be
179   // reset after a successful read/write.
180   private volatile long ioErrorStartTime = -1;
181 
182   /**
183    * A ReentrantReadWriteLock to lock on a particular block identified by offset.
184    * The purpose of this is to avoid freeing the block which is being read.
185    */
186   @VisibleForTesting
187   final IdReadWriteLock offsetLock = new IdReadWriteLock();
188 
189   private final ConcurrentIndex<String, BlockCacheKey> blocksByHFile =
190       new ConcurrentIndex<String, BlockCacheKey>(new Comparator<BlockCacheKey>() {
191         @Override
192         public int compare(BlockCacheKey a, BlockCacheKey b) {
193           if (a.getOffset() == b.getOffset()) {
194             return 0;
195           } else if (a.getOffset() < b.getOffset()) {
196             return -1;
197           }
198           return 1;
199         }
200       });
201 
202   /** Statistics thread schedule pool (for heavy debugging, could remove) */
203   private final ScheduledExecutorService scheduleThreadPool = Executors.newScheduledThreadPool(1,
204     new ThreadFactoryBuilder().setNameFormat("BucketCacheStatsExecutor").setDaemon(true).build());
205 
206   // Allocate or free space for the block
207   private BucketAllocator bucketAllocator;
208 
209   public BucketCache(String ioEngineName, long capacity, int blockSize, int[] bucketSizes,
210       int writerThreadNum, int writerQLen, String persistencePath) throws FileNotFoundException,
211       IOException {
212     this(ioEngineName, capacity, blockSize, bucketSizes, writerThreadNum, writerQLen,
213       persistencePath, DEFAULT_ERROR_TOLERATION_DURATION);
214   }
215 
216   public BucketCache(String ioEngineName, long capacity, int blockSize, int[] bucketSizes,
217       int writerThreadNum, int writerQLen, String persistencePath, int ioErrorsTolerationDuration)
218       throws FileNotFoundException, IOException {
219     this.ioEngine = getIOEngineFromName(ioEngineName, capacity);
220     this.writerThreads = new WriterThread[writerThreadNum];
221     long blockNumCapacity = capacity / blockSize;
222     if (blockNumCapacity >= Integer.MAX_VALUE) {
223       // Enough for about 32TB of cache!
224       throw new IllegalArgumentException("Cache capacity is too large, only support 32TB now");
225     }
226 
227     this.cacheCapacity = capacity;
228     this.persistencePath = persistencePath;
229     this.blockSize = blockSize;
230     this.ioErrorsTolerationDuration = ioErrorsTolerationDuration;
231 
232     bucketAllocator = new BucketAllocator(capacity, bucketSizes);
233     for (int i = 0; i < writerThreads.length; ++i) {
234       writerQueues.add(new ArrayBlockingQueue<RAMQueueEntry>(writerQLen));
235     }
236 
237     assert writerQueues.size() == writerThreads.length;
238     this.ramCache = new ConcurrentHashMap<BlockCacheKey, RAMQueueEntry>();
239 
240     this.backingMap = new ConcurrentHashMap<BlockCacheKey, BucketEntry>((int) blockNumCapacity);
241 
242     if (ioEngine.isPersistent() && persistencePath != null) {
243       try {
244         retrieveFromFile(bucketSizes);
245       } catch (IOException ioex) {
246         LOG.error("Can't restore from file because of", ioex);
247       } catch (ClassNotFoundException cnfe) {
248         LOG.error("Can't restore from file in rebuild because can't deserialise",cnfe);
249         throw new RuntimeException(cnfe);
250       }
251     }
252     final String threadName = Thread.currentThread().getName();
253     this.cacheEnabled = true;
254     for (int i = 0; i < writerThreads.length; ++i) {
255       writerThreads[i] = new WriterThread(writerQueues.get(i));
256       writerThreads[i].setName(threadName + "-BucketCacheWriter-" + i);
257       writerThreads[i].setDaemon(true);
258     }
259     startWriterThreads();
260 
261     // Run the statistics thread periodically to print the cache statistics log
262     // TODO: Add means of turning this off.  Bit obnoxious running thread just to make a log
263     // every five minutes.
264     this.scheduleThreadPool.scheduleAtFixedRate(new StatisticsThread(this),
265         statThreadPeriod, statThreadPeriod, TimeUnit.SECONDS);
266     LOG.info("Started bucket cache; ioengine=" + ioEngineName +
267         ", capacity=" + StringUtils.byteDesc(capacity) +
268       ", blockSize=" + StringUtils.byteDesc(blockSize) + ", writerThreadNum=" +
269         writerThreadNum + ", writerQLen=" + writerQLen + ", persistencePath=" +
270       persistencePath + ", bucketAllocator=" + this.bucketAllocator.getClass().getName());
271   }
272 
273   /**
274    * Called by the constructor to start the writer threads. Used by tests that need to override
275    * starting the threads.
276    */
277   @VisibleForTesting
278   protected void startWriterThreads() {
279     for (WriterThread thread : writerThreads) {
280       thread.start();
281     }
282   }
283 
284   @VisibleForTesting
285   boolean isCacheEnabled() {
286     return this.cacheEnabled;
287   }
288 
289   public long getMaxSize() {
290     return this.cacheCapacity;
291   }
292 
293   public String getIoEngine() {
294     return ioEngine.toString();
295   }
296 
297   /**
298    * Get the IOEngine from the IO engine name
299    * @param ioEngineName
300    * @param capacity
301    * @return the IOEngine
302    * @throws IOException
303    */
304   private IOEngine getIOEngineFromName(String ioEngineName, long capacity)
305       throws IOException {
306     if (ioEngineName.startsWith("file:"))
307       return new FileIOEngine(ioEngineName.substring(5), capacity);
308     else if (ioEngineName.startsWith("offheap"))
309       return new ByteBufferIOEngine(capacity, true);
310     else if (ioEngineName.startsWith("heap"))
311       return new ByteBufferIOEngine(capacity, false);
312     else
313       throw new IllegalArgumentException(
314           "Don't understand io engine name for cache - prefix with file:, heap or offheap");
315   }
316 
317   /**
318    * Cache the block with the specified name and buffer.
319    * @param cacheKey block's cache key
320    * @param buf block buffer
321    */
322   @Override
323   public void cacheBlock(BlockCacheKey cacheKey, Cacheable buf) {
324     cacheBlock(cacheKey, buf, false, false);
325   }
326 
327   /**
328    * Cache the block with the specified name and buffer.
329    * @param cacheKey block's cache key
330    * @param cachedItem block buffer
331    * @param inMemory if block is in-memory
332    * @param cacheDataInL1
333    */
334   @Override
335   public void cacheBlock(BlockCacheKey cacheKey, Cacheable cachedItem, boolean inMemory,
336       final boolean cacheDataInL1) {
337     cacheBlockWithWait(cacheKey, cachedItem, inMemory, wait_when_cache);
338   }
339 
340   /**
341    * Cache the block to ramCache
342    * @param cacheKey block's cache key
343    * @param cachedItem block buffer
344    * @param inMemory if block is in-memory
345    * @param wait if true, blocking wait when queue is full
346    */
347   public void cacheBlockWithWait(BlockCacheKey cacheKey, Cacheable cachedItem, boolean inMemory,
348       boolean wait) {
349     if (!cacheEnabled) {
350       return;
351     }
352 
353     if (backingMap.containsKey(cacheKey)) {
354       return;
355     }
356 
357     /*
358      * Stuff the entry into the RAM cache so it can get drained to the persistent store
359      */
360     RAMQueueEntry re =
361         new RAMQueueEntry(cacheKey, cachedItem, accessCount.incrementAndGet(), inMemory);
362     if (ramCache.putIfAbsent(cacheKey, re) != null) {
363       return;
364     }
365     int queueNum = (cacheKey.hashCode() & 0x7FFFFFFF) % writerQueues.size();
366     BlockingQueue<RAMQueueEntry> bq = writerQueues.get(queueNum);
367     boolean successfulAddition = false;
368     if (wait) {
369       try {
370         successfulAddition = bq.offer(re, DEFAULT_CACHE_WAIT_TIME, TimeUnit.MILLISECONDS);
371       } catch (InterruptedException e) {
372         Thread.currentThread().interrupt();
373       }
374     } else {
375       successfulAddition = bq.offer(re);
376     }
377     if (!successfulAddition) {
378       ramCache.remove(cacheKey);
379       cacheStats.failInsert();
380     } else {
381       this.blockNumber.incrementAndGet();
382       this.heapSize.addAndGet(cachedItem.heapSize());
383       blocksByHFile.put(cacheKey.getHfileName(), cacheKey);
384     }
385   }
386 
387   /**
388    * Get the buffer of the block with the specified key.
389    * @param key block's cache key
390    * @param caching true if the caller caches blocks on cache misses
391    * @param repeat Whether this is a repeat lookup for the same block
392    * @param updateCacheMetrics Whether we should update cache metrics or not
393    * @return buffer of specified cache key, or null if not in cache
394    */
395   @Override
396   public Cacheable getBlock(BlockCacheKey key, boolean caching, boolean repeat,
397       boolean updateCacheMetrics) {
398     if (!cacheEnabled) {
399       return null;
400     }
401     RAMQueueEntry re = ramCache.get(key);
402     if (re != null) {
403       if (updateCacheMetrics) {
404         cacheStats.hit(caching, key.isPrimary());
405       }
406       re.access(accessCount.incrementAndGet());
407       return re.getData();
408     }
409     BucketEntry bucketEntry = backingMap.get(key);
410     if (bucketEntry != null) {
411       long start = System.nanoTime();
412       ReentrantReadWriteLock lock = offsetLock.getLock(bucketEntry.offset());
413       try {
414         lock.readLock().lock();
415         // We can not read here even if backingMap does contain the given key because its offset
416         // maybe changed. If we lock BlockCacheKey instead of offset, then we can only check
417         // existence here.
418         if (bucketEntry.equals(backingMap.get(key))) {
419           // TODO : change this area - should be removed after server cells and
420           // 12295 are available
421           int len = bucketEntry.getLength();
422           Cacheable cachedBlock = ioEngine.read(bucketEntry.offset(), len,
423               bucketEntry.deserializerReference(this.deserialiserMap));
424           long timeTaken = System.nanoTime() - start;
425           if (updateCacheMetrics) {
426             cacheStats.hit(caching, key.isPrimary());
427             cacheStats.ioHit(timeTaken);
428           }
429           if (cachedBlock.getMemoryType() == MemoryType.SHARED) {
430             bucketEntry.refCount.incrementAndGet();
431           }
432           bucketEntry.access(accessCount.incrementAndGet());
433           if (this.ioErrorStartTime > 0) {
434             ioErrorStartTime = -1;
435           }
436           return cachedBlock;
437         }
438       } catch (IOException ioex) {
439         LOG.error("Failed reading block " + key + " from bucket cache", ioex);
440         checkIOErrorIsTolerated();
441       } finally {
442         lock.readLock().unlock();
443       }
444     }
445     if (!repeat && updateCacheMetrics) {
446       cacheStats.miss(caching, key.isPrimary());
447     }
448     return null;
449   }
450 
451   @VisibleForTesting
452   void blockEvicted(BlockCacheKey cacheKey, BucketEntry bucketEntry, boolean decrementBlockNumber) {
453     bucketAllocator.freeBlock(bucketEntry.offset());
454     realCacheSize.addAndGet(-1 * bucketEntry.getLength());
455     blocksByHFile.remove(cacheKey.getHfileName(), cacheKey);
456     if (decrementBlockNumber) {
457       this.blockNumber.decrementAndGet();
458     }
459   }
460 
461   @Override
462   public boolean evictBlock(BlockCacheKey cacheKey) {
463     return evictBlock(cacheKey, true);
464   }
465 
466   // does not check for the ref count. Just tries to evict it if found in the
467   // bucket map
468   private boolean forceEvict(BlockCacheKey cacheKey) {
469     if (!cacheEnabled) {
470       return false;
471     }
472     RAMQueueEntry removedBlock = checkRamCache(cacheKey);
473     BucketEntry bucketEntry = backingMap.get(cacheKey);
474     if (bucketEntry == null) {
475       if (removedBlock != null) {
476         cacheStats.evicted(0, cacheKey.isPrimary());
477         return true;
478       } else {
479         return false;
480       }
481     }
482     ReentrantReadWriteLock lock = offsetLock.getLock(bucketEntry.offset());
483     try {
484       lock.writeLock().lock();
485       if (backingMap.remove(cacheKey, bucketEntry)) {
486         blockEvicted(cacheKey, bucketEntry, removedBlock == null);
487       } else {
488         return false;
489       }
490     } finally {
491       lock.writeLock().unlock();
492     }
493     cacheStats.evicted(bucketEntry.getCachedTime(), cacheKey.isPrimary());
494     return true;
495   }
496 
497   private RAMQueueEntry checkRamCache(BlockCacheKey cacheKey) {
498     RAMQueueEntry removedBlock = ramCache.remove(cacheKey);
499     if (removedBlock != null) {
500       this.blockNumber.decrementAndGet();
501       this.heapSize.addAndGet(-1 * removedBlock.getData().heapSize());
502     }
503     return removedBlock;
504   }
505 
506   public boolean evictBlock(BlockCacheKey cacheKey, boolean deletedBlock) {
507     if (!cacheEnabled) {
508       return false;
509     }
510     RAMQueueEntry removedBlock = checkRamCache(cacheKey);
511     BucketEntry bucketEntry = backingMap.get(cacheKey);
512     if (bucketEntry == null) {
513       if (removedBlock != null) {
514         cacheStats.evicted(0, cacheKey.isPrimary());
515         return true;
516       } else {
517         return false;
518       }
519     }
520     ReentrantReadWriteLock lock = offsetLock.getLock(bucketEntry.offset());
521     try {
522       lock.writeLock().lock();
523       int refCount = bucketEntry.refCount.get();
524       if(refCount == 0) {
525         if (backingMap.remove(cacheKey, bucketEntry)) {
526           blockEvicted(cacheKey, bucketEntry, removedBlock == null);
527         } else {
528           return false;
529         }
530       } else {
531         if(!deletedBlock) {
532           if (LOG.isDebugEnabled()) {
533             LOG.debug("This block " + cacheKey + " is still referred by " + refCount
534                 + " readers. Can not be freed now");
535           }
536           return false;
537         } else {
538           if (LOG.isDebugEnabled()) {
539             LOG.debug("This block " + cacheKey + " is still referred by " + refCount
540                 + " readers. Can not be freed now. Hence will mark this"
541                 + " for evicting at a later point");
542           }
543           bucketEntry.markedForEvict = true;
544         }
545       }
546     } finally {
547       lock.writeLock().unlock();
548     }
549     cacheStats.evicted(bucketEntry.getCachedTime(), cacheKey.isPrimary());
550     return true;
551   }
552 
553   /*
554    * Statistics thread.  Periodically output cache statistics to the log.
555    */
556   private static class StatisticsThread extends Thread {
557     private final BucketCache bucketCache;
558 
559     public StatisticsThread(BucketCache bucketCache) {
560       super("BucketCacheStatsThread");
561       setDaemon(true);
562       this.bucketCache = bucketCache;
563     }
564 
565     @Override
566     public void run() {
567       bucketCache.logStats();
568     }
569   }
570 
571   public void logStats() {
572     long totalSize = bucketAllocator.getTotalSize();
573     long usedSize = bucketAllocator.getUsedSize();
574     long freeSize = totalSize - usedSize;
575     long cacheSize = getRealCacheSize();
576     LOG.info("failedBlockAdditions=" + cacheStats.getFailedInserts() + ", " +
577         "totalSize=" + StringUtils.byteDesc(totalSize) + ", " +
578         "freeSize=" + StringUtils.byteDesc(freeSize) + ", " +
579         "usedSize=" + StringUtils.byteDesc(usedSize) +", " +
580         "cacheSize=" + StringUtils.byteDesc(cacheSize) +", " +
581         "accesses=" + cacheStats.getRequestCount() + ", " +
582         "hits=" + cacheStats.getHitCount() + ", " +
583         "IOhitsPerSecond=" + cacheStats.getIOHitsPerSecond() + ", " +
584         "IOTimePerHit=" + String.format("%.2f", cacheStats.getIOTimePerHit())+ ", " +
585         "hitRatio=" + (cacheStats.getHitCount() == 0 ? "0," :
586           (StringUtils.formatPercent(cacheStats.getHitRatio(), 2)+ ", ")) +
587         "cachingAccesses=" + cacheStats.getRequestCachingCount() + ", " +
588         "cachingHits=" + cacheStats.getHitCachingCount() + ", " +
589         "cachingHitsRatio=" +(cacheStats.getHitCachingCount() == 0 ? "0," :
590           (StringUtils.formatPercent(cacheStats.getHitCachingRatio(), 2)+ ", ")) +
591         "evictions=" + cacheStats.getEvictionCount() + ", " +
592         "evicted=" + cacheStats.getEvictedCount() + ", " +
593         "evictedPerRun=" + cacheStats.evictedPerEviction());
594     cacheStats.reset();
595   }
596 
597   public long getRealCacheSize() {
598     return this.realCacheSize.get();
599   }
600 
601   private long acceptableSize() {
602     return (long) Math.floor(bucketAllocator.getTotalSize() * DEFAULT_ACCEPT_FACTOR);
603   }
604 
605   private long singleSize() {
606     return (long) Math.floor(bucketAllocator.getTotalSize()
607         * DEFAULT_SINGLE_FACTOR * DEFAULT_MIN_FACTOR);
608   }
609 
610   private long multiSize() {
611     return (long) Math.floor(bucketAllocator.getTotalSize() * DEFAULT_MULTI_FACTOR
612         * DEFAULT_MIN_FACTOR);
613   }
614 
615   private long memorySize() {
616     return (long) Math.floor(bucketAllocator.getTotalSize() * DEFAULT_MEMORY_FACTOR
617         * DEFAULT_MIN_FACTOR);
618   }
619 
620   /**
621    * Free the space if the used size reaches acceptableSize() or one size block
622    * couldn't be allocated. When freeing the space, we use the LRU algorithm and
623    * ensure there must be some blocks evicted
624    * @param why Why we are being called
625    */
626   private void freeSpace(final String why) {
627     // Ensure only one freeSpace progress at a time
628     if (!freeSpaceLock.tryLock()) return;
629     try {
630       freeInProgress = true;
631       long bytesToFreeWithoutExtra = 0;
632       // Calculate free byte for each bucketSizeinfo
633       StringBuffer msgBuffer = LOG.isDebugEnabled()? new StringBuffer(): null;
634       BucketAllocator.IndexStatistics[] stats = bucketAllocator.getIndexStatistics();
635       long[] bytesToFreeForBucket = new long[stats.length];
636       for (int i = 0; i < stats.length; i++) {
637         bytesToFreeForBucket[i] = 0;
638         long freeGoal = (long) Math.floor(stats[i].totalCount() * (1 - DEFAULT_MIN_FACTOR));
639         freeGoal = Math.max(freeGoal, 1);
640         if (stats[i].freeCount() < freeGoal) {
641           bytesToFreeForBucket[i] = stats[i].itemSize() * (freeGoal - stats[i].freeCount());
642           bytesToFreeWithoutExtra += bytesToFreeForBucket[i];
643           if (msgBuffer != null) {
644             msgBuffer.append("Free for bucketSize(" + stats[i].itemSize() + ")="
645               + StringUtils.byteDesc(bytesToFreeForBucket[i]) + ", ");
646           }
647         }
648       }
649       if (msgBuffer != null) {
650         msgBuffer.append("Free for total=" + StringUtils.byteDesc(bytesToFreeWithoutExtra) + ", ");
651       }
652 
653       if (bytesToFreeWithoutExtra <= 0) {
654         return;
655       }
656       long currentSize = bucketAllocator.getUsedSize();
657       long totalSize=bucketAllocator.getTotalSize();
658       if (LOG.isDebugEnabled() && msgBuffer != null) {
659         LOG.debug("Free started because \"" + why + "\"; " + msgBuffer.toString() +
660           " of current used=" + StringUtils.byteDesc(currentSize) + ", actual cacheSize=" +
661           StringUtils.byteDesc(realCacheSize.get()) + ", total=" + StringUtils.byteDesc(totalSize));
662       }
663 
664       long bytesToFreeWithExtra = (long) Math.floor(bytesToFreeWithoutExtra
665           * (1 + DEFAULT_EXTRA_FREE_FACTOR));
666 
667       // Instantiate priority buckets
668       BucketEntryGroup bucketSingle = new BucketEntryGroup(bytesToFreeWithExtra,
669           blockSize, singleSize());
670       BucketEntryGroup bucketMulti = new BucketEntryGroup(bytesToFreeWithExtra,
671           blockSize, multiSize());
672       BucketEntryGroup bucketMemory = new BucketEntryGroup(bytesToFreeWithExtra,
673           blockSize, memorySize());
674 
675       // Scan entire map putting bucket entry into appropriate bucket entry
676       // group
677       for (Map.Entry<BlockCacheKey, BucketEntry> bucketEntryWithKey : backingMap.entrySet()) {
678         switch (bucketEntryWithKey.getValue().getPriority()) {
679           case SINGLE: {
680             bucketSingle.add(bucketEntryWithKey);
681             break;
682           }
683           case MULTI: {
684             bucketMulti.add(bucketEntryWithKey);
685             break;
686           }
687           case MEMORY: {
688             bucketMemory.add(bucketEntryWithKey);
689             break;
690           }
691         }
692       }
693 
694       PriorityQueue<BucketEntryGroup> bucketQueue = new PriorityQueue<BucketEntryGroup>(3);
695 
696       bucketQueue.add(bucketSingle);
697       bucketQueue.add(bucketMulti);
698       bucketQueue.add(bucketMemory);
699 
700       int remainingBuckets = 3;
701       long bytesFreed = 0;
702 
703       BucketEntryGroup bucketGroup;
704       while ((bucketGroup = bucketQueue.poll()) != null) {
705         long overflow = bucketGroup.overflow();
706         if (overflow > 0) {
707           long bucketBytesToFree = Math.min(overflow,
708               (bytesToFreeWithoutExtra - bytesFreed) / remainingBuckets);
709           bytesFreed += bucketGroup.free(bucketBytesToFree);
710         }
711         remainingBuckets--;
712       }
713 
714       /**
715        * Check whether need extra free because some bucketSizeinfo still needs
716        * free space
717        */
718       stats = bucketAllocator.getIndexStatistics();
719       boolean needFreeForExtra = false;
720       for (int i = 0; i < stats.length; i++) {
721         long freeGoal = (long) Math.floor(stats[i].totalCount() * (1 - DEFAULT_MIN_FACTOR));
722         freeGoal = Math.max(freeGoal, 1);
723         if (stats[i].freeCount() < freeGoal) {
724           needFreeForExtra = true;
725           break;
726         }
727       }
728 
729       if (needFreeForExtra) {
730         bucketQueue.clear();
731         remainingBuckets = 2;
732 
733         bucketQueue.add(bucketSingle);
734         bucketQueue.add(bucketMulti);
735 
736         while ((bucketGroup = bucketQueue.poll()) != null) {
737           long bucketBytesToFree = (bytesToFreeWithExtra - bytesFreed) / remainingBuckets;
738           bytesFreed += bucketGroup.free(bucketBytesToFree);
739           remainingBuckets--;
740         }
741       }
742 
743       if (LOG.isDebugEnabled()) {
744         long single = bucketSingle.totalSize();
745         long multi = bucketMulti.totalSize();
746         long memory = bucketMemory.totalSize();
747         if (LOG.isDebugEnabled()) {
748           LOG.debug("Bucket cache free space completed; " + "freed="
749             + StringUtils.byteDesc(bytesFreed) + ", " + "total="
750             + StringUtils.byteDesc(totalSize) + ", " + "single="
751             + StringUtils.byteDesc(single) + ", " + "multi="
752             + StringUtils.byteDesc(multi) + ", " + "memory="
753             + StringUtils.byteDesc(memory));
754         }
755       }
756 
757     } catch (Throwable t) {
758       LOG.warn("Failed freeing space", t);
759     } finally {
760       cacheStats.evict();
761       freeInProgress = false;
762       freeSpaceLock.unlock();
763     }
764   }
765 
766   // This handles flushing the RAM cache to IOEngine.
767   @VisibleForTesting
768   class WriterThread extends HasThread {
769     private final BlockingQueue<RAMQueueEntry> inputQueue;
770     private volatile boolean writerEnabled = true;
771 
772     WriterThread(BlockingQueue<RAMQueueEntry> queue) {
773       this.inputQueue = queue;
774     }
775 
776     // Used for test
777     @VisibleForTesting
778     void disableWriter() {
779       this.writerEnabled = false;
780     }
781 
782     public void run() {
783       List<RAMQueueEntry> entries = new ArrayList<RAMQueueEntry>();
784       try {
785         while (cacheEnabled && writerEnabled) {
786           try {
787             try {
788               // Blocks
789               entries = getRAMQueueEntries(inputQueue, entries);
790             } catch (InterruptedException ie) {
791               if (!cacheEnabled) break;
792             }
793             doDrain(entries);
794           } catch (Exception ioe) {
795             LOG.error("WriterThread encountered error", ioe);
796           }
797         }
798       } catch (Throwable t) {
799         LOG.warn("Failed doing drain", t);
800       }
801       LOG.info(this.getName() + " exiting, cacheEnabled=" + cacheEnabled);
802     }
803 
804     /**
805      * Flush the entries in ramCache to IOEngine and add bucket entry to backingMap.
806      * Process all that are passed in even if failure being sure to remove from ramCache else we'll
807      * never undo the references and we'll OOME.
808      * @param entries Presumes list passed in here will be processed by this invocation only. No
809      *   interference expected.
810      * @throws InterruptedException
811      */
812     @VisibleForTesting
813     void doDrain(final List<RAMQueueEntry> entries) throws InterruptedException {
814       if (entries.isEmpty()) {
815         return;
816       }
817       // This method is a little hard to follow. We run through the passed in entries and for each
818       // successful add, we add a non-null BucketEntry to the below bucketEntries.  Later we must
819       // do cleanup making sure we've cleared ramCache of all entries regardless of whether we
820       // successfully added the item to the bucketcache; if we don't do the cleanup, we'll OOME by
821       // filling ramCache.  We do the clean up by again running through the passed in entries
822       // doing extra work when we find a non-null bucketEntries corresponding entry.
823       final int size = entries.size();
824       BucketEntry[] bucketEntries = new BucketEntry[size];
825       // Index updated inside loop if success or if we can't succeed. We retry if cache is full
826       // when we go to add an entry by going around the loop again without upping the index.
827       int index = 0;
828       while (cacheEnabled && index < size) {
829         RAMQueueEntry re = null;
830         try {
831           re = entries.get(index);
832           if (re == null) {
833             LOG.warn("Couldn't get entry or changed on us; who else is messing with it?");
834             index++;
835             continue;
836           }
837           BucketEntry bucketEntry =
838             re.writeToCache(ioEngine, bucketAllocator, deserialiserMap, realCacheSize);
839           // Successfully added.  Up index and add bucketEntry. Clear io exceptions.
840           bucketEntries[index] = bucketEntry;
841           if (ioErrorStartTime > 0) {
842             ioErrorStartTime = -1;
843           }
844           index++;
845         } catch (BucketAllocatorException fle) {
846           LOG.warn("Failed allocation for " + (re == null ? "" : re.getKey()) + "; " + fle);
847           // Presume can't add. Too big? Move index on. Entry will be cleared from ramCache below.
848           bucketEntries[index] = null;
849           index++;
850         } catch (CacheFullException cfe) {
851           // Cache full when we tried to add. Try freeing space and then retrying (don't up index)
852           if (!freeInProgress) {
853             freeSpace("Full!");
854           } else {
855             Thread.sleep(50);
856           }
857         } catch (IOException ioex) {
858           // Hopefully transient. Retry. checkIOErrorIsTolerated disables cache if problem.
859           LOG.error("Failed writing to bucket cache", ioex);
860           checkIOErrorIsTolerated();
861         }
862       }
863 
864       // Make sure data pages are written are on media before we update maps.
865       try {
866         ioEngine.sync();
867       } catch (IOException ioex) {
868         LOG.error("Failed syncing IO engine", ioex);
869         checkIOErrorIsTolerated();
870         // Since we failed sync, free the blocks in bucket allocator
871         for (int i = 0; i < entries.size(); ++i) {
872           if (bucketEntries[i] != null) {
873             bucketAllocator.freeBlock(bucketEntries[i].offset());
874             bucketEntries[i] = null;
875           }
876         }
877       }
878 
879       // Now add to backingMap if successfully added to bucket cache.  Remove from ramCache if
880       // success or error.
881       for (int i = 0; i < size; ++i) {
882         BlockCacheKey key = entries.get(i).getKey();
883         // Only add if non-null entry.
884         if (bucketEntries[i] != null) {
885           backingMap.put(key, bucketEntries[i]);
886         }
887         // Always remove from ramCache even if we failed adding it to the block cache above.
888         RAMQueueEntry ramCacheEntry = ramCache.remove(key);
889         if (ramCacheEntry != null) {
890           heapSize.addAndGet(-1 * entries.get(i).getData().heapSize());
891         } else if (bucketEntries[i] != null){
892           // Block should have already been evicted. Remove it and free space.
893           ReentrantReadWriteLock lock = offsetLock.getLock(bucketEntries[i].offset());
894           try {
895             lock.writeLock().lock();
896             if (backingMap.remove(key, bucketEntries[i])) {
897               blockEvicted(key, bucketEntries[i], false);
898             }
899           } finally {
900             lock.writeLock().unlock();
901           }
902         }
903       }
904 
905       long used = bucketAllocator.getUsedSize();
906       if (used > acceptableSize()) {
907         freeSpace("Used=" + used + " > acceptable=" + acceptableSize());
908       }
909       return;
910     }
911   }
912 
913   /**
914    * Blocks until elements available in {@code q} then tries to grab as many as possible
915    * before returning.
916    * @param receptacle Where to stash the elements taken from queue. We clear before we use it
917    *     just in case.
918    * @param q The queue to take from.
919    * @return {@code receptacle} laden with elements taken from the queue or empty if none found.
920    */
921   @VisibleForTesting
922   static List<RAMQueueEntry> getRAMQueueEntries(final BlockingQueue<RAMQueueEntry> q,
923       final List<RAMQueueEntry> receptacle)
924   throws InterruptedException {
925     // Clear sets all entries to null and sets size to 0. We retain allocations. Presume it
926     // ok even if list grew to accommodate thousands.
927     receptacle.clear();
928     receptacle.add(q.take());
929     q.drainTo(receptacle);
930     return receptacle;
931   }
932 
933   private void persistToFile() throws IOException {
934     assert !cacheEnabled;
935     FileOutputStream fos = null;
936     ObjectOutputStream oos = null;
937     try {
938       if (!ioEngine.isPersistent())
939         throw new IOException(
940             "Attempt to persist non-persistent cache mappings!");
941       fos = new FileOutputStream(persistencePath, false);
942       oos = new ObjectOutputStream(fos);
943       oos.writeLong(cacheCapacity);
944       oos.writeUTF(ioEngine.getClass().getName());
945       oos.writeUTF(backingMap.getClass().getName());
946       oos.writeObject(deserialiserMap);
947       oos.writeObject(backingMap);
948     } finally {
949       if (oos != null) oos.close();
950       if (fos != null) fos.close();
951     }
952   }
953 
954   @SuppressWarnings("unchecked")
955   private void retrieveFromFile(int[] bucketSizes) throws IOException, BucketAllocatorException,
956       ClassNotFoundException {
957     File persistenceFile = new File(persistencePath);
958     if (!persistenceFile.exists()) {
959       return;
960     }
961     assert !cacheEnabled;
962     FileInputStream fis = null;
963     ObjectInputStream ois = null;
964     try {
965       if (!ioEngine.isPersistent())
966         throw new IOException(
967             "Attempt to restore non-persistent cache mappings!");
968       fis = new FileInputStream(persistencePath);
969       ois = new ObjectInputStream(fis);
970       long capacitySize = ois.readLong();
971       if (capacitySize != cacheCapacity)
972         throw new IOException("Mismatched cache capacity:"
973             + StringUtils.byteDesc(capacitySize) + ", expected: "
974             + StringUtils.byteDesc(cacheCapacity));
975       String ioclass = ois.readUTF();
976       String mapclass = ois.readUTF();
977       if (!ioEngine.getClass().getName().equals(ioclass))
978         throw new IOException("Class name for IO engine mismatch: " + ioclass
979             + ", expected:" + ioEngine.getClass().getName());
980       if (!backingMap.getClass().getName().equals(mapclass))
981         throw new IOException("Class name for cache map mismatch: " + mapclass
982             + ", expected:" + backingMap.getClass().getName());
983       UniqueIndexMap<Integer> deserMap = (UniqueIndexMap<Integer>) ois
984           .readObject();
985       BucketAllocator allocator = new BucketAllocator(cacheCapacity, bucketSizes,
986           backingMap, realCacheSize);
987       backingMap = (ConcurrentHashMap<BlockCacheKey, BucketEntry>) ois
988           .readObject();
989       bucketAllocator = allocator;
990       deserialiserMap = deserMap;
991     } finally {
992       if (ois != null) ois.close();
993       if (fis != null) fis.close();
994       if (!persistenceFile.delete()) {
995         throw new IOException("Failed deleting persistence file "
996             + persistenceFile.getAbsolutePath());
997       }
998     }
999   }
1000 
1001   /**
1002    * Check whether we tolerate IO error this time. If the duration of IOEngine
1003    * throwing errors exceeds ioErrorsDurationTimeTolerated, we will disable the
1004    * cache
1005    */
1006   private void checkIOErrorIsTolerated() {
1007     long now = EnvironmentEdgeManager.currentTime();
1008     if (this.ioErrorStartTime > 0) {
1009       if (cacheEnabled && (now - ioErrorStartTime) > this.ioErrorsTolerationDuration) {
1010         LOG.error("IO errors duration time has exceeded " + ioErrorsTolerationDuration +
1011           "ms, disabing cache, please check your IOEngine");
1012         disableCache();
1013       }
1014     } else {
1015       this.ioErrorStartTime = now;
1016     }
1017   }
1018 
1019   /**
1020    * Used to shut down the cache -or- turn it off in the case of something
1021    * broken.
1022    */
1023   private void disableCache() {
1024     if (!cacheEnabled)
1025       return;
1026     cacheEnabled = false;
1027     ioEngine.shutdown();
1028     this.scheduleThreadPool.shutdown();
1029     for (int i = 0; i < writerThreads.length; ++i)
1030       writerThreads[i].interrupt();
1031     this.ramCache.clear();
1032     if (!ioEngine.isPersistent() || persistencePath == null) {
1033       this.backingMap.clear();
1034     }
1035   }
1036 
1037   private void join() throws InterruptedException {
1038     for (int i = 0; i < writerThreads.length; ++i)
1039       writerThreads[i].join();
1040   }
1041 
1042   @Override
1043   public void shutdown() {
1044     disableCache();
1045     LOG.info("Shutdown bucket cache: IO persistent=" + ioEngine.isPersistent()
1046         + "; path to write=" + persistencePath);
1047     if (ioEngine.isPersistent() && persistencePath != null) {
1048       try {
1049         join();
1050         persistToFile();
1051       } catch (IOException ex) {
1052         LOG.error("Unable to persist data on exit: " + ex.toString(), ex);
1053       } catch (InterruptedException e) {
1054         LOG.warn("Failed to persist data on exit", e);
1055       }
1056     }
1057   }
1058 
1059   @Override
1060   public CacheStats getStats() {
1061     return cacheStats;
1062   }
1063 
1064   public BucketAllocator getAllocator() {
1065     return this.bucketAllocator;
1066   }
1067 
1068   @Override
1069   public long heapSize() {
1070     return this.heapSize.get();
1071   }
1072 
1073   @Override
1074   public long size() {
1075     return this.realCacheSize.get();
1076   }
1077 
1078   @Override
1079   public long getFreeSize() {
1080     return this.bucketAllocator.getFreeSize();
1081   }
1082 
1083   @Override
1084   public long getBlockCount() {
1085     return this.blockNumber.get();
1086   }
1087 
1088   @Override
1089   public long getCurrentSize() {
1090     return this.bucketAllocator.getUsedSize();
1091   }
1092 
1093   /**
1094    * Evicts all blocks for a specific HFile.
1095    * <p>
1096    * This is used for evict-on-close to remove all blocks of a specific HFile.
1097    *
1098    * @return the number of blocks evicted
1099    */
1100   @Override
1101   public int evictBlocksByHfileName(String hfileName) {
1102     // Copy the list to avoid ConcurrentModificationException
1103     // as evictBlockKey removes the key from the index
1104     Set<BlockCacheKey> keySet = blocksByHFile.values(hfileName);
1105     if (keySet == null) {
1106       return 0;
1107     }
1108     int numEvicted = 0;
1109     List<BlockCacheKey> keysForHFile = ImmutableList.copyOf(keySet);
1110     for (BlockCacheKey key : keysForHFile) {
1111       if (evictBlock(key)) {
1112           ++numEvicted;
1113       }
1114     }
1115 
1116     return numEvicted;
1117   }
1118 
1119   /**
1120    * Item in cache. We expect this to be where most memory goes. Java uses 8
1121    * bytes just for object headers; after this, we want to use as little as
1122    * possible - so we only use 8 bytes, but in order to do so we end up messing
1123    * around with all this Java casting stuff. Offset stored as 5 bytes that make
1124    * up the long. Doubt we'll see devices this big for ages. Offsets are divided
1125    * by 256. So 5 bytes gives us 256TB or so.
1126    */
1127   static class BucketEntry implements Serializable {
1128     private static final long serialVersionUID = -6741504807982257534L;
1129 
1130     // access counter comparator, descending order
1131     static final Comparator<BucketEntry> COMPARATOR = new Comparator<BucketCache.BucketEntry>() {
1132 
1133       @Override
1134       public int compare(BucketEntry o1, BucketEntry o2) {
1135         long accessCounter1 = o1.accessCounter;
1136         long accessCounter2 = o2.accessCounter;
1137         return accessCounter1 < accessCounter2 ? 1 : accessCounter1 == accessCounter2 ? 0 : -1;
1138       }
1139     };
1140 
1141     private int offsetBase;
1142     private int length;
1143     private byte offset1;
1144     byte deserialiserIndex;
1145     private volatile long accessCounter;
1146     private BlockPriority priority;
1147     // Set this when we were not able to forcefully evict the block
1148     private volatile boolean markedForEvict;
1149     private AtomicInteger refCount = new AtomicInteger(0);
1150 
1151     /**
1152      * Time this block was cached.  Presumes we are created just before we are added to the cache.
1153      */
1154     private final long cachedTime = System.nanoTime();
1155 
1156     BucketEntry(long offset, int length, long accessCounter, boolean inMemory) {
1157       setOffset(offset);
1158       this.length = length;
1159       this.accessCounter = accessCounter;
1160       if (inMemory) {
1161         this.priority = BlockPriority.MEMORY;
1162       } else {
1163         this.priority = BlockPriority.SINGLE;
1164       }
1165     }
1166 
1167     long offset() { // Java has no unsigned numbers
1168       long o = ((long) offsetBase) & 0xFFFFFFFF;
1169       o += (((long) (offset1)) & 0xFF) << 32;
1170       return o << 8;
1171     }
1172 
1173     private void setOffset(long value) {
1174       assert (value & 0xFF) == 0;
1175       value >>= 8;
1176       offsetBase = (int) value;
1177       offset1 = (byte) (value >> 32);
1178     }
1179 
1180     public int getLength() {
1181       return length;
1182     }
1183 
1184     protected CacheableDeserializer<Cacheable> deserializerReference(
1185         UniqueIndexMap<Integer> deserialiserMap) {
1186       return CacheableDeserializerIdManager.getDeserializer(deserialiserMap
1187           .unmap(deserialiserIndex));
1188     }
1189 
1190     protected void setDeserialiserReference(
1191         CacheableDeserializer<Cacheable> deserializer,
1192         UniqueIndexMap<Integer> deserialiserMap) {
1193       this.deserialiserIndex = ((byte) deserialiserMap.map(deserializer
1194           .getDeserialiserIdentifier()));
1195     }
1196 
1197     /**
1198      * Block has been accessed. Update its local access counter.
1199      */
1200     public void access(long accessCounter) {
1201       this.accessCounter = accessCounter;
1202       if (this.priority == BlockPriority.SINGLE) {
1203         this.priority = BlockPriority.MULTI;
1204       }
1205     }
1206 
1207     public BlockPriority getPriority() {
1208       return this.priority;
1209     }
1210 
1211     public long getCachedTime() {
1212       return cachedTime;
1213     }
1214   }
1215 
1216   /**
1217    * Used to group bucket entries into priority buckets. There will be a
1218    * BucketEntryGroup for each priority (single, multi, memory). Once bucketed,
1219    * the eviction algorithm takes the appropriate number of elements out of each
1220    * according to configuration parameters and their relative sizes.
1221    */
1222   private class BucketEntryGroup implements Comparable<BucketEntryGroup> {
1223 
1224     private CachedEntryQueue queue;
1225     private long totalSize = 0;
1226     private long bucketSize;
1227 
1228     public BucketEntryGroup(long bytesToFree, long blockSize, long bucketSize) {
1229       this.bucketSize = bucketSize;
1230       queue = new CachedEntryQueue(bytesToFree, blockSize);
1231       totalSize = 0;
1232     }
1233 
1234     public void add(Map.Entry<BlockCacheKey, BucketEntry> block) {
1235       totalSize += block.getValue().getLength();
1236       queue.add(block);
1237     }
1238 
1239     public long free(long toFree) {
1240       Map.Entry<BlockCacheKey, BucketEntry> entry;
1241       long freedBytes = 0;
1242       // TODO avoid a cycling siutation. We find no block which is not in use and so no way to free
1243       // What to do then? Caching attempt fail? Need some changes in cacheBlock API?
1244       while ((entry = queue.pollLast()) != null) {
1245         if (evictBlock(entry.getKey(), false)) {
1246           freedBytes += entry.getValue().getLength();
1247         }
1248         if (freedBytes >= toFree) {
1249           return freedBytes;
1250         }
1251       }
1252       return freedBytes;
1253     }
1254 
1255     public long overflow() {
1256       return totalSize - bucketSize;
1257     }
1258 
1259     public long totalSize() {
1260       return totalSize;
1261     }
1262 
1263     @Override
1264     public int compareTo(BucketEntryGroup that) {
1265       if (this.overflow() == that.overflow())
1266         return 0;
1267       return this.overflow() > that.overflow() ? 1 : -1;
1268     }
1269 
1270     @Override
1271     public boolean equals(Object that) {
1272       return this == that;
1273     }
1274 
1275   }
1276 
1277   /**
1278    * Block Entry stored in the memory with key,data and so on
1279    */
1280   @VisibleForTesting
1281   static class RAMQueueEntry {
1282     private BlockCacheKey key;
1283     private Cacheable data;
1284     private long accessCounter;
1285     private boolean inMemory;
1286 
1287     public RAMQueueEntry(BlockCacheKey bck, Cacheable data, long accessCounter,
1288         boolean inMemory) {
1289       this.key = bck;
1290       this.data = data;
1291       this.accessCounter = accessCounter;
1292       this.inMemory = inMemory;
1293     }
1294 
1295     public Cacheable getData() {
1296       return data;
1297     }
1298 
1299     public BlockCacheKey getKey() {
1300       return key;
1301     }
1302 
1303     public void access(long accessCounter) {
1304       this.accessCounter = accessCounter;
1305     }
1306 
1307     public BucketEntry writeToCache(final IOEngine ioEngine,
1308         final BucketAllocator bucketAllocator,
1309         final UniqueIndexMap<Integer> deserialiserMap,
1310         final AtomicLong realCacheSize) throws CacheFullException, IOException,
1311         BucketAllocatorException {
1312       int len = data.getSerializedLength();
1313       // This cacheable thing can't be serialized...
1314       if (len == 0) return null;
1315       long offset = bucketAllocator.allocateBlock(len);
1316       BucketEntry bucketEntry = new BucketEntry(offset, len, accessCounter, inMemory);
1317       bucketEntry.setDeserialiserReference(data.getDeserializer(), deserialiserMap);
1318       try {
1319         if (data instanceof HFileBlock) {
1320           HFileBlock block = (HFileBlock) data;
1321           ByteBuff sliceBuf = block.getBufferReadOnlyWithHeader();
1322           sliceBuf.rewind();
1323           assert len == sliceBuf.limit() + HFileBlock.EXTRA_SERIALIZATION_SPACE ||
1324             len == sliceBuf.limit() + block.headerSize() + HFileBlock.EXTRA_SERIALIZATION_SPACE;
1325           ByteBuffer extraInfoBuffer = ByteBuffer.allocate(HFileBlock.EXTRA_SERIALIZATION_SPACE);
1326           block.serializeExtraInfo(extraInfoBuffer);
1327           ioEngine.write(sliceBuf, offset);
1328           ioEngine.write(extraInfoBuffer, offset + len - HFileBlock.EXTRA_SERIALIZATION_SPACE);
1329         } else {
1330           ByteBuffer bb = ByteBuffer.allocate(len);
1331           data.serialize(bb);
1332           ioEngine.write(bb, offset);
1333         }
1334       } catch (IOException ioe) {
1335         // free it in bucket allocator
1336         bucketAllocator.freeBlock(offset);
1337         throw ioe;
1338       }
1339 
1340       realCacheSize.addAndGet(len);
1341       return bucketEntry;
1342     }
1343   }
1344 
1345   /**
1346    * Only used in test
1347    * @throws InterruptedException
1348    */
1349   void stopWriterThreads() throws InterruptedException {
1350     for (WriterThread writerThread : writerThreads) {
1351       writerThread.disableWriter();
1352       writerThread.interrupt();
1353       writerThread.join();
1354     }
1355   }
1356 
1357   @Override
1358   public Iterator<CachedBlock> iterator() {
1359     // Don't bother with ramcache since stuff is in here only a little while.
1360     final Iterator<Map.Entry<BlockCacheKey, BucketEntry>> i =
1361         this.backingMap.entrySet().iterator();
1362     return new Iterator<CachedBlock>() {
1363       private final long now = System.nanoTime();
1364 
1365       @Override
1366       public boolean hasNext() {
1367         return i.hasNext();
1368       }
1369 
1370       @Override
1371       public CachedBlock next() {
1372         final Map.Entry<BlockCacheKey, BucketEntry> e = i.next();
1373         return new CachedBlock() {
1374           @Override
1375           public String toString() {
1376             return BlockCacheUtil.toString(this, now);
1377           }
1378 
1379           @Override
1380           public BlockPriority getBlockPriority() {
1381             return e.getValue().getPriority();
1382           }
1383 
1384           @Override
1385           public BlockType getBlockType() {
1386             // Not held by BucketEntry.  Could add it if wanted on BucketEntry creation.
1387             return null;
1388           }
1389 
1390           @Override
1391           public long getOffset() {
1392             return e.getKey().getOffset();
1393           }
1394 
1395           @Override
1396           public long getSize() {
1397             return e.getValue().getLength();
1398           }
1399 
1400           @Override
1401           public long getCachedTime() {
1402             return e.getValue().getCachedTime();
1403           }
1404 
1405           @Override
1406           public String getFilename() {
1407             return e.getKey().getHfileName();
1408           }
1409 
1410           @Override
1411           public int compareTo(CachedBlock other) {
1412             int diff = this.getFilename().compareTo(other.getFilename());
1413             if (diff != 0) return diff;
1414             diff = (int)(this.getOffset() - other.getOffset());
1415             if (diff != 0) return diff;
1416             if (other.getCachedTime() < 0 || this.getCachedTime() < 0) {
1417               throw new IllegalStateException("" + this.getCachedTime() + ", " +
1418                 other.getCachedTime());
1419             }
1420             return (int)(other.getCachedTime() - this.getCachedTime());
1421           }
1422 
1423           @Override
1424           public int hashCode() {
1425             return e.getKey().hashCode();
1426           }
1427 
1428           @Override
1429           public boolean equals(Object obj) {
1430             if (obj instanceof CachedBlock) {
1431               CachedBlock cb = (CachedBlock)obj;
1432               return compareTo(cb) == 0;
1433             } else {
1434               return false;
1435             }
1436           }
1437         };
1438       }
1439 
1440       @Override
1441       public void remove() {
1442         throw new UnsupportedOperationException();
1443       }
1444     };
1445   }
1446 
1447   @Override
1448   public BlockCache[] getBlockCaches() {
1449     return null;
1450   }
1451 
1452   @Override
1453   public void returnBlock(BlockCacheKey cacheKey, Cacheable block) {
1454     if (block.getMemoryType() == MemoryType.SHARED) {
1455       BucketEntry bucketEntry = backingMap.get(cacheKey);
1456       if (bucketEntry != null) {
1457         int refCount = bucketEntry.refCount.decrementAndGet();
1458         if (bucketEntry.markedForEvict && refCount == 0) {
1459           forceEvict(cacheKey);
1460         }
1461       }
1462     }
1463   }
1464 
1465   @VisibleForTesting
1466   public int getRefCount(BlockCacheKey cacheKey) {
1467     BucketEntry bucketEntry = backingMap.get(cacheKey);
1468     if (bucketEntry != null) {
1469       return bucketEntry.refCount.get();
1470     }
1471     return 0;
1472   }
1473 }