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   * BucketCache#ramCache and 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 if (ioEngineName.startsWith("mmap:")) {
313       return new FileMmapEngine(ioEngineName.substring(5), capacity);
314     } else {
315       throw new IllegalArgumentException(
316           "Don't understand io engine name for cache - prefix with file:, heap or offheap");
317     }
318   }
319 
320   /**
321    * Cache the block with the specified name and buffer.
322    * @param cacheKey block's cache key
323    * @param buf block buffer
324    */
325   @Override
326   public void cacheBlock(BlockCacheKey cacheKey, Cacheable buf) {
327     cacheBlock(cacheKey, buf, false, false);
328   }
329 
330   /**
331    * Cache the block with the specified name and buffer.
332    * @param cacheKey block's cache key
333    * @param cachedItem block buffer
334    * @param inMemory if block is in-memory
335    * @param cacheDataInL1
336    */
337   @Override
338   public void cacheBlock(BlockCacheKey cacheKey, Cacheable cachedItem, boolean inMemory,
339       final boolean cacheDataInL1) {
340     cacheBlockWithWait(cacheKey, cachedItem, inMemory, wait_when_cache);
341   }
342 
343   /**
344    * Cache the block to ramCache
345    * @param cacheKey block's cache key
346    * @param cachedItem block buffer
347    * @param inMemory if block is in-memory
348    * @param wait if true, blocking wait when queue is full
349    */
350   public void cacheBlockWithWait(BlockCacheKey cacheKey, Cacheable cachedItem, boolean inMemory,
351       boolean wait) {
352     if (LOG.isTraceEnabled()) LOG.trace("Caching key=" + cacheKey + ", item=" + cachedItem);
353     if (!cacheEnabled) {
354       return;
355     }
356 
357     if (backingMap.containsKey(cacheKey)) {
358       return;
359     }
360 
361     /*
362      * Stuff the entry into the RAM cache so it can get drained to the persistent store
363      */
364     RAMQueueEntry re =
365         new RAMQueueEntry(cacheKey, cachedItem, accessCount.incrementAndGet(), inMemory);
366     if (ramCache.putIfAbsent(cacheKey, re) != null) {
367       return;
368     }
369     int queueNum = (cacheKey.hashCode() & 0x7FFFFFFF) % writerQueues.size();
370     BlockingQueue<RAMQueueEntry> bq = writerQueues.get(queueNum);
371     boolean successfulAddition = false;
372     if (wait) {
373       try {
374         successfulAddition = bq.offer(re, DEFAULT_CACHE_WAIT_TIME, TimeUnit.MILLISECONDS);
375       } catch (InterruptedException e) {
376         Thread.currentThread().interrupt();
377       }
378     } else {
379       successfulAddition = bq.offer(re);
380     }
381     if (!successfulAddition) {
382       ramCache.remove(cacheKey);
383       cacheStats.failInsert();
384     } else {
385       this.blockNumber.incrementAndGet();
386       this.heapSize.addAndGet(cachedItem.heapSize());
387       blocksByHFile.put(cacheKey.getHfileName(), cacheKey);
388     }
389   }
390 
391   /**
392    * Get the buffer of the block with the specified key.
393    * @param key block's cache key
394    * @param caching true if the caller caches blocks on cache misses
395    * @param repeat Whether this is a repeat lookup for the same block
396    * @param updateCacheMetrics Whether we should update cache metrics or not
397    * @return buffer of specified cache key, or null if not in cache
398    */
399   @Override
400   public Cacheable getBlock(BlockCacheKey key, boolean caching, boolean repeat,
401       boolean updateCacheMetrics) {
402     if (!cacheEnabled) {
403       return null;
404     }
405     RAMQueueEntry re = ramCache.get(key);
406     if (re != null) {
407       if (updateCacheMetrics) {
408         cacheStats.hit(caching, key.isPrimary(), key.getBlockType());
409       }
410       re.access(accessCount.incrementAndGet());
411       return re.getData();
412     }
413     BucketEntry bucketEntry = backingMap.get(key);
414     if (bucketEntry != null) {
415       long start = System.nanoTime();
416       ReentrantReadWriteLock lock = offsetLock.getLock(bucketEntry.offset());
417       try {
418         lock.readLock().lock();
419         // We can not read here even if backingMap does contain the given key because its offset
420         // maybe changed. If we lock BlockCacheKey instead of offset, then we can only check
421         // existence here.
422         if (bucketEntry.equals(backingMap.get(key))) {
423           // TODO : change this area - should be removed after server cells and
424           // 12295 are available
425           int len = bucketEntry.getLength();
426           if (LOG.isTraceEnabled()) {
427             LOG.trace("Read offset=" + bucketEntry.offset() + ", len=" + len);
428           }
429           Cacheable cachedBlock = ioEngine.read(bucketEntry.offset(), len,
430               bucketEntry.deserializerReference(this.deserialiserMap));
431           long timeTaken = System.nanoTime() - start;
432           if (updateCacheMetrics) {
433             cacheStats.hit(caching, key.isPrimary(), key.getBlockType());
434             cacheStats.ioHit(timeTaken);
435           }
436           if (cachedBlock.getMemoryType() == MemoryType.SHARED) {
437             bucketEntry.refCount.incrementAndGet();
438           }
439           bucketEntry.access(accessCount.incrementAndGet());
440           if (this.ioErrorStartTime > 0) {
441             ioErrorStartTime = -1;
442           }
443           return cachedBlock;
444         }
445       } catch (IOException ioex) {
446         LOG.error("Failed reading block " + key + " from bucket cache", ioex);
447         checkIOErrorIsTolerated();
448       } finally {
449         lock.readLock().unlock();
450       }
451     }
452     if (!repeat && updateCacheMetrics) {
453       cacheStats.miss(caching, key.isPrimary(), key.getBlockType());
454     }
455     return null;
456   }
457 
458   @VisibleForTesting
459   void blockEvicted(BlockCacheKey cacheKey, BucketEntry bucketEntry, boolean decrementBlockNumber) {
460     bucketAllocator.freeBlock(bucketEntry.offset());
461     realCacheSize.addAndGet(-1 * bucketEntry.getLength());
462     blocksByHFile.remove(cacheKey.getHfileName(), cacheKey);
463     if (decrementBlockNumber) {
464       this.blockNumber.decrementAndGet();
465     }
466   }
467 
468   @Override
469   public boolean evictBlock(BlockCacheKey cacheKey) {
470     return evictBlock(cacheKey, true);
471   }
472 
473   // does not check for the ref count. Just tries to evict it if found in the
474   // bucket map
475   private boolean forceEvict(BlockCacheKey cacheKey) {
476     if (!cacheEnabled) {
477       return false;
478     }
479     RAMQueueEntry removedBlock = checkRamCache(cacheKey);
480     BucketEntry bucketEntry = backingMap.get(cacheKey);
481     if (bucketEntry == null) {
482       if (removedBlock != null) {
483         cacheStats.evicted(0, cacheKey.isPrimary());
484         return true;
485       } else {
486         return false;
487       }
488     }
489     ReentrantReadWriteLock lock = offsetLock.getLock(bucketEntry.offset());
490     try {
491       lock.writeLock().lock();
492       if (backingMap.remove(cacheKey, bucketEntry)) {
493         blockEvicted(cacheKey, bucketEntry, removedBlock == null);
494       } else {
495         return false;
496       }
497     } finally {
498       lock.writeLock().unlock();
499     }
500     cacheStats.evicted(bucketEntry.getCachedTime(), cacheKey.isPrimary());
501     return true;
502   }
503 
504   private RAMQueueEntry checkRamCache(BlockCacheKey cacheKey) {
505     RAMQueueEntry removedBlock = ramCache.remove(cacheKey);
506     if (removedBlock != null) {
507       this.blockNumber.decrementAndGet();
508       this.heapSize.addAndGet(-1 * removedBlock.getData().heapSize());
509     }
510     return removedBlock;
511   }
512 
513   public boolean evictBlock(BlockCacheKey cacheKey, boolean deletedBlock) {
514     if (!cacheEnabled) {
515       return false;
516     }
517     RAMQueueEntry removedBlock = checkRamCache(cacheKey);
518     BucketEntry bucketEntry = backingMap.get(cacheKey);
519     if (bucketEntry == null) {
520       if (removedBlock != null) {
521         cacheStats.evicted(0, cacheKey.isPrimary());
522         return true;
523       } else {
524         return false;
525       }
526     }
527     ReentrantReadWriteLock lock = offsetLock.getLock(bucketEntry.offset());
528     try {
529       lock.writeLock().lock();
530       int refCount = bucketEntry.refCount.get();
531       if(refCount == 0) {
532         if (backingMap.remove(cacheKey, bucketEntry)) {
533           blockEvicted(cacheKey, bucketEntry, removedBlock == null);
534         } else {
535           return false;
536         }
537       } else {
538         if(!deletedBlock) {
539           if (LOG.isDebugEnabled()) {
540             LOG.debug("This block " + cacheKey + " is still referred by " + refCount
541                 + " readers. Can not be freed now");
542           }
543           return false;
544         } else {
545           if (LOG.isDebugEnabled()) {
546             LOG.debug("This block " + cacheKey + " is still referred by " + refCount
547                 + " readers. Can not be freed now. Hence will mark this"
548                 + " for evicting at a later point");
549           }
550           bucketEntry.markedForEvict = true;
551         }
552       }
553     } finally {
554       lock.writeLock().unlock();
555     }
556     cacheStats.evicted(bucketEntry.getCachedTime(), cacheKey.isPrimary());
557     return true;
558   }
559 
560   /*
561    * Statistics thread.  Periodically output cache statistics to the log.
562    */
563   private static class StatisticsThread extends Thread {
564     private final BucketCache bucketCache;
565 
566     public StatisticsThread(BucketCache bucketCache) {
567       super("BucketCacheStatsThread");
568       setDaemon(true);
569       this.bucketCache = bucketCache;
570     }
571 
572     @Override
573     public void run() {
574       bucketCache.logStats();
575     }
576   }
577 
578   public void logStats() {
579     long totalSize = bucketAllocator.getTotalSize();
580     long usedSize = bucketAllocator.getUsedSize();
581     long freeSize = totalSize - usedSize;
582     long cacheSize = getRealCacheSize();
583     LOG.info("failedBlockAdditions=" + cacheStats.getFailedInserts() + ", " +
584         "totalSize=" + StringUtils.byteDesc(totalSize) + ", " +
585         "freeSize=" + StringUtils.byteDesc(freeSize) + ", " +
586         "usedSize=" + StringUtils.byteDesc(usedSize) +", " +
587         "cacheSize=" + StringUtils.byteDesc(cacheSize) +", " +
588         "accesses=" + cacheStats.getRequestCount() + ", " +
589         "hits=" + cacheStats.getHitCount() + ", " +
590         "IOhitsPerSecond=" + cacheStats.getIOHitsPerSecond() + ", " +
591         "IOTimePerHit=" + String.format("%.2f", cacheStats.getIOTimePerHit())+ ", " +
592         "hitRatio=" + (cacheStats.getHitCount() == 0 ? "0," :
593           (StringUtils.formatPercent(cacheStats.getHitRatio(), 2)+ ", ")) +
594         "cachingAccesses=" + cacheStats.getRequestCachingCount() + ", " +
595         "cachingHits=" + cacheStats.getHitCachingCount() + ", " +
596         "cachingHitsRatio=" +(cacheStats.getHitCachingCount() == 0 ? "0," :
597           (StringUtils.formatPercent(cacheStats.getHitCachingRatio(), 2)+ ", ")) +
598         "evictions=" + cacheStats.getEvictionCount() + ", " +
599         "evicted=" + cacheStats.getEvictedCount() + ", " +
600         "evictedPerRun=" + cacheStats.evictedPerEviction());
601     cacheStats.reset();
602   }
603 
604   public long getRealCacheSize() {
605     return this.realCacheSize.get();
606   }
607 
608   private long acceptableSize() {
609     return (long) Math.floor(bucketAllocator.getTotalSize() * DEFAULT_ACCEPT_FACTOR);
610   }
611 
612   private long singleSize() {
613     return (long) Math.floor(bucketAllocator.getTotalSize()
614         * DEFAULT_SINGLE_FACTOR * DEFAULT_MIN_FACTOR);
615   }
616 
617   private long multiSize() {
618     return (long) Math.floor(bucketAllocator.getTotalSize() * DEFAULT_MULTI_FACTOR
619         * DEFAULT_MIN_FACTOR);
620   }
621 
622   private long memorySize() {
623     return (long) Math.floor(bucketAllocator.getTotalSize() * DEFAULT_MEMORY_FACTOR
624         * DEFAULT_MIN_FACTOR);
625   }
626 
627   /**
628    * Free the space if the used size reaches acceptableSize() or one size block
629    * couldn't be allocated. When freeing the space, we use the LRU algorithm and
630    * ensure there must be some blocks evicted
631    * @param why Why we are being called
632    */
633   private void freeSpace(final String why) {
634     // Ensure only one freeSpace progress at a time
635     if (!freeSpaceLock.tryLock()) {
636       return;
637     }
638     try {
639       freeInProgress = true;
640       long bytesToFreeWithoutExtra = 0;
641       // Calculate free byte for each bucketSizeinfo
642       StringBuffer msgBuffer = LOG.isDebugEnabled()? new StringBuffer(): null;
643       BucketAllocator.IndexStatistics[] stats = bucketAllocator.getIndexStatistics();
644       long[] bytesToFreeForBucket = new long[stats.length];
645       for (int i = 0; i < stats.length; i++) {
646         bytesToFreeForBucket[i] = 0;
647         long freeGoal = (long) Math.floor(stats[i].totalCount() * (1 - DEFAULT_MIN_FACTOR));
648         freeGoal = Math.max(freeGoal, 1);
649         if (stats[i].freeCount() < freeGoal) {
650           bytesToFreeForBucket[i] = stats[i].itemSize() * (freeGoal - stats[i].freeCount());
651           bytesToFreeWithoutExtra += bytesToFreeForBucket[i];
652           if (msgBuffer != null) {
653             msgBuffer.append("Free for bucketSize(" + stats[i].itemSize() + ")="
654               + StringUtils.byteDesc(bytesToFreeForBucket[i]) + ", ");
655           }
656         }
657       }
658       if (msgBuffer != null) {
659         msgBuffer.append("Free for total=" + StringUtils.byteDesc(bytesToFreeWithoutExtra) + ", ");
660       }
661 
662       if (bytesToFreeWithoutExtra <= 0) {
663         return;
664       }
665       long currentSize = bucketAllocator.getUsedSize();
666       long totalSize = bucketAllocator.getTotalSize();
667       if (LOG.isDebugEnabled() && msgBuffer != null) {
668         LOG.debug("Free started because \"" + why + "\"; " + msgBuffer.toString() +
669           " of current used=" + StringUtils.byteDesc(currentSize) + ", actual cacheSize=" +
670           StringUtils.byteDesc(realCacheSize.get()) + ", total=" + StringUtils.byteDesc(totalSize));
671       }
672 
673       long bytesToFreeWithExtra = (long) Math.floor(bytesToFreeWithoutExtra
674           * (1 + DEFAULT_EXTRA_FREE_FACTOR));
675 
676       // Instantiate priority buckets
677       BucketEntryGroup bucketSingle = new BucketEntryGroup(bytesToFreeWithExtra,
678           blockSize, singleSize());
679       BucketEntryGroup bucketMulti = new BucketEntryGroup(bytesToFreeWithExtra,
680           blockSize, multiSize());
681       BucketEntryGroup bucketMemory = new BucketEntryGroup(bytesToFreeWithExtra,
682           blockSize, memorySize());
683 
684       // Scan entire map putting bucket entry into appropriate bucket entry
685       // group
686       for (Map.Entry<BlockCacheKey, BucketEntry> bucketEntryWithKey : backingMap.entrySet()) {
687         switch (bucketEntryWithKey.getValue().getPriority()) {
688           case SINGLE: {
689             bucketSingle.add(bucketEntryWithKey);
690             break;
691           }
692           case MULTI: {
693             bucketMulti.add(bucketEntryWithKey);
694             break;
695           }
696           case MEMORY: {
697             bucketMemory.add(bucketEntryWithKey);
698             break;
699           }
700         }
701       }
702 
703       PriorityQueue<BucketEntryGroup> bucketQueue = new PriorityQueue<BucketEntryGroup>(3);
704 
705       bucketQueue.add(bucketSingle);
706       bucketQueue.add(bucketMulti);
707       bucketQueue.add(bucketMemory);
708 
709       int remainingBuckets = 3;
710       long bytesFreed = 0;
711 
712       BucketEntryGroup bucketGroup;
713       while ((bucketGroup = bucketQueue.poll()) != null) {
714         long overflow = bucketGroup.overflow();
715         if (overflow > 0) {
716           long bucketBytesToFree = Math.min(overflow,
717               (bytesToFreeWithoutExtra - bytesFreed) / remainingBuckets);
718           bytesFreed += bucketGroup.free(bucketBytesToFree);
719         }
720         remainingBuckets--;
721       }
722 
723       /**
724        * Check whether need extra free because some bucketSizeinfo still needs
725        * free space
726        */
727       stats = bucketAllocator.getIndexStatistics();
728       boolean needFreeForExtra = false;
729       for (int i = 0; i < stats.length; i++) {
730         long freeGoal = (long) Math.floor(stats[i].totalCount() * (1 - DEFAULT_MIN_FACTOR));
731         freeGoal = Math.max(freeGoal, 1);
732         if (stats[i].freeCount() < freeGoal) {
733           needFreeForExtra = true;
734           break;
735         }
736       }
737 
738       if (needFreeForExtra) {
739         bucketQueue.clear();
740         remainingBuckets = 2;
741 
742         bucketQueue.add(bucketSingle);
743         bucketQueue.add(bucketMulti);
744 
745         while ((bucketGroup = bucketQueue.poll()) != null) {
746           long bucketBytesToFree = (bytesToFreeWithExtra - bytesFreed) / remainingBuckets;
747           bytesFreed += bucketGroup.free(bucketBytesToFree);
748           remainingBuckets--;
749         }
750       }
751 
752       if (LOG.isDebugEnabled()) {
753         long single = bucketSingle.totalSize();
754         long multi = bucketMulti.totalSize();
755         long memory = bucketMemory.totalSize();
756         if (LOG.isDebugEnabled()) {
757           LOG.debug("Bucket cache free space completed; " + "freed="
758             + StringUtils.byteDesc(bytesFreed) + ", " + "total="
759             + StringUtils.byteDesc(totalSize) + ", " + "single="
760             + StringUtils.byteDesc(single) + ", " + "multi="
761             + StringUtils.byteDesc(multi) + ", " + "memory="
762             + StringUtils.byteDesc(memory));
763         }
764       }
765 
766     } catch (Throwable t) {
767       LOG.warn("Failed freeing space", t);
768     } finally {
769       cacheStats.evict();
770       freeInProgress = false;
771       freeSpaceLock.unlock();
772     }
773   }
774 
775   // This handles flushing the RAM cache to IOEngine.
776   @VisibleForTesting
777   class WriterThread extends HasThread {
778     private final BlockingQueue<RAMQueueEntry> inputQueue;
779     private volatile boolean writerEnabled = true;
780 
781     WriterThread(BlockingQueue<RAMQueueEntry> queue) {
782       this.inputQueue = queue;
783     }
784 
785     // Used for test
786     @VisibleForTesting
787     void disableWriter() {
788       this.writerEnabled = false;
789     }
790 
791     public void run() {
792       List<RAMQueueEntry> entries = new ArrayList<RAMQueueEntry>();
793       try {
794         while (cacheEnabled && writerEnabled) {
795           try {
796             try {
797               // Blocks
798               entries = getRAMQueueEntries(inputQueue, entries);
799             } catch (InterruptedException ie) {
800               if (!cacheEnabled) break;
801             }
802             doDrain(entries);
803           } catch (Exception ioe) {
804             LOG.error("WriterThread encountered error", ioe);
805           }
806         }
807       } catch (Throwable t) {
808         LOG.warn("Failed doing drain", t);
809       }
810       LOG.info(this.getName() + " exiting, cacheEnabled=" + cacheEnabled);
811     }
812 
813     /**
814      * Flush the entries in ramCache to IOEngine and add bucket entry to backingMap.
815      * Process all that are passed in even if failure being sure to remove from ramCache else we'll
816      * never undo the references and we'll OOME.
817      * @param entries Presumes list passed in here will be processed by this invocation only. No
818      *   interference expected.
819      * @throws InterruptedException
820      */
821     @VisibleForTesting
822     void doDrain(final List<RAMQueueEntry> entries) throws InterruptedException {
823       if (entries.isEmpty()) {
824         return;
825       }
826       // This method is a little hard to follow. We run through the passed in entries and for each
827       // successful add, we add a non-null BucketEntry to the below bucketEntries.  Later we must
828       // do cleanup making sure we've cleared ramCache of all entries regardless of whether we
829       // successfully added the item to the bucketcache; if we don't do the cleanup, we'll OOME by
830       // filling ramCache.  We do the clean up by again running through the passed in entries
831       // doing extra work when we find a non-null bucketEntries corresponding entry.
832       final int size = entries.size();
833       BucketEntry[] bucketEntries = new BucketEntry[size];
834       // Index updated inside loop if success or if we can't succeed. We retry if cache is full
835       // when we go to add an entry by going around the loop again without upping the index.
836       int index = 0;
837       while (cacheEnabled && index < size) {
838         RAMQueueEntry re = null;
839         try {
840           re = entries.get(index);
841           if (re == null) {
842             LOG.warn("Couldn't get entry or changed on us; who else is messing with it?");
843             index++;
844             continue;
845           }
846           BucketEntry bucketEntry =
847             re.writeToCache(ioEngine, bucketAllocator, deserialiserMap, realCacheSize);
848           // Successfully added.  Up index and add bucketEntry. Clear io exceptions.
849           bucketEntries[index] = bucketEntry;
850           if (ioErrorStartTime > 0) {
851             ioErrorStartTime = -1;
852           }
853           index++;
854         } catch (BucketAllocatorException fle) {
855           LOG.warn("Failed allocation for " + (re == null ? "" : re.getKey()) + "; " + fle);
856           // Presume can't add. Too big? Move index on. Entry will be cleared from ramCache below.
857           bucketEntries[index] = null;
858           index++;
859         } catch (CacheFullException cfe) {
860           // Cache full when we tried to add. Try freeing space and then retrying (don't up index)
861           if (!freeInProgress) {
862             freeSpace("Full!");
863           } else {
864             Thread.sleep(50);
865           }
866         } catch (IOException ioex) {
867           // Hopefully transient. Retry. checkIOErrorIsTolerated disables cache if problem.
868           LOG.error("Failed writing to bucket cache", ioex);
869           checkIOErrorIsTolerated();
870         }
871       }
872 
873       // Make sure data pages are written on media before we update maps.
874       try {
875         ioEngine.sync();
876       } catch (IOException ioex) {
877         LOG.error("Failed syncing IO engine", ioex);
878         checkIOErrorIsTolerated();
879         // Since we failed sync, free the blocks in bucket allocator
880         for (int i = 0; i < entries.size(); ++i) {
881           if (bucketEntries[i] != null) {
882             bucketAllocator.freeBlock(bucketEntries[i].offset());
883             bucketEntries[i] = null;
884           }
885         }
886       }
887 
888       // Now add to backingMap if successfully added to bucket cache.  Remove from ramCache if
889       // success or error.
890       for (int i = 0; i < size; ++i) {
891         BlockCacheKey key = entries.get(i).getKey();
892         // Only add if non-null entry.
893         if (bucketEntries[i] != null) {
894           backingMap.put(key, bucketEntries[i]);
895         }
896         // Always remove from ramCache even if we failed adding it to the block cache above.
897         RAMQueueEntry ramCacheEntry = ramCache.remove(key);
898         if (ramCacheEntry != null) {
899           heapSize.addAndGet(-1 * entries.get(i).getData().heapSize());
900         } else if (bucketEntries[i] != null){
901           // Block should have already been evicted. Remove it and free space.
902           ReentrantReadWriteLock lock = offsetLock.getLock(bucketEntries[i].offset());
903           try {
904             lock.writeLock().lock();
905             if (backingMap.remove(key, bucketEntries[i])) {
906               blockEvicted(key, bucketEntries[i], false);
907             }
908           } finally {
909             lock.writeLock().unlock();
910           }
911         }
912       }
913 
914       long used = bucketAllocator.getUsedSize();
915       if (used > acceptableSize()) {
916         freeSpace("Used=" + used + " > acceptable=" + acceptableSize());
917       }
918       return;
919     }
920   }
921 
922   /**
923    * Blocks until elements available in {@code q} then tries to grab as many as possible
924    * before returning.
925    * @param receptacle Where to stash the elements taken from queue. We clear before we use it
926    *     just in case.
927    * @param q The queue to take from.
928    * @return {@code receptacle} laden with elements taken from the queue or empty if none found.
929    */
930   @VisibleForTesting
931   static List<RAMQueueEntry> getRAMQueueEntries(final BlockingQueue<RAMQueueEntry> q,
932       final List<RAMQueueEntry> receptacle)
933   throws InterruptedException {
934     // Clear sets all entries to null and sets size to 0. We retain allocations. Presume it
935     // ok even if list grew to accommodate thousands.
936     receptacle.clear();
937     receptacle.add(q.take());
938     q.drainTo(receptacle);
939     return receptacle;
940   }
941 
942   private void persistToFile() throws IOException {
943     assert !cacheEnabled;
944     FileOutputStream fos = null;
945     ObjectOutputStream oos = null;
946     try {
947       if (!ioEngine.isPersistent()) {
948         throw new IOException("Attempt to persist non-persistent cache mappings!");
949       }
950       fos = new FileOutputStream(persistencePath, false);
951       oos = new ObjectOutputStream(fos);
952       oos.writeLong(cacheCapacity);
953       oos.writeUTF(ioEngine.getClass().getName());
954       oos.writeUTF(backingMap.getClass().getName());
955       oos.writeObject(deserialiserMap);
956       oos.writeObject(backingMap);
957     } finally {
958       if (oos != null) oos.close();
959       if (fos != null) fos.close();
960     }
961   }
962 
963   @SuppressWarnings("unchecked")
964   private void retrieveFromFile(int[] bucketSizes) throws IOException, BucketAllocatorException,
965       ClassNotFoundException {
966     File persistenceFile = new File(persistencePath);
967     if (!persistenceFile.exists()) {
968       return;
969     }
970     assert !cacheEnabled;
971     FileInputStream fis = null;
972     ObjectInputStream ois = null;
973     try {
974       if (!ioEngine.isPersistent())
975         throw new IOException(
976             "Attempt to restore non-persistent cache mappings!");
977       fis = new FileInputStream(persistencePath);
978       ois = new ObjectInputStream(fis);
979       long capacitySize = ois.readLong();
980       if (capacitySize != cacheCapacity)
981         throw new IOException("Mismatched cache capacity:"
982             + StringUtils.byteDesc(capacitySize) + ", expected: "
983             + StringUtils.byteDesc(cacheCapacity));
984       String ioclass = ois.readUTF();
985       String mapclass = ois.readUTF();
986       if (!ioEngine.getClass().getName().equals(ioclass))
987         throw new IOException("Class name for IO engine mismatch: " + ioclass
988             + ", expected:" + ioEngine.getClass().getName());
989       if (!backingMap.getClass().getName().equals(mapclass))
990         throw new IOException("Class name for cache map mismatch: " + mapclass
991             + ", expected:" + backingMap.getClass().getName());
992       UniqueIndexMap<Integer> deserMap = (UniqueIndexMap<Integer>) ois
993           .readObject();
994       BucketAllocator allocator = new BucketAllocator(cacheCapacity, bucketSizes,
995           backingMap, realCacheSize);
996       backingMap = (ConcurrentHashMap<BlockCacheKey, BucketEntry>) ois
997           .readObject();
998       bucketAllocator = allocator;
999       deserialiserMap = deserMap;
1000     } finally {
1001       if (ois != null) ois.close();
1002       if (fis != null) fis.close();
1003       if (!persistenceFile.delete()) {
1004         throw new IOException("Failed deleting persistence file "
1005             + persistenceFile.getAbsolutePath());
1006       }
1007     }
1008   }
1009 
1010   /**
1011    * Check whether we tolerate IO error this time. If the duration of IOEngine
1012    * throwing errors exceeds ioErrorsDurationTimeTolerated, we will disable the
1013    * cache
1014    */
1015   private void checkIOErrorIsTolerated() {
1016     long now = EnvironmentEdgeManager.currentTime();
1017     if (this.ioErrorStartTime > 0) {
1018       if (cacheEnabled && (now - ioErrorStartTime) > this.ioErrorsTolerationDuration) {
1019         LOG.error("IO errors duration time has exceeded " + ioErrorsTolerationDuration +
1020           "ms, disabing cache, please check your IOEngine");
1021         disableCache();
1022       }
1023     } else {
1024       this.ioErrorStartTime = now;
1025     }
1026   }
1027 
1028   /**
1029    * Used to shut down the cache -or- turn it off in the case of something broken.
1030    */
1031   private void disableCache() {
1032     if (!cacheEnabled) return;
1033     cacheEnabled = false;
1034     ioEngine.shutdown();
1035     this.scheduleThreadPool.shutdown();
1036     for (int i = 0; i < writerThreads.length; ++i) writerThreads[i].interrupt();
1037     this.ramCache.clear();
1038     if (!ioEngine.isPersistent() || persistencePath == null) {
1039       // If persistent ioengine and a path, we will serialize out the backingMap.
1040       this.backingMap.clear();
1041     }
1042   }
1043 
1044   private void join() throws InterruptedException {
1045     for (int i = 0; i < writerThreads.length; ++i)
1046       writerThreads[i].join();
1047   }
1048 
1049   @Override
1050   public void shutdown() {
1051     disableCache();
1052     LOG.info("Shutdown bucket cache: IO persistent=" + ioEngine.isPersistent()
1053         + "; path to write=" + persistencePath);
1054     if (ioEngine.isPersistent() && persistencePath != null) {
1055       try {
1056         join();
1057         persistToFile();
1058       } catch (IOException ex) {
1059         LOG.error("Unable to persist data on exit: " + ex.toString(), ex);
1060       } catch (InterruptedException e) {
1061         LOG.warn("Failed to persist data on exit", e);
1062       }
1063     }
1064   }
1065 
1066   @Override
1067   public CacheStats getStats() {
1068     return cacheStats;
1069   }
1070 
1071   public BucketAllocator getAllocator() {
1072     return this.bucketAllocator;
1073   }
1074 
1075   @Override
1076   public long heapSize() {
1077     return this.heapSize.get();
1078   }
1079 
1080   @Override
1081   public long size() {
1082     return this.realCacheSize.get();
1083   }
1084 
1085   @Override
1086   public long getFreeSize() {
1087     return this.bucketAllocator.getFreeSize();
1088   }
1089 
1090   @Override
1091   public long getBlockCount() {
1092     return this.blockNumber.get();
1093   }
1094 
1095   @Override
1096   public long getCurrentSize() {
1097     return this.bucketAllocator.getUsedSize();
1098   }
1099 
1100   /**
1101    * Evicts all blocks for a specific HFile.
1102    * <p>
1103    * This is used for evict-on-close to remove all blocks of a specific HFile.
1104    *
1105    * @return the number of blocks evicted
1106    */
1107   @Override
1108   public int evictBlocksByHfileName(String hfileName) {
1109     // Copy the list to avoid ConcurrentModificationException
1110     // as evictBlockKey removes the key from the index
1111     Set<BlockCacheKey> keySet = blocksByHFile.values(hfileName);
1112     if (keySet == null) {
1113       return 0;
1114     }
1115     int numEvicted = 0;
1116     List<BlockCacheKey> keysForHFile = ImmutableList.copyOf(keySet);
1117     for (BlockCacheKey key : keysForHFile) {
1118       if (evictBlock(key)) {
1119           ++numEvicted;
1120       }
1121     }
1122 
1123     return numEvicted;
1124   }
1125 
1126   /**
1127    * Item in cache. We expect this to be where most memory goes. Java uses 8
1128    * bytes just for object headers; after this, we want to use as little as
1129    * possible - so we only use 8 bytes, but in order to do so we end up messing
1130    * around with all this Java casting stuff. Offset stored as 5 bytes that make
1131    * up the long. Doubt we'll see devices this big for ages. Offsets are divided
1132    * by 256. So 5 bytes gives us 256TB or so.
1133    */
1134   static class BucketEntry implements Serializable {
1135     private static final long serialVersionUID = -6741504807982257534L;
1136 
1137     // access counter comparator, descending order
1138     static final Comparator<BucketEntry> COMPARATOR = new Comparator<BucketCache.BucketEntry>() {
1139 
1140       @Override
1141       public int compare(BucketEntry o1, BucketEntry o2) {
1142         long accessCounter1 = o1.accessCounter;
1143         long accessCounter2 = o2.accessCounter;
1144         return accessCounter1 < accessCounter2 ? 1 : accessCounter1 == accessCounter2 ? 0 : -1;
1145       }
1146     };
1147 
1148     private int offsetBase;
1149     private int length;
1150     private byte offset1;
1151     byte deserialiserIndex;
1152     private volatile long accessCounter;
1153     private BlockPriority priority;
1154     // Set this when we were not able to forcefully evict the block
1155     private volatile boolean markedForEvict;
1156     private AtomicInteger refCount = new AtomicInteger(0);
1157 
1158     /**
1159      * Time this block was cached.  Presumes we are created just before we are added to the cache.
1160      */
1161     private final long cachedTime = System.nanoTime();
1162 
1163     BucketEntry(long offset, int length, long accessCounter, boolean inMemory) {
1164       setOffset(offset);
1165       this.length = length;
1166       this.accessCounter = accessCounter;
1167       if (inMemory) {
1168         this.priority = BlockPriority.MEMORY;
1169       } else {
1170         this.priority = BlockPriority.SINGLE;
1171       }
1172     }
1173 
1174     long offset() { // Java has no unsigned numbers
1175       long o = ((long) offsetBase) & 0xFFFFFFFF;
1176       o += (((long) (offset1)) & 0xFF) << 32;
1177       return o << 8;
1178     }
1179 
1180     private void setOffset(long value) {
1181       assert (value & 0xFF) == 0;
1182       value >>= 8;
1183       offsetBase = (int) value;
1184       offset1 = (byte) (value >> 32);
1185     }
1186 
1187     public int getLength() {
1188       return length;
1189     }
1190 
1191     protected CacheableDeserializer<Cacheable> deserializerReference(
1192         UniqueIndexMap<Integer> deserialiserMap) {
1193       return CacheableDeserializerIdManager.getDeserializer(deserialiserMap
1194           .unmap(deserialiserIndex));
1195     }
1196 
1197     protected void setDeserialiserReference(
1198         CacheableDeserializer<Cacheable> deserializer,
1199         UniqueIndexMap<Integer> deserialiserMap) {
1200       this.deserialiserIndex = ((byte) deserialiserMap.map(deserializer
1201           .getDeserialiserIdentifier()));
1202     }
1203 
1204     /**
1205      * Block has been accessed. Update its local access counter.
1206      */
1207     public void access(long accessCounter) {
1208       this.accessCounter = accessCounter;
1209       if (this.priority == BlockPriority.SINGLE) {
1210         this.priority = BlockPriority.MULTI;
1211       }
1212     }
1213 
1214     public BlockPriority getPriority() {
1215       return this.priority;
1216     }
1217 
1218     public long getCachedTime() {
1219       return cachedTime;
1220     }
1221   }
1222 
1223   /**
1224    * Used to group bucket entries into priority buckets. There will be a
1225    * BucketEntryGroup for each priority (single, multi, memory). Once bucketed,
1226    * the eviction algorithm takes the appropriate number of elements out of each
1227    * according to configuration parameters and their relative sizes.
1228    */
1229   private class BucketEntryGroup implements Comparable<BucketEntryGroup> {
1230 
1231     private CachedEntryQueue queue;
1232     private long totalSize = 0;
1233     private long bucketSize;
1234 
1235     public BucketEntryGroup(long bytesToFree, long blockSize, long bucketSize) {
1236       this.bucketSize = bucketSize;
1237       queue = new CachedEntryQueue(bytesToFree, blockSize);
1238       totalSize = 0;
1239     }
1240 
1241     public void add(Map.Entry<BlockCacheKey, BucketEntry> block) {
1242       totalSize += block.getValue().getLength();
1243       queue.add(block);
1244     }
1245 
1246     public long free(long toFree) {
1247       Map.Entry<BlockCacheKey, BucketEntry> entry;
1248       long freedBytes = 0;
1249       // TODO avoid a cycling siutation. We find no block which is not in use and so no way to free
1250       // What to do then? Caching attempt fail? Need some changes in cacheBlock API?
1251       while ((entry = queue.pollLast()) != null) {
1252         if (evictBlock(entry.getKey(), false)) {
1253           freedBytes += entry.getValue().getLength();
1254         }
1255         if (freedBytes >= toFree) {
1256           return freedBytes;
1257         }
1258       }
1259       return freedBytes;
1260     }
1261 
1262     public long overflow() {
1263       return totalSize - bucketSize;
1264     }
1265 
1266     public long totalSize() {
1267       return totalSize;
1268     }
1269 
1270     @Override
1271     public int compareTo(BucketEntryGroup that) {
1272       if (this.overflow() == that.overflow())
1273         return 0;
1274       return this.overflow() > that.overflow() ? 1 : -1;
1275     }
1276 
1277     @Override
1278     public boolean equals(Object that) {
1279       return this == that;
1280     }
1281 
1282   }
1283 
1284   /**
1285    * Block Entry stored in the memory with key,data and so on
1286    */
1287   @VisibleForTesting
1288   static class RAMQueueEntry {
1289     private BlockCacheKey key;
1290     private Cacheable data;
1291     private long accessCounter;
1292     private boolean inMemory;
1293 
1294     public RAMQueueEntry(BlockCacheKey bck, Cacheable data, long accessCounter,
1295         boolean inMemory) {
1296       this.key = bck;
1297       this.data = data;
1298       this.accessCounter = accessCounter;
1299       this.inMemory = inMemory;
1300     }
1301 
1302     public Cacheable getData() {
1303       return data;
1304     }
1305 
1306     public BlockCacheKey getKey() {
1307       return key;
1308     }
1309 
1310     public void access(long accessCounter) {
1311       this.accessCounter = accessCounter;
1312     }
1313 
1314     public BucketEntry writeToCache(final IOEngine ioEngine,
1315         final BucketAllocator bucketAllocator,
1316         final UniqueIndexMap<Integer> deserialiserMap,
1317         final AtomicLong realCacheSize) throws CacheFullException, IOException,
1318         BucketAllocatorException {
1319       int len = data.getSerializedLength();
1320       // This cacheable thing can't be serialized
1321       if (len == 0) return null;
1322       long offset = bucketAllocator.allocateBlock(len);
1323       BucketEntry bucketEntry = new BucketEntry(offset, len, accessCounter, inMemory);
1324       bucketEntry.setDeserialiserReference(data.getDeserializer(), deserialiserMap);
1325       try {
1326         if (data instanceof HFileBlock) {
1327           // If an instance of HFileBlock, save on some allocations.
1328           HFileBlock block = (HFileBlock)data;
1329           ByteBuff sliceBuf = block.getBufferReadOnly();
1330           ByteBuffer metadata = block.getMetaData();
1331           if (LOG.isTraceEnabled()) {
1332             LOG.trace("Write offset=" + offset + ", len=" + len);
1333           }
1334           ioEngine.write(sliceBuf, offset);
1335           ioEngine.write(metadata, offset + len - metadata.limit());
1336         } else {
1337           ByteBuffer bb = ByteBuffer.allocate(len);
1338           data.serialize(bb);
1339           ioEngine.write(bb, offset);
1340         }
1341       } catch (IOException ioe) {
1342         // free it in bucket allocator
1343         bucketAllocator.freeBlock(offset);
1344         throw ioe;
1345       }
1346 
1347       realCacheSize.addAndGet(len);
1348       return bucketEntry;
1349     }
1350   }
1351 
1352   /**
1353    * Only used in test
1354    * @throws InterruptedException
1355    */
1356   void stopWriterThreads() throws InterruptedException {
1357     for (WriterThread writerThread : writerThreads) {
1358       writerThread.disableWriter();
1359       writerThread.interrupt();
1360       writerThread.join();
1361     }
1362   }
1363 
1364   @Override
1365   public Iterator<CachedBlock> iterator() {
1366     // Don't bother with ramcache since stuff is in here only a little while.
1367     final Iterator<Map.Entry<BlockCacheKey, BucketEntry>> i =
1368         this.backingMap.entrySet().iterator();
1369     return new Iterator<CachedBlock>() {
1370       private final long now = System.nanoTime();
1371 
1372       @Override
1373       public boolean hasNext() {
1374         return i.hasNext();
1375       }
1376 
1377       @Override
1378       public CachedBlock next() {
1379         final Map.Entry<BlockCacheKey, BucketEntry> e = i.next();
1380         return new CachedBlock() {
1381           @Override
1382           public String toString() {
1383             return BlockCacheUtil.toString(this, now);
1384           }
1385 
1386           @Override
1387           public BlockPriority getBlockPriority() {
1388             return e.getValue().getPriority();
1389           }
1390 
1391           @Override
1392           public BlockType getBlockType() {
1393             // Not held by BucketEntry.  Could add it if wanted on BucketEntry creation.
1394             return null;
1395           }
1396 
1397           @Override
1398           public long getOffset() {
1399             return e.getKey().getOffset();
1400           }
1401 
1402           @Override
1403           public long getSize() {
1404             return e.getValue().getLength();
1405           }
1406 
1407           @Override
1408           public long getCachedTime() {
1409             return e.getValue().getCachedTime();
1410           }
1411 
1412           @Override
1413           public String getFilename() {
1414             return e.getKey().getHfileName();
1415           }
1416 
1417           @Override
1418           public int compareTo(CachedBlock other) {
1419             int diff = this.getFilename().compareTo(other.getFilename());
1420             if (diff != 0) return diff;
1421             diff = (int)(this.getOffset() - other.getOffset());
1422             if (diff != 0) return diff;
1423             if (other.getCachedTime() < 0 || this.getCachedTime() < 0) {
1424               throw new IllegalStateException("" + this.getCachedTime() + ", " +
1425                 other.getCachedTime());
1426             }
1427             return (int)(other.getCachedTime() - this.getCachedTime());
1428           }
1429 
1430           @Override
1431           public int hashCode() {
1432             return e.getKey().hashCode();
1433           }
1434 
1435           @Override
1436           public boolean equals(Object obj) {
1437             if (obj instanceof CachedBlock) {
1438               CachedBlock cb = (CachedBlock)obj;
1439               return compareTo(cb) == 0;
1440             } else {
1441               return false;
1442             }
1443           }
1444         };
1445       }
1446 
1447       @Override
1448       public void remove() {
1449         throw new UnsupportedOperationException();
1450       }
1451     };
1452   }
1453 
1454   @Override
1455   public BlockCache[] getBlockCaches() {
1456     return null;
1457   }
1458 
1459   @Override
1460   public void returnBlock(BlockCacheKey cacheKey, Cacheable block) {
1461     if (block.getMemoryType() == MemoryType.SHARED) {
1462       BucketEntry bucketEntry = backingMap.get(cacheKey);
1463       if (bucketEntry != null) {
1464         int refCount = bucketEntry.refCount.decrementAndGet();
1465         if (bucketEntry.markedForEvict && refCount == 0) {
1466           forceEvict(cacheKey);
1467         }
1468       }
1469     }
1470   }
1471 
1472   @VisibleForTesting
1473   public int getRefCount(BlockCacheKey cacheKey) {
1474     BucketEntry bucketEntry = backingMap.get(cacheKey);
1475     if (bucketEntry != null) {
1476       return bucketEntry.refCount.get();
1477     }
1478     return 0;
1479   }
1480 }