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.regionserver; 019 020import static org.apache.hadoop.hbase.HConstants.HFILE_BLOCK_CACHE_MEMORY_SIZE_KEY; 021import static org.apache.hadoop.hbase.HConstants.HFILE_BLOCK_CACHE_SIZE_KEY; 022 023import java.lang.management.MemoryUsage; 024import java.util.ArrayList; 025import java.util.List; 026import java.util.concurrent.atomic.AtomicLong; 027import org.apache.hadoop.conf.Configuration; 028import org.apache.hadoop.hbase.ChoreService; 029import org.apache.hadoop.hbase.HConstants; 030import org.apache.hadoop.hbase.ScheduledChore; 031import org.apache.hadoop.hbase.Server; 032import org.apache.hadoop.hbase.io.hfile.BlockCache; 033import org.apache.hadoop.hbase.io.hfile.CombinedBlockCache; 034import org.apache.hadoop.hbase.io.hfile.ResizableBlockCache; 035import org.apache.hadoop.hbase.io.util.MemorySizeUtil; 036import org.apache.hadoop.util.ReflectionUtils; 037import org.apache.yetus.audience.InterfaceAudience; 038import org.slf4j.Logger; 039import org.slf4j.LoggerFactory; 040 041/** 042 * Manages tuning of Heap memory using <code>HeapMemoryTuner</code>. Most part of the heap memory is 043 * split between Memstores and BlockCache. This manager helps in tuning sizes of both these 044 * dynamically, as per the R/W load on the servers. 045 */ 046@InterfaceAudience.Private 047public class HeapMemoryManager { 048 private static final Logger LOG = LoggerFactory.getLogger(HeapMemoryManager.class); 049 private static final int CONVERT_TO_PERCENTAGE = 100; 050 051 public static final String BLOCK_CACHE_SIZE_MAX_RANGE_KEY = "hfile.block.cache.size.max.range"; 052 public static final String BLOCK_CACHE_SIZE_MIN_RANGE_KEY = "hfile.block.cache.size.min.range"; 053 public static final String MEMSTORE_SIZE_MAX_RANGE_KEY = 054 "hbase.regionserver.global.memstore.size.max.range"; 055 public static final String MEMSTORE_SIZE_MIN_RANGE_KEY = 056 "hbase.regionserver.global.memstore.size.min.range"; 057 public static final String HBASE_RS_HEAP_MEMORY_TUNER_PERIOD = 058 "hbase.regionserver.heapmemory.tuner.period"; 059 public static final int HBASE_RS_HEAP_MEMORY_TUNER_DEFAULT_PERIOD = 60 * 1000; 060 public static final String HBASE_RS_HEAP_MEMORY_TUNER_CLASS = 061 "hbase.regionserver.heapmemory.tuner.class"; 062 063 public static final float HEAP_OCCUPANCY_ERROR_VALUE = -0.0f; 064 065 private float globalMemStorePercent; 066 private float globalMemStorePercentMinRange; 067 private float globalMemStorePercentMaxRange; 068 069 private float blockCachePercent; 070 private float blockCachePercentMinRange; 071 private float blockCachePercentMaxRange; 072 073 private float heapOccupancyPercent; 074 075 private final ResizableBlockCache blockCache; 076 // TODO : remove this and mark regionServerAccounting as the observer directly 077 private final FlushRequester memStoreFlusher; 078 private final Server server; 079 private final RegionServerAccounting regionServerAccounting; 080 081 private HeapMemoryTunerChore heapMemTunerChore = null; 082 private final boolean tunerOn; 083 private final int defaultChorePeriod; 084 private final float heapOccupancyLowWatermark; 085 private final float minFreeHeapFraction; 086 087 private final long maxHeapSize; 088 { 089 // note that this initialization still isn't threadsafe, because updating a long isn't atomic. 090 long tempMaxHeap = -1L; 091 try { 092 final MemoryUsage usage = MemorySizeUtil.safeGetHeapMemoryUsage(); 093 if (usage != null) { 094 tempMaxHeap = usage.getMax(); 095 } 096 } finally { 097 maxHeapSize = tempMaxHeap; 098 } 099 } 100 101 private MetricsHeapMemoryManager metricsHeapMemoryManager; 102 103 private List<HeapMemoryTuneObserver> tuneObservers = new ArrayList<>(); 104 105 HeapMemoryManager(BlockCache blockCache, FlushRequester memStoreFlusher, Server server, 106 RegionServerAccounting regionServerAccounting) { 107 Configuration conf = server.getConfiguration(); 108 this.blockCache = toResizableBlockCache(blockCache); 109 this.memStoreFlusher = memStoreFlusher; 110 this.server = server; 111 this.regionServerAccounting = regionServerAccounting; 112 this.minFreeHeapFraction = MemorySizeUtil.getRegionServerMinFreeHeapFraction(conf); 113 this.tunerOn = doInit(conf); 114 this.defaultChorePeriod = 115 conf.getInt(HBASE_RS_HEAP_MEMORY_TUNER_PERIOD, HBASE_RS_HEAP_MEMORY_TUNER_DEFAULT_PERIOD); 116 this.heapOccupancyLowWatermark = conf.getFloat(HConstants.HEAP_OCCUPANCY_LOW_WATERMARK_KEY, 117 HConstants.DEFAULT_HEAP_OCCUPANCY_LOW_WATERMARK); 118 metricsHeapMemoryManager = new MetricsHeapMemoryManager(); 119 } 120 121 private ResizableBlockCache toResizableBlockCache(BlockCache blockCache) { 122 if (blockCache instanceof CombinedBlockCache) { 123 return (ResizableBlockCache) ((CombinedBlockCache) blockCache).getFirstLevelCache(); 124 } else { 125 return (ResizableBlockCache) blockCache; 126 } 127 } 128 129 private boolean doInit(Configuration conf) { 130 boolean tuningEnabled = true; 131 globalMemStorePercent = MemorySizeUtil.getGlobalMemStoreHeapPercent(conf, false); 132 blockCachePercent = MemorySizeUtil.getBlockCacheHeapPercent(conf); 133 MemorySizeUtil.validateRegionServerHeapMemoryAllocation(conf); 134 // Initialize max and min range for memstore heap space 135 globalMemStorePercentMinRange = 136 conf.getFloat(MEMSTORE_SIZE_MIN_RANGE_KEY, globalMemStorePercent); 137 globalMemStorePercentMaxRange = 138 conf.getFloat(MEMSTORE_SIZE_MAX_RANGE_KEY, globalMemStorePercent); 139 if (globalMemStorePercent < globalMemStorePercentMinRange) { 140 LOG.warn("Setting " + MEMSTORE_SIZE_MIN_RANGE_KEY + " to " + globalMemStorePercent 141 + ", same value as " + MemorySizeUtil.MEMSTORE_SIZE_KEY 142 + " because supplied value greater than initial memstore size value."); 143 globalMemStorePercentMinRange = globalMemStorePercent; 144 conf.setFloat(MEMSTORE_SIZE_MIN_RANGE_KEY, globalMemStorePercentMinRange); 145 } 146 if (globalMemStorePercent > globalMemStorePercentMaxRange) { 147 LOG.warn("Setting " + MEMSTORE_SIZE_MAX_RANGE_KEY + " to " + globalMemStorePercent 148 + ", same value as " + MemorySizeUtil.MEMSTORE_SIZE_KEY 149 + " because supplied value less than initial memstore size value."); 150 globalMemStorePercentMaxRange = globalMemStorePercent; 151 conf.setFloat(MEMSTORE_SIZE_MAX_RANGE_KEY, globalMemStorePercentMaxRange); 152 } 153 if ( 154 globalMemStorePercent == globalMemStorePercentMinRange 155 && globalMemStorePercent == globalMemStorePercentMaxRange 156 ) { 157 tuningEnabled = false; 158 } 159 // Initialize max and min range for block cache 160 blockCachePercentMinRange = conf.getFloat(BLOCK_CACHE_SIZE_MIN_RANGE_KEY, blockCachePercent); 161 blockCachePercentMaxRange = conf.getFloat(BLOCK_CACHE_SIZE_MAX_RANGE_KEY, blockCachePercent); 162 if (blockCachePercent < blockCachePercentMinRange) { 163 LOG.warn( 164 "Setting {} to {} (lookup order: {} -> {}), " 165 + "because supplied value greater than initial block cache size.", 166 BLOCK_CACHE_SIZE_MIN_RANGE_KEY, blockCachePercent, HFILE_BLOCK_CACHE_MEMORY_SIZE_KEY, 167 HFILE_BLOCK_CACHE_SIZE_KEY); 168 blockCachePercentMinRange = blockCachePercent; 169 conf.setFloat(BLOCK_CACHE_SIZE_MIN_RANGE_KEY, blockCachePercentMinRange); 170 } 171 if (blockCachePercent > blockCachePercentMaxRange) { 172 LOG.warn( 173 "Setting {} to {} (lookup order: {} -> {}), " 174 + "because supplied value less than initial block cache size.", 175 BLOCK_CACHE_SIZE_MAX_RANGE_KEY, blockCachePercent, HFILE_BLOCK_CACHE_MEMORY_SIZE_KEY, 176 HFILE_BLOCK_CACHE_SIZE_KEY); 177 blockCachePercentMaxRange = blockCachePercent; 178 conf.setFloat(BLOCK_CACHE_SIZE_MAX_RANGE_KEY, blockCachePercentMaxRange); 179 } 180 if ( 181 tuningEnabled && blockCachePercent == blockCachePercentMinRange 182 && blockCachePercent == blockCachePercentMaxRange 183 ) { 184 tuningEnabled = false; 185 } 186 187 checkHeapMemoryLimits(MEMSTORE_SIZE_MAX_RANGE_KEY, globalMemStorePercentMaxRange, 188 BLOCK_CACHE_SIZE_MIN_RANGE_KEY, blockCachePercentMinRange); 189 checkHeapMemoryLimits(MEMSTORE_SIZE_MIN_RANGE_KEY, globalMemStorePercentMinRange, 190 BLOCK_CACHE_SIZE_MAX_RANGE_KEY, blockCachePercentMaxRange); 191 192 return tuningEnabled; 193 } 194 195 public void start(ChoreService service) { 196 LOG.info("Starting, tuneOn={}", this.tunerOn); 197 this.heapMemTunerChore = new HeapMemoryTunerChore(); 198 service.scheduleChore(heapMemTunerChore); 199 if (tunerOn) { 200 // Register HeapMemoryTuner as a memstore flush listener 201 memStoreFlusher.registerFlushRequestListener(heapMemTunerChore); 202 } 203 } 204 205 public void stop() { 206 // The thread is Daemon. Just interrupting the ongoing process. 207 LOG.info("Stopping"); 208 this.heapMemTunerChore.shutdown(true); 209 } 210 211 public void registerTuneObserver(HeapMemoryTuneObserver observer) { 212 this.tuneObservers.add(observer); 213 } 214 215 // Used by the test cases. 216 boolean isTunerOn() { 217 return this.tunerOn; 218 } 219 220 /** Returns heap occupancy percentage, 0 <= n <= 1. or -0.0 for error asking JVM */ 221 public float getHeapOccupancyPercent() { 222 return this.heapOccupancyPercent == Float.MAX_VALUE 223 ? HEAP_OCCUPANCY_ERROR_VALUE 224 : this.heapOccupancyPercent; 225 } 226 227 private boolean isHeapMemoryUsageExceedingLimit(float memStoreFraction, 228 float blockCacheFraction) { 229 // Use integer percentage to avoid subtle float precision issues and ensure consistent 230 // comparison. This also maintains backward compatibility with previous logic relying on int 231 // truncation. 232 int memStorePercent = (int) (memStoreFraction * CONVERT_TO_PERCENTAGE); 233 int blockCachePercent = (int) (blockCacheFraction * CONVERT_TO_PERCENTAGE); 234 int minFreeHeapPercent = (int) (this.minFreeHeapFraction * CONVERT_TO_PERCENTAGE); 235 236 return memStorePercent + blockCachePercent + minFreeHeapPercent > CONVERT_TO_PERCENTAGE; 237 } 238 239 private void checkHeapMemoryLimits(String memStoreConfKey, float memStoreFraction, 240 String blockCacheConfKey, float blockCacheFraction) { 241 if (isHeapMemoryUsageExceedingLimit(memStoreFraction, blockCacheFraction)) { 242 throw new RuntimeException(String.format( 243 "Current heap configuration for MemStore and BlockCache exceeds the allowed heap usage. " 244 + "At least %.2f of the heap must remain free to ensure stable RegionServer operation. " 245 + "Please check the settings for %s and %s in your configuration. " 246 + "%s is %.2f and %s is %.2f", 247 minFreeHeapFraction, memStoreConfKey, blockCacheConfKey, memStoreConfKey, memStoreFraction, 248 blockCacheConfKey, blockCacheFraction)); 249 } 250 } 251 252 private class HeapMemoryTunerChore extends ScheduledChore implements FlushRequestListener { 253 private HeapMemoryTuner heapMemTuner; 254 private AtomicLong blockedFlushCount = new AtomicLong(); 255 private AtomicLong unblockedFlushCount = new AtomicLong(); 256 private long evictCount = 0L; 257 private long cacheMissCount = 0L; 258 private TunerContext tunerContext = new TunerContext(); 259 private boolean alarming = false; 260 261 public HeapMemoryTunerChore() { 262 super(server.getServerName() + "-HeapMemoryTunerChore", server, defaultChorePeriod); 263 Class<? extends HeapMemoryTuner> tunerKlass = server.getConfiguration().getClass( 264 HBASE_RS_HEAP_MEMORY_TUNER_CLASS, DefaultHeapMemoryTuner.class, HeapMemoryTuner.class); 265 heapMemTuner = ReflectionUtils.newInstance(tunerKlass, server.getConfiguration()); 266 tunerContext.setOffheapMemStore(regionServerAccounting.isOffheap()); 267 } 268 269 @Override 270 protected void chore() { 271 // Sample heap occupancy 272 final MemoryUsage usage = MemorySizeUtil.safeGetHeapMemoryUsage(); 273 if (usage != null) { 274 heapOccupancyPercent = (float) usage.getUsed() / (float) usage.getCommitted(); 275 } else { 276 // previously, an exception would have meant death for the tuning chore 277 // so switch to alarming so that we similarly stop tuning until we get 278 // heap usage information again. 279 heapOccupancyPercent = Float.MAX_VALUE; 280 } 281 // If we are above the heap occupancy alarm low watermark, switch to short 282 // sleeps for close monitoring. Stop autotuning, we are in a danger zone. 283 if (heapOccupancyPercent >= heapOccupancyLowWatermark) { 284 if (!alarming) { 285 LOG.warn("heapOccupancyPercent " + heapOccupancyPercent 286 + " is above heap occupancy alarm watermark (" + heapOccupancyLowWatermark + ")"); 287 alarming = true; 288 } 289 metricsHeapMemoryManager.increaseAboveHeapOccupancyLowWatermarkCounter(); 290 triggerNow(); 291 try { 292 // Need to sleep ourselves since we've told the chore's sleeper 293 // to skip the next sleep cycle. 294 Thread.sleep(1000); 295 } catch (InterruptedException e) { 296 // Interrupted, propagate 297 Thread.currentThread().interrupt(); 298 } 299 } else { 300 if (alarming) { 301 LOG.info("heapOccupancyPercent " + heapOccupancyPercent 302 + " is now below the heap occupancy alarm watermark (" + heapOccupancyLowWatermark 303 + ")"); 304 alarming = false; 305 } 306 } 307 // Autotune if tuning is enabled and allowed 308 if (tunerOn && !alarming) { 309 tune(); 310 } 311 } 312 313 private void tune() { 314 // TODO check if we can increase the memory boundaries 315 // while remaining in the limits 316 long curEvictCount; 317 long curCacheMisCount; 318 long blockedFlushCnt; 319 long unblockedFlushCnt; 320 curEvictCount = blockCache.getStats().getEvictedCount(); 321 tunerContext.setEvictCount(curEvictCount - evictCount); 322 evictCount = curEvictCount; 323 curCacheMisCount = blockCache.getStats().getMissCachingCount(); 324 tunerContext.setCacheMissCount(curCacheMisCount - cacheMissCount); 325 cacheMissCount = curCacheMisCount; 326 blockedFlushCnt = blockedFlushCount.getAndSet(0); 327 tunerContext.setBlockedFlushCount(blockedFlushCnt); 328 metricsHeapMemoryManager.updateBlockedFlushCount(blockedFlushCnt); 329 unblockedFlushCnt = unblockedFlushCount.getAndSet(0); 330 tunerContext.setUnblockedFlushCount(unblockedFlushCnt); 331 metricsHeapMemoryManager.updateUnblockedFlushCount(unblockedFlushCnt); 332 tunerContext.setCurBlockCacheUsed((float) blockCache.getCurrentSize() / maxHeapSize); 333 metricsHeapMemoryManager.setCurBlockCacheSizeGauge(blockCache.getCurrentSize()); 334 long globalMemstoreDataSize = regionServerAccounting.getGlobalMemStoreDataSize(); 335 long globalMemstoreHeapSize = regionServerAccounting.getGlobalMemStoreHeapSize(); 336 long globalMemStoreOffHeapSize = regionServerAccounting.getGlobalMemStoreOffHeapSize(); 337 tunerContext.setCurMemStoreUsed((float) globalMemstoreHeapSize / maxHeapSize); 338 metricsHeapMemoryManager.setCurMemStoreSizeGauge(globalMemstoreDataSize); 339 metricsHeapMemoryManager.setCurMemStoreOnHeapSizeGauge(globalMemstoreHeapSize); 340 metricsHeapMemoryManager.setCurMemStoreOffHeapSizeGauge(globalMemStoreOffHeapSize); 341 tunerContext.setCurBlockCacheSize(blockCachePercent); 342 tunerContext.setCurMemStoreSize(globalMemStorePercent); 343 TunerResult result = null; 344 try { 345 result = this.heapMemTuner.tune(tunerContext); 346 } catch (Throwable t) { 347 LOG.error("Exception thrown from the HeapMemoryTuner implementation", t); 348 } 349 if (result != null && result.needsTuning()) { 350 float memstoreSize = result.getMemStoreSize(); 351 float blockCacheSize = result.getBlockCacheSize(); 352 LOG.debug("From HeapMemoryTuner new memstoreSize: " + memstoreSize 353 + ". new blockCacheSize: " + blockCacheSize); 354 if (memstoreSize < globalMemStorePercentMinRange) { 355 LOG.info("New memstoreSize from HeapMemoryTuner " + memstoreSize + " is below min level " 356 + globalMemStorePercentMinRange + ". Resetting memstoreSize to min size"); 357 memstoreSize = globalMemStorePercentMinRange; 358 } else if (memstoreSize > globalMemStorePercentMaxRange) { 359 LOG.info("New memstoreSize from HeapMemoryTuner " + memstoreSize + " is above max level " 360 + globalMemStorePercentMaxRange + ". Resetting memstoreSize to max size"); 361 memstoreSize = globalMemStorePercentMaxRange; 362 } 363 if (blockCacheSize < blockCachePercentMinRange) { 364 LOG.info( 365 "New blockCacheSize from HeapMemoryTuner " + blockCacheSize + " is below min level " 366 + blockCachePercentMinRange + ". Resetting blockCacheSize to min size"); 367 blockCacheSize = blockCachePercentMinRange; 368 } else if (blockCacheSize > blockCachePercentMaxRange) { 369 LOG.info( 370 "New blockCacheSize from HeapMemoryTuner " + blockCacheSize + " is above max level " 371 + blockCachePercentMaxRange + ". Resetting blockCacheSize to min size"); 372 blockCacheSize = blockCachePercentMaxRange; 373 } 374 375 if (isHeapMemoryUsageExceedingLimit(memstoreSize, blockCacheSize)) { 376 LOG.info("Current heap configuration from HeapMemoryTuner exceeds " 377 + "the allowed heap usage. At least " + minFreeHeapFraction 378 + " of the heap must remain free to ensure stable RegionServer operation. " 379 + MemorySizeUtil.MEMSTORE_SIZE_KEY + " is " + memstoreSize + " and " 380 + HFILE_BLOCK_CACHE_SIZE_KEY + " is " + blockCacheSize); 381 // NOTE: In the future, we might adjust values to not exceed limits, 382 // but for now tuning is skipped if over threshold. 383 } else { 384 int memStoreDeltaSize = 385 (int) ((memstoreSize - globalMemStorePercent) * CONVERT_TO_PERCENTAGE); 386 int blockCacheDeltaSize = 387 (int) ((blockCacheSize - blockCachePercent) * CONVERT_TO_PERCENTAGE); 388 metricsHeapMemoryManager.updateMemStoreDeltaSizeHistogram(memStoreDeltaSize); 389 metricsHeapMemoryManager.updateBlockCacheDeltaSizeHistogram(blockCacheDeltaSize); 390 long newBlockCacheSize = (long) (maxHeapSize * blockCacheSize); 391 // we could have got an increase or decrease in size for the offheap memstore 392 // also if the flush had happened due to heap overhead. In that case it is ok 393 // to adjust the onheap memstore limit configs 394 long newMemstoreSize = (long) (maxHeapSize * memstoreSize); 395 LOG.info("Setting block cache heap size to " + newBlockCacheSize 396 + " and memstore heap size to " + newMemstoreSize); 397 blockCachePercent = blockCacheSize; 398 blockCache.setMaxSize(newBlockCacheSize); 399 globalMemStorePercent = memstoreSize; 400 // Internally sets it to RegionServerAccounting 401 // TODO : Set directly on RSAccounting?? 402 memStoreFlusher.setGlobalMemStoreLimit(newMemstoreSize); 403 for (HeapMemoryTuneObserver observer : tuneObservers) { 404 // Risky.. If this newMemstoreSize decreases we reduce the count in offheap chunk pool 405 observer.onHeapMemoryTune(newMemstoreSize, newBlockCacheSize); 406 } 407 } 408 } else { 409 metricsHeapMemoryManager.increaseTunerDoNothingCounter(); 410 if (LOG.isDebugEnabled()) { 411 LOG.debug("No changes made by HeapMemoryTuner."); 412 } 413 } 414 } 415 416 @Override 417 public void flushRequested(FlushType type, Region region) { 418 switch (type) { 419 case ABOVE_ONHEAP_HIGHER_MARK: 420 blockedFlushCount.incrementAndGet(); 421 break; 422 case ABOVE_ONHEAP_LOWER_MARK: 423 unblockedFlushCount.incrementAndGet(); 424 break; 425 // Removed the counting of the offheap related flushes (after reviews). Will add later if 426 // needed 427 default: 428 // In case of any other flush don't do any action. 429 break; 430 } 431 } 432 } 433 434 /** 435 * POJO to pass all the relevant information required to do the heap memory tuning. It holds the 436 * flush counts and block cache evictions happened within the interval. Also holds the current 437 * heap percentage allocated for memstore and block cache. 438 */ 439 public static final class TunerContext { 440 private long blockedFlushCount; 441 private long unblockedFlushCount; 442 private long evictCount; 443 private long cacheMissCount; 444 private float curBlockCacheUsed; 445 private float curMemStoreUsed; 446 private float curMemStoreSize; 447 private float curBlockCacheSize; 448 private boolean offheapMemstore; 449 450 public long getBlockedFlushCount() { 451 return blockedFlushCount; 452 } 453 454 public void setBlockedFlushCount(long blockedFlushCount) { 455 this.blockedFlushCount = blockedFlushCount; 456 } 457 458 public long getUnblockedFlushCount() { 459 return unblockedFlushCount; 460 } 461 462 public void setUnblockedFlushCount(long unblockedFlushCount) { 463 this.unblockedFlushCount = unblockedFlushCount; 464 } 465 466 public long getEvictCount() { 467 return evictCount; 468 } 469 470 public void setEvictCount(long evictCount) { 471 this.evictCount = evictCount; 472 } 473 474 public float getCurMemStoreSize() { 475 return curMemStoreSize; 476 } 477 478 public void setCurMemStoreSize(float curMemStoreSize) { 479 this.curMemStoreSize = curMemStoreSize; 480 } 481 482 public float getCurBlockCacheSize() { 483 return curBlockCacheSize; 484 } 485 486 public void setCurBlockCacheSize(float curBlockCacheSize) { 487 this.curBlockCacheSize = curBlockCacheSize; 488 } 489 490 public long getCacheMissCount() { 491 return cacheMissCount; 492 } 493 494 public void setCacheMissCount(long cacheMissCount) { 495 this.cacheMissCount = cacheMissCount; 496 } 497 498 public float getCurBlockCacheUsed() { 499 return curBlockCacheUsed; 500 } 501 502 public void setCurBlockCacheUsed(float curBlockCacheUsed) { 503 this.curBlockCacheUsed = curBlockCacheUsed; 504 } 505 506 public float getCurMemStoreUsed() { 507 return curMemStoreUsed; 508 } 509 510 public void setCurMemStoreUsed(float d) { 511 this.curMemStoreUsed = d; 512 } 513 514 public void setOffheapMemStore(boolean offheapMemstore) { 515 this.offheapMemstore = offheapMemstore; 516 } 517 518 public boolean isOffheapMemStore() { 519 return this.offheapMemstore; 520 } 521 } 522 523 /** 524 * POJO which holds the result of memory tuning done by HeapMemoryTuner implementation. It 525 * includes the new heap percentage for memstore and block cache. 526 */ 527 public static final class TunerResult { 528 private float memstoreSize; 529 private float blockCacheSize; 530 private final boolean needsTuning; 531 532 public TunerResult(boolean needsTuning) { 533 this.needsTuning = needsTuning; 534 } 535 536 public float getMemStoreSize() { 537 return memstoreSize; 538 } 539 540 public void setMemStoreSize(float memstoreSize) { 541 this.memstoreSize = memstoreSize; 542 } 543 544 public float getBlockCacheSize() { 545 return blockCacheSize; 546 } 547 548 public void setBlockCacheSize(float blockCacheSize) { 549 this.blockCacheSize = blockCacheSize; 550 } 551 552 public boolean needsTuning() { 553 return needsTuning; 554 } 555 } 556 557 /** 558 * Every class that wants to observe heap memory tune actions must implement this interface. 559 */ 560 public static interface HeapMemoryTuneObserver { 561 562 /** 563 * This method would be called by HeapMemoryManger when a heap memory tune action took place. 564 * @param newMemstoreSize The newly calculated global memstore size 565 * @param newBlockCacheSize The newly calculated global blockcache size 566 */ 567 void onHeapMemoryTune(long newMemstoreSize, long newBlockCacheSize); 568 } 569}