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