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