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