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