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