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