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