001/*
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018package org.apache.hadoop.hbase.io.hfile.bucket;
019
020import static org.apache.hadoop.hbase.io.hfile.BlockCacheFactory.BLOCKCACHE_STATS_PERIODS;
021import static org.apache.hadoop.hbase.io.hfile.BlockCacheFactory.BLOCKCACHE_STATS_PERIOD_MINUTES_KEY;
022import static org.apache.hadoop.hbase.io.hfile.BlockCacheFactory.DEFAULT_BLOCKCACHE_STATS_PERIODS;
023import static org.apache.hadoop.hbase.io.hfile.BlockCacheFactory.DEFAULT_BLOCKCACHE_STATS_PERIOD_MINUTES;
024import static org.apache.hadoop.hbase.io.hfile.CacheConfig.BUCKETCACHE_PERSIST_INTERVAL_KEY;
025
026import java.io.File;
027import java.io.FileInputStream;
028import java.io.FileOutputStream;
029import java.io.IOException;
030import java.nio.ByteBuffer;
031import java.util.ArrayList;
032import java.util.Arrays;
033import java.util.Collections;
034import java.util.Comparator;
035import java.util.HashSet;
036import java.util.Iterator;
037import java.util.List;
038import java.util.Map;
039import java.util.NavigableSet;
040import java.util.Optional;
041import java.util.PriorityQueue;
042import java.util.Set;
043import java.util.concurrent.ArrayBlockingQueue;
044import java.util.concurrent.BlockingQueue;
045import java.util.concurrent.ConcurrentHashMap;
046import java.util.concurrent.ConcurrentMap;
047import java.util.concurrent.ConcurrentSkipListSet;
048import java.util.concurrent.Executors;
049import java.util.concurrent.ScheduledExecutorService;
050import java.util.concurrent.TimeUnit;
051import java.util.concurrent.atomic.AtomicBoolean;
052import java.util.concurrent.atomic.AtomicLong;
053import java.util.concurrent.atomic.LongAdder;
054import java.util.concurrent.locks.Lock;
055import java.util.concurrent.locks.ReentrantLock;
056import java.util.concurrent.locks.ReentrantReadWriteLock;
057import java.util.function.Consumer;
058import java.util.function.Function;
059import org.apache.commons.io.IOUtils;
060import org.apache.commons.lang3.mutable.MutableInt;
061import org.apache.hadoop.conf.Configuration;
062import org.apache.hadoop.fs.Path;
063import org.apache.hadoop.hbase.HBaseConfiguration;
064import org.apache.hadoop.hbase.HBaseIOException;
065import org.apache.hadoop.hbase.TableName;
066import org.apache.hadoop.hbase.client.Admin;
067import org.apache.hadoop.hbase.io.ByteBuffAllocator;
068import org.apache.hadoop.hbase.io.ByteBuffAllocator.Recycler;
069import org.apache.hadoop.hbase.io.HeapSize;
070import org.apache.hadoop.hbase.io.hfile.BlockCache;
071import org.apache.hadoop.hbase.io.hfile.BlockCacheKey;
072import org.apache.hadoop.hbase.io.hfile.BlockCacheUtil;
073import org.apache.hadoop.hbase.io.hfile.BlockPriority;
074import org.apache.hadoop.hbase.io.hfile.BlockType;
075import org.apache.hadoop.hbase.io.hfile.CacheConfig;
076import org.apache.hadoop.hbase.io.hfile.CacheStats;
077import org.apache.hadoop.hbase.io.hfile.Cacheable;
078import org.apache.hadoop.hbase.io.hfile.CachedBlock;
079import org.apache.hadoop.hbase.io.hfile.CombinedBlockCache;
080import org.apache.hadoop.hbase.io.hfile.HFileBlock;
081import org.apache.hadoop.hbase.io.hfile.HFileContext;
082import org.apache.hadoop.hbase.io.hfile.HFileInfo;
083import org.apache.hadoop.hbase.nio.ByteBuff;
084import org.apache.hadoop.hbase.nio.RefCnt;
085import org.apache.hadoop.hbase.protobuf.ProtobufMagic;
086import org.apache.hadoop.hbase.regionserver.DataTieringManager;
087import org.apache.hadoop.hbase.regionserver.HRegion;
088import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
089import org.apache.hadoop.hbase.util.Bytes;
090import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
091import org.apache.hadoop.hbase.util.IdReadWriteLock;
092import org.apache.hadoop.hbase.util.IdReadWriteLockStrongRef;
093import org.apache.hadoop.hbase.util.IdReadWriteLockWithObjectPool;
094import org.apache.hadoop.hbase.util.IdReadWriteLockWithObjectPool.ReferenceType;
095import org.apache.hadoop.hbase.util.Pair;
096import org.apache.hadoop.util.StringUtils;
097import org.apache.yetus.audience.InterfaceAudience;
098import org.slf4j.Logger;
099import org.slf4j.LoggerFactory;
100
101import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
102import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
103
104import org.apache.hadoop.hbase.shaded.protobuf.generated.BucketCacheProtos;
105
106/**
107 * BucketCache uses {@link BucketAllocator} to allocate/free blocks, and uses BucketCache#ramCache
108 * and BucketCache#backingMap in order to determine if a given element is in the cache. The bucket
109 * cache can use off-heap memory {@link ByteBufferIOEngine} or mmap
110 * {@link ExclusiveMemoryMmapIOEngine} or pmem {@link SharedMemoryMmapIOEngine} or local files
111 * {@link FileIOEngine} to store/read the block data.
112 * <p>
113 * Eviction is via a similar algorithm as used in
114 * {@link org.apache.hadoop.hbase.io.hfile.LruBlockCache}
115 * <p>
116 * BucketCache can be used as mainly a block cache (see
117 * {@link org.apache.hadoop.hbase.io.hfile.CombinedBlockCache}), combined with a BlockCache to
118 * decrease CMS GC and heap fragmentation.
119 * <p>
120 * It also can be used as a secondary cache (e.g. using a file on ssd/fusionio to store blocks) to
121 * enlarge cache space via a victim cache.
122 */
123@InterfaceAudience.Private
124public class BucketCache implements BlockCache, HeapSize {
125  private static final Logger LOG = LoggerFactory.getLogger(BucketCache.class);
126
127  /** Priority buckets config */
128  static final String SINGLE_FACTOR_CONFIG_NAME = "hbase.bucketcache.single.factor";
129  static final String MULTI_FACTOR_CONFIG_NAME = "hbase.bucketcache.multi.factor";
130  static final String MEMORY_FACTOR_CONFIG_NAME = "hbase.bucketcache.memory.factor";
131  static final String EXTRA_FREE_FACTOR_CONFIG_NAME = "hbase.bucketcache.extrafreefactor";
132  static final String ACCEPT_FACTOR_CONFIG_NAME = "hbase.bucketcache.acceptfactor";
133  static final String MIN_FACTOR_CONFIG_NAME = "hbase.bucketcache.minfactor";
134  static final String BACKING_MAP_PERSISTENCE_CHUNK_SIZE =
135    "hbase.bucketcache.persistence.chunksize";
136
137  /** Use strong reference for offsetLock or not */
138  private static final String STRONG_REF_KEY = "hbase.bucketcache.offsetlock.usestrongref";
139  private static final boolean STRONG_REF_DEFAULT = false;
140
141  /** The cache age of blocks to check if the related file is present on any online regions. */
142  static final String BLOCK_ORPHAN_GRACE_PERIOD =
143    "hbase.bucketcache.block.orphan.evictgraceperiod.seconds";
144
145  static final long BLOCK_ORPHAN_GRACE_PERIOD_DEFAULT = 24 * 60 * 60 * 1000L;
146
147  /** Priority buckets */
148  static final float DEFAULT_SINGLE_FACTOR = 0.25f;
149  static final float DEFAULT_MULTI_FACTOR = 0.50f;
150  static final float DEFAULT_MEMORY_FACTOR = 0.25f;
151  static final float DEFAULT_MIN_FACTOR = 0.85f;
152
153  static final float DEFAULT_EXTRA_FREE_FACTOR = 0.10f;
154  static final float DEFAULT_ACCEPT_FACTOR = 0.95f;
155
156  // Number of blocks to clear for each of the bucket size that is full
157  static final int DEFAULT_FREE_ENTIRE_BLOCK_FACTOR = 2;
158
159  /** Statistics thread */
160  private static final int statThreadPeriod = 5 * 60;
161
162  final static int DEFAULT_WRITER_THREADS = 3;
163  final static int DEFAULT_WRITER_QUEUE_ITEMS = 64;
164
165  final static long DEFAULT_BACKING_MAP_PERSISTENCE_CHUNK_SIZE = 10000;
166
167  // Store/read block data
168  transient final IOEngine ioEngine;
169
170  // Store the block in this map before writing it to cache
171  transient final RAMCache ramCache;
172
173  // In this map, store the block's meta data like offset, length
174  transient Map<BlockCacheKey, BucketEntry> backingMap;
175
176  private AtomicBoolean backingMapValidated = new AtomicBoolean(false);
177
178  /**
179   * Map of hFile -> Region -> File size. This map is used to track all files completed prefetch,
180   * together with the region those belong to and the total cached size for the
181   * region.TestBlockEvictionOnRegionMovement
182   */
183  transient final Map<String, Pair<String, Long>> fullyCachedFiles = new ConcurrentHashMap<>();
184  /**
185   * Map of region -> total size of the region prefetched on this region server. This is the total
186   * size of hFiles for this region prefetched on this region server
187   */
188  final Map<String, Long> regionCachedSize = new ConcurrentHashMap<>();
189
190  private transient BucketCachePersister cachePersister;
191
192  /**
193   * Enum to represent the state of cache
194   */
195  protected enum CacheState {
196    // Initializing: State when the cache is being initialised from persistence.
197    INITIALIZING,
198    // Enabled: State when cache is initialised and is ready.
199    ENABLED,
200    // Disabled: State when the cache is disabled.
201    DISABLED
202  }
203
204  /**
205   * Flag if the cache is enabled or not... We shut it off if there are IO errors for some time, so
206   * that Bucket IO exceptions/errors don't bring down the HBase server.
207   */
208  private volatile CacheState cacheState;
209
210  /**
211   * A list of writer queues. We have a queue per {@link WriterThread} we have running. In other
212   * words, the work adding blocks to the BucketCache is divided up amongst the running
213   * WriterThreads. Its done by taking hash of the cache key modulo queue count. WriterThread when
214   * it runs takes whatever has been recently added and 'drains' the entries to the BucketCache. It
215   * then updates the ramCache and backingMap accordingly.
216   */
217  transient final ArrayList<BlockingQueue<RAMQueueEntry>> writerQueues = new ArrayList<>();
218  transient final WriterThread[] writerThreads;
219
220  /** Volatile boolean to track if free space is in process or not */
221  private volatile boolean freeInProgress = false;
222  private transient final Lock freeSpaceLock = new ReentrantLock();
223
224  private final LongAdder realCacheSize = new LongAdder();
225  private final LongAdder heapSize = new LongAdder();
226  /** Current number of cached elements */
227  private final LongAdder blockNumber = new LongAdder();
228
229  /** Cache access count (sequential ID) */
230  private final AtomicLong accessCount = new AtomicLong();
231
232  private static final int DEFAULT_CACHE_WAIT_TIME = 50;
233
234  private final BucketCacheStats cacheStats;
235  private final String persistencePath;
236  static AtomicBoolean isCacheInconsistent = new AtomicBoolean(false);
237  private final long cacheCapacity;
238  /** Approximate block size */
239  private final long blockSize;
240
241  /** Duration of IO errors tolerated before we disable cache, 1 min as default */
242  private final int ioErrorsTolerationDuration;
243  // 1 min
244  public static final int DEFAULT_ERROR_TOLERATION_DURATION = 60 * 1000;
245
246  // Start time of first IO error when reading or writing IO Engine, it will be
247  // reset after a successful read/write.
248  private volatile long ioErrorStartTime = -1;
249
250  private transient Configuration conf;
251
252  /**
253   * A ReentrantReadWriteLock to lock on a particular block identified by offset. The purpose of
254   * this is to avoid freeing the block which is being read.
255   * <p>
256   */
257  transient final IdReadWriteLock<Long> offsetLock;
258
259  transient NavigableSet<BlockCacheKey> blocksByHFile = new ConcurrentSkipListSet<>(
260    Comparator.comparing(BlockCacheKey::getHfileName).thenComparingLong(BlockCacheKey::getOffset));
261
262  /** Statistics thread schedule pool (for heavy debugging, could remove) */
263  private transient final ScheduledExecutorService scheduleThreadPool =
264    Executors.newScheduledThreadPool(1,
265      new ThreadFactoryBuilder().setNameFormat("BucketCacheStatsExecutor").setDaemon(true).build());
266
267  // Allocate or free space for the block
268  private transient BucketAllocator bucketAllocator;
269
270  /** Acceptable size of cache (no evictions if size < acceptable) */
271  private float acceptableFactor;
272
273  /** Minimum threshold of cache (when evicting, evict until size < min) */
274  private float minFactor;
275
276  /**
277   * Free this floating point factor of extra blocks when evicting. For example free the number of
278   * blocks requested * (1 + extraFreeFactor)
279   */
280  private float extraFreeFactor;
281
282  /** Single access bucket size */
283  private float singleFactor;
284
285  /** Multiple access bucket size */
286  private float multiFactor;
287
288  /** In-memory bucket size */
289  private float memoryFactor;
290
291  private long bucketcachePersistInterval;
292
293  private static final String FILE_VERIFY_ALGORITHM =
294    "hbase.bucketcache.persistent.file.integrity.check.algorithm";
295  private static final String DEFAULT_FILE_VERIFY_ALGORITHM = "MD5";
296
297  static final String QUEUE_ADDITION_WAIT_TIME = "hbase.bucketcache.queue.addition.waittime";
298  private static final long DEFAULT_QUEUE_ADDITION_WAIT_TIME = 0;
299  private long queueAdditionWaitTime;
300  /**
301   * Use {@link java.security.MessageDigest} class's encryption algorithms to check persistent file
302   * integrity, default algorithm is MD5
303   */
304  private String algorithm;
305
306  private long persistenceChunkSize;
307
308  /* Tracing failed Bucket Cache allocations. */
309  private long allocFailLogPrevTs; // time of previous log event for allocation failure.
310  private static final int ALLOCATION_FAIL_LOG_TIME_PERIOD = 60000; // Default 1 minute.
311
312  private transient Map<String, HRegion> onlineRegions;
313
314  private long orphanBlockGracePeriod = 0;
315
316  public BucketCache(String ioEngineName, long capacity, int blockSize, int[] bucketSizes,
317    int writerThreadNum, int writerQLen, String persistencePath) throws IOException {
318    this(ioEngineName, capacity, blockSize, bucketSizes, writerThreadNum, writerQLen,
319      persistencePath, DEFAULT_ERROR_TOLERATION_DURATION, HBaseConfiguration.create());
320  }
321
322  public BucketCache(String ioEngineName, long capacity, int blockSize, int[] bucketSizes,
323    int writerThreadNum, int writerQLen, String persistencePath, int ioErrorsTolerationDuration,
324    Configuration conf) throws IOException {
325    this(ioEngineName, capacity, blockSize, bucketSizes, writerThreadNum, writerQLen,
326      persistencePath, ioErrorsTolerationDuration, conf, null);
327  }
328
329  public BucketCache(String ioEngineName, long capacity, int blockSize, int[] bucketSizes,
330    int writerThreadNum, int writerQLen, String persistencePath, int ioErrorsTolerationDuration,
331    Configuration conf, Map<String, HRegion> onlineRegions) throws IOException {
332    Preconditions.checkArgument(blockSize > 0,
333      "BucketCache capacity is set to " + blockSize + ", can not be less than 0");
334    boolean useStrongRef = conf.getBoolean(STRONG_REF_KEY, STRONG_REF_DEFAULT);
335    if (useStrongRef) {
336      this.offsetLock = new IdReadWriteLockStrongRef<>();
337    } else {
338      this.offsetLock = new IdReadWriteLockWithObjectPool<>(ReferenceType.SOFT);
339    }
340    this.conf = conf;
341    this.algorithm = conf.get(FILE_VERIFY_ALGORITHM, DEFAULT_FILE_VERIFY_ALGORITHM);
342    this.ioEngine = getIOEngineFromName(ioEngineName, capacity, persistencePath);
343    this.writerThreads = new WriterThread[writerThreadNum];
344    this.onlineRegions = onlineRegions;
345    this.orphanBlockGracePeriod =
346      conf.getLong(BLOCK_ORPHAN_GRACE_PERIOD, BLOCK_ORPHAN_GRACE_PERIOD_DEFAULT);
347    long blockNumCapacity = capacity / blockSize;
348    if (blockNumCapacity >= Integer.MAX_VALUE) {
349      // Enough for about 32TB of cache!
350      throw new IllegalArgumentException("Cache capacity is too large, only support 32TB now");
351    }
352
353    // these sets the dynamic configs
354    this.onConfigurationChange(conf);
355    this.cacheStats =
356      new BucketCacheStats(conf.getInt(BLOCKCACHE_STATS_PERIODS, DEFAULT_BLOCKCACHE_STATS_PERIODS),
357        conf.getInt(BLOCKCACHE_STATS_PERIOD_MINUTES_KEY, DEFAULT_BLOCKCACHE_STATS_PERIOD_MINUTES));
358
359    LOG.info("Instantiating BucketCache with acceptableFactor: " + acceptableFactor
360      + ", minFactor: " + minFactor + ", extraFreeFactor: " + extraFreeFactor + ", singleFactor: "
361      + singleFactor + ", multiFactor: " + multiFactor + ", memoryFactor: " + memoryFactor
362      + ", useStrongRef: " + useStrongRef);
363
364    this.cacheCapacity = capacity;
365    this.persistencePath = persistencePath;
366    this.blockSize = blockSize;
367    this.ioErrorsTolerationDuration = ioErrorsTolerationDuration;
368    this.cacheState = CacheState.INITIALIZING;
369
370    this.allocFailLogPrevTs = 0;
371
372    for (int i = 0; i < writerThreads.length; ++i) {
373      writerQueues.add(new ArrayBlockingQueue<>(writerQLen));
374    }
375
376    assert writerQueues.size() == writerThreads.length;
377    this.ramCache = new RAMCache();
378
379    this.backingMap = new ConcurrentHashMap<>((int) blockNumCapacity);
380    instantiateWriterThreads();
381
382    if (isCachePersistent()) {
383      if (ioEngine instanceof FileIOEngine) {
384        startBucketCachePersisterThread();
385      }
386      startPersistenceRetriever(bucketSizes, capacity);
387    } else {
388      bucketAllocator = new BucketAllocator(capacity, bucketSizes);
389      this.cacheState = CacheState.ENABLED;
390      startWriterThreads();
391    }
392
393    // Run the statistics thread periodically to print the cache statistics log
394    // TODO: Add means of turning this off. Bit obnoxious running thread just to make a log
395    // every five minutes.
396    this.scheduleThreadPool.scheduleAtFixedRate(new StatisticsThread(this), statThreadPeriod,
397      statThreadPeriod, TimeUnit.SECONDS);
398    LOG.info("Started bucket cache; ioengine=" + ioEngineName + ", capacity="
399      + StringUtils.byteDesc(capacity) + ", blockSize=" + StringUtils.byteDesc(blockSize)
400      + ", writerThreadNum=" + writerThreadNum + ", writerQLen=" + writerQLen + ", persistencePath="
401      + persistencePath + ", bucketAllocator=" + BucketAllocator.class.getName());
402  }
403
404  private void startPersistenceRetriever(int[] bucketSizes, long capacity) {
405    Runnable persistentCacheRetriever = () -> {
406      try {
407        retrieveFromFile(bucketSizes);
408        LOG.info("Persistent bucket cache recovery from {} is complete.", persistencePath);
409      } catch (Throwable ex) {
410        LOG.warn("Can't restore from file[{}]. The bucket cache will be reset and rebuilt."
411          + " Exception seen: ", persistencePath, ex);
412        backingMap.clear();
413        FilePathStringPool.getInstance().clear();
414        fullyCachedFiles.clear();
415        backingMapValidated.set(true);
416        regionCachedSize.clear();
417        try {
418          bucketAllocator = new BucketAllocator(capacity, bucketSizes);
419        } catch (BucketAllocatorException allocatorException) {
420          LOG.error("Exception during Bucket Allocation", allocatorException);
421        }
422      } finally {
423        this.cacheState = CacheState.ENABLED;
424        startWriterThreads();
425      }
426    };
427    Thread t = new Thread(persistentCacheRetriever);
428    t.start();
429  }
430
431  private void sanityCheckConfigs() {
432    Preconditions.checkArgument(acceptableFactor <= 1 && acceptableFactor >= 0,
433      ACCEPT_FACTOR_CONFIG_NAME + " must be between 0.0 and 1.0");
434    Preconditions.checkArgument(minFactor <= 1 && minFactor >= 0,
435      MIN_FACTOR_CONFIG_NAME + " must be between 0.0 and 1.0");
436    Preconditions.checkArgument(minFactor <= acceptableFactor,
437      MIN_FACTOR_CONFIG_NAME + " must be <= " + ACCEPT_FACTOR_CONFIG_NAME);
438    Preconditions.checkArgument(extraFreeFactor >= 0,
439      EXTRA_FREE_FACTOR_CONFIG_NAME + " must be greater than 0.0");
440    Preconditions.checkArgument(singleFactor <= 1 && singleFactor >= 0,
441      SINGLE_FACTOR_CONFIG_NAME + " must be between 0.0 and 1.0");
442    Preconditions.checkArgument(multiFactor <= 1 && multiFactor >= 0,
443      MULTI_FACTOR_CONFIG_NAME + " must be between 0.0 and 1.0");
444    Preconditions.checkArgument(memoryFactor <= 1 && memoryFactor >= 0,
445      MEMORY_FACTOR_CONFIG_NAME + " must be between 0.0 and 1.0");
446    Preconditions.checkArgument((singleFactor + multiFactor + memoryFactor) == 1,
447      SINGLE_FACTOR_CONFIG_NAME + ", " + MULTI_FACTOR_CONFIG_NAME + ", and "
448        + MEMORY_FACTOR_CONFIG_NAME + " segments must add up to 1.0");
449    if (this.persistenceChunkSize <= 0) {
450      persistenceChunkSize = DEFAULT_BACKING_MAP_PERSISTENCE_CHUNK_SIZE;
451    }
452  }
453
454  /**
455   * Called by the constructor to instantiate the writer threads.
456   */
457  private void instantiateWriterThreads() {
458    final String threadName = Thread.currentThread().getName();
459    for (int i = 0; i < this.writerThreads.length; ++i) {
460      this.writerThreads[i] = new WriterThread(this.writerQueues.get(i));
461      this.writerThreads[i].setName(threadName + "-BucketCacheWriter-" + i);
462      this.writerThreads[i].setDaemon(true);
463    }
464  }
465
466  /**
467   * Called by the constructor to start the writer threads. Used by tests that need to override
468   * starting the threads.
469   */
470  protected void startWriterThreads() {
471    for (WriterThread thread : writerThreads) {
472      thread.start();
473    }
474  }
475
476  void startBucketCachePersisterThread() {
477    LOG.info("Starting BucketCachePersisterThread");
478    cachePersister = new BucketCachePersister(this, bucketcachePersistInterval);
479    cachePersister.setDaemon(true);
480    cachePersister.start();
481  }
482
483  @Override
484  public boolean isCacheEnabled() {
485    return this.cacheState == CacheState.ENABLED;
486  }
487
488  @Override
489  public long getMaxSize() {
490    return this.cacheCapacity;
491  }
492
493  public String getIoEngine() {
494    return ioEngine.toString();
495  }
496
497  /**
498   * Get the IOEngine from the IO engine name
499   * @return the IOEngine
500   */
501  private IOEngine getIOEngineFromName(String ioEngineName, long capacity, String persistencePath)
502    throws IOException {
503    if (ioEngineName.startsWith("file:") || ioEngineName.startsWith("files:")) {
504      // In order to make the usage simple, we only need the prefix 'files:' in
505      // document whether one or multiple file(s), but also support 'file:' for
506      // the compatibility
507      String[] filePaths =
508        ioEngineName.substring(ioEngineName.indexOf(":") + 1).split(FileIOEngine.FILE_DELIMITER);
509      return new FileIOEngine(capacity, persistencePath != null, filePaths);
510    } else if (ioEngineName.startsWith("offheap")) {
511      return new ByteBufferIOEngine(capacity);
512    } else if (ioEngineName.startsWith("mmap:")) {
513      return new ExclusiveMemoryMmapIOEngine(ioEngineName.substring(5), capacity);
514    } else if (ioEngineName.startsWith("pmem:")) {
515      // This mode of bucket cache creates an IOEngine over a file on the persistent memory
516      // device. Since the persistent memory device has its own address space the contents
517      // mapped to this address space does not get swapped out like in the case of mmapping
518      // on to DRAM. Hence the cells created out of the hfile blocks in the pmem bucket cache
519      // can be directly referred to without having to copy them onheap. Once the RPC is done,
520      // the blocks can be returned back as in case of ByteBufferIOEngine.
521      return new SharedMemoryMmapIOEngine(ioEngineName.substring(5), capacity);
522    } else {
523      throw new IllegalArgumentException(
524        "Don't understand io engine name for cache- prefix with file:, files:, mmap: or offheap");
525    }
526  }
527
528  public boolean isCachePersistenceEnabled() {
529    return persistencePath != null;
530  }
531
532  /**
533   * Cache the block with the specified name and buffer.
534   * @param cacheKey block's cache key
535   * @param buf      block buffer
536   */
537  @Override
538  public void cacheBlock(BlockCacheKey cacheKey, Cacheable buf) {
539    cacheBlock(cacheKey, buf, false);
540  }
541
542  /**
543   * Cache the block with the specified name and buffer.
544   * @param cacheKey   block's cache key
545   * @param cachedItem block buffer
546   * @param inMemory   if block is in-memory
547   */
548  @Override
549  public void cacheBlock(BlockCacheKey cacheKey, Cacheable cachedItem, boolean inMemory) {
550    cacheBlockWithWait(cacheKey, cachedItem, inMemory, false);
551  }
552
553  /**
554   * Cache the block with the specified name and buffer.
555   * @param cacheKey   block's cache key
556   * @param cachedItem block buffer
557   * @param inMemory   if block is in-memory
558   */
559  @Override
560  public void cacheBlock(BlockCacheKey cacheKey, Cacheable cachedItem, boolean inMemory,
561    boolean waitWhenCache) {
562    cacheBlockWithWait(cacheKey, cachedItem, inMemory, waitWhenCache && queueAdditionWaitTime > 0);
563  }
564
565  /**
566   * Cache the block to ramCache
567   * @param cacheKey   block's cache key
568   * @param cachedItem block buffer
569   * @param inMemory   if block is in-memory
570   * @param wait       if true, blocking wait when queue is full
571   */
572  public void cacheBlockWithWait(BlockCacheKey cacheKey, Cacheable cachedItem, boolean inMemory,
573    boolean wait) {
574    if (isCacheEnabled()) {
575      if (backingMap.containsKey(cacheKey) || ramCache.containsKey(cacheKey)) {
576        if (shouldReplaceExistingCacheBlock(cacheKey, cachedItem)) {
577          BucketEntry bucketEntry = backingMap.get(cacheKey);
578          if (bucketEntry != null && bucketEntry.isRpcRef()) {
579            // avoid replace when there are RPC refs for the bucket entry in bucket cache
580            return;
581          }
582          cacheBlockWithWaitInternal(cacheKey, cachedItem, inMemory, wait);
583        }
584      } else {
585        cacheBlockWithWaitInternal(cacheKey, cachedItem, inMemory, wait);
586      }
587    }
588  }
589
590  protected boolean shouldReplaceExistingCacheBlock(BlockCacheKey cacheKey, Cacheable newBlock) {
591    return BlockCacheUtil.shouldReplaceExistingCacheBlock(this, cacheKey, newBlock);
592  }
593
594  protected void cacheBlockWithWaitInternal(BlockCacheKey cacheKey, Cacheable cachedItem,
595    boolean inMemory, boolean wait) {
596    if (!isCacheEnabled()) {
597      return;
598    }
599    if (cacheKey.getBlockType() == null && cachedItem.getBlockType() != null) {
600      cacheKey.setBlockType(cachedItem.getBlockType());
601    }
602    LOG.debug("Caching key={}, item={}, key heap size={}", cacheKey, cachedItem,
603      cacheKey.heapSize());
604    // Stuff the entry into the RAM cache so it can get drained to the persistent store
605    RAMQueueEntry re = new RAMQueueEntry(cacheKey, cachedItem, accessCount.incrementAndGet(),
606      inMemory, isCachePersistent() && ioEngine instanceof FileIOEngine, wait);
607    /**
608     * Don't use ramCache.put(cacheKey, re) here. because there may be a existing entry with same
609     * key in ramCache, the heap size of bucket cache need to update if replacing entry from
610     * ramCache. But WriterThread will also remove entry from ramCache and update heap size, if
611     * using ramCache.put(), It's possible that the removed entry in WriterThread is not the correct
612     * one, then the heap size will mess up (HBASE-20789)
613     */
614    if (ramCache.putIfAbsent(cacheKey, re) != null) {
615      return;
616    }
617    int queueNum = (cacheKey.hashCode() & 0x7FFFFFFF) % writerQueues.size();
618    BlockingQueue<RAMQueueEntry> bq = writerQueues.get(queueNum);
619    boolean successfulAddition = false;
620    if (wait) {
621      try {
622        successfulAddition = bq.offer(re, queueAdditionWaitTime, TimeUnit.MILLISECONDS);
623      } catch (InterruptedException e) {
624        LOG.error("Thread interrupted: ", e);
625        Thread.currentThread().interrupt();
626      }
627    } else {
628      successfulAddition = bq.offer(re);
629    }
630    if (!successfulAddition) {
631      LOG.debug("Failed to insert block {} into the cache writers queue", cacheKey);
632      ramCache.remove(cacheKey);
633      cacheStats.failInsert();
634    } else {
635      this.blockNumber.increment();
636      this.heapSize.add(cachedItem.heapSize());
637    }
638  }
639
640  /**
641   * If the passed cache key relates to a reference (&lt;hfile&gt;.&lt;parentEncRegion&gt;), this
642   * method looks for the block from the referred file, in the cache. If present in the cache, the
643   * block for the referred file is returned, otherwise, this method returns null. It will also
644   * return null if the passed cache key doesn't relate to a reference.
645   * @param key the BlockCacheKey instance to look for in the cache.
646   * @return the cached block from the referred file, null if there's no such block in the cache or
647   *         the passed key doesn't relate to a reference.
648   */
649  public BucketEntry getBlockForReference(BlockCacheKey key) {
650    BucketEntry foundEntry = null;
651    String referredFileName = null;
652    if (StoreFileInfo.isReference(key.getHfileName())) {
653      referredFileName = StoreFileInfo.getReferredToRegionAndFile(key.getHfileName()).getSecond();
654    }
655    if (referredFileName != null) {
656      // Since we just need this key for a lookup, it's enough to use only name and offset
657      BlockCacheKey convertedCacheKey = new BlockCacheKey(referredFileName, key.getOffset());
658      foundEntry = backingMap.get(convertedCacheKey);
659      LOG.debug("Got a link/ref: {}. Related cacheKey: {}. Found entry: {}", key.getHfileName(),
660        convertedCacheKey, foundEntry);
661    }
662    return foundEntry;
663  }
664
665  /**
666   * Get the buffer of the block with the specified key.
667   * @param key                block's cache key
668   * @param caching            true if the caller caches blocks on cache misses
669   * @param repeat             Whether this is a repeat lookup for the same block
670   * @param updateCacheMetrics Whether we should update cache metrics or not
671   * @return buffer of specified cache key, or null if not in cache
672   */
673  @Override
674  public Cacheable getBlock(BlockCacheKey key, boolean caching, boolean repeat,
675    boolean updateCacheMetrics) {
676    if (!isCacheEnabled()) {
677      cacheStats.miss(caching, key.isPrimary(), key.getBlockType());
678      return null;
679    }
680    RAMQueueEntry re = ramCache.get(key);
681    if (re != null) {
682      if (updateCacheMetrics) {
683        cacheStats.hit(caching, key.isPrimary(), key.getBlockType());
684      }
685      re.access(accessCount.incrementAndGet());
686      return re.getData();
687    }
688    BucketEntry bucketEntry = backingMap.get(key);
689    LOG.debug("bucket entry for key {}: {}", key,
690      bucketEntry == null ? null : bucketEntry.offset());
691    if (bucketEntry == null) {
692      bucketEntry = getBlockForReference(key);
693    }
694    if (bucketEntry != null) {
695      long start = System.nanoTime();
696      ReentrantReadWriteLock lock = offsetLock.getLock(bucketEntry.offset());
697      try {
698        lock.readLock().lock();
699        // We can not read here even if backingMap does contain the given key because its offset
700        // maybe changed. If we lock BlockCacheKey instead of offset, then we can only check
701        // existence here.
702        if (
703          bucketEntry.equals(backingMap.get(key)) || bucketEntry.equals(getBlockForReference(key))
704        ) {
705          // Read the block from IOEngine based on the bucketEntry's offset and length, NOTICE: the
706          // block will use the refCnt of bucketEntry, which means if two HFileBlock mapping to
707          // the same BucketEntry, then all of the three will share the same refCnt.
708          Cacheable cachedBlock = ioEngine.read(bucketEntry);
709          if (ioEngine.usesSharedMemory()) {
710            // If IOEngine use shared memory, cachedBlock and BucketEntry will share the
711            // same RefCnt, do retain here, in order to count the number of RPC references
712            cachedBlock.retain();
713          }
714          // Update the cache statistics.
715          if (updateCacheMetrics) {
716            cacheStats.hit(caching, key.isPrimary(), key.getBlockType());
717            cacheStats.ioHit(System.nanoTime() - start);
718          }
719          bucketEntry.access(accessCount.incrementAndGet());
720          if (this.ioErrorStartTime > 0) {
721            ioErrorStartTime = -1;
722          }
723          return cachedBlock;
724        }
725      } catch (HBaseIOException hioex) {
726        // When using file io engine persistent cache,
727        // the cache map state might differ from the actual cache. If we reach this block,
728        // we should remove the cache key entry from the backing map
729        backingMap.remove(key);
730        fileNotFullyCached(key, bucketEntry);
731        LOG.debug("Failed to fetch block for cache key: {}.", key, hioex);
732      } catch (IOException ioex) {
733        LOG.error("Failed reading block " + key + " from bucket cache", ioex);
734        checkIOErrorIsTolerated();
735      } finally {
736        lock.readLock().unlock();
737      }
738    }
739    if (!repeat && updateCacheMetrics) {
740      cacheStats.miss(caching, key.isPrimary(), key.getBlockType());
741    }
742    return null;
743  }
744
745  /**
746   * This method is invoked after the bucketEntry is removed from {@link BucketCache#backingMap}
747   */
748  void blockEvicted(BlockCacheKey cacheKey, BucketEntry bucketEntry, boolean decrementBlockNumber,
749    boolean evictedByEvictionProcess) {
750    bucketEntry.markAsEvicted();
751    blocksByHFile.remove(cacheKey);
752    if (decrementBlockNumber) {
753      this.blockNumber.decrement();
754      if (ioEngine.isPersistent()) {
755        fileNotFullyCached(cacheKey, bucketEntry);
756      }
757    }
758    if (evictedByEvictionProcess) {
759      cacheStats.evicted(bucketEntry.getCachedTime(), cacheKey.isPrimary());
760    }
761    if (ioEngine.isPersistent()) {
762      setCacheInconsistent(true);
763    }
764  }
765
766  private void fileNotFullyCached(BlockCacheKey key, BucketEntry entry) {
767    // Update the updateRegionCachedSize before removing the file from fullyCachedFiles.
768    // This computation should happen even if the file is not in fullyCachedFiles map.
769    updateRegionCachedSize(key, (entry.getLength() * -1));
770    fullyCachedFiles.remove(key.getHfileName());
771  }
772
773  public void fileCacheCompleted(Path filePath, long size) {
774    Pair<String, Long> pair = new Pair<>();
775    // sets the region name
776    String regionName = filePath.getParent().getParent().getName();
777    pair.setFirst(regionName);
778    pair.setSecond(size);
779    fullyCachedFiles.put(filePath.getName(), pair);
780  }
781
782  private void updateRegionCachedSize(BlockCacheKey key, long cachedSize) {
783    if (key.getRegionName() != null) {
784      if (key.isArchived()) {
785        LOG.trace("Skipping region cached size update for archived file:{} from region: {}",
786          key.getHfileName(), key.getRegionName());
787      } else {
788        String regionName = key.getRegionName();
789        regionCachedSize.merge(regionName, cachedSize,
790          (previousSize, newBlockSize) -> previousSize + newBlockSize);
791        LOG.trace("Updating region cached size for region: {}", regionName);
792        // If all the blocks for a region are evicted from the cache,
793        // remove the entry for that region from regionCachedSize map.
794        if (regionCachedSize.get(regionName) <= 0) {
795          regionCachedSize.remove(regionName);
796          FilePathStringPool.getInstance().remove(regionName);
797        }
798      }
799    }
800  }
801
802  /**
803   * Free the {{@link BucketEntry} actually,which could only be invoked when the
804   * {@link BucketEntry#refCnt} becoming 0.
805   */
806  void freeBucketEntry(BucketEntry bucketEntry) {
807    bucketAllocator.freeBlock(bucketEntry.offset(), bucketEntry.getLength());
808    realCacheSize.add(-1 * bucketEntry.getLength());
809  }
810
811  /**
812   * Try to evict the block from {@link BlockCache} by force. We'll call this in few cases:<br>
813   * 1. Close an HFile, and clear all cached blocks. <br>
814   * 2. Call {@link Admin#clearBlockCache(TableName)} to clear all blocks for a given table.<br>
815   * <p>
816   * Firstly, we'll try to remove the block from RAMCache,and then try to evict from backingMap.
817   * Here we evict the block from backingMap immediately, but only free the reference from bucket
818   * cache by calling {@link BucketEntry#markedAsEvicted}. If there're still some RPC referring this
819   * block, block can only be de-allocated when all of them release the block.
820   * <p>
821   * NOTICE: we need to grab the write offset lock firstly before releasing the reference from
822   * bucket cache. if we don't, we may read an {@link BucketEntry} with refCnt = 0 when
823   * {@link BucketCache#getBlock(BlockCacheKey, boolean, boolean, boolean)}, it's a memory leak.
824   * @param cacheKey Block to evict
825   * @return true to indicate whether we've evicted successfully or not.
826   */
827  @Override
828  public boolean evictBlock(BlockCacheKey cacheKey) {
829    return doEvictBlock(cacheKey, null, false);
830  }
831
832  /**
833   * Evict the {@link BlockCacheKey} and {@link BucketEntry} from {@link BucketCache#backingMap} and
834   * {@link BucketCache#ramCache}. <br/>
835   * NOTE:When Evict from {@link BucketCache#backingMap},only the matched {@link BlockCacheKey} and
836   * {@link BucketEntry} could be removed.
837   * @param cacheKey    {@link BlockCacheKey} to evict.
838   * @param bucketEntry {@link BucketEntry} matched {@link BlockCacheKey} to evict.
839   * @return true to indicate whether we've evicted successfully or not.
840   */
841  private boolean doEvictBlock(BlockCacheKey cacheKey, BucketEntry bucketEntry,
842    boolean evictedByEvictionProcess) {
843    if (!isCacheEnabled()) {
844      return false;
845    }
846    boolean existedInRamCache = removeFromRamCache(cacheKey);
847    if (bucketEntry == null) {
848      bucketEntry = backingMap.get(cacheKey);
849    }
850    final BucketEntry bucketEntryToUse = bucketEntry;
851
852    if (bucketEntryToUse == null) {
853      if (existedInRamCache && evictedByEvictionProcess) {
854        cacheStats.evicted(0, cacheKey.isPrimary());
855      }
856      return existedInRamCache;
857    } else {
858      return bucketEntryToUse.withWriteLock(offsetLock, () -> {
859        if (backingMap.remove(cacheKey, bucketEntryToUse)) {
860          LOG.debug("removed key {} from back map with offset lock {} in the evict process",
861            cacheKey, bucketEntryToUse.offset());
862          blockEvicted(cacheKey, bucketEntryToUse, !existedInRamCache, evictedByEvictionProcess);
863          return true;
864        }
865        return false;
866      });
867    }
868  }
869
870  /**
871   * <pre>
872   * Create the {@link Recycler} for {@link BucketEntry#refCnt},which would be used as
873   * {@link RefCnt#recycler} of {@link HFileBlock#buf} returned from {@link BucketCache#getBlock}.
874   * NOTE: for {@link BucketCache#getBlock},the {@link RefCnt#recycler} of {@link HFileBlock#buf}
875   * from {@link BucketCache#backingMap} and {@link BucketCache#ramCache} are different:
876   * 1.For {@link RefCnt#recycler} of {@link HFileBlock#buf} from {@link BucketCache#backingMap},
877   *   it is the return value of current {@link BucketCache#createRecycler} method.
878   *
879   * 2.For {@link RefCnt#recycler} of {@link HFileBlock#buf} from {@link BucketCache#ramCache},
880   *   it is {@link ByteBuffAllocator#putbackBuffer}.
881   * </pre>
882   */
883  public Recycler createRecycler(final BucketEntry bucketEntry) {
884    return () -> {
885      freeBucketEntry(bucketEntry);
886      return;
887    };
888  }
889
890  /**
891   * NOTE: This method is only for test.
892   */
893  public boolean evictBlockIfNoRpcReferenced(BlockCacheKey blockCacheKey) {
894    BucketEntry bucketEntry = backingMap.get(blockCacheKey);
895    if (bucketEntry == null) {
896      return false;
897    }
898    return evictBucketEntryIfNoRpcReferenced(blockCacheKey, bucketEntry);
899  }
900
901  /**
902   * Evict {@link BlockCacheKey} and its corresponding {@link BucketEntry} only if
903   * {@link BucketEntry#isRpcRef} is false. <br/>
904   * NOTE:When evict from {@link BucketCache#backingMap},only the matched {@link BlockCacheKey} and
905   * {@link BucketEntry} could be removed.
906   * @param blockCacheKey {@link BlockCacheKey} to evict.
907   * @param bucketEntry   {@link BucketEntry} matched {@link BlockCacheKey} to evict.
908   * @return true to indicate whether we've evicted successfully or not.
909   */
910  boolean evictBucketEntryIfNoRpcReferenced(BlockCacheKey blockCacheKey, BucketEntry bucketEntry) {
911    if (!bucketEntry.isRpcRef()) {
912      return doEvictBlock(blockCacheKey, bucketEntry, true);
913    }
914    return false;
915  }
916
917  /**
918   * Since HBASE-29249, the following properties governin freeSpace behaviour and block priorities
919   * were made dynamically configurable: - hbase.bucketcache.acceptfactor -
920   * hbase.bucketcache.minfactor - hbase.bucketcache.extrafreefactor -
921   * hbase.bucketcache.single.factor - hbase.bucketcache.multi.factor -
922   * hbase.bucketcache.multi.factor - hbase.bucketcache.memory.factor The
923   * hbase.bucketcache.queue.addition.waittime property allows for introducing a delay in the
924   * publishing of blocks for the cache writer threads during prefetch reads only (client reads
925   * wouldn't get delayed). It has also been made dynamic configurable since HBASE-29249. The
926   * hbase.bucketcache.persist.intervalinmillis propperty determines the frequency for saving the
927   * persistent cache, and it has also been made dynamically configurable since HBASE-29249. The
928   * hbase.bucketcache.persistence.chunksize property determines the size of the persistent file
929   * splits (due to the limitation of maximum allowed protobuff size), and it has also been made
930   * dynamically configurable since HBASE-29249.
931   * @param config the new configuration to be updated.
932   */
933  @Override
934  public void onConfigurationChange(Configuration config) {
935    this.acceptableFactor = conf.getFloat(ACCEPT_FACTOR_CONFIG_NAME, DEFAULT_ACCEPT_FACTOR);
936    this.minFactor = conf.getFloat(MIN_FACTOR_CONFIG_NAME, DEFAULT_MIN_FACTOR);
937    this.extraFreeFactor = conf.getFloat(EXTRA_FREE_FACTOR_CONFIG_NAME, DEFAULT_EXTRA_FREE_FACTOR);
938    this.singleFactor = conf.getFloat(SINGLE_FACTOR_CONFIG_NAME, DEFAULT_SINGLE_FACTOR);
939    this.multiFactor = conf.getFloat(MULTI_FACTOR_CONFIG_NAME, DEFAULT_MULTI_FACTOR);
940    this.memoryFactor = conf.getFloat(MEMORY_FACTOR_CONFIG_NAME, DEFAULT_MEMORY_FACTOR);
941    this.queueAdditionWaitTime =
942      conf.getLong(QUEUE_ADDITION_WAIT_TIME, DEFAULT_QUEUE_ADDITION_WAIT_TIME);
943    this.bucketcachePersistInterval = conf.getLong(BUCKETCACHE_PERSIST_INTERVAL_KEY, 1000);
944    this.persistenceChunkSize =
945      conf.getLong(BACKING_MAP_PERSISTENCE_CHUNK_SIZE, DEFAULT_BACKING_MAP_PERSISTENCE_CHUNK_SIZE);
946    sanityCheckConfigs();
947  }
948
949  protected boolean removeFromRamCache(BlockCacheKey cacheKey) {
950    return ramCache.remove(cacheKey, re -> {
951      if (re != null) {
952        this.blockNumber.decrement();
953        this.heapSize.add(-1 * re.getData().heapSize());
954      }
955    });
956  }
957
958  public boolean isCacheInconsistent() {
959    return isCacheInconsistent.get();
960  }
961
962  public void setCacheInconsistent(boolean setCacheInconsistent) {
963    isCacheInconsistent.set(setCacheInconsistent);
964  }
965
966  protected void setCacheState(CacheState state) {
967    cacheState = state;
968  }
969
970  /*
971   * Statistics thread. Periodically output cache statistics to the log.
972   */
973  private static class StatisticsThread extends Thread {
974    private final BucketCache bucketCache;
975
976    public StatisticsThread(BucketCache bucketCache) {
977      super("BucketCacheStatsThread");
978      setDaemon(true);
979      this.bucketCache = bucketCache;
980    }
981
982    @Override
983    public void run() {
984      bucketCache.logStats();
985    }
986  }
987
988  public void logStats() {
989    if (!isCacheInitialized("BucketCache::logStats")) {
990      return;
991    }
992
993    long totalSize = bucketAllocator.getTotalSize();
994    long usedSize = bucketAllocator.getUsedSize();
995    long freeSize = totalSize - usedSize;
996    long cacheSize = getRealCacheSize();
997    LOG.info("failedBlockAdditions=" + cacheStats.getFailedInserts() + ", " + "totalSize="
998      + StringUtils.byteDesc(totalSize) + ", " + "freeSize=" + StringUtils.byteDesc(freeSize) + ", "
999      + "usedSize=" + StringUtils.byteDesc(usedSize) + ", " + "cacheSize="
1000      + StringUtils.byteDesc(cacheSize) + ", " + "accesses=" + cacheStats.getRequestCount() + ", "
1001      + "hits=" + cacheStats.getHitCount() + ", " + "IOhitsPerSecond="
1002      + cacheStats.getIOHitsPerSecond() + ", " + "IOTimePerHit="
1003      + String.format("%.2f", cacheStats.getIOTimePerHit()) + ", " + "hitRatio="
1004      + (cacheStats.getHitCount() == 0
1005        ? "0,"
1006        : (StringUtils.formatPercent(cacheStats.getHitRatio(), 2) + ", "))
1007      + "cachingAccesses=" + cacheStats.getRequestCachingCount() + ", " + "cachingHits="
1008      + cacheStats.getHitCachingCount() + ", " + "cachingHitsRatio="
1009      + (cacheStats.getHitCachingCount() == 0
1010        ? "0,"
1011        : (StringUtils.formatPercent(cacheStats.getHitCachingRatio(), 2) + ", "))
1012      + "evictions=" + cacheStats.getEvictionCount() + ", " + "evicted="
1013      + cacheStats.getEvictedCount() + ", " + "evictedPerRun=" + cacheStats.evictedPerEviction()
1014      + ", " + "allocationFailCount=" + cacheStats.getAllocationFailCount() + ", blocksCount="
1015      + backingMap.size());
1016    LOG.info(FilePathStringPool.getInstance().getPoolStats());
1017    cacheStats.reset();
1018
1019    bucketAllocator.logDebugStatistics();
1020  }
1021
1022  public long getRealCacheSize() {
1023    return this.realCacheSize.sum();
1024  }
1025
1026  public long acceptableSize() {
1027    if (!isCacheInitialized("BucketCache::acceptableSize")) {
1028      return 0;
1029    }
1030    return (long) Math.floor(bucketAllocator.getTotalSize() * acceptableFactor);
1031  }
1032
1033  long getPartitionSize(float partitionFactor) {
1034    if (!isCacheInitialized("BucketCache::getPartitionSize")) {
1035      return 0;
1036    }
1037
1038    return (long) Math.floor(bucketAllocator.getTotalSize() * partitionFactor * minFactor);
1039  }
1040
1041  /**
1042   * Return the count of bucketSizeinfos still need free space
1043   */
1044  private int bucketSizesAboveThresholdCount(float minFactor) {
1045    if (!isCacheInitialized("BucketCache::bucketSizesAboveThresholdCount")) {
1046      return 0;
1047    }
1048
1049    BucketAllocator.IndexStatistics[] stats = bucketAllocator.getIndexStatistics();
1050    int fullCount = 0;
1051    for (int i = 0; i < stats.length; i++) {
1052      long freeGoal = (long) Math.floor(stats[i].totalCount() * (1 - minFactor));
1053      freeGoal = Math.max(freeGoal, 1);
1054      if (stats[i].freeCount() < freeGoal) {
1055        fullCount++;
1056      }
1057    }
1058    return fullCount;
1059  }
1060
1061  /**
1062   * This method will find the buckets that are minimally occupied and are not reference counted and
1063   * will free them completely without any constraint on the access times of the elements, and as a
1064   * process will completely free at most the number of buckets passed, sometimes it might not due
1065   * to changing refCounts
1066   * @param completelyFreeBucketsNeeded number of buckets to free
1067   **/
1068  private void freeEntireBuckets(int completelyFreeBucketsNeeded) {
1069    if (!isCacheInitialized("BucketCache::freeEntireBuckets")) {
1070      return;
1071    }
1072
1073    if (completelyFreeBucketsNeeded != 0) {
1074      // First we will build a set where the offsets are reference counted, usually
1075      // this set is small around O(Handler Count) unless something else is wrong
1076      Set<Integer> inUseBuckets = new HashSet<>();
1077      backingMap.forEach((k, be) -> {
1078        if (be.isRpcRef()) {
1079          inUseBuckets.add(bucketAllocator.getBucketIndex(be.offset()));
1080        }
1081      });
1082      Set<Integer> candidateBuckets =
1083        bucketAllocator.getLeastFilledBuckets(inUseBuckets, completelyFreeBucketsNeeded);
1084      for (Map.Entry<BlockCacheKey, BucketEntry> entry : backingMap.entrySet()) {
1085        if (candidateBuckets.contains(bucketAllocator.getBucketIndex(entry.getValue().offset()))) {
1086          evictBucketEntryIfNoRpcReferenced(entry.getKey(), entry.getValue());
1087        }
1088      }
1089    }
1090  }
1091
1092  private long calculateBytesToFree(StringBuilder msgBuffer) {
1093    long bytesToFreeWithoutExtra = 0;
1094    BucketAllocator.IndexStatistics[] stats = bucketAllocator.getIndexStatistics();
1095    long[] bytesToFreeForBucket = new long[stats.length];
1096    for (int i = 0; i < stats.length; i++) {
1097      bytesToFreeForBucket[i] = 0;
1098      long freeGoal = (long) Math.floor(stats[i].totalCount() * (1 - minFactor));
1099      freeGoal = Math.max(freeGoal, 1);
1100      if (stats[i].freeCount() < freeGoal) {
1101        bytesToFreeForBucket[i] = stats[i].itemSize() * (freeGoal - stats[i].freeCount());
1102        bytesToFreeWithoutExtra += bytesToFreeForBucket[i];
1103        if (msgBuffer != null) {
1104          msgBuffer.append("Free for bucketSize(" + stats[i].itemSize() + ")="
1105            + StringUtils.byteDesc(bytesToFreeForBucket[i]) + ", ");
1106        }
1107      }
1108    }
1109    if (msgBuffer != null) {
1110      msgBuffer.append("Free for total=" + StringUtils.byteDesc(bytesToFreeWithoutExtra) + ", ");
1111    }
1112    return bytesToFreeWithoutExtra;
1113  }
1114
1115  /**
1116   * Free the space if the used size reaches acceptableSize() or one size block couldn't be
1117   * allocated. When freeing the space, we use the LRU algorithm and ensure there must be some
1118   * blocks evicted
1119   * @param why Why we are being called
1120   */
1121  void freeSpace(final String why) {
1122    if (!isCacheInitialized("BucketCache::freeSpace")) {
1123      return;
1124    }
1125    // Ensure only one freeSpace progress at a time
1126    if (!freeSpaceLock.tryLock()) {
1127      return;
1128    }
1129    try {
1130      freeInProgress = true;
1131      StringBuilder msgBuffer = LOG.isDebugEnabled() ? new StringBuilder() : null;
1132      long bytesToFreeWithoutExtra = calculateBytesToFree(msgBuffer);
1133      if (bytesToFreeWithoutExtra <= 0) {
1134        return;
1135      }
1136      long currentSize = bucketAllocator.getUsedSize();
1137      long totalSize = bucketAllocator.getTotalSize();
1138      if (LOG.isDebugEnabled() && msgBuffer != null) {
1139        LOG.debug("Free started because \"" + why + "\"; " + msgBuffer + " of current used="
1140          + StringUtils.byteDesc(currentSize) + ", actual cacheSize="
1141          + StringUtils.byteDesc(realCacheSize.sum()) + ", total="
1142          + StringUtils.byteDesc(totalSize));
1143      }
1144      long bytesToFreeWithExtra =
1145        (long) Math.floor(bytesToFreeWithoutExtra * (1 + extraFreeFactor));
1146      // Instantiate priority buckets
1147      BucketEntryGroup bucketSingle =
1148        new BucketEntryGroup(bytesToFreeWithExtra, blockSize, getPartitionSize(singleFactor));
1149      BucketEntryGroup bucketMulti =
1150        new BucketEntryGroup(bytesToFreeWithExtra, blockSize, getPartitionSize(multiFactor));
1151      BucketEntryGroup bucketMemory =
1152        new BucketEntryGroup(bytesToFreeWithExtra, blockSize, getPartitionSize(memoryFactor));
1153
1154      Set<String> allValidFiles = null;
1155      // We need the region/stores/files tree, in order to figure out if a block is "orphan" or not.
1156      // See further comments below for more details.
1157      if (onlineRegions != null) {
1158        allValidFiles = BlockCacheUtil.listAllFilesNames(onlineRegions);
1159      }
1160      // the cached time is recored in nanos, so we need to convert the grace period accordingly
1161      long orphanGracePeriodNanos = orphanBlockGracePeriod * 1000000;
1162      long bytesFreed = 0;
1163      // Check the list of files to determine the cold files which can be readily evicted.
1164      Map<String, String> coldFiles = null;
1165
1166      DataTieringManager dataTieringManager = DataTieringManager.getInstance();
1167      if (dataTieringManager != null) {
1168        coldFiles = dataTieringManager.getColdFilesList();
1169      }
1170      // Scan entire map putting bucket entry into appropriate bucket entry
1171      // group
1172      for (Map.Entry<BlockCacheKey, BucketEntry> bucketEntryWithKey : backingMap.entrySet()) {
1173        BlockCacheKey key = bucketEntryWithKey.getKey();
1174        BucketEntry entry = bucketEntryWithKey.getValue();
1175        // Under certain conditions, blocks for regions not on the current region server might
1176        // be hanging on the cache. For example, when using the persistent cache feature, if the
1177        // RS crashes, then if not the same regions are assigned back once its online again, blocks
1178        // for the previous online regions would be recovered and stay in the cache. These would be
1179        // "orphan" blocks, as the files these blocks belong to are not in any of the online
1180        // regions.
1181        // "Orphan" blocks are a pure waste of cache space and should be evicted first during
1182        // the freespace run.
1183        // Compactions and Flushes may cache blocks before its files are completely written. In
1184        // these cases the file won't be found in any of the online regions stores, but the block
1185        // shouldn't be evicted. To avoid this, we defined this
1186        // hbase.bucketcache.block.orphan.evictgraceperiod property, to account for a grace
1187        // period (default 24 hours) where a block should be checked if it's an orphan block.
1188        if (
1189          allValidFiles != null
1190            && entry.getCachedTime() < (System.nanoTime() - orphanGracePeriodNanos)
1191        ) {
1192          if (!allValidFiles.contains(key.getHfileName())) {
1193            if (evictBucketEntryIfNoRpcReferenced(key, entry)) {
1194              // We calculate the freed bytes, but we don't stop if the goal was reached because
1195              // these are orphan blocks anyway, so let's leverage this run of freeSpace
1196              // to get rid of all orphans at once.
1197              bytesFreed += entry.getLength();
1198              continue;
1199            }
1200          }
1201        }
1202
1203        if (
1204          bytesFreed < bytesToFreeWithExtra && coldFiles != null
1205            && coldFiles.containsKey(bucketEntryWithKey.getKey().getHfileName())
1206        ) {
1207          int freedBlockSize = bucketEntryWithKey.getValue().getLength();
1208          if (evictBlockIfNoRpcReferenced(bucketEntryWithKey.getKey())) {
1209            bytesFreed += freedBlockSize;
1210          }
1211          continue;
1212        }
1213
1214        switch (entry.getPriority()) {
1215          case SINGLE: {
1216            bucketSingle.add(bucketEntryWithKey);
1217            break;
1218          }
1219          case MULTI: {
1220            bucketMulti.add(bucketEntryWithKey);
1221            break;
1222          }
1223          case MEMORY: {
1224            bucketMemory.add(bucketEntryWithKey);
1225            break;
1226          }
1227        }
1228      }
1229
1230      // Check if the cold file eviction is sufficient to create enough space.
1231      bytesToFreeWithExtra -= bytesFreed;
1232      if (bytesToFreeWithExtra <= 0) {
1233        LOG.debug("Bucket cache free space completed; freed space : {} bytes of cold data blocks.",
1234          StringUtils.byteDesc(bytesFreed));
1235        return;
1236      }
1237
1238      if (LOG.isDebugEnabled()) {
1239        LOG.debug(
1240          "Bucket cache free space completed; freed space : {} "
1241            + "bytes of cold data blocks. {} more bytes required to be freed.",
1242          StringUtils.byteDesc(bytesFreed), bytesToFreeWithExtra);
1243      }
1244
1245      PriorityQueue<BucketEntryGroup> bucketQueue =
1246        new PriorityQueue<>(3, Comparator.comparingLong(BucketEntryGroup::overflow));
1247
1248      bucketQueue.add(bucketSingle);
1249      bucketQueue.add(bucketMulti);
1250      bucketQueue.add(bucketMemory);
1251
1252      int remainingBuckets = bucketQueue.size();
1253      BucketEntryGroup bucketGroup;
1254      while ((bucketGroup = bucketQueue.poll()) != null) {
1255        long overflow = bucketGroup.overflow();
1256        if (overflow > 0) {
1257          long bucketBytesToFree =
1258            Math.min(overflow, (bytesToFreeWithoutExtra - bytesFreed) / remainingBuckets);
1259          bytesFreed += bucketGroup.free(bucketBytesToFree);
1260        }
1261        remainingBuckets--;
1262      }
1263
1264      // Check and free if there are buckets that still need freeing of space
1265      if (bucketSizesAboveThresholdCount(minFactor) > 0) {
1266        bucketQueue.clear();
1267        remainingBuckets = 3;
1268        bucketQueue.add(bucketSingle);
1269        bucketQueue.add(bucketMulti);
1270        bucketQueue.add(bucketMemory);
1271        while ((bucketGroup = bucketQueue.poll()) != null) {
1272          long bucketBytesToFree = (bytesToFreeWithExtra - bytesFreed) / remainingBuckets;
1273          bytesFreed += bucketGroup.free(bucketBytesToFree);
1274          remainingBuckets--;
1275        }
1276      }
1277      // Even after the above free we might still need freeing because of the
1278      // De-fragmentation of the buckets (also called Slab Calcification problem), i.e
1279      // there might be some buckets where the occupancy is very sparse and thus are not
1280      // yielding the free for the other bucket sizes, the fix for this to evict some
1281      // of the buckets, we do this by evicting the buckets that are least fulled
1282      freeEntireBuckets(DEFAULT_FREE_ENTIRE_BLOCK_FACTOR * bucketSizesAboveThresholdCount(1.0f));
1283
1284      if (LOG.isDebugEnabled()) {
1285        long single = bucketSingle.totalSize();
1286        long multi = bucketMulti.totalSize();
1287        long memory = bucketMemory.totalSize();
1288        if (LOG.isDebugEnabled()) {
1289          LOG.debug("Bucket cache free space completed; " + "freed="
1290            + StringUtils.byteDesc(bytesFreed) + ", " + "total=" + StringUtils.byteDesc(totalSize)
1291            + ", " + "single=" + StringUtils.byteDesc(single) + ", " + "multi="
1292            + StringUtils.byteDesc(multi) + ", " + "memory=" + StringUtils.byteDesc(memory));
1293        }
1294      }
1295    } catch (Throwable t) {
1296      LOG.warn("Failed freeing space", t);
1297    } finally {
1298      cacheStats.evict();
1299      freeInProgress = false;
1300      freeSpaceLock.unlock();
1301    }
1302  }
1303
1304  // This handles flushing the RAM cache to IOEngine.
1305  class WriterThread extends Thread {
1306    private final BlockingQueue<RAMQueueEntry> inputQueue;
1307    private volatile boolean writerEnabled = true;
1308    private final ByteBuffer metaBuff = ByteBuffer.allocate(HFileBlock.BLOCK_METADATA_SPACE);
1309
1310    WriterThread(BlockingQueue<RAMQueueEntry> queue) {
1311      super("BucketCacheWriterThread");
1312      this.inputQueue = queue;
1313    }
1314
1315    // Used for test
1316    void disableWriter() {
1317      this.writerEnabled = false;
1318    }
1319
1320    @Override
1321    public void run() {
1322      List<RAMQueueEntry> entries = new ArrayList<>();
1323      try {
1324        while (isCacheEnabled() && writerEnabled) {
1325          try {
1326            try {
1327              // Blocks
1328              entries = getRAMQueueEntries(inputQueue, entries);
1329            } catch (InterruptedException ie) {
1330              if (!isCacheEnabled() || !writerEnabled) {
1331                break;
1332              }
1333            }
1334            doDrain(entries, metaBuff);
1335          } catch (Exception ioe) {
1336            LOG.error("WriterThread encountered error", ioe);
1337          }
1338        }
1339      } catch (Throwable t) {
1340        LOG.warn("Failed doing drain", t);
1341      }
1342      LOG.info(this.getName() + " exiting, cacheEnabled=" + isCacheEnabled());
1343    }
1344  }
1345
1346  /**
1347   * Put the new bucket entry into backingMap. Notice that we are allowed to replace the existing
1348   * cache with a new block for the same cache key. there's a corner case: one thread cache a block
1349   * in ramCache, copy to io-engine and add a bucket entry to backingMap. Caching another new block
1350   * with the same cache key do the same thing for the same cache key, so if not evict the previous
1351   * bucket entry, then memory leak happen because the previous bucketEntry is gone but the
1352   * bucketAllocator do not free its memory.
1353   * @see BlockCacheUtil#shouldReplaceExistingCacheBlock(BlockCache blockCache,BlockCacheKey
1354   *      cacheKey, Cacheable newBlock)
1355   * @param key         Block cache key
1356   * @param bucketEntry Bucket entry to put into backingMap.
1357   */
1358  protected void putIntoBackingMap(BlockCacheKey key, BucketEntry bucketEntry) {
1359    BucketEntry previousEntry = backingMap.put(key, bucketEntry);
1360    blocksByHFile.add(key);
1361    updateRegionCachedSize(key, bucketEntry.getLength());
1362    if (previousEntry != null && previousEntry != bucketEntry) {
1363      previousEntry.withWriteLock(offsetLock, () -> {
1364        blockEvicted(key, previousEntry, false, false);
1365        return null;
1366      });
1367    }
1368  }
1369
1370  /**
1371   * Prepare and return a warning message for Bucket Allocator Exception
1372   * @param fle The exception
1373   * @param re  The RAMQueueEntry for which the exception was thrown.
1374   * @return A warning message created from the input RAMQueueEntry object.
1375   */
1376  private static String getAllocationFailWarningMessage(final BucketAllocatorException fle,
1377    final RAMQueueEntry re) {
1378    final StringBuilder sb = new StringBuilder();
1379    sb.append("Most recent failed allocation after ");
1380    sb.append(ALLOCATION_FAIL_LOG_TIME_PERIOD);
1381    sb.append(" ms;");
1382    if (re != null) {
1383      if (re.getData() instanceof HFileBlock) {
1384        final HFileContext fileContext = ((HFileBlock) re.getData()).getHFileContext();
1385        final String columnFamily = Bytes.toString(fileContext.getColumnFamily());
1386        final String tableName = Bytes.toString(fileContext.getTableName());
1387        if (tableName != null) {
1388          sb.append(" Table: ");
1389          sb.append(tableName);
1390        }
1391        if (columnFamily != null) {
1392          sb.append(" CF: ");
1393          sb.append(columnFamily);
1394        }
1395        sb.append(" HFile: ");
1396        if (fileContext.getHFileName() != null) {
1397          sb.append(fileContext.getHFileName());
1398        } else {
1399          sb.append(re.getKey());
1400        }
1401      } else {
1402        sb.append(" HFile: ");
1403        sb.append(re.getKey());
1404      }
1405    }
1406    sb.append(" Message: ");
1407    sb.append(fle.getMessage());
1408    return sb.toString();
1409  }
1410
1411  /**
1412   * Flush the entries in ramCache to IOEngine and add bucket entry to backingMap. Process all that
1413   * are passed in even if failure being sure to remove from ramCache else we'll never undo the
1414   * references and we'll OOME.
1415   * @param entries Presumes list passed in here will be processed by this invocation only. No
1416   *                interference expected.
1417   */
1418  void doDrain(final List<RAMQueueEntry> entries, ByteBuffer metaBuff) throws InterruptedException {
1419    if (entries.isEmpty()) {
1420      return;
1421    }
1422    // This method is a little hard to follow. We run through the passed in entries and for each
1423    // successful add, we add a non-null BucketEntry to the below bucketEntries. Later we must
1424    // do cleanup making sure we've cleared ramCache of all entries regardless of whether we
1425    // successfully added the item to the bucketcache; if we don't do the cleanup, we'll OOME by
1426    // filling ramCache. We do the clean up by again running through the passed in entries
1427    // doing extra work when we find a non-null bucketEntries corresponding entry.
1428    final int size = entries.size();
1429    BucketEntry[] bucketEntries = new BucketEntry[size];
1430    // Index updated inside loop if success or if we can't succeed. We retry if cache is full
1431    // when we go to add an entry by going around the loop again without upping the index.
1432    int index = 0;
1433    while (isCacheEnabled() && index < size) {
1434      RAMQueueEntry re = null;
1435      try {
1436        re = entries.get(index);
1437        if (re == null) {
1438          LOG.warn("Couldn't get entry or changed on us; who else is messing with it?");
1439          index++;
1440          continue;
1441        }
1442        // Reset the position for reuse.
1443        // It should be guaranteed that the data in the metaBuff has been transferred to the
1444        // ioEngine safely. Otherwise, this reuse is problematic. Fortunately, the data is already
1445        // transferred with our current IOEngines. Should take care, when we have new kinds of
1446        // IOEngine in the future.
1447        metaBuff.clear();
1448        BucketEntry bucketEntry = re.writeToCache(ioEngine, bucketAllocator, realCacheSize,
1449          this::createRecycler, metaBuff, acceptableSize());
1450        // Successfully added. Up index and add bucketEntry. Clear io exceptions.
1451        bucketEntries[index] = bucketEntry;
1452        if (ioErrorStartTime > 0) {
1453          ioErrorStartTime = -1;
1454        }
1455        index++;
1456      } catch (BucketAllocatorException fle) {
1457        long currTs = EnvironmentEdgeManager.currentTime();
1458        cacheStats.allocationFailed(); // Record the warning.
1459        if (
1460          allocFailLogPrevTs == 0 || (currTs - allocFailLogPrevTs) > ALLOCATION_FAIL_LOG_TIME_PERIOD
1461        ) {
1462          LOG.warn(getAllocationFailWarningMessage(fle, re));
1463          allocFailLogPrevTs = currTs;
1464        }
1465        // Presume can't add. Too big? Move index on. Entry will be cleared from ramCache below.
1466        bucketEntries[index] = null;
1467        index++;
1468      } catch (CacheFullException cfe) {
1469        // Cache full when we tried to add. Try freeing space and then retrying (don't up index)
1470        if (!freeInProgress && !re.isPrefetch()) {
1471          freeSpace("Full!");
1472        } else if (re.isPrefetch()) {
1473          bucketEntries[index] = null;
1474          index++;
1475        } else {
1476          Thread.sleep(50);
1477        }
1478      } catch (IOException ioex) {
1479        // Hopefully transient. Retry. checkIOErrorIsTolerated disables cache if problem.
1480        LOG.error("Failed writing to bucket cache", ioex);
1481        checkIOErrorIsTolerated();
1482      }
1483    }
1484
1485    // Make sure data pages are written on media before we update maps.
1486    try {
1487      ioEngine.sync();
1488    } catch (IOException ioex) {
1489      LOG.error("Failed syncing IO engine", ioex);
1490      checkIOErrorIsTolerated();
1491      // Since we failed sync, free the blocks in bucket allocator
1492      for (int i = 0; i < entries.size(); ++i) {
1493        BucketEntry bucketEntry = bucketEntries[i];
1494        if (bucketEntry != null) {
1495          bucketAllocator.freeBlock(bucketEntry.offset(), bucketEntry.getLength());
1496          bucketEntries[i] = null;
1497        }
1498      }
1499    }
1500
1501    // Now add to backingMap if successfully added to bucket cache. Remove from ramCache if
1502    // success or error.
1503    for (int i = 0; i < size; ++i) {
1504      BlockCacheKey key = entries.get(i).getKey();
1505      // Only add if non-null entry.
1506      if (bucketEntries[i] != null) {
1507        putIntoBackingMap(key, bucketEntries[i]);
1508        if (ioEngine.isPersistent()) {
1509          setCacheInconsistent(true);
1510        }
1511      }
1512      // Always remove from ramCache even if we failed adding it to the block cache above.
1513      boolean existed = ramCache.remove(key, re -> {
1514        if (re != null) {
1515          heapSize.add(-1 * re.getData().heapSize());
1516        }
1517      });
1518      if (!existed && bucketEntries[i] != null) {
1519        // Block should have already been evicted. Remove it and free space.
1520        final BucketEntry bucketEntry = bucketEntries[i];
1521        bucketEntry.withWriteLock(offsetLock, () -> {
1522          if (backingMap.remove(key, bucketEntry)) {
1523            blockEvicted(key, bucketEntry, false, false);
1524          }
1525          return null;
1526        });
1527      }
1528      long used = bucketAllocator.getUsedSize();
1529      if (!entries.get(i).isPrefetch() && used > acceptableSize()) {
1530        LOG.debug("Calling freeSpace for block: {}", entries.get(i).getKey());
1531        freeSpace("Used=" + used + " > acceptable=" + acceptableSize());
1532      }
1533    }
1534
1535  }
1536
1537  /**
1538   * Blocks until elements available in {@code q} then tries to grab as many as possible before
1539   * returning.
1540   * @param receptacle Where to stash the elements taken from queue. We clear before we use it just
1541   *                   in case.
1542   * @param q          The queue to take from.
1543   * @return {@code receptacle} laden with elements taken from the queue or empty if none found.
1544   */
1545  static List<RAMQueueEntry> getRAMQueueEntries(BlockingQueue<RAMQueueEntry> q,
1546    List<RAMQueueEntry> receptacle) throws InterruptedException {
1547    // Clear sets all entries to null and sets size to 0. We retain allocations. Presume it
1548    // ok even if list grew to accommodate thousands.
1549    receptacle.clear();
1550    receptacle.add(q.take());
1551    q.drainTo(receptacle);
1552    return receptacle;
1553  }
1554
1555  /**
1556   * @see #retrieveFromFile(int[])
1557   */
1558  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "OBL_UNSATISFIED_OBLIGATION",
1559      justification = "false positive, try-with-resources ensures close is called.")
1560  void persistToFile() throws IOException {
1561    LOG.debug("Thread {} started persisting bucket cache to file",
1562      Thread.currentThread().getName());
1563    if (!isCachePersistent()) {
1564      throw new IOException("Attempt to persist non-persistent cache mappings!");
1565    }
1566    File tempPersistencePath = new File(persistencePath + EnvironmentEdgeManager.currentTime());
1567    try (FileOutputStream fos = new FileOutputStream(tempPersistencePath, false)) {
1568      LOG.debug("Persist in new chunked persistence format.");
1569
1570      persistChunkedBackingMap(fos);
1571
1572      LOG.debug(
1573        "PersistToFile: after persisting backing map size: {}, fullycachedFiles size: {},"
1574          + " file name: {}",
1575        backingMap.size(), fullyCachedFiles.size(), tempPersistencePath.getName());
1576    } catch (IOException e) {
1577      LOG.error("Failed to persist bucket cache to file", e);
1578      throw e;
1579    } catch (Throwable e) {
1580      LOG.error("Failed during persist bucket cache to file: ", e);
1581      throw e;
1582    }
1583    LOG.debug("Thread {} finished persisting bucket cache to file, renaming",
1584      Thread.currentThread().getName());
1585    if (!tempPersistencePath.renameTo(new File(persistencePath))) {
1586      LOG.warn("Failed to commit cache persistent file. We might lose cached blocks if "
1587        + "RS crashes/restarts before we successfully checkpoint again.");
1588    }
1589  }
1590
1591  public boolean isCachePersistent() {
1592    return ioEngine.isPersistent() && persistencePath != null;
1593  }
1594
1595  @Override
1596  public Optional<Map<String, Long>> getRegionCachedInfo() {
1597    return Optional.of(Collections.unmodifiableMap(regionCachedSize));
1598  }
1599
1600  /**
1601   * @see #persistToFile()
1602   */
1603  private void retrieveFromFile(int[] bucketSizes) throws IOException {
1604    LOG.info("Started retrieving bucket cache from file");
1605    File persistenceFile = new File(persistencePath);
1606    if (!persistenceFile.exists()) {
1607      LOG.warn("Persistence file missing! "
1608        + "It's ok if it's first run after enabling persistent cache.");
1609      bucketAllocator = new BucketAllocator(cacheCapacity, bucketSizes, backingMap, realCacheSize);
1610      blockNumber.add(backingMap.size());
1611      backingMapValidated.set(true);
1612      return;
1613    }
1614    assert !isCacheEnabled();
1615
1616    try (FileInputStream in = new FileInputStream(persistenceFile)) {
1617      int pblen = ProtobufMagic.lengthOfPBMagic();
1618      byte[] pbuf = new byte[pblen];
1619      IOUtils.readFully(in, pbuf, 0, pblen);
1620      if (ProtobufMagic.isPBMagicPrefix(pbuf)) {
1621        LOG.info("Reading old format of persistence.");
1622        // The old non-chunked version of backing map persistence.
1623        parsePB(BucketCacheProtos.BucketCacheEntry.parseDelimitedFrom(in));
1624      } else if (Arrays.equals(pbuf, BucketProtoUtils.PB_MAGIC_V2)) {
1625        // The new persistence format of chunked persistence.
1626        LOG.info("Reading new chunked format of persistence.");
1627        retrieveChunkedBackingMap(in);
1628      } else {
1629        // In 3.0 we have enough flexibility to dump the old cache data.
1630        // TODO: In 2.x line, this might need to be filled in to support reading the old format
1631        throw new IOException(
1632          "Persistence file does not start with protobuf magic number. " + persistencePath);
1633      }
1634      bucketAllocator = new BucketAllocator(cacheCapacity, bucketSizes, backingMap, realCacheSize);
1635      blockNumber.add(backingMap.size());
1636      LOG.info("Bucket cache retrieved from file successfully with size: {}", backingMap.size());
1637    }
1638  }
1639
1640  private void updateRegionSizeMapWhileRetrievingFromFile() {
1641    // Update the regionCachedSize with the region size while restarting the region server
1642    if (LOG.isDebugEnabled()) {
1643      LOG.debug("Updating region size map after retrieving cached file list");
1644      dumpPrefetchList();
1645    }
1646    regionCachedSize.clear();
1647    fullyCachedFiles.forEach((hFileName, hFileSize) -> {
1648      // Get the region name for each file
1649      String regionEncodedName = hFileSize.getFirst();
1650      long cachedFileSize = hFileSize.getSecond();
1651      regionCachedSize.merge(regionEncodedName, cachedFileSize,
1652        (oldpf, fileSize) -> oldpf + fileSize);
1653    });
1654  }
1655
1656  private void dumpPrefetchList() {
1657    for (Map.Entry<String, Pair<String, Long>> outerEntry : fullyCachedFiles.entrySet()) {
1658      LOG.debug("Cached File Entry:<{},<{},{}>>", outerEntry.getKey(),
1659        outerEntry.getValue().getFirst(), outerEntry.getValue().getSecond());
1660    }
1661  }
1662
1663  private void verifyCapacityAndClasses(long capacitySize, String ioclass, String mapclass)
1664    throws IOException {
1665    if (capacitySize != cacheCapacity) {
1666      throw new IOException("Mismatched cache capacity:" + StringUtils.byteDesc(capacitySize)
1667        + ", expected: " + StringUtils.byteDesc(cacheCapacity));
1668    }
1669    if (!ioEngine.getClass().getName().equals(ioclass)) {
1670      throw new IOException("Class name for IO engine mismatch: " + ioclass + ", expected:"
1671        + ioEngine.getClass().getName());
1672    }
1673    if (!backingMap.getClass().getName().equals(mapclass)) {
1674      throw new IOException("Class name for cache map mismatch: " + mapclass + ", expected:"
1675        + backingMap.getClass().getName());
1676    }
1677  }
1678
1679  private void verifyFileIntegrity(BucketCacheProtos.BucketCacheEntry proto) {
1680    try {
1681      if (proto.hasChecksum()) {
1682        ((PersistentIOEngine) ioEngine).verifyFileIntegrity(proto.getChecksum().toByteArray(),
1683          algorithm);
1684      }
1685      backingMapValidated.set(true);
1686    } catch (IOException e) {
1687      LOG.warn("Checksum for cache file failed. "
1688        + "We need to validate each cache key in the backing map. "
1689        + "This may take some time, so we'll do it in a background thread,");
1690
1691      Runnable cacheValidator = () -> {
1692        while (bucketAllocator == null) {
1693          try {
1694            Thread.sleep(50);
1695          } catch (InterruptedException ex) {
1696            throw new RuntimeException(ex);
1697          }
1698        }
1699        long startTime = EnvironmentEdgeManager.currentTime();
1700        int totalKeysOriginally = backingMap.size();
1701        for (Map.Entry<BlockCacheKey, BucketEntry> keyEntry : backingMap.entrySet()) {
1702          try {
1703            ((FileIOEngine) ioEngine).checkCacheTime(keyEntry.getValue());
1704          } catch (IOException e1) {
1705            LOG.debug("Check for key {} failed. Evicting.", keyEntry.getKey());
1706            evictBlock(keyEntry.getKey());
1707            fileNotFullyCached(keyEntry.getKey(), keyEntry.getValue());
1708          }
1709        }
1710        backingMapValidated.set(true);
1711        LOG.info("Finished validating {} keys in the backing map. Recovered: {}. This took {}ms.",
1712          totalKeysOriginally, backingMap.size(),
1713          (EnvironmentEdgeManager.currentTime() - startTime));
1714      };
1715      Thread t = new Thread(cacheValidator);
1716      t.setDaemon(true);
1717      t.start();
1718    }
1719  }
1720
1721  private void updateCacheIndex(BucketCacheProtos.BackingMap chunk,
1722    java.util.Map<java.lang.Integer, java.lang.String> deserializer) throws IOException {
1723    Pair<ConcurrentHashMap<BlockCacheKey, BucketEntry>, NavigableSet<BlockCacheKey>> pair2 =
1724      BucketProtoUtils.fromPB(deserializer, chunk, this::createRecycler);
1725    backingMap.putAll(pair2.getFirst());
1726    blocksByHFile.addAll(pair2.getSecond());
1727  }
1728
1729  private void parsePB(BucketCacheProtos.BucketCacheEntry proto) throws IOException {
1730    Pair<ConcurrentHashMap<BlockCacheKey, BucketEntry>, NavigableSet<BlockCacheKey>> pair =
1731      BucketProtoUtils.fromPB(proto.getDeserializersMap(), proto.getBackingMap(),
1732        this::createRecycler);
1733    backingMap = pair.getFirst();
1734    blocksByHFile = pair.getSecond();
1735    fullyCachedFiles.clear();
1736    fullyCachedFiles.putAll(BucketProtoUtils.fromPB(proto.getCachedFilesMap()));
1737
1738    LOG.info("After retrieval Backing map size: {}, fullyCachedFiles size: {}", backingMap.size(),
1739      fullyCachedFiles.size());
1740
1741    verifyFileIntegrity(proto);
1742    updateRegionSizeMapWhileRetrievingFromFile();
1743    verifyCapacityAndClasses(proto.getCacheCapacity(), proto.getIoClass(), proto.getMapClass());
1744  }
1745
1746  private void persistChunkedBackingMap(FileOutputStream fos) throws IOException {
1747    LOG.debug(
1748      "persistToFile: before persisting backing map size: {}, "
1749        + "fullycachedFiles size: {}, chunkSize: {}",
1750      backingMap.size(), fullyCachedFiles.size(), persistenceChunkSize);
1751
1752    BucketProtoUtils.serializeAsPB(this, fos, persistenceChunkSize);
1753
1754    LOG.debug(
1755      "persistToFile: after persisting backing map size: {}, " + "fullycachedFiles size: {}",
1756      backingMap.size(), fullyCachedFiles.size());
1757  }
1758
1759  private void retrieveChunkedBackingMap(FileInputStream in) throws IOException {
1760
1761    // Read the first chunk that has all the details.
1762    BucketCacheProtos.BucketCacheEntry cacheEntry =
1763      BucketCacheProtos.BucketCacheEntry.parseDelimitedFrom(in);
1764
1765    fullyCachedFiles.clear();
1766    fullyCachedFiles.putAll(BucketProtoUtils.fromPB(cacheEntry.getCachedFilesMap()));
1767
1768    backingMap.clear();
1769    blocksByHFile.clear();
1770    FilePathStringPool.getInstance().clear();
1771
1772    // Read the backing map entries in batches.
1773    int numChunks = 0;
1774    while (in.available() > 0) {
1775      updateCacheIndex(BucketCacheProtos.BackingMap.parseDelimitedFrom(in),
1776        cacheEntry.getDeserializersMap());
1777      numChunks++;
1778    }
1779
1780    LOG.info("Retrieved {} of chunks with blockCount = {}.", numChunks, backingMap.size());
1781    verifyFileIntegrity(cacheEntry);
1782    verifyCapacityAndClasses(cacheEntry.getCacheCapacity(), cacheEntry.getIoClass(),
1783      cacheEntry.getMapClass());
1784    updateRegionSizeMapWhileRetrievingFromFile();
1785  }
1786
1787  /**
1788   * Check whether we tolerate IO error this time. If the duration of IOEngine throwing errors
1789   * exceeds ioErrorsDurationTimeTolerated, we will disable the cache
1790   */
1791  private void checkIOErrorIsTolerated() {
1792    long now = EnvironmentEdgeManager.currentTime();
1793    // Do a single read to a local variable to avoid timing issue - HBASE-24454
1794    long ioErrorStartTimeTmp = this.ioErrorStartTime;
1795    if (ioErrorStartTimeTmp > 0) {
1796      if (isCacheEnabled() && (now - ioErrorStartTimeTmp) > this.ioErrorsTolerationDuration) {
1797        LOG.error("IO errors duration time has exceeded " + ioErrorsTolerationDuration
1798          + "ms, disabling cache, please check your IOEngine");
1799        disableCache();
1800      }
1801    } else {
1802      this.ioErrorStartTime = now;
1803    }
1804  }
1805
1806  /**
1807   * Used to shut down the cache -or- turn it off in the case of something broken.
1808   */
1809  private void disableCache() {
1810    if (!isCacheEnabled()) {
1811      return;
1812    }
1813    LOG.info("Disabling cache");
1814    cacheState = CacheState.DISABLED;
1815    ioEngine.shutdown();
1816    this.scheduleThreadPool.shutdown();
1817    for (int i = 0; i < writerThreads.length; ++i)
1818      writerThreads[i].interrupt();
1819    this.ramCache.clear();
1820    if (!ioEngine.isPersistent() || persistencePath == null) {
1821      // If persistent ioengine and a path, we will serialize out the backingMap.
1822      this.backingMap.clear();
1823      this.blocksByHFile.clear();
1824      this.fullyCachedFiles.clear();
1825      this.regionCachedSize.clear();
1826      FilePathStringPool.getInstance().clear();
1827    }
1828    if (cacheStats.getMetricsRollerScheduler() != null) {
1829      cacheStats.getMetricsRollerScheduler().shutdownNow();
1830    }
1831  }
1832
1833  private void join() throws InterruptedException {
1834    for (int i = 0; i < writerThreads.length; ++i)
1835      writerThreads[i].join();
1836  }
1837
1838  @Override
1839  public void shutdown() {
1840    if (isCacheEnabled()) {
1841      disableCache();
1842      LOG.info("Shutdown bucket cache: IO persistent=" + ioEngine.isPersistent()
1843        + "; path to write=" + persistencePath);
1844      if (ioEngine.isPersistent() && persistencePath != null) {
1845        try {
1846          join();
1847          if (cachePersister != null) {
1848            LOG.info("Shutting down cache persister thread.");
1849            cachePersister.shutdown();
1850            while (cachePersister.isAlive()) {
1851              Thread.sleep(10);
1852            }
1853          }
1854          persistToFile();
1855          FilePathStringPool.getInstance().clear();
1856        } catch (IOException ex) {
1857          LOG.error("Unable to persist data on exit: " + ex.toString(), ex);
1858        } catch (InterruptedException e) {
1859          LOG.warn("Failed to persist data on exit", e);
1860        }
1861      }
1862    }
1863  }
1864
1865  /**
1866   * Needed mostly for UTs that might run in the same VM and create different BucketCache instances
1867   * on different UT methods.
1868   */
1869  @Override
1870  protected void finalize() {
1871    if (cachePersister != null && !cachePersister.isInterrupted()) {
1872      cachePersister.interrupt();
1873    }
1874  }
1875
1876  @Override
1877  public CacheStats getStats() {
1878    return cacheStats;
1879  }
1880
1881  public BucketAllocator getAllocator() {
1882    return this.bucketAllocator;
1883  }
1884
1885  @Override
1886  public long heapSize() {
1887    return this.heapSize.sum();
1888  }
1889
1890  @Override
1891  public long size() {
1892    return this.realCacheSize.sum();
1893  }
1894
1895  @Override
1896  public long getCurrentDataSize() {
1897    return size();
1898  }
1899
1900  @Override
1901  public long getFreeSize() {
1902    if (!isCacheInitialized("BucketCache:getFreeSize")) {
1903      return 0;
1904    }
1905    return this.bucketAllocator.getFreeSize();
1906  }
1907
1908  @Override
1909  public long getBlockCount() {
1910    return this.blockNumber.sum();
1911  }
1912
1913  @Override
1914  public long getDataBlockCount() {
1915    return getBlockCount();
1916  }
1917
1918  @Override
1919  public long getCurrentSize() {
1920    if (!isCacheInitialized("BucketCache::getCurrentSize")) {
1921      return 0;
1922    }
1923    return this.bucketAllocator.getUsedSize();
1924  }
1925
1926  protected String getAlgorithm() {
1927    return algorithm;
1928  }
1929
1930  /**
1931   * Evicts all blocks for a specific HFile.
1932   * <p>
1933   * This is used for evict-on-close to remove all blocks of a specific HFile.
1934   * @return the number of blocks evicted
1935   */
1936  @Override
1937  public int evictBlocksByHfileName(String hfileName) {
1938    return evictBlocksRangeByHfileName(hfileName, 0, Long.MAX_VALUE);
1939  }
1940
1941  @Override
1942  public int evictBlocksRangeByHfileName(String hfileName, long initOffset, long endOffset) {
1943    Set<BlockCacheKey> keySet = getAllCacheKeysForFile(hfileName, initOffset, endOffset);
1944    // We need to make sure whether we are evicting all blocks for this given file. In case of
1945    // split references, we might be evicting just half of the blocks
1946    int totalFileKeys = (endOffset == Long.MAX_VALUE)
1947      ? keySet.size()
1948      : getAllCacheKeysForFile(hfileName, 0, Long.MAX_VALUE).size();
1949    LOG.debug("found {} blocks for file {}, starting offset: {}, end offset: {}", keySet.size(),
1950      hfileName, initOffset, endOffset);
1951    int numEvicted = 0;
1952    for (BlockCacheKey key : keySet) {
1953      if (evictBlock(key)) {
1954        ++numEvicted;
1955      }
1956    }
1957    if (numEvicted > 0) {
1958      if (totalFileKeys == numEvicted) {
1959        FilePathStringPool.getInstance().remove(hfileName);
1960      }
1961    }
1962    return numEvicted;
1963  }
1964
1965  private Set<BlockCacheKey> getAllCacheKeysForFile(String hfileName, long init, long end) {
1966    // These keys are just for comparison and are short lived, so we need only file name and offset
1967    return blocksByHFile.subSet(new BlockCacheKey(hfileName, init), true,
1968      new BlockCacheKey(hfileName, end), true);
1969  }
1970
1971  /**
1972   * Used to group bucket entries into priority buckets. There will be a BucketEntryGroup for each
1973   * priority (single, multi, memory). Once bucketed, the eviction algorithm takes the appropriate
1974   * number of elements out of each according to configuration parameters and their relative sizes.
1975   */
1976  private class BucketEntryGroup {
1977
1978    private CachedEntryQueue queue;
1979    private long totalSize = 0;
1980    private long bucketSize;
1981
1982    public BucketEntryGroup(long bytesToFree, long blockSize, long bucketSize) {
1983      this.bucketSize = bucketSize;
1984      queue = new CachedEntryQueue(bytesToFree, blockSize);
1985      totalSize = 0;
1986    }
1987
1988    public void add(Map.Entry<BlockCacheKey, BucketEntry> block) {
1989      totalSize += block.getValue().getLength();
1990      queue.add(block);
1991    }
1992
1993    public long free(long toFree) {
1994      Map.Entry<BlockCacheKey, BucketEntry> entry;
1995      long freedBytes = 0;
1996      // TODO avoid a cycling siutation. We find no block which is not in use and so no way to free
1997      // What to do then? Caching attempt fail? Need some changes in cacheBlock API?
1998      while ((entry = queue.pollLast()) != null) {
1999        BlockCacheKey blockCacheKey = entry.getKey();
2000        BucketEntry be = entry.getValue();
2001        if (evictBucketEntryIfNoRpcReferenced(blockCacheKey, be)) {
2002          freedBytes += be.getLength();
2003        }
2004        if (freedBytes >= toFree) {
2005          return freedBytes;
2006        }
2007      }
2008      return freedBytes;
2009    }
2010
2011    public long overflow() {
2012      return totalSize - bucketSize;
2013    }
2014
2015    public long totalSize() {
2016      return totalSize;
2017    }
2018  }
2019
2020  /**
2021   * Block Entry stored in the memory with key,data and so on
2022   */
2023  static class RAMQueueEntry {
2024    private final BlockCacheKey key;
2025    private final Cacheable data;
2026    private long accessCounter;
2027    private boolean inMemory;
2028    private boolean isCachePersistent;
2029
2030    private boolean isPrefetch;
2031
2032    RAMQueueEntry(BlockCacheKey bck, Cacheable data, long accessCounter, boolean inMemory,
2033      boolean isCachePersistent, boolean isPrefetch) {
2034      this.key = bck;
2035      this.data = data;
2036      this.accessCounter = accessCounter;
2037      this.inMemory = inMemory;
2038      this.isCachePersistent = isCachePersistent;
2039      this.isPrefetch = isPrefetch;
2040    }
2041
2042    public Cacheable getData() {
2043      return data;
2044    }
2045
2046    public BlockCacheKey getKey() {
2047      return key;
2048    }
2049
2050    public boolean isPrefetch() {
2051      return isPrefetch;
2052    }
2053
2054    public void access(long accessCounter) {
2055      this.accessCounter = accessCounter;
2056    }
2057
2058    private ByteBuffAllocator getByteBuffAllocator() {
2059      if (data instanceof HFileBlock) {
2060        return ((HFileBlock) data).getByteBuffAllocator();
2061      }
2062      return ByteBuffAllocator.HEAP;
2063    }
2064
2065    public BucketEntry writeToCache(final IOEngine ioEngine, final BucketAllocator alloc,
2066      final LongAdder realCacheSize, Function<BucketEntry, Recycler> createRecycler,
2067      ByteBuffer metaBuff, final Long acceptableSize) throws IOException {
2068      int len = data.getSerializedLength();
2069      // This cacheable thing can't be serialized
2070      if (len == 0) {
2071        return null;
2072      }
2073      if (isCachePersistent && data instanceof HFileBlock) {
2074        len += Long.BYTES; // we need to record the cache time for consistency check in case of
2075                           // recovery
2076      }
2077      long offset = alloc.allocateBlock(len);
2078      // In the case of prefetch, we want to avoid freeSpace runs when the cache is full.
2079      // this makes the cache allocation more predictable, and is particularly important
2080      // when persistent cache is enabled, as it won't trigger evictions of the recovered blocks,
2081      // which are likely the most accessed and relevant blocks in the cache.
2082      if (isPrefetch() && alloc.getUsedSize() > acceptableSize) {
2083        alloc.freeBlock(offset, len);
2084        return null;
2085      }
2086      boolean succ = false;
2087      BucketEntry bucketEntry = null;
2088      try {
2089        int diskSizeWithHeader = (data instanceof HFileBlock)
2090          ? ((HFileBlock) data).getOnDiskSizeWithHeader()
2091          : data.getSerializedLength();
2092        bucketEntry = new BucketEntry(offset, len, diskSizeWithHeader, accessCounter, inMemory,
2093          createRecycler, getByteBuffAllocator());
2094        bucketEntry.setDeserializerReference(data.getDeserializer());
2095        if (data instanceof HFileBlock) {
2096          // If an instance of HFileBlock, save on some allocations.
2097          HFileBlock block = (HFileBlock) data;
2098          ByteBuff sliceBuf = block.getBufferReadOnly();
2099          block.getMetaData(metaBuff);
2100          // adds the cache time prior to the block and metadata part
2101          if (isCachePersistent) {
2102            ByteBuffer buffer = ByteBuffer.allocate(Long.BYTES);
2103            buffer.putLong(bucketEntry.getCachedTime());
2104            buffer.rewind();
2105            ioEngine.write(buffer, offset);
2106            ioEngine.write(sliceBuf, (offset + Long.BYTES));
2107          } else {
2108            ioEngine.write(sliceBuf, offset);
2109          }
2110          ioEngine.write(metaBuff, offset + len - metaBuff.limit());
2111        } else {
2112          // Only used for testing.
2113          ByteBuffer bb = ByteBuffer.allocate(len);
2114          data.serialize(bb, true);
2115          ioEngine.write(bb, offset);
2116        }
2117        succ = true;
2118      } finally {
2119        if (!succ) {
2120          alloc.freeBlock(offset, len);
2121        }
2122      }
2123      realCacheSize.add(len);
2124      return bucketEntry;
2125    }
2126  }
2127
2128  /**
2129   * Only used in test
2130   */
2131  void stopWriterThreads() throws InterruptedException {
2132    for (WriterThread writerThread : writerThreads) {
2133      writerThread.disableWriter();
2134      writerThread.interrupt();
2135      writerThread.join();
2136    }
2137  }
2138
2139  @Override
2140  public Iterator<CachedBlock> iterator() {
2141    // Don't bother with ramcache since stuff is in here only a little while.
2142    final Iterator<Map.Entry<BlockCacheKey, BucketEntry>> i = this.backingMap.entrySet().iterator();
2143    return new Iterator<CachedBlock>() {
2144      private final long now = System.nanoTime();
2145
2146      @Override
2147      public boolean hasNext() {
2148        return i.hasNext();
2149      }
2150
2151      @Override
2152      public CachedBlock next() {
2153        final Map.Entry<BlockCacheKey, BucketEntry> e = i.next();
2154        return new CachedBlock() {
2155          @Override
2156          public String toString() {
2157            return BlockCacheUtil.toString(this, now);
2158          }
2159
2160          @Override
2161          public BlockPriority getBlockPriority() {
2162            return e.getValue().getPriority();
2163          }
2164
2165          @Override
2166          public BlockType getBlockType() {
2167            // Not held by BucketEntry. Could add it if wanted on BucketEntry creation.
2168            return null;
2169          }
2170
2171          @Override
2172          public long getOffset() {
2173            return e.getKey().getOffset();
2174          }
2175
2176          @Override
2177          public long getSize() {
2178            return e.getValue().getLength();
2179          }
2180
2181          @Override
2182          public long getCachedTime() {
2183            return e.getValue().getCachedTime();
2184          }
2185
2186          @Override
2187          public String getFilename() {
2188            return e.getKey().getHfileName();
2189          }
2190
2191          @Override
2192          public int compareTo(CachedBlock other) {
2193            int diff = this.getFilename().compareTo(other.getFilename());
2194            if (diff != 0) return diff;
2195
2196            diff = Long.compare(this.getOffset(), other.getOffset());
2197            if (diff != 0) return diff;
2198            if (other.getCachedTime() < 0 || this.getCachedTime() < 0) {
2199              throw new IllegalStateException(
2200                "" + this.getCachedTime() + ", " + other.getCachedTime());
2201            }
2202            return Long.compare(other.getCachedTime(), this.getCachedTime());
2203          }
2204
2205          @Override
2206          public int hashCode() {
2207            return e.getKey().hashCode();
2208          }
2209
2210          @Override
2211          public boolean equals(Object obj) {
2212            if (obj instanceof CachedBlock) {
2213              CachedBlock cb = (CachedBlock) obj;
2214              return compareTo(cb) == 0;
2215            } else {
2216              return false;
2217            }
2218          }
2219        };
2220      }
2221
2222      @Override
2223      public void remove() {
2224        throw new UnsupportedOperationException();
2225      }
2226    };
2227  }
2228
2229  @Override
2230  public BlockCache[] getBlockCaches() {
2231    return null;
2232  }
2233
2234  public int getRpcRefCount(BlockCacheKey cacheKey) {
2235    BucketEntry bucketEntry = backingMap.get(cacheKey);
2236    if (bucketEntry != null) {
2237      return bucketEntry.refCnt() - (bucketEntry.markedAsEvicted.get() ? 0 : 1);
2238    }
2239    return 0;
2240  }
2241
2242  float getAcceptableFactor() {
2243    return acceptableFactor;
2244  }
2245
2246  float getMinFactor() {
2247    return minFactor;
2248  }
2249
2250  float getExtraFreeFactor() {
2251    return extraFreeFactor;
2252  }
2253
2254  float getSingleFactor() {
2255    return singleFactor;
2256  }
2257
2258  float getMultiFactor() {
2259    return multiFactor;
2260  }
2261
2262  float getMemoryFactor() {
2263    return memoryFactor;
2264  }
2265
2266  long getQueueAdditionWaitTime() {
2267    return queueAdditionWaitTime;
2268  }
2269
2270  long getPersistenceChunkSize() {
2271    return persistenceChunkSize;
2272  }
2273
2274  long getBucketcachePersistInterval() {
2275    return bucketcachePersistInterval;
2276  }
2277
2278  public String getPersistencePath() {
2279    return persistencePath;
2280  }
2281
2282  /**
2283   * Wrapped the delegate ConcurrentMap with maintaining its block's reference count.
2284   */
2285  static class RAMCache {
2286    /**
2287     * Defined the map as {@link ConcurrentHashMap} explicitly here, because in
2288     * {@link RAMCache#get(BlockCacheKey)} and
2289     * {@link RAMCache#putIfAbsent(BlockCacheKey, BucketCache.RAMQueueEntry)} , we need to guarantee
2290     * the atomicity of map#computeIfPresent(key, func) and map#putIfAbsent(key, func). Besides, the
2291     * func method can execute exactly once only when the key is present(or absent) and under the
2292     * lock context. Otherwise, the reference count of block will be messed up. Notice that the
2293     * {@link java.util.concurrent.ConcurrentSkipListMap} can not guarantee that.
2294     */
2295    final ConcurrentHashMap<BlockCacheKey, RAMQueueEntry> delegate = new ConcurrentHashMap<>();
2296
2297    public boolean containsKey(BlockCacheKey key) {
2298      return delegate.containsKey(key);
2299    }
2300
2301    public RAMQueueEntry get(BlockCacheKey key) {
2302      return delegate.computeIfPresent(key, (k, re) -> {
2303        // It'll be referenced by RPC, so retain atomically here. if the get and retain is not
2304        // atomic, another thread may remove and release the block, when retaining in this thread we
2305        // may retain a block with refCnt=0 which is disallowed. (see HBASE-22422)
2306        re.getData().retain();
2307        return re;
2308      });
2309    }
2310
2311    /**
2312     * Return the previous associated value, or null if absent. It has the same meaning as
2313     * {@link ConcurrentMap#putIfAbsent(Object, Object)}
2314     */
2315    public RAMQueueEntry putIfAbsent(BlockCacheKey key, RAMQueueEntry entry) {
2316      AtomicBoolean absent = new AtomicBoolean(false);
2317      RAMQueueEntry re = delegate.computeIfAbsent(key, k -> {
2318        // The RAMCache reference to this entry, so reference count should be increment.
2319        entry.getData().retain();
2320        absent.set(true);
2321        return entry;
2322      });
2323      return absent.get() ? null : re;
2324    }
2325
2326    public boolean remove(BlockCacheKey key) {
2327      return remove(key, re -> {
2328      });
2329    }
2330
2331    /**
2332     * Defined an {@link Consumer} here, because once the removed entry release its reference count,
2333     * then it's ByteBuffers may be recycled and accessing it outside this method will be thrown an
2334     * exception. the consumer will access entry to remove before release its reference count.
2335     * Notice, don't change its reference count in the {@link Consumer}
2336     */
2337    public boolean remove(BlockCacheKey key, Consumer<RAMQueueEntry> action) {
2338      RAMQueueEntry previous = delegate.remove(key);
2339      action.accept(previous);
2340      if (previous != null) {
2341        previous.getData().release();
2342      }
2343      return previous != null;
2344    }
2345
2346    public boolean isEmpty() {
2347      return delegate.isEmpty();
2348    }
2349
2350    public void clear() {
2351      Iterator<Map.Entry<BlockCacheKey, RAMQueueEntry>> it = delegate.entrySet().iterator();
2352      while (it.hasNext()) {
2353        RAMQueueEntry re = it.next().getValue();
2354        it.remove();
2355        re.getData().release();
2356      }
2357    }
2358
2359    public boolean hasBlocksForFile(String fileName) {
2360      return delegate.keySet().stream().filter(key -> key.getHfileName().equals(fileName))
2361        .findFirst().isPresent();
2362    }
2363  }
2364
2365  public Map<BlockCacheKey, BucketEntry> getBackingMap() {
2366    return backingMap;
2367  }
2368
2369  public AtomicBoolean getBackingMapValidated() {
2370    return backingMapValidated;
2371  }
2372
2373  @Override
2374  public Optional<Map<String, Pair<String, Long>>> getFullyCachedFiles() {
2375    return Optional.of(fullyCachedFiles);
2376  }
2377
2378  public static Optional<BucketCache> getBucketCacheFromCacheConfig(CacheConfig cacheConf) {
2379    if (cacheConf.getBlockCache().isPresent()) {
2380      BlockCache bc = cacheConf.getBlockCache().get();
2381      if (bc instanceof CombinedBlockCache) {
2382        BlockCache l2 = ((CombinedBlockCache) bc).getSecondLevelCache();
2383        if (l2 instanceof BucketCache) {
2384          return Optional.of((BucketCache) l2);
2385        }
2386      } else if (bc instanceof BucketCache) {
2387        return Optional.of((BucketCache) bc);
2388      }
2389    }
2390    return Optional.empty();
2391  }
2392
2393  @Override
2394  public void notifyFileCachingCompleted(Path fileName, int totalBlockCount, int dataBlockCount,
2395    long size) {
2396    // block eviction may be happening in the background as prefetch runs,
2397    // so we need to count all blocks for this file in the backing map under
2398    // a read lock for the block offset
2399    final List<ReentrantReadWriteLock> locks = new ArrayList<>();
2400    LOG.debug("Notifying caching completed for file {}, with total blocks {}, and data blocks {}",
2401      fileName, totalBlockCount, dataBlockCount);
2402    try {
2403      final MutableInt count = new MutableInt();
2404      LOG.debug("iterating over {} entries in the backing map", backingMap.size());
2405      Set<BlockCacheKey> result = getAllCacheKeysForFile(fileName.getName(), 0, Long.MAX_VALUE);
2406      if (result.isEmpty() && StoreFileInfo.isReference(fileName)) {
2407        result = getAllCacheKeysForFile(
2408          StoreFileInfo.getReferredToRegionAndFile(fileName.getName()).getSecond(), 0,
2409          Long.MAX_VALUE);
2410      }
2411      result.stream().forEach(entry -> {
2412        LOG.debug("found block for file {} in the backing map. Acquiring read lock for offset {}",
2413          fileName.getName(), entry.getOffset());
2414        ReentrantReadWriteLock lock = offsetLock.getLock(entry.getOffset());
2415        lock.readLock().lock();
2416        locks.add(lock);
2417        if (backingMap.containsKey(entry) && entry.getBlockType().isData()) {
2418          count.increment();
2419        }
2420      });
2421      // BucketCache would only have data blocks
2422      if (dataBlockCount == count.getValue()) {
2423        LOG.debug("File {} has now been fully cached.", fileName);
2424        fileCacheCompleted(fileName, size);
2425      } else {
2426        LOG.debug(
2427          "Prefetch executor completed for {}, but only {} data blocks were cached. "
2428            + "Total data blocks for file: {}. "
2429            + "Checking for blocks pending cache in cache writer queue.",
2430          fileName, count.getValue(), dataBlockCount);
2431        if (ramCache.hasBlocksForFile(fileName.getName())) {
2432          for (ReentrantReadWriteLock lock : locks) {
2433            lock.readLock().unlock();
2434          }
2435          locks.clear();
2436          LOG.debug("There are still blocks pending caching for file {}. Will sleep 100ms "
2437            + "and try the verification again.", fileName);
2438          Thread.sleep(100);
2439          notifyFileCachingCompleted(fileName, totalBlockCount, dataBlockCount, size);
2440        } else {
2441          LOG.info(
2442            "The total block count was {}. We found only {} data blocks cached from "
2443              + "a total of {} data blocks for file {}, "
2444              + "but no blocks pending caching. Maybe cache is full or evictions "
2445              + "happened concurrently to cache prefetch.",
2446            totalBlockCount, count, dataBlockCount, fileName);
2447        }
2448      }
2449    } catch (InterruptedException e) {
2450      throw new RuntimeException(e);
2451    } finally {
2452      for (ReentrantReadWriteLock lock : locks) {
2453        lock.readLock().unlock();
2454      }
2455    }
2456  }
2457
2458  @Override
2459  public Optional<Boolean> blockFitsIntoTheCache(HFileBlock block) {
2460    if (!isCacheInitialized("blockFitsIntoTheCache")) {
2461      return Optional.of(false);
2462    }
2463
2464    long currentUsed = bucketAllocator.getUsedSize();
2465    boolean result = (currentUsed + block.getOnDiskSizeWithHeader()) < acceptableSize();
2466    return Optional.of(result);
2467  }
2468
2469  @Override
2470  public Optional<Boolean> shouldCacheFile(HFileInfo hFileInfo, Configuration conf) {
2471    String fileName = hFileInfo.getHFileContext().getHFileName();
2472    DataTieringManager dataTieringManager = DataTieringManager.getInstance();
2473    if (dataTieringManager != null && !dataTieringManager.isHotData(hFileInfo, conf)) {
2474      LOG.debug("Data tiering is enabled for file: '{}' and it is not hot data", fileName);
2475      return Optional.of(false);
2476    }
2477    // if we don't have the file in fullyCachedFiles, we should cache it
2478    return Optional.of(!fullyCachedFiles.containsKey(fileName));
2479  }
2480
2481  @Override
2482  public Optional<Boolean> shouldCacheBlock(BlockCacheKey key, long maxTimestamp,
2483    Configuration conf) {
2484    DataTieringManager dataTieringManager = DataTieringManager.getInstance();
2485    if (dataTieringManager != null && !dataTieringManager.isHotData(maxTimestamp, conf)) {
2486      LOG.debug("Data tiering is enabled for file: '{}' and it is not hot data",
2487        key.getHfileName());
2488      return Optional.of(false);
2489    }
2490    return Optional.of(true);
2491  }
2492
2493  @Override
2494  public Optional<Boolean> isAlreadyCached(BlockCacheKey key) {
2495    boolean foundKey = backingMap.containsKey(key);
2496    // if there's no entry for the key itself, we need to check if this key is for a reference,
2497    // and if so, look for a block from the referenced file using this getBlockForReference method.
2498    return Optional.of(foundKey ? true : getBlockForReference(key) != null);
2499  }
2500
2501  @Override
2502  public Optional<Integer> getBlockSize(BlockCacheKey key) {
2503    BucketEntry entry = backingMap.get(key);
2504    if (entry == null) {
2505      // the key might be for a reference tha we had found the block from the referenced file in
2506      // the cache when we first tried to cache it.
2507      entry = getBlockForReference(key);
2508      return entry == null ? Optional.empty() : Optional.of(entry.getOnDiskSizeWithHeader());
2509    } else {
2510      return Optional.of(entry.getOnDiskSizeWithHeader());
2511    }
2512
2513  }
2514
2515  boolean isCacheInitialized(String api) {
2516    if (cacheState == CacheState.INITIALIZING) {
2517      LOG.warn("Bucket initialisation pending at {}", api);
2518      return false;
2519    }
2520    return true;
2521  }
2522
2523  @Override
2524  public boolean waitForCacheInitialization(long timeout) {
2525    try {
2526      while (cacheState == CacheState.INITIALIZING) {
2527        if (timeout <= 0) {
2528          break;
2529        }
2530        Thread.sleep(100);
2531        timeout -= 100;
2532      }
2533    } finally {
2534      return isCacheEnabled();
2535    }
2536  }
2537}