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