001/**
002 * Copyright The Apache Software Foundation
003 *
004 * Licensed to the Apache Software Foundation (ASF) under one
005 * or more contributor license agreements.  See the NOTICE file
006 * distributed with this work for additional information
007 * regarding copyright ownership.  The ASF licenses this file
008 * to you under the Apache License, Version 2.0 (the
009 * "License"); you may not use this file except in compliance
010 * with the License.  You may obtain a copy of the License at
011 *
012 *     http://www.apache.org/licenses/LICENSE-2.0
013 *
014 * Unless required by applicable law or agreed to in writing, software
015
016 * distributed under the License is distributed on an "AS IS" BASIS,
017 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
018 * See the License for the specific language governing permissions and
019 * limitations under the License.
020 */
021package org.apache.hadoop.hbase.io.hfile.bucket;
022
023import java.io.File;
024import java.io.FileInputStream;
025import java.io.FileOutputStream;
026import java.io.IOException;
027import java.nio.ByteBuffer;
028import java.util.ArrayList;
029import java.util.Comparator;
030import java.util.HashSet;
031import java.util.Iterator;
032import java.util.List;
033import java.util.Map;
034import java.util.NavigableSet;
035import java.util.PriorityQueue;
036import java.util.Set;
037import java.util.concurrent.ArrayBlockingQueue;
038import java.util.concurrent.BlockingQueue;
039import java.util.concurrent.ConcurrentHashMap;
040import java.util.concurrent.ConcurrentMap;
041import java.util.concurrent.ConcurrentSkipListSet;
042import java.util.concurrent.Executors;
043import java.util.concurrent.ScheduledExecutorService;
044import java.util.concurrent.TimeUnit;
045import java.util.concurrent.atomic.AtomicBoolean;
046import java.util.concurrent.atomic.AtomicLong;
047import java.util.concurrent.atomic.LongAdder;
048import java.util.concurrent.locks.Lock;
049import java.util.concurrent.locks.ReentrantLock;
050import java.util.concurrent.locks.ReentrantReadWriteLock;
051import java.util.function.Consumer;
052import org.apache.hadoop.conf.Configuration;
053import org.apache.hadoop.hbase.HBaseConfiguration;
054import org.apache.hadoop.hbase.TableName;
055import org.apache.hadoop.hbase.client.Admin;
056import org.apache.hadoop.hbase.io.ByteBuffAllocator;
057import org.apache.hadoop.hbase.io.ByteBuffAllocator.Recycler;
058import org.apache.hadoop.hbase.io.HeapSize;
059import org.apache.hadoop.hbase.io.hfile.BlockCache;
060import org.apache.hadoop.hbase.io.hfile.BlockCacheKey;
061import org.apache.hadoop.hbase.io.hfile.BlockCacheUtil;
062import org.apache.hadoop.hbase.io.hfile.BlockPriority;
063import org.apache.hadoop.hbase.io.hfile.BlockType;
064import org.apache.hadoop.hbase.io.hfile.CacheStats;
065import org.apache.hadoop.hbase.io.hfile.Cacheable;
066import org.apache.hadoop.hbase.io.hfile.CachedBlock;
067import org.apache.hadoop.hbase.io.hfile.HFileBlock;
068import org.apache.hadoop.hbase.nio.ByteBuff;
069import org.apache.hadoop.hbase.nio.RefCnt;
070import org.apache.hadoop.hbase.protobuf.ProtobufMagic;
071import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
072import org.apache.hadoop.hbase.util.IdReadWriteLock;
073import org.apache.hadoop.hbase.util.IdReadWriteLockStrongRef;
074import org.apache.hadoop.hbase.util.IdReadWriteLockWithObjectPool;
075import org.apache.hadoop.hbase.util.IdReadWriteLockWithObjectPool.ReferenceType;
076import org.apache.hadoop.util.StringUtils;
077import org.apache.yetus.audience.InterfaceAudience;
078import org.slf4j.Logger;
079import org.slf4j.LoggerFactory;
080
081import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
082import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
083
084import org.apache.hadoop.hbase.shaded.protobuf.generated.BucketCacheProtos;
085
086/**
087 * BucketCache uses {@link BucketAllocator} to allocate/free blocks, and uses
088 * BucketCache#ramCache and BucketCache#backingMap in order to
089 * determine if a given element is in the cache. The bucket cache can use on-heap or
090 * off-heap memory {@link ByteBufferIOEngine} or in a file {@link FileIOEngine} to
091 * store/read the block data.
092 *
093 * <p>Eviction is via a similar algorithm as used in
094 * {@link org.apache.hadoop.hbase.io.hfile.LruBlockCache}
095 *
096 * <p>BucketCache can be used as mainly a block cache (see
097 * {@link org.apache.hadoop.hbase.io.hfile.CombinedBlockCache}), combined with
098 * a BlockCache to decrease CMS GC and heap fragmentation.
099 *
100 * <p>It also can be used as a secondary cache (e.g. using a file on ssd/fusionio to store
101 * blocks) to enlarge cache space via a victim cache.
102 */
103@InterfaceAudience.Private
104public class BucketCache implements BlockCache, HeapSize {
105  private static final Logger LOG = LoggerFactory.getLogger(BucketCache.class);
106
107  /** Priority buckets config */
108  static final String SINGLE_FACTOR_CONFIG_NAME = "hbase.bucketcache.single.factor";
109  static final String MULTI_FACTOR_CONFIG_NAME = "hbase.bucketcache.multi.factor";
110  static final String MEMORY_FACTOR_CONFIG_NAME = "hbase.bucketcache.memory.factor";
111  static final String EXTRA_FREE_FACTOR_CONFIG_NAME = "hbase.bucketcache.extrafreefactor";
112  static final String ACCEPT_FACTOR_CONFIG_NAME = "hbase.bucketcache.acceptfactor";
113  static final String MIN_FACTOR_CONFIG_NAME = "hbase.bucketcache.minfactor";
114
115  /** Use strong reference for offsetLock or not */
116  private static final String STRONG_REF_KEY = "hbase.bucketcache.offsetlock.usestrongref";
117  private static final boolean STRONG_REF_DEFAULT = false;
118
119  /** Priority buckets */
120  static final float DEFAULT_SINGLE_FACTOR = 0.25f;
121  static final float DEFAULT_MULTI_FACTOR = 0.50f;
122  static final float DEFAULT_MEMORY_FACTOR = 0.25f;
123  static final float DEFAULT_MIN_FACTOR = 0.85f;
124
125  private static final float DEFAULT_EXTRA_FREE_FACTOR = 0.10f;
126  private static final float DEFAULT_ACCEPT_FACTOR = 0.95f;
127
128  // Number of blocks to clear for each of the bucket size that is full
129  private static final int DEFAULT_FREE_ENTIRE_BLOCK_FACTOR = 2;
130
131  /** Statistics thread */
132  private static final int statThreadPeriod = 5 * 60;
133
134  final static int DEFAULT_WRITER_THREADS = 3;
135  final static int DEFAULT_WRITER_QUEUE_ITEMS = 64;
136
137  // Store/read block data
138  transient final IOEngine ioEngine;
139
140  // Store the block in this map before writing it to cache
141  transient final RAMCache ramCache;
142  // In this map, store the block's meta data like offset, length
143  transient ConcurrentHashMap<BlockCacheKey, BucketEntry> backingMap;
144
145  /**
146   * Flag if the cache is enabled or not... We shut it off if there are IO
147   * errors for some time, so that Bucket IO exceptions/errors don't bring down
148   * the HBase server.
149   */
150  private volatile boolean cacheEnabled;
151
152  /**
153   * A list of writer queues.  We have a queue per {@link WriterThread} we have running.
154   * In other words, the work adding blocks to the BucketCache is divided up amongst the
155   * running WriterThreads.  Its done by taking hash of the cache key modulo queue count.
156   * WriterThread when it runs takes whatever has been recently added and 'drains' the entries
157   * to the BucketCache.  It then updates the ramCache and backingMap accordingly.
158   */
159  transient final ArrayList<BlockingQueue<RAMQueueEntry>> writerQueues = new ArrayList<>();
160  transient final WriterThread[] writerThreads;
161
162  /** Volatile boolean to track if free space is in process or not */
163  private volatile boolean freeInProgress = false;
164  private transient final Lock freeSpaceLock = new ReentrantLock();
165
166  private final LongAdder realCacheSize = new LongAdder();
167  private final LongAdder heapSize = new LongAdder();
168  /** Current number of cached elements */
169  private final LongAdder blockNumber = new LongAdder();
170
171  /** Cache access count (sequential ID) */
172  private final AtomicLong accessCount = new AtomicLong();
173
174  private static final int DEFAULT_CACHE_WAIT_TIME = 50;
175
176  /**
177   * Used in tests. If this flag is false and the cache speed is very fast,
178   * bucket cache will skip some blocks when caching. If the flag is true, we
179   * will wait until blocks are flushed to IOEngine.
180   */
181  boolean wait_when_cache = false;
182
183  private final BucketCacheStats cacheStats = new BucketCacheStats();
184
185  private final String persistencePath;
186  private final long cacheCapacity;
187  /** Approximate block size */
188  private final long blockSize;
189
190  /** Duration of IO errors tolerated before we disable cache, 1 min as default */
191  private final int ioErrorsTolerationDuration;
192  // 1 min
193  public static final int DEFAULT_ERROR_TOLERATION_DURATION = 60 * 1000;
194
195  // Start time of first IO error when reading or writing IO Engine, it will be
196  // reset after a successful read/write.
197  private volatile long ioErrorStartTime = -1;
198
199  /**
200   * A ReentrantReadWriteLock to lock on a particular block identified by offset.
201   * The purpose of this is to avoid freeing the block which is being read.
202   * <p>
203   */
204  transient final IdReadWriteLock<Long> offsetLock;
205
206  private final NavigableSet<BlockCacheKey> blocksByHFile = new ConcurrentSkipListSet<>((a, b) -> {
207    int nameComparison = a.getHfileName().compareTo(b.getHfileName());
208    if (nameComparison != 0) {
209      return nameComparison;
210    }
211    return Long.compare(a.getOffset(), b.getOffset());
212  });
213
214  /** Statistics thread schedule pool (for heavy debugging, could remove) */
215  private transient final ScheduledExecutorService scheduleThreadPool =
216    Executors.newScheduledThreadPool(1,
217      new ThreadFactoryBuilder().setNameFormat("BucketCacheStatsExecutor").setDaemon(true).build());
218
219  // Allocate or free space for the block
220  private transient BucketAllocator bucketAllocator;
221
222  /** Acceptable size of cache (no evictions if size < acceptable) */
223  private float acceptableFactor;
224
225  /** Minimum threshold of cache (when evicting, evict until size < min) */
226  private float minFactor;
227
228  /** Free this floating point factor of extra blocks when evicting. For example free the number of blocks requested * (1 + extraFreeFactor) */
229  private float extraFreeFactor;
230
231  /** Single access bucket size */
232  private float singleFactor;
233
234  /** Multiple access bucket size */
235  private float multiFactor;
236
237  /** In-memory bucket size */
238  private float memoryFactor;
239
240  private static final String FILE_VERIFY_ALGORITHM =
241    "hbase.bucketcache.persistent.file.integrity.check.algorithm";
242  private static final String DEFAULT_FILE_VERIFY_ALGORITHM = "MD5";
243
244  /**
245   * Use {@link java.security.MessageDigest} class's encryption algorithms to check
246   * persistent file integrity, default algorithm is MD5
247   * */
248  private String algorithm;
249
250  public BucketCache(String ioEngineName, long capacity, int blockSize, int[] bucketSizes,
251      int writerThreadNum, int writerQLen, String persistencePath) throws IOException {
252    this(ioEngineName, capacity, blockSize, bucketSizes, writerThreadNum, writerQLen,
253        persistencePath, DEFAULT_ERROR_TOLERATION_DURATION, HBaseConfiguration.create());
254  }
255
256  public BucketCache(String ioEngineName, long capacity, int blockSize, int[] bucketSizes,
257      int writerThreadNum, int writerQLen, String persistencePath, int ioErrorsTolerationDuration,
258      Configuration conf) throws IOException {
259    boolean useStrongRef = conf.getBoolean(STRONG_REF_KEY, STRONG_REF_DEFAULT);
260    if (useStrongRef) {
261      this.offsetLock = new IdReadWriteLockStrongRef<>();
262    } else {
263      this.offsetLock = new IdReadWriteLockWithObjectPool<>(ReferenceType.SOFT);
264    }
265    this.algorithm = conf.get(FILE_VERIFY_ALGORITHM, DEFAULT_FILE_VERIFY_ALGORITHM);
266    this.ioEngine = getIOEngineFromName(ioEngineName, capacity, persistencePath);
267    this.writerThreads = new WriterThread[writerThreadNum];
268    long blockNumCapacity = capacity / blockSize;
269    if (blockNumCapacity >= Integer.MAX_VALUE) {
270      // Enough for about 32TB of cache!
271      throw new IllegalArgumentException("Cache capacity is too large, only support 32TB now");
272    }
273
274    this.acceptableFactor = conf.getFloat(ACCEPT_FACTOR_CONFIG_NAME, DEFAULT_ACCEPT_FACTOR);
275    this.minFactor = conf.getFloat(MIN_FACTOR_CONFIG_NAME, DEFAULT_MIN_FACTOR);
276    this.extraFreeFactor = conf.getFloat(EXTRA_FREE_FACTOR_CONFIG_NAME, DEFAULT_EXTRA_FREE_FACTOR);
277    this.singleFactor = conf.getFloat(SINGLE_FACTOR_CONFIG_NAME, DEFAULT_SINGLE_FACTOR);
278    this.multiFactor = conf.getFloat(MULTI_FACTOR_CONFIG_NAME, DEFAULT_MULTI_FACTOR);
279    this.memoryFactor = conf.getFloat(MEMORY_FACTOR_CONFIG_NAME, DEFAULT_MEMORY_FACTOR);
280
281    sanityCheckConfigs();
282
283    LOG.info("Instantiating BucketCache with acceptableFactor: " + acceptableFactor + ", minFactor: " + minFactor +
284        ", extraFreeFactor: " + extraFreeFactor + ", singleFactor: " + singleFactor + ", multiFactor: " + multiFactor +
285        ", memoryFactor: " + memoryFactor + ", useStrongRef: " + useStrongRef);
286
287    this.cacheCapacity = capacity;
288    this.persistencePath = persistencePath;
289    this.blockSize = blockSize;
290    this.ioErrorsTolerationDuration = ioErrorsTolerationDuration;
291
292    bucketAllocator = new BucketAllocator(capacity, bucketSizes);
293    for (int i = 0; i < writerThreads.length; ++i) {
294      writerQueues.add(new ArrayBlockingQueue<>(writerQLen));
295    }
296
297    assert writerQueues.size() == writerThreads.length;
298    this.ramCache = new RAMCache();
299
300    this.backingMap = new ConcurrentHashMap<>((int) blockNumCapacity);
301
302    if (ioEngine.isPersistent() && persistencePath != null) {
303      try {
304        retrieveFromFile(bucketSizes);
305      } catch (IOException ioex) {
306        LOG.error("Can't restore from file[" + persistencePath + "] because of ", ioex);
307      }
308    }
309    final String threadName = Thread.currentThread().getName();
310    this.cacheEnabled = true;
311    for (int i = 0; i < writerThreads.length; ++i) {
312      writerThreads[i] = new WriterThread(writerQueues.get(i));
313      writerThreads[i].setName(threadName + "-BucketCacheWriter-" + i);
314      writerThreads[i].setDaemon(true);
315    }
316    startWriterThreads();
317
318    // Run the statistics thread periodically to print the cache statistics log
319    // TODO: Add means of turning this off.  Bit obnoxious running thread just to make a log
320    // every five minutes.
321    this.scheduleThreadPool.scheduleAtFixedRate(new StatisticsThread(this),
322        statThreadPeriod, statThreadPeriod, TimeUnit.SECONDS);
323    LOG.info("Started bucket cache; ioengine=" + ioEngineName +
324        ", capacity=" + StringUtils.byteDesc(capacity) +
325      ", blockSize=" + StringUtils.byteDesc(blockSize) + ", writerThreadNum=" +
326        writerThreadNum + ", writerQLen=" + writerQLen + ", persistencePath=" +
327      persistencePath + ", bucketAllocator=" + this.bucketAllocator.getClass().getName());
328  }
329
330  private void sanityCheckConfigs() {
331    Preconditions.checkArgument(acceptableFactor <= 1 && acceptableFactor >= 0, ACCEPT_FACTOR_CONFIG_NAME + " must be between 0.0 and 1.0");
332    Preconditions.checkArgument(minFactor <= 1 && minFactor >= 0, MIN_FACTOR_CONFIG_NAME + " must be between 0.0 and 1.0");
333    Preconditions.checkArgument(minFactor <= acceptableFactor, MIN_FACTOR_CONFIG_NAME + " must be <= " + ACCEPT_FACTOR_CONFIG_NAME);
334    Preconditions.checkArgument(extraFreeFactor >= 0, EXTRA_FREE_FACTOR_CONFIG_NAME + " must be greater than 0.0");
335    Preconditions.checkArgument(singleFactor <= 1 && singleFactor >= 0, SINGLE_FACTOR_CONFIG_NAME + " must be between 0.0 and 1.0");
336    Preconditions.checkArgument(multiFactor <= 1 && multiFactor >= 0, MULTI_FACTOR_CONFIG_NAME + " must be between 0.0 and 1.0");
337    Preconditions.checkArgument(memoryFactor <= 1 && memoryFactor >= 0, MEMORY_FACTOR_CONFIG_NAME + " must be between 0.0 and 1.0");
338    Preconditions.checkArgument((singleFactor + multiFactor + memoryFactor) == 1, SINGLE_FACTOR_CONFIG_NAME + ", " +
339        MULTI_FACTOR_CONFIG_NAME + ", and " + MEMORY_FACTOR_CONFIG_NAME + " segments must add up to 1.0");
340  }
341
342  /**
343   * Called by the constructor to start the writer threads. Used by tests that need to override
344   * starting the threads.
345   */
346  protected void startWriterThreads() {
347    for (WriterThread thread : writerThreads) {
348      thread.start();
349    }
350  }
351
352  boolean isCacheEnabled() {
353    return this.cacheEnabled;
354  }
355
356  @Override
357  public long getMaxSize() {
358    return this.cacheCapacity;
359  }
360
361  public String getIoEngine() {
362    return ioEngine.toString();
363  }
364
365  /**
366   * Get the IOEngine from the IO engine name
367   * @param ioEngineName
368   * @param capacity
369   * @param persistencePath
370   * @return the IOEngine
371   * @throws IOException
372   */
373  private IOEngine getIOEngineFromName(String ioEngineName, long capacity, String persistencePath)
374      throws IOException {
375    if (ioEngineName.startsWith("file:") || ioEngineName.startsWith("files:")) {
376      // In order to make the usage simple, we only need the prefix 'files:' in
377      // document whether one or multiple file(s), but also support 'file:' for
378      // the compatibility
379      String[] filePaths = ioEngineName.substring(ioEngineName.indexOf(":") + 1)
380          .split(FileIOEngine.FILE_DELIMITER);
381      return new FileIOEngine(capacity, persistencePath != null, filePaths);
382    } else if (ioEngineName.startsWith("offheap")) {
383      return new ByteBufferIOEngine(capacity);
384    } else if (ioEngineName.startsWith("mmap:")) {
385      return new ExclusiveMemoryMmapIOEngine(ioEngineName.substring(5), capacity);
386    } else if (ioEngineName.startsWith("pmem:")) {
387      // This mode of bucket cache creates an IOEngine over a file on the persistent memory
388      // device. Since the persistent memory device has its own address space the contents
389      // mapped to this address space does not get swapped out like in the case of mmapping
390      // on to DRAM. Hence the cells created out of the hfile blocks in the pmem bucket cache
391      // can be directly referred to without having to copy them onheap. Once the RPC is done,
392      // the blocks can be returned back as in case of ByteBufferIOEngine.
393      return new SharedMemoryMmapIOEngine(ioEngineName.substring(5), capacity);
394    } else {
395      throw new IllegalArgumentException(
396          "Don't understand io engine name for cache- prefix with file:, files:, mmap: or offheap");
397    }
398  }
399
400  /**
401   * Cache the block with the specified name and buffer.
402   * @param cacheKey block's cache key
403   * @param buf block buffer
404   */
405  @Override
406  public void cacheBlock(BlockCacheKey cacheKey, Cacheable buf) {
407    cacheBlock(cacheKey, buf, false);
408  }
409
410  /**
411   * Cache the block with the specified name and buffer.
412   * @param cacheKey block's cache key
413   * @param cachedItem block buffer
414   * @param inMemory if block is in-memory
415   */
416  @Override
417  public void cacheBlock(BlockCacheKey cacheKey, Cacheable cachedItem, boolean inMemory) {
418    cacheBlockWithWait(cacheKey, cachedItem, inMemory, wait_when_cache);
419  }
420
421  /**
422   * Cache the block to ramCache
423   * @param cacheKey block's cache key
424   * @param cachedItem block buffer
425   * @param inMemory if block is in-memory
426   * @param wait if true, blocking wait when queue is full
427   */
428  public void cacheBlockWithWait(BlockCacheKey cacheKey, Cacheable cachedItem, boolean inMemory,
429      boolean wait) {
430    if (cacheEnabled) {
431      if (backingMap.containsKey(cacheKey) || ramCache.containsKey(cacheKey)) {
432        if (BlockCacheUtil.shouldReplaceExistingCacheBlock(this, cacheKey, cachedItem)) {
433          cacheBlockWithWaitInternal(cacheKey, cachedItem, inMemory, wait);
434        }
435      } else {
436        cacheBlockWithWaitInternal(cacheKey, cachedItem, inMemory, wait);
437      }
438    }
439  }
440
441  private void cacheBlockWithWaitInternal(BlockCacheKey cacheKey, Cacheable cachedItem,
442      boolean inMemory, boolean wait) {
443    if (!cacheEnabled) {
444      return;
445    }
446    LOG.trace("Caching key={}, item={}", cacheKey, cachedItem);
447    // Stuff the entry into the RAM cache so it can get drained to the persistent store
448    RAMQueueEntry re =
449        new RAMQueueEntry(cacheKey, cachedItem, accessCount.incrementAndGet(), inMemory,
450              createRecycler(cacheKey));
451    /**
452     * Don't use ramCache.put(cacheKey, re) here. because there may be a existing entry with same
453     * key in ramCache, the heap size of bucket cache need to update if replacing entry from
454     * ramCache. But WriterThread will also remove entry from ramCache and update heap size, if
455     * using ramCache.put(), It's possible that the removed entry in WriterThread is not the correct
456     * one, then the heap size will mess up (HBASE-20789)
457     */
458    if (ramCache.putIfAbsent(cacheKey, re) != null) {
459      return;
460    }
461    int queueNum = (cacheKey.hashCode() & 0x7FFFFFFF) % writerQueues.size();
462    BlockingQueue<RAMQueueEntry> bq = writerQueues.get(queueNum);
463    boolean successfulAddition = false;
464    if (wait) {
465      try {
466        successfulAddition = bq.offer(re, DEFAULT_CACHE_WAIT_TIME, TimeUnit.MILLISECONDS);
467      } catch (InterruptedException e) {
468        Thread.currentThread().interrupt();
469      }
470    } else {
471      successfulAddition = bq.offer(re);
472    }
473    if (!successfulAddition) {
474      ramCache.remove(cacheKey);
475      cacheStats.failInsert();
476    } else {
477      this.blockNumber.increment();
478      this.heapSize.add(cachedItem.heapSize());
479      blocksByHFile.add(cacheKey);
480    }
481  }
482
483  /**
484   * Get the buffer of the block with the specified key.
485   * @param key block's cache key
486   * @param caching true if the caller caches blocks on cache misses
487   * @param repeat Whether this is a repeat lookup for the same block
488   * @param updateCacheMetrics Whether we should update cache metrics or not
489   * @return buffer of specified cache key, or null if not in cache
490   */
491  @Override
492  public Cacheable getBlock(BlockCacheKey key, boolean caching, boolean repeat,
493      boolean updateCacheMetrics) {
494    if (!cacheEnabled) {
495      return null;
496    }
497    RAMQueueEntry re = ramCache.get(key);
498    if (re != null) {
499      if (updateCacheMetrics) {
500        cacheStats.hit(caching, key.isPrimary(), key.getBlockType());
501      }
502      re.access(accessCount.incrementAndGet());
503      return re.getData();
504    }
505    BucketEntry bucketEntry = backingMap.get(key);
506    if (bucketEntry != null) {
507      long start = System.nanoTime();
508      ReentrantReadWriteLock lock = offsetLock.getLock(bucketEntry.offset());
509      try {
510        lock.readLock().lock();
511        // We can not read here even if backingMap does contain the given key because its offset
512        // maybe changed. If we lock BlockCacheKey instead of offset, then we can only check
513        // existence here.
514        if (bucketEntry.equals(backingMap.get(key))) {
515          // Read the block from IOEngine based on the bucketEntry's offset and length, NOTICE: the
516          // block will use the refCnt of bucketEntry, which means if two HFileBlock mapping to
517          // the same BucketEntry, then all of the three will share the same refCnt.
518          Cacheable cachedBlock = ioEngine.read(bucketEntry);
519          if (ioEngine.usesSharedMemory()) {
520            // If IOEngine use shared memory, cachedBlock and BucketEntry will share the
521            // same RefCnt, do retain here, in order to count the number of RPC references
522            cachedBlock.retain();
523          }
524          // Update the cache statistics.
525          if (updateCacheMetrics) {
526            cacheStats.hit(caching, key.isPrimary(), key.getBlockType());
527            cacheStats.ioHit(System.nanoTime() - start);
528          }
529          bucketEntry.access(accessCount.incrementAndGet());
530          if (this.ioErrorStartTime > 0) {
531            ioErrorStartTime = -1;
532          }
533          return cachedBlock;
534        }
535      } catch (IOException ioex) {
536        LOG.error("Failed reading block " + key + " from bucket cache", ioex);
537        checkIOErrorIsTolerated();
538      } finally {
539        lock.readLock().unlock();
540      }
541    }
542    if (!repeat && updateCacheMetrics) {
543      cacheStats.miss(caching, key.isPrimary(), key.getBlockType());
544    }
545    return null;
546  }
547
548  void blockEvicted(BlockCacheKey cacheKey, BucketEntry bucketEntry, boolean decrementBlockNumber) {
549    bucketAllocator.freeBlock(bucketEntry.offset());
550    realCacheSize.add(-1 * bucketEntry.getLength());
551    blocksByHFile.remove(cacheKey);
552    if (decrementBlockNumber) {
553      this.blockNumber.decrement();
554    }
555  }
556
557  /**
558   * Try to evict the block from {@link BlockCache} by force. We'll call this in few cases:<br>
559   * 1. Close an HFile, and clear all cached blocks. <br>
560   * 2. Call {@link Admin#clearBlockCache(TableName)} to clear all blocks for a given table.<br>
561   * <p>
562   * Firstly, we'll try to remove the block from RAMCache. If it doesn't exist in RAMCache, then try
563   * to evict from backingMap. Here we only need to free the reference from bucket cache by calling
564   * {@link BucketEntry#markedAsEvicted}. If there're still some RPC referring this block, block can
565   * only be de-allocated when all of them release the block.
566   * <p>
567   * NOTICE: we need to grab the write offset lock firstly before releasing the reference from
568   * bucket cache. if we don't, we may read an {@link BucketEntry} with refCnt = 0 when
569   * {@link BucketCache#getBlock(BlockCacheKey, boolean, boolean, boolean)}, it's a memory leak.
570   * @param cacheKey Block to evict
571   * @return true to indicate whether we've evicted successfully or not.
572   */
573  @Override
574  public boolean evictBlock(BlockCacheKey cacheKey) {
575    if (!cacheEnabled) {
576      return false;
577    }
578    boolean existed = removeFromRamCache(cacheKey);
579    BucketEntry be = backingMap.get(cacheKey);
580    if (be == null) {
581      if (existed) {
582        cacheStats.evicted(0, cacheKey.isPrimary());
583      }
584      return existed;
585    } else {
586      return be.withWriteLock(offsetLock, be::markAsEvicted);
587    }
588  }
589
590  private Recycler createRecycler(BlockCacheKey cacheKey) {
591    return () -> {
592      if (!cacheEnabled) {
593        return;
594      }
595      boolean existed = removeFromRamCache(cacheKey);
596      BucketEntry be = backingMap.get(cacheKey);
597      if (be == null && existed) {
598        cacheStats.evicted(0, cacheKey.isPrimary());
599      } else if (be != null) {
600        be.withWriteLock(offsetLock, () -> {
601          if (backingMap.remove(cacheKey, be)) {
602            blockEvicted(cacheKey, be, !existed);
603            cacheStats.evicted(be.getCachedTime(), cacheKey.isPrimary());
604          }
605          return null;
606        });
607      }
608    };
609  }
610
611  private boolean removeFromRamCache(BlockCacheKey cacheKey) {
612    return ramCache.remove(cacheKey, re -> {
613      if (re != null) {
614        this.blockNumber.decrement();
615        this.heapSize.add(-1 * re.getData().heapSize());
616      }
617    });
618  }
619
620  /*
621   * Statistics thread.  Periodically output cache statistics to the log.
622   */
623  private static class StatisticsThread extends Thread {
624    private final BucketCache bucketCache;
625
626    public StatisticsThread(BucketCache bucketCache) {
627      super("BucketCacheStatsThread");
628      setDaemon(true);
629      this.bucketCache = bucketCache;
630    }
631
632    @Override
633    public void run() {
634      bucketCache.logStats();
635    }
636  }
637
638  public void logStats() {
639    long totalSize = bucketAllocator.getTotalSize();
640    long usedSize = bucketAllocator.getUsedSize();
641    long freeSize = totalSize - usedSize;
642    long cacheSize = getRealCacheSize();
643    LOG.info("failedBlockAdditions=" + cacheStats.getFailedInserts() + ", " +
644        "totalSize=" + StringUtils.byteDesc(totalSize) + ", " +
645        "freeSize=" + StringUtils.byteDesc(freeSize) + ", " +
646        "usedSize=" + StringUtils.byteDesc(usedSize) +", " +
647        "cacheSize=" + StringUtils.byteDesc(cacheSize) +", " +
648        "accesses=" + cacheStats.getRequestCount() + ", " +
649        "hits=" + cacheStats.getHitCount() + ", " +
650        "IOhitsPerSecond=" + cacheStats.getIOHitsPerSecond() + ", " +
651        "IOTimePerHit=" + String.format("%.2f", cacheStats.getIOTimePerHit())+ ", " +
652        "hitRatio=" + (cacheStats.getHitCount() == 0 ? "0," :
653          (StringUtils.formatPercent(cacheStats.getHitRatio(), 2)+ ", ")) +
654        "cachingAccesses=" + cacheStats.getRequestCachingCount() + ", " +
655        "cachingHits=" + cacheStats.getHitCachingCount() + ", " +
656        "cachingHitsRatio=" +(cacheStats.getHitCachingCount() == 0 ? "0," :
657          (StringUtils.formatPercent(cacheStats.getHitCachingRatio(), 2)+ ", ")) +
658        "evictions=" + cacheStats.getEvictionCount() + ", " +
659        "evicted=" + cacheStats.getEvictedCount() + ", " +
660        "evictedPerRun=" + cacheStats.evictedPerEviction());
661    cacheStats.reset();
662  }
663
664  public long getRealCacheSize() {
665    return this.realCacheSize.sum();
666  }
667
668  public long acceptableSize() {
669    return (long) Math.floor(bucketAllocator.getTotalSize() * acceptableFactor);
670  }
671
672  long getPartitionSize(float partitionFactor) {
673    return (long) Math.floor(bucketAllocator.getTotalSize() * partitionFactor * minFactor);
674  }
675
676  /**
677   * Return the count of bucketSizeinfos still need free space
678   */
679  private int bucketSizesAboveThresholdCount(float minFactor) {
680    BucketAllocator.IndexStatistics[] stats = bucketAllocator.getIndexStatistics();
681    int fullCount = 0;
682    for (int i = 0; i < stats.length; i++) {
683      long freeGoal = (long) Math.floor(stats[i].totalCount() * (1 - minFactor));
684      freeGoal = Math.max(freeGoal, 1);
685      if (stats[i].freeCount() < freeGoal) {
686        fullCount++;
687      }
688    }
689    return fullCount;
690  }
691
692  /**
693   * This method will find the buckets that are minimally occupied
694   * and are not reference counted and will free them completely
695   * without any constraint on the access times of the elements,
696   * and as a process will completely free at most the number of buckets
697   * passed, sometimes it might not due to changing refCounts
698   *
699   * @param completelyFreeBucketsNeeded number of buckets to free
700   **/
701  private void freeEntireBuckets(int completelyFreeBucketsNeeded) {
702    if (completelyFreeBucketsNeeded != 0) {
703      // First we will build a set where the offsets are reference counted, usually
704      // this set is small around O(Handler Count) unless something else is wrong
705      Set<Integer> inUseBuckets = new HashSet<>();
706      backingMap.forEach((k, be) -> {
707        if (be.isRpcRef()) {
708          inUseBuckets.add(bucketAllocator.getBucketIndex(be.offset()));
709        }
710      });
711      Set<Integer> candidateBuckets =
712          bucketAllocator.getLeastFilledBuckets(inUseBuckets, completelyFreeBucketsNeeded);
713      for (Map.Entry<BlockCacheKey, BucketEntry> entry : backingMap.entrySet()) {
714        if (candidateBuckets.contains(bucketAllocator.getBucketIndex(entry.getValue().offset()))) {
715          entry.getValue().withWriteLock(offsetLock, entry.getValue()::markStaleAsEvicted);
716        }
717      }
718    }
719  }
720
721  /**
722   * Free the space if the used size reaches acceptableSize() or one size block
723   * couldn't be allocated. When freeing the space, we use the LRU algorithm and
724   * ensure there must be some blocks evicted
725   * @param why Why we are being called
726   */
727  private void freeSpace(final String why) {
728    // Ensure only one freeSpace progress at a time
729    if (!freeSpaceLock.tryLock()) {
730      return;
731    }
732    try {
733      freeInProgress = true;
734      long bytesToFreeWithoutExtra = 0;
735      // Calculate free byte for each bucketSizeinfo
736      StringBuilder msgBuffer = LOG.isDebugEnabled()? new StringBuilder(): null;
737      BucketAllocator.IndexStatistics[] stats = bucketAllocator.getIndexStatistics();
738      long[] bytesToFreeForBucket = new long[stats.length];
739      for (int i = 0; i < stats.length; i++) {
740        bytesToFreeForBucket[i] = 0;
741        long freeGoal = (long) Math.floor(stats[i].totalCount() * (1 - minFactor));
742        freeGoal = Math.max(freeGoal, 1);
743        if (stats[i].freeCount() < freeGoal) {
744          bytesToFreeForBucket[i] = stats[i].itemSize() * (freeGoal - stats[i].freeCount());
745          bytesToFreeWithoutExtra += bytesToFreeForBucket[i];
746          if (msgBuffer != null) {
747            msgBuffer.append("Free for bucketSize(" + stats[i].itemSize() + ")="
748              + StringUtils.byteDesc(bytesToFreeForBucket[i]) + ", ");
749          }
750        }
751      }
752      if (msgBuffer != null) {
753        msgBuffer.append("Free for total=" + StringUtils.byteDesc(bytesToFreeWithoutExtra) + ", ");
754      }
755
756      if (bytesToFreeWithoutExtra <= 0) {
757        return;
758      }
759      long currentSize = bucketAllocator.getUsedSize();
760      long totalSize = bucketAllocator.getTotalSize();
761      if (LOG.isDebugEnabled() && msgBuffer != null) {
762        LOG.debug("Free started because \"" + why + "\"; " + msgBuffer.toString() +
763          " of current used=" + StringUtils.byteDesc(currentSize) + ", actual cacheSize=" +
764          StringUtils.byteDesc(realCacheSize.sum()) + ", total=" + StringUtils.byteDesc(totalSize));
765      }
766
767      long bytesToFreeWithExtra = (long) Math.floor(bytesToFreeWithoutExtra
768          * (1 + extraFreeFactor));
769
770      // Instantiate priority buckets
771      BucketEntryGroup bucketSingle = new BucketEntryGroup(bytesToFreeWithExtra,
772          blockSize, getPartitionSize(singleFactor));
773      BucketEntryGroup bucketMulti = new BucketEntryGroup(bytesToFreeWithExtra,
774          blockSize, getPartitionSize(multiFactor));
775      BucketEntryGroup bucketMemory = new BucketEntryGroup(bytesToFreeWithExtra,
776          blockSize, getPartitionSize(memoryFactor));
777
778      // Scan entire map putting bucket entry into appropriate bucket entry
779      // group
780      for (Map.Entry<BlockCacheKey, BucketEntry> bucketEntryWithKey : backingMap.entrySet()) {
781        switch (bucketEntryWithKey.getValue().getPriority()) {
782          case SINGLE: {
783            bucketSingle.add(bucketEntryWithKey);
784            break;
785          }
786          case MULTI: {
787            bucketMulti.add(bucketEntryWithKey);
788            break;
789          }
790          case MEMORY: {
791            bucketMemory.add(bucketEntryWithKey);
792            break;
793          }
794        }
795      }
796
797      PriorityQueue<BucketEntryGroup> bucketQueue = new PriorityQueue<>(3,
798          Comparator.comparingLong(BucketEntryGroup::overflow));
799
800      bucketQueue.add(bucketSingle);
801      bucketQueue.add(bucketMulti);
802      bucketQueue.add(bucketMemory);
803
804      int remainingBuckets = bucketQueue.size();
805      long bytesFreed = 0;
806
807      BucketEntryGroup bucketGroup;
808      while ((bucketGroup = bucketQueue.poll()) != null) {
809        long overflow = bucketGroup.overflow();
810        if (overflow > 0) {
811          long bucketBytesToFree = Math.min(overflow,
812              (bytesToFreeWithoutExtra - bytesFreed) / remainingBuckets);
813          bytesFreed += bucketGroup.free(bucketBytesToFree);
814        }
815        remainingBuckets--;
816      }
817
818      // Check and free if there are buckets that still need freeing of space
819      if (bucketSizesAboveThresholdCount(minFactor) > 0) {
820        bucketQueue.clear();
821        remainingBuckets = 3;
822
823        bucketQueue.add(bucketSingle);
824        bucketQueue.add(bucketMulti);
825        bucketQueue.add(bucketMemory);
826
827        while ((bucketGroup = bucketQueue.poll()) != null) {
828          long bucketBytesToFree = (bytesToFreeWithExtra - bytesFreed) / remainingBuckets;
829          bytesFreed += bucketGroup.free(bucketBytesToFree);
830          remainingBuckets--;
831        }
832      }
833
834      // Even after the above free we might still need freeing because of the
835      // De-fragmentation of the buckets (also called Slab Calcification problem), i.e
836      // there might be some buckets where the occupancy is very sparse and thus are not
837      // yielding the free for the other bucket sizes, the fix for this to evict some
838      // of the buckets, we do this by evicting the buckets that are least fulled
839      freeEntireBuckets(DEFAULT_FREE_ENTIRE_BLOCK_FACTOR *
840          bucketSizesAboveThresholdCount(1.0f));
841
842      if (LOG.isDebugEnabled()) {
843        long single = bucketSingle.totalSize();
844        long multi = bucketMulti.totalSize();
845        long memory = bucketMemory.totalSize();
846        if (LOG.isDebugEnabled()) {
847          LOG.debug("Bucket cache free space completed; " + "freed="
848            + StringUtils.byteDesc(bytesFreed) + ", " + "total="
849            + StringUtils.byteDesc(totalSize) + ", " + "single="
850            + StringUtils.byteDesc(single) + ", " + "multi="
851            + StringUtils.byteDesc(multi) + ", " + "memory="
852            + StringUtils.byteDesc(memory));
853        }
854      }
855
856    } catch (Throwable t) {
857      LOG.warn("Failed freeing space", t);
858    } finally {
859      cacheStats.evict();
860      freeInProgress = false;
861      freeSpaceLock.unlock();
862    }
863  }
864
865  // This handles flushing the RAM cache to IOEngine.
866  class WriterThread extends Thread {
867    private final BlockingQueue<RAMQueueEntry> inputQueue;
868    private volatile boolean writerEnabled = true;
869
870    WriterThread(BlockingQueue<RAMQueueEntry> queue) {
871      super("BucketCacheWriterThread");
872      this.inputQueue = queue;
873    }
874
875    // Used for test
876    void disableWriter() {
877      this.writerEnabled = false;
878    }
879
880    @Override
881    public void run() {
882      List<RAMQueueEntry> entries = new ArrayList<>();
883      try {
884        while (cacheEnabled && writerEnabled) {
885          try {
886            try {
887              // Blocks
888              entries = getRAMQueueEntries(inputQueue, entries);
889            } catch (InterruptedException ie) {
890              if (!cacheEnabled || !writerEnabled) {
891                break;
892              }
893            }
894            doDrain(entries);
895          } catch (Exception ioe) {
896            LOG.error("WriterThread encountered error", ioe);
897          }
898        }
899      } catch (Throwable t) {
900        LOG.warn("Failed doing drain", t);
901      }
902      LOG.info(this.getName() + " exiting, cacheEnabled=" + cacheEnabled);
903    }
904
905    /**
906     * Put the new bucket entry into backingMap. Notice that we are allowed to replace the existing
907     * cache with a new block for the same cache key. there's a corner case: one thread cache a
908     * block in ramCache, copy to io-engine and add a bucket entry to backingMap. Caching another
909     * new block with the same cache key do the same thing for the same cache key, so if not evict
910     * the previous bucket entry, then memory leak happen because the previous bucketEntry is gone
911     * but the bucketAllocator do not free its memory.
912     * @see BlockCacheUtil#shouldReplaceExistingCacheBlock(BlockCache blockCache,BlockCacheKey
913     *      cacheKey, Cacheable newBlock)
914     * @param key Block cache key
915     * @param bucketEntry Bucket entry to put into backingMap.
916     */
917    private void putIntoBackingMap(BlockCacheKey key, BucketEntry bucketEntry) {
918      BucketEntry previousEntry = backingMap.put(key, bucketEntry);
919      if (previousEntry != null && previousEntry != bucketEntry) {
920        previousEntry.withWriteLock(offsetLock, () -> {
921          blockEvicted(key, previousEntry, false);
922          return null;
923        });
924      }
925    }
926
927    /**
928     * Flush the entries in ramCache to IOEngine and add bucket entry to backingMap.
929     * Process all that are passed in even if failure being sure to remove from ramCache else we'll
930     * never undo the references and we'll OOME.
931     * @param entries Presumes list passed in here will be processed by this invocation only. No
932     *   interference expected.
933     * @throws InterruptedException
934     */
935    void doDrain(final List<RAMQueueEntry> entries) throws InterruptedException {
936      if (entries.isEmpty()) {
937        return;
938      }
939      // This method is a little hard to follow. We run through the passed in entries and for each
940      // successful add, we add a non-null BucketEntry to the below bucketEntries.  Later we must
941      // do cleanup making sure we've cleared ramCache of all entries regardless of whether we
942      // successfully added the item to the bucketcache; if we don't do the cleanup, we'll OOME by
943      // filling ramCache.  We do the clean up by again running through the passed in entries
944      // doing extra work when we find a non-null bucketEntries corresponding entry.
945      final int size = entries.size();
946      BucketEntry[] bucketEntries = new BucketEntry[size];
947      // Index updated inside loop if success or if we can't succeed. We retry if cache is full
948      // when we go to add an entry by going around the loop again without upping the index.
949      int index = 0;
950      while (cacheEnabled && index < size) {
951        RAMQueueEntry re = null;
952        try {
953          re = entries.get(index);
954          if (re == null) {
955            LOG.warn("Couldn't get entry or changed on us; who else is messing with it?");
956            index++;
957            continue;
958          }
959          BucketEntry bucketEntry = re.writeToCache(ioEngine, bucketAllocator, realCacheSize);
960          // Successfully added. Up index and add bucketEntry. Clear io exceptions.
961          bucketEntries[index] = bucketEntry;
962          if (ioErrorStartTime > 0) {
963            ioErrorStartTime = -1;
964          }
965          index++;
966        } catch (BucketAllocatorException fle) {
967          LOG.warn("Failed allocation for " + (re == null ? "" : re.getKey()) + "; " + fle);
968          // Presume can't add. Too big? Move index on. Entry will be cleared from ramCache below.
969          bucketEntries[index] = null;
970          index++;
971        } catch (CacheFullException cfe) {
972          // Cache full when we tried to add. Try freeing space and then retrying (don't up index)
973          if (!freeInProgress) {
974            freeSpace("Full!");
975          } else {
976            Thread.sleep(50);
977          }
978        } catch (IOException ioex) {
979          // Hopefully transient. Retry. checkIOErrorIsTolerated disables cache if problem.
980          LOG.error("Failed writing to bucket cache", ioex);
981          checkIOErrorIsTolerated();
982        }
983      }
984
985      // Make sure data pages are written on media before we update maps.
986      try {
987        ioEngine.sync();
988      } catch (IOException ioex) {
989        LOG.error("Failed syncing IO engine", ioex);
990        checkIOErrorIsTolerated();
991        // Since we failed sync, free the blocks in bucket allocator
992        for (int i = 0; i < entries.size(); ++i) {
993          if (bucketEntries[i] != null) {
994            bucketAllocator.freeBlock(bucketEntries[i].offset());
995            bucketEntries[i] = null;
996          }
997        }
998      }
999
1000      // Now add to backingMap if successfully added to bucket cache.  Remove from ramCache if
1001      // success or error.
1002      for (int i = 0; i < size; ++i) {
1003        BlockCacheKey key = entries.get(i).getKey();
1004        // Only add if non-null entry.
1005        if (bucketEntries[i] != null) {
1006          putIntoBackingMap(key, bucketEntries[i]);
1007        }
1008        // Always remove from ramCache even if we failed adding it to the block cache above.
1009        boolean existed = ramCache.remove(key, re -> {
1010          if (re != null) {
1011            heapSize.add(-1 * re.getData().heapSize());
1012          }
1013        });
1014        if (!existed && bucketEntries[i] != null) {
1015          // Block should have already been evicted. Remove it and free space.
1016          final BucketEntry bucketEntry = bucketEntries[i];
1017          bucketEntry.withWriteLock(offsetLock, () -> {
1018            if (backingMap.remove(key, bucketEntry)) {
1019              blockEvicted(key, bucketEntry, false);
1020            }
1021            return null;
1022          });
1023        }
1024      }
1025
1026      long used = bucketAllocator.getUsedSize();
1027      if (used > acceptableSize()) {
1028        freeSpace("Used=" + used + " > acceptable=" + acceptableSize());
1029      }
1030      return;
1031    }
1032  }
1033
1034  /**
1035   * Blocks until elements available in {@code q} then tries to grab as many as possible before
1036   * returning.
1037   * @param receptacle Where to stash the elements taken from queue. We clear before we use it just
1038   *          in case.
1039   * @param q The queue to take from.
1040   * @return {@code receptacle} laden with elements taken from the queue or empty if none found.
1041   */
1042  static List<RAMQueueEntry> getRAMQueueEntries(BlockingQueue<RAMQueueEntry> q,
1043      List<RAMQueueEntry> receptacle) throws InterruptedException {
1044    // Clear sets all entries to null and sets size to 0. We retain allocations. Presume it
1045    // ok even if list grew to accommodate thousands.
1046    receptacle.clear();
1047    receptacle.add(q.take());
1048    q.drainTo(receptacle);
1049    return receptacle;
1050  }
1051
1052  /**
1053   * @see #retrieveFromFile(int[])
1054   */
1055  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="OBL_UNSATISFIED_OBLIGATION",
1056      justification = "false positive, try-with-resources ensures close is called.")
1057  private void persistToFile() throws IOException {
1058    assert !cacheEnabled;
1059    if (!ioEngine.isPersistent()) {
1060      throw new IOException("Attempt to persist non-persistent cache mappings!");
1061    }
1062    try (FileOutputStream fos = new FileOutputStream(persistencePath, false)) {
1063      fos.write(ProtobufMagic.PB_MAGIC);
1064      BucketProtoUtils.toPB(this).writeDelimitedTo(fos);
1065    }
1066  }
1067
1068  /**
1069   * @see #persistToFile()
1070   */
1071  private void retrieveFromFile(int[] bucketSizes) throws IOException {
1072    File persistenceFile = new File(persistencePath);
1073    if (!persistenceFile.exists()) {
1074      return;
1075    }
1076    assert !cacheEnabled;
1077
1078    try (FileInputStream in = deleteFileOnClose(persistenceFile)) {
1079      int pblen = ProtobufMagic.lengthOfPBMagic();
1080      byte[] pbuf = new byte[pblen];
1081      int read = in.read(pbuf);
1082      if (read != pblen) {
1083        throw new IOException("Incorrect number of bytes read while checking for protobuf magic "
1084            + "number. Requested=" + pblen + ", Received= " + read + ", File=" + persistencePath);
1085      }
1086      if (! ProtobufMagic.isPBMagicPrefix(pbuf)) {
1087        // In 3.0 we have enough flexibility to dump the old cache data.
1088        // TODO: In 2.x line, this might need to be filled in to support reading the old format
1089        throw new IOException("Persistence file does not start with protobuf magic number. " +
1090            persistencePath);
1091      }
1092      parsePB(BucketCacheProtos.BucketCacheEntry.parseDelimitedFrom(in));
1093      bucketAllocator = new BucketAllocator(cacheCapacity, bucketSizes, backingMap, realCacheSize);
1094      blockNumber.add(backingMap.size());
1095    }
1096  }
1097
1098  /**
1099   * Create an input stream that deletes the file after reading it. Use in try-with-resources to
1100   * avoid this pattern where an exception thrown from a finally block may mask earlier exceptions:
1101   * <pre>
1102   *   File f = ...
1103   *   try (FileInputStream fis = new FileInputStream(f)) {
1104   *     // use the input stream
1105   *   } finally {
1106   *     if (!f.delete()) throw new IOException("failed to delete");
1107   *   }
1108   * </pre>
1109   * @param file the file to read and delete
1110   * @return a FileInputStream for the given file
1111   * @throws IOException if there is a problem creating the stream
1112   */
1113  private FileInputStream deleteFileOnClose(final File file) throws IOException {
1114    return new FileInputStream(file) {
1115      @Override
1116      public void close() throws IOException {
1117        super.close();
1118        if (!file.delete()) {
1119          throw new IOException("Failed deleting persistence file " + file.getAbsolutePath());
1120        }
1121      }
1122    };
1123  }
1124
1125  private void verifyCapacityAndClasses(long capacitySize, String ioclass, String mapclass)
1126      throws IOException {
1127    if (capacitySize != cacheCapacity) {
1128      throw new IOException("Mismatched cache capacity:"
1129          + StringUtils.byteDesc(capacitySize) + ", expected: "
1130          + StringUtils.byteDesc(cacheCapacity));
1131    }
1132    if (!ioEngine.getClass().getName().equals(ioclass)) {
1133      throw new IOException("Class name for IO engine mismatch: " + ioclass
1134          + ", expected:" + ioEngine.getClass().getName());
1135    }
1136    if (!backingMap.getClass().getName().equals(mapclass)) {
1137      throw new IOException("Class name for cache map mismatch: " + mapclass
1138          + ", expected:" + backingMap.getClass().getName());
1139    }
1140  }
1141
1142  private void parsePB(BucketCacheProtos.BucketCacheEntry proto) throws IOException {
1143    if (proto.hasChecksum()) {
1144      ((PersistentIOEngine) ioEngine).verifyFileIntegrity(proto.getChecksum().toByteArray(),
1145        algorithm);
1146    } else {
1147      // if has not checksum, it means the persistence file is old format
1148      LOG.info("Persistent file is old format, it does not support verifying file integrity!");
1149    }
1150    verifyCapacityAndClasses(proto.getCacheCapacity(), proto.getIoClass(), proto.getMapClass());
1151    backingMap = BucketProtoUtils.fromPB(proto.getDeserializersMap(), proto.getBackingMap());
1152  }
1153
1154  /**
1155   * Check whether we tolerate IO error this time. If the duration of IOEngine
1156   * throwing errors exceeds ioErrorsDurationTimeTolerated, we will disable the
1157   * cache
1158   */
1159  private void checkIOErrorIsTolerated() {
1160    long now = EnvironmentEdgeManager.currentTime();
1161    // Do a single read to a local variable to avoid timing issue - HBASE-24454
1162    long ioErrorStartTimeTmp = this.ioErrorStartTime;
1163    if (ioErrorStartTimeTmp > 0) {
1164      if (cacheEnabled && (now - ioErrorStartTimeTmp) > this.ioErrorsTolerationDuration) {
1165        LOG.error("IO errors duration time has exceeded " + ioErrorsTolerationDuration +
1166          "ms, disabling cache, please check your IOEngine");
1167        disableCache();
1168      }
1169    } else {
1170      this.ioErrorStartTime = now;
1171    }
1172  }
1173
1174  /**
1175   * Used to shut down the cache -or- turn it off in the case of something broken.
1176   */
1177  private void disableCache() {
1178    if (!cacheEnabled) return;
1179    cacheEnabled = false;
1180    ioEngine.shutdown();
1181    this.scheduleThreadPool.shutdown();
1182    for (int i = 0; i < writerThreads.length; ++i) writerThreads[i].interrupt();
1183    this.ramCache.clear();
1184    if (!ioEngine.isPersistent() || persistencePath == null) {
1185      // If persistent ioengine and a path, we will serialize out the backingMap.
1186      this.backingMap.clear();
1187    }
1188  }
1189
1190  private void join() throws InterruptedException {
1191    for (int i = 0; i < writerThreads.length; ++i)
1192      writerThreads[i].join();
1193  }
1194
1195  @Override
1196  public void shutdown() {
1197    disableCache();
1198    LOG.info("Shutdown bucket cache: IO persistent=" + ioEngine.isPersistent()
1199        + "; path to write=" + persistencePath);
1200    if (ioEngine.isPersistent() && persistencePath != null) {
1201      try {
1202        join();
1203        persistToFile();
1204      } catch (IOException ex) {
1205        LOG.error("Unable to persist data on exit: " + ex.toString(), ex);
1206      } catch (InterruptedException e) {
1207        LOG.warn("Failed to persist data on exit", e);
1208      }
1209    }
1210  }
1211
1212  @Override
1213  public CacheStats getStats() {
1214    return cacheStats;
1215  }
1216
1217  public BucketAllocator getAllocator() {
1218    return this.bucketAllocator;
1219  }
1220
1221  @Override
1222  public long heapSize() {
1223    return this.heapSize.sum();
1224  }
1225
1226  @Override
1227  public long size() {
1228    return this.realCacheSize.sum();
1229  }
1230
1231  @Override
1232  public long getCurrentDataSize() {
1233    return size();
1234  }
1235
1236  @Override
1237  public long getFreeSize() {
1238    return this.bucketAllocator.getFreeSize();
1239  }
1240
1241  @Override
1242  public long getBlockCount() {
1243    return this.blockNumber.sum();
1244  }
1245
1246  @Override
1247  public long getDataBlockCount() {
1248    return getBlockCount();
1249  }
1250
1251  @Override
1252  public long getCurrentSize() {
1253    return this.bucketAllocator.getUsedSize();
1254  }
1255
1256  protected String getAlgorithm() {
1257    return algorithm;
1258  }
1259
1260  /**
1261   * Evicts all blocks for a specific HFile.
1262   * <p>
1263   * This is used for evict-on-close to remove all blocks of a specific HFile.
1264   *
1265   * @return the number of blocks evicted
1266   */
1267  @Override
1268  public int evictBlocksByHfileName(String hfileName) {
1269    Set<BlockCacheKey> keySet = blocksByHFile.subSet(
1270        new BlockCacheKey(hfileName, Long.MIN_VALUE), true,
1271        new BlockCacheKey(hfileName, Long.MAX_VALUE), true);
1272
1273    int numEvicted = 0;
1274    for (BlockCacheKey key : keySet) {
1275      if (evictBlock(key)) {
1276          ++numEvicted;
1277      }
1278    }
1279
1280    return numEvicted;
1281  }
1282
1283  /**
1284   * Used to group bucket entries into priority buckets. There will be a
1285   * BucketEntryGroup for each priority (single, multi, memory). Once bucketed,
1286   * the eviction algorithm takes the appropriate number of elements out of each
1287   * according to configuration parameters and their relative sizes.
1288   */
1289  private class BucketEntryGroup {
1290
1291    private CachedEntryQueue queue;
1292    private long totalSize = 0;
1293    private long bucketSize;
1294
1295    public BucketEntryGroup(long bytesToFree, long blockSize, long bucketSize) {
1296      this.bucketSize = bucketSize;
1297      queue = new CachedEntryQueue(bytesToFree, blockSize);
1298      totalSize = 0;
1299    }
1300
1301    public void add(Map.Entry<BlockCacheKey, BucketEntry> block) {
1302      totalSize += block.getValue().getLength();
1303      queue.add(block);
1304    }
1305
1306    public long free(long toFree) {
1307      Map.Entry<BlockCacheKey, BucketEntry> entry;
1308      long freedBytes = 0;
1309      // TODO avoid a cycling siutation. We find no block which is not in use and so no way to free
1310      // What to do then? Caching attempt fail? Need some changes in cacheBlock API?
1311      while ((entry = queue.pollLast()) != null) {
1312        BucketEntry be = entry.getValue();
1313        if (be.withWriteLock(offsetLock, be::markStaleAsEvicted)) {
1314          freedBytes += be.getLength();
1315        }
1316        if (freedBytes >= toFree) {
1317          return freedBytes;
1318        }
1319      }
1320      return freedBytes;
1321    }
1322
1323    public long overflow() {
1324      return totalSize - bucketSize;
1325    }
1326
1327    public long totalSize() {
1328      return totalSize;
1329    }
1330  }
1331
1332  /**
1333   * Block Entry stored in the memory with key,data and so on
1334   */
1335  static class RAMQueueEntry {
1336    private final BlockCacheKey key;
1337    private final Cacheable data;
1338    private long accessCounter;
1339    private boolean inMemory;
1340    private final Recycler recycler;
1341
1342    RAMQueueEntry(BlockCacheKey bck, Cacheable data, long accessCounter, boolean inMemory,
1343        Recycler recycler) {
1344      this.key = bck;
1345      this.data = data;
1346      this.accessCounter = accessCounter;
1347      this.inMemory = inMemory;
1348      this.recycler = recycler;
1349    }
1350
1351    public Cacheable getData() {
1352      return data;
1353    }
1354
1355    public BlockCacheKey getKey() {
1356      return key;
1357    }
1358
1359    public void access(long accessCounter) {
1360      this.accessCounter = accessCounter;
1361    }
1362
1363    private ByteBuffAllocator getByteBuffAllocator() {
1364      if (data instanceof HFileBlock) {
1365        return ((HFileBlock) data).getByteBuffAllocator();
1366      }
1367      return ByteBuffAllocator.HEAP;
1368    }
1369
1370    public BucketEntry writeToCache(final IOEngine ioEngine, final BucketAllocator alloc,
1371        final LongAdder realCacheSize) throws IOException {
1372      int len = data.getSerializedLength();
1373      // This cacheable thing can't be serialized
1374      if (len == 0) {
1375        return null;
1376      }
1377      long offset = alloc.allocateBlock(len);
1378      boolean succ = false;
1379      BucketEntry bucketEntry = null;
1380      try {
1381        bucketEntry = new BucketEntry(offset, len, accessCounter, inMemory, RefCnt.create(recycler),
1382            getByteBuffAllocator());
1383        bucketEntry.setDeserializerReference(data.getDeserializer());
1384        if (data instanceof HFileBlock) {
1385          // If an instance of HFileBlock, save on some allocations.
1386          HFileBlock block = (HFileBlock) data;
1387          ByteBuff sliceBuf = block.getBufferReadOnly();
1388          ByteBuffer metadata = block.getMetaData();
1389          ioEngine.write(sliceBuf, offset);
1390          ioEngine.write(metadata, offset + len - metadata.limit());
1391        } else {
1392          // Only used for testing.
1393          ByteBuffer bb = ByteBuffer.allocate(len);
1394          data.serialize(bb, true);
1395          ioEngine.write(bb, offset);
1396        }
1397        succ = true;
1398      } finally {
1399        if (!succ) {
1400          alloc.freeBlock(offset);
1401        }
1402      }
1403      realCacheSize.add(len);
1404      return bucketEntry;
1405    }
1406  }
1407
1408  /**
1409   * Only used in test
1410   * @throws InterruptedException
1411   */
1412  void stopWriterThreads() throws InterruptedException {
1413    for (WriterThread writerThread : writerThreads) {
1414      writerThread.disableWriter();
1415      writerThread.interrupt();
1416      writerThread.join();
1417    }
1418  }
1419
1420  @Override
1421  public Iterator<CachedBlock> iterator() {
1422    // Don't bother with ramcache since stuff is in here only a little while.
1423    final Iterator<Map.Entry<BlockCacheKey, BucketEntry>> i =
1424        this.backingMap.entrySet().iterator();
1425    return new Iterator<CachedBlock>() {
1426      private final long now = System.nanoTime();
1427
1428      @Override
1429      public boolean hasNext() {
1430        return i.hasNext();
1431      }
1432
1433      @Override
1434      public CachedBlock next() {
1435        final Map.Entry<BlockCacheKey, BucketEntry> e = i.next();
1436        return new CachedBlock() {
1437          @Override
1438          public String toString() {
1439            return BlockCacheUtil.toString(this, now);
1440          }
1441
1442          @Override
1443          public BlockPriority getBlockPriority() {
1444            return e.getValue().getPriority();
1445          }
1446
1447          @Override
1448          public BlockType getBlockType() {
1449            // Not held by BucketEntry.  Could add it if wanted on BucketEntry creation.
1450            return null;
1451          }
1452
1453          @Override
1454          public long getOffset() {
1455            return e.getKey().getOffset();
1456          }
1457
1458          @Override
1459          public long getSize() {
1460            return e.getValue().getLength();
1461          }
1462
1463          @Override
1464          public long getCachedTime() {
1465            return e.getValue().getCachedTime();
1466          }
1467
1468          @Override
1469          public String getFilename() {
1470            return e.getKey().getHfileName();
1471          }
1472
1473          @Override
1474          public int compareTo(CachedBlock other) {
1475            int diff = this.getFilename().compareTo(other.getFilename());
1476            if (diff != 0) return diff;
1477
1478            diff = Long.compare(this.getOffset(), other.getOffset());
1479            if (diff != 0) return diff;
1480            if (other.getCachedTime() < 0 || this.getCachedTime() < 0) {
1481              throw new IllegalStateException("" + this.getCachedTime() + ", " +
1482                other.getCachedTime());
1483            }
1484            return Long.compare(other.getCachedTime(), this.getCachedTime());
1485          }
1486
1487          @Override
1488          public int hashCode() {
1489            return e.getKey().hashCode();
1490          }
1491
1492          @Override
1493          public boolean equals(Object obj) {
1494            if (obj instanceof CachedBlock) {
1495              CachedBlock cb = (CachedBlock)obj;
1496              return compareTo(cb) == 0;
1497            } else {
1498              return false;
1499            }
1500          }
1501        };
1502      }
1503
1504      @Override
1505      public void remove() {
1506        throw new UnsupportedOperationException();
1507      }
1508    };
1509  }
1510
1511  @Override
1512  public BlockCache[] getBlockCaches() {
1513    return null;
1514  }
1515
1516  public int getRpcRefCount(BlockCacheKey cacheKey) {
1517    BucketEntry bucketEntry = backingMap.get(cacheKey);
1518    if (bucketEntry != null) {
1519      return bucketEntry.refCnt() - (bucketEntry.markedAsEvicted.get() ? 0 : 1);
1520    }
1521    return 0;
1522  }
1523
1524  float getAcceptableFactor() {
1525    return acceptableFactor;
1526  }
1527
1528  float getMinFactor() {
1529    return minFactor;
1530  }
1531
1532  float getExtraFreeFactor() {
1533    return extraFreeFactor;
1534  }
1535
1536  float getSingleFactor() {
1537    return singleFactor;
1538  }
1539
1540  float getMultiFactor() {
1541    return multiFactor;
1542  }
1543
1544  float getMemoryFactor() {
1545    return memoryFactor;
1546  }
1547
1548  /**
1549   * Wrapped the delegate ConcurrentMap with maintaining its block's reference count.
1550   */
1551  static class RAMCache {
1552    /**
1553     * Defined the map as {@link ConcurrentHashMap} explicitly here, because in
1554     * {@link RAMCache#get(BlockCacheKey)} and
1555     * {@link RAMCache#putIfAbsent(BlockCacheKey, BucketCache.RAMQueueEntry)} , we need to
1556     * guarantee the atomicity of map#computeIfPresent(key, func) and map#putIfAbsent(key, func).
1557     * Besides, the func method can execute exactly once only when the key is present(or absent)
1558     * and under the lock context. Otherwise, the reference count of block will be messed up.
1559     * Notice that the {@link java.util.concurrent.ConcurrentSkipListMap} can not guarantee that.
1560     */
1561    final ConcurrentHashMap<BlockCacheKey, RAMQueueEntry> delegate = new ConcurrentHashMap<>();
1562
1563    public boolean containsKey(BlockCacheKey key) {
1564      return delegate.containsKey(key);
1565    }
1566
1567    public RAMQueueEntry get(BlockCacheKey key) {
1568      return delegate.computeIfPresent(key, (k, re) -> {
1569        // It'll be referenced by RPC, so retain atomically here. if the get and retain is not
1570        // atomic, another thread may remove and release the block, when retaining in this thread we
1571        // may retain a block with refCnt=0 which is disallowed. (see HBASE-22422)
1572        re.getData().retain();
1573        return re;
1574      });
1575    }
1576
1577    /**
1578     * Return the previous associated value, or null if absent. It has the same meaning as
1579     * {@link ConcurrentMap#putIfAbsent(Object, Object)}
1580     */
1581    public RAMQueueEntry putIfAbsent(BlockCacheKey key, RAMQueueEntry entry) {
1582      AtomicBoolean absent = new AtomicBoolean(false);
1583      RAMQueueEntry re = delegate.computeIfAbsent(key, k -> {
1584        // The RAMCache reference to this entry, so reference count should be increment.
1585        entry.getData().retain();
1586        absent.set(true);
1587        return entry;
1588      });
1589      return absent.get() ? null : re;
1590    }
1591
1592    public boolean remove(BlockCacheKey key) {
1593      return remove(key, re->{});
1594    }
1595
1596    /**
1597     * Defined an {@link Consumer} here, because once the removed entry release its reference count,
1598     * then it's ByteBuffers may be recycled and accessing it outside this method will be thrown an
1599     * exception. the consumer will access entry to remove before release its reference count.
1600     * Notice, don't change its reference count in the {@link Consumer}
1601     */
1602    public boolean remove(BlockCacheKey key, Consumer<RAMQueueEntry> action) {
1603      RAMQueueEntry previous = delegate.remove(key);
1604      action.accept(previous);
1605      if (previous != null) {
1606        previous.getData().release();
1607      }
1608      return previous != null;
1609    }
1610
1611    public boolean isEmpty() {
1612      return delegate.isEmpty();
1613    }
1614
1615    public void clear() {
1616      Iterator<Map.Entry<BlockCacheKey, RAMQueueEntry>> it = delegate.entrySet().iterator();
1617      while (it.hasNext()) {
1618        RAMQueueEntry re = it.next().getValue();
1619        it.remove();
1620        re.getData().release();
1621      }
1622    }
1623  }
1624}