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