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