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