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