View Javadoc

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