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