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