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 private static final int CLUSTER_MINIMUM_MEMORY_THRESHOLD = 051 (int) (CONVERT_TO_PERCENTAGE * HConstants.HBASE_CLUSTER_MINIMUM_MEMORY_THRESHOLD); 052 053 public static final String BLOCK_CACHE_SIZE_MAX_RANGE_KEY = "hfile.block.cache.size.max.range"; 054 public static final String BLOCK_CACHE_SIZE_MIN_RANGE_KEY = "hfile.block.cache.size.min.range"; 055 public static final String MEMSTORE_SIZE_MAX_RANGE_KEY = 056 "hbase.regionserver.global.memstore.size.max.range"; 057 public static final String MEMSTORE_SIZE_MIN_RANGE_KEY = 058 "hbase.regionserver.global.memstore.size.min.range"; 059 public static final String HBASE_RS_HEAP_MEMORY_TUNER_PERIOD = 060 "hbase.regionserver.heapmemory.tuner.period"; 061 public static final int HBASE_RS_HEAP_MEMORY_TUNER_DEFAULT_PERIOD = 60 * 1000; 062 public static final String HBASE_RS_HEAP_MEMORY_TUNER_CLASS = 063 "hbase.regionserver.heapmemory.tuner.class"; 064 065 public static final float HEAP_OCCUPANCY_ERROR_VALUE = -0.0f; 066 067 private float globalMemStorePercent; 068 private float globalMemStorePercentMinRange; 069 private float globalMemStorePercentMaxRange; 070 071 private float blockCachePercent; 072 private float blockCachePercentMinRange; 073 private float blockCachePercentMaxRange; 074 075 private float heapOccupancyPercent; 076 077 private final ResizableBlockCache blockCache; 078 // TODO : remove this and mark regionServerAccounting as the observer directly 079 private final FlushRequester memStoreFlusher; 080 private final Server server; 081 private final RegionServerAccounting regionServerAccounting; 082 083 private HeapMemoryTunerChore heapMemTunerChore = null; 084 private final boolean tunerOn; 085 private final int defaultChorePeriod; 086 private final float heapOccupancyLowWatermark; 087 088 private final long maxHeapSize; 089 { 090 // note that this initialization still isn't threadsafe, because updating a long isn't atomic. 091 long tempMaxHeap = -1L; 092 try { 093 final MemoryUsage usage = MemorySizeUtil.safeGetHeapMemoryUsage(); 094 if (usage != null) { 095 tempMaxHeap = usage.getMax(); 096 } 097 } finally { 098 maxHeapSize = tempMaxHeap; 099 } 100 } 101 102 private MetricsHeapMemoryManager metricsHeapMemoryManager; 103 104 private List<HeapMemoryTuneObserver> tuneObservers = new ArrayList<>(); 105 106 HeapMemoryManager(BlockCache blockCache, FlushRequester memStoreFlusher, Server server, 107 RegionServerAccounting regionServerAccounting) { 108 Configuration conf = server.getConfiguration(); 109 this.blockCache = toResizableBlockCache(blockCache); 110 this.memStoreFlusher = memStoreFlusher; 111 this.server = server; 112 this.regionServerAccounting = regionServerAccounting; 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.checkForClusterFreeHeapMemoryLimit(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 int gml = (int) (globalMemStorePercentMaxRange * CONVERT_TO_PERCENTAGE); 188 int bcul = (int) ((blockCachePercentMinRange) * CONVERT_TO_PERCENTAGE); 189 if (CONVERT_TO_PERCENTAGE - (gml + bcul) < CLUSTER_MINIMUM_MEMORY_THRESHOLD) { 190 throw new RuntimeException("Current heap configuration for MemStore and BlockCache exceeds " 191 + "the threshold required for successful cluster operation. " 192 + "The combined value cannot exceed 0.8. Please check the settings for " 193 + MEMSTORE_SIZE_MAX_RANGE_KEY + " and " + BLOCK_CACHE_SIZE_MIN_RANGE_KEY 194 + " in your configuration. " + MEMSTORE_SIZE_MAX_RANGE_KEY + " is " 195 + globalMemStorePercentMaxRange + " and " + BLOCK_CACHE_SIZE_MIN_RANGE_KEY + " is " 196 + blockCachePercentMinRange); 197 } 198 gml = (int) (globalMemStorePercentMinRange * CONVERT_TO_PERCENTAGE); 199 bcul = (int) ((blockCachePercentMaxRange) * CONVERT_TO_PERCENTAGE); 200 if (CONVERT_TO_PERCENTAGE - (gml + bcul) < CLUSTER_MINIMUM_MEMORY_THRESHOLD) { 201 throw new RuntimeException("Current heap configuration for MemStore and BlockCache exceeds " 202 + "the threshold required for successful cluster operation. " 203 + "The combined value cannot exceed 0.8. Please check the settings for " 204 + MEMSTORE_SIZE_MIN_RANGE_KEY + " and " + BLOCK_CACHE_SIZE_MAX_RANGE_KEY 205 + " in your configuration. " + MEMSTORE_SIZE_MIN_RANGE_KEY + " is " 206 + globalMemStorePercentMinRange + " and " + BLOCK_CACHE_SIZE_MAX_RANGE_KEY + " is " 207 + blockCachePercentMaxRange); 208 } 209 return tuningEnabled; 210 } 211 212 public void start(ChoreService service) { 213 LOG.info("Starting, tuneOn={}", this.tunerOn); 214 this.heapMemTunerChore = new HeapMemoryTunerChore(); 215 service.scheduleChore(heapMemTunerChore); 216 if (tunerOn) { 217 // Register HeapMemoryTuner as a memstore flush listener 218 memStoreFlusher.registerFlushRequestListener(heapMemTunerChore); 219 } 220 } 221 222 public void stop() { 223 // The thread is Daemon. Just interrupting the ongoing process. 224 LOG.info("Stopping"); 225 this.heapMemTunerChore.shutdown(true); 226 } 227 228 public void registerTuneObserver(HeapMemoryTuneObserver observer) { 229 this.tuneObservers.add(observer); 230 } 231 232 // Used by the test cases. 233 boolean isTunerOn() { 234 return this.tunerOn; 235 } 236 237 /** Returns heap occupancy percentage, 0 <= n <= 1. or -0.0 for error asking JVM */ 238 public float getHeapOccupancyPercent() { 239 return this.heapOccupancyPercent == Float.MAX_VALUE 240 ? HEAP_OCCUPANCY_ERROR_VALUE 241 : this.heapOccupancyPercent; 242 } 243 244 private class HeapMemoryTunerChore extends ScheduledChore implements FlushRequestListener { 245 private HeapMemoryTuner heapMemTuner; 246 private AtomicLong blockedFlushCount = new AtomicLong(); 247 private AtomicLong unblockedFlushCount = new AtomicLong(); 248 private long evictCount = 0L; 249 private long cacheMissCount = 0L; 250 private TunerContext tunerContext = new TunerContext(); 251 private boolean alarming = false; 252 253 public HeapMemoryTunerChore() { 254 super(server.getServerName() + "-HeapMemoryTunerChore", server, defaultChorePeriod); 255 Class<? extends HeapMemoryTuner> tunerKlass = server.getConfiguration().getClass( 256 HBASE_RS_HEAP_MEMORY_TUNER_CLASS, DefaultHeapMemoryTuner.class, HeapMemoryTuner.class); 257 heapMemTuner = ReflectionUtils.newInstance(tunerKlass, server.getConfiguration()); 258 tunerContext.setOffheapMemStore(regionServerAccounting.isOffheap()); 259 } 260 261 @Override 262 protected void chore() { 263 // Sample heap occupancy 264 final MemoryUsage usage = MemorySizeUtil.safeGetHeapMemoryUsage(); 265 if (usage != null) { 266 heapOccupancyPercent = (float) usage.getUsed() / (float) usage.getCommitted(); 267 } else { 268 // previously, an exception would have meant death for the tuning chore 269 // so switch to alarming so that we similarly stop tuning until we get 270 // heap usage information again. 271 heapOccupancyPercent = Float.MAX_VALUE; 272 } 273 // If we are above the heap occupancy alarm low watermark, switch to short 274 // sleeps for close monitoring. Stop autotuning, we are in a danger zone. 275 if (heapOccupancyPercent >= heapOccupancyLowWatermark) { 276 if (!alarming) { 277 LOG.warn("heapOccupancyPercent " + heapOccupancyPercent 278 + " is above heap occupancy alarm watermark (" + heapOccupancyLowWatermark + ")"); 279 alarming = true; 280 } 281 metricsHeapMemoryManager.increaseAboveHeapOccupancyLowWatermarkCounter(); 282 triggerNow(); 283 try { 284 // Need to sleep ourselves since we've told the chore's sleeper 285 // to skip the next sleep cycle. 286 Thread.sleep(1000); 287 } catch (InterruptedException e) { 288 // Interrupted, propagate 289 Thread.currentThread().interrupt(); 290 } 291 } else { 292 if (alarming) { 293 LOG.info("heapOccupancyPercent " + heapOccupancyPercent 294 + " is now below the heap occupancy alarm watermark (" + heapOccupancyLowWatermark 295 + ")"); 296 alarming = false; 297 } 298 } 299 // Autotune if tuning is enabled and allowed 300 if (tunerOn && !alarming) { 301 tune(); 302 } 303 } 304 305 private void tune() { 306 // TODO check if we can increase the memory boundaries 307 // while remaining in the limits 308 long curEvictCount; 309 long curCacheMisCount; 310 long blockedFlushCnt; 311 long unblockedFlushCnt; 312 curEvictCount = blockCache.getStats().getEvictedCount(); 313 tunerContext.setEvictCount(curEvictCount - evictCount); 314 evictCount = curEvictCount; 315 curCacheMisCount = blockCache.getStats().getMissCachingCount(); 316 tunerContext.setCacheMissCount(curCacheMisCount - cacheMissCount); 317 cacheMissCount = curCacheMisCount; 318 blockedFlushCnt = blockedFlushCount.getAndSet(0); 319 tunerContext.setBlockedFlushCount(blockedFlushCnt); 320 metricsHeapMemoryManager.updateBlockedFlushCount(blockedFlushCnt); 321 unblockedFlushCnt = unblockedFlushCount.getAndSet(0); 322 tunerContext.setUnblockedFlushCount(unblockedFlushCnt); 323 metricsHeapMemoryManager.updateUnblockedFlushCount(unblockedFlushCnt); 324 tunerContext.setCurBlockCacheUsed((float) blockCache.getCurrentSize() / maxHeapSize); 325 metricsHeapMemoryManager.setCurBlockCacheSizeGauge(blockCache.getCurrentSize()); 326 long globalMemstoreDataSize = regionServerAccounting.getGlobalMemStoreDataSize(); 327 long globalMemstoreHeapSize = regionServerAccounting.getGlobalMemStoreHeapSize(); 328 long globalMemStoreOffHeapSize = regionServerAccounting.getGlobalMemStoreOffHeapSize(); 329 tunerContext.setCurMemStoreUsed((float) globalMemstoreHeapSize / maxHeapSize); 330 metricsHeapMemoryManager.setCurMemStoreSizeGauge(globalMemstoreDataSize); 331 metricsHeapMemoryManager.setCurMemStoreOnHeapSizeGauge(globalMemstoreHeapSize); 332 metricsHeapMemoryManager.setCurMemStoreOffHeapSizeGauge(globalMemStoreOffHeapSize); 333 tunerContext.setCurBlockCacheSize(blockCachePercent); 334 tunerContext.setCurMemStoreSize(globalMemStorePercent); 335 TunerResult result = null; 336 try { 337 result = this.heapMemTuner.tune(tunerContext); 338 } catch (Throwable t) { 339 LOG.error("Exception thrown from the HeapMemoryTuner implementation", t); 340 } 341 if (result != null && result.needsTuning()) { 342 float memstoreSize = result.getMemStoreSize(); 343 float blockCacheSize = result.getBlockCacheSize(); 344 LOG.debug("From HeapMemoryTuner new memstoreSize: " + memstoreSize 345 + ". new blockCacheSize: " + blockCacheSize); 346 if (memstoreSize < globalMemStorePercentMinRange) { 347 LOG.info("New memstoreSize from HeapMemoryTuner " + memstoreSize + " is below min level " 348 + globalMemStorePercentMinRange + ". Resetting memstoreSize to min size"); 349 memstoreSize = globalMemStorePercentMinRange; 350 } else if (memstoreSize > globalMemStorePercentMaxRange) { 351 LOG.info("New memstoreSize from HeapMemoryTuner " + memstoreSize + " is above max level " 352 + globalMemStorePercentMaxRange + ". Resetting memstoreSize to max size"); 353 memstoreSize = globalMemStorePercentMaxRange; 354 } 355 if (blockCacheSize < blockCachePercentMinRange) { 356 LOG.info( 357 "New blockCacheSize from HeapMemoryTuner " + blockCacheSize + " is below min level " 358 + blockCachePercentMinRange + ". Resetting blockCacheSize to min size"); 359 blockCacheSize = blockCachePercentMinRange; 360 } else if (blockCacheSize > blockCachePercentMaxRange) { 361 LOG.info( 362 "New blockCacheSize from HeapMemoryTuner " + blockCacheSize + " is above max level " 363 + blockCachePercentMaxRange + ". Resetting blockCacheSize to min size"); 364 blockCacheSize = blockCachePercentMaxRange; 365 } 366 int gml = (int) (memstoreSize * CONVERT_TO_PERCENTAGE); 367 int bcul = (int) ((blockCacheSize) * CONVERT_TO_PERCENTAGE); 368 if (CONVERT_TO_PERCENTAGE - (gml + bcul) < CLUSTER_MINIMUM_MEMORY_THRESHOLD) { 369 LOG.info("Current heap configuration from HeapMemoryTuner exceeds " 370 + "the threshold required for successful cluster operation. " 371 + "The combined value cannot exceed 0.8. " + MemorySizeUtil.MEMSTORE_SIZE_KEY + " is " 372 + memstoreSize + " and " + HFILE_BLOCK_CACHE_SIZE_KEY + " is " + blockCacheSize); 373 // TODO can adjust the value so as not exceed 80%. Is that correct? may be. 374 } else { 375 int memStoreDeltaSize = 376 (int) ((memstoreSize - globalMemStorePercent) * CONVERT_TO_PERCENTAGE); 377 int blockCacheDeltaSize = 378 (int) ((blockCacheSize - blockCachePercent) * CONVERT_TO_PERCENTAGE); 379 metricsHeapMemoryManager.updateMemStoreDeltaSizeHistogram(memStoreDeltaSize); 380 metricsHeapMemoryManager.updateBlockCacheDeltaSizeHistogram(blockCacheDeltaSize); 381 long newBlockCacheSize = (long) (maxHeapSize * blockCacheSize); 382 // we could have got an increase or decrease in size for the offheap memstore 383 // also if the flush had happened due to heap overhead. In that case it is ok 384 // to adjust the onheap memstore limit configs 385 long newMemstoreSize = (long) (maxHeapSize * memstoreSize); 386 LOG.info("Setting block cache heap size to " + newBlockCacheSize 387 + " and memstore heap size to " + newMemstoreSize); 388 blockCachePercent = blockCacheSize; 389 blockCache.setMaxSize(newBlockCacheSize); 390 globalMemStorePercent = memstoreSize; 391 // Internally sets it to RegionServerAccounting 392 // TODO : Set directly on RSAccounting?? 393 memStoreFlusher.setGlobalMemStoreLimit(newMemstoreSize); 394 for (HeapMemoryTuneObserver observer : tuneObservers) { 395 // Risky.. If this newMemstoreSize decreases we reduce the count in offheap chunk pool 396 observer.onHeapMemoryTune(newMemstoreSize, newBlockCacheSize); 397 } 398 } 399 } else { 400 metricsHeapMemoryManager.increaseTunerDoNothingCounter(); 401 if (LOG.isDebugEnabled()) { 402 LOG.debug("No changes made by HeapMemoryTuner."); 403 } 404 } 405 } 406 407 @Override 408 public void flushRequested(FlushType type, Region region) { 409 switch (type) { 410 case ABOVE_ONHEAP_HIGHER_MARK: 411 blockedFlushCount.incrementAndGet(); 412 break; 413 case ABOVE_ONHEAP_LOWER_MARK: 414 unblockedFlushCount.incrementAndGet(); 415 break; 416 // Removed the counting of the offheap related flushes (after reviews). Will add later if 417 // needed 418 default: 419 // In case of any other flush don't do any action. 420 break; 421 } 422 } 423 } 424 425 /** 426 * POJO to pass all the relevant information required to do the heap memory tuning. It holds the 427 * flush counts and block cache evictions happened within the interval. Also holds the current 428 * heap percentage allocated for memstore and block cache. 429 */ 430 public static final class TunerContext { 431 private long blockedFlushCount; 432 private long unblockedFlushCount; 433 private long evictCount; 434 private long cacheMissCount; 435 private float curBlockCacheUsed; 436 private float curMemStoreUsed; 437 private float curMemStoreSize; 438 private float curBlockCacheSize; 439 private boolean offheapMemstore; 440 441 public long getBlockedFlushCount() { 442 return blockedFlushCount; 443 } 444 445 public void setBlockedFlushCount(long blockedFlushCount) { 446 this.blockedFlushCount = blockedFlushCount; 447 } 448 449 public long getUnblockedFlushCount() { 450 return unblockedFlushCount; 451 } 452 453 public void setUnblockedFlushCount(long unblockedFlushCount) { 454 this.unblockedFlushCount = unblockedFlushCount; 455 } 456 457 public long getEvictCount() { 458 return evictCount; 459 } 460 461 public void setEvictCount(long evictCount) { 462 this.evictCount = evictCount; 463 } 464 465 public float getCurMemStoreSize() { 466 return curMemStoreSize; 467 } 468 469 public void setCurMemStoreSize(float curMemStoreSize) { 470 this.curMemStoreSize = curMemStoreSize; 471 } 472 473 public float getCurBlockCacheSize() { 474 return curBlockCacheSize; 475 } 476 477 public void setCurBlockCacheSize(float curBlockCacheSize) { 478 this.curBlockCacheSize = curBlockCacheSize; 479 } 480 481 public long getCacheMissCount() { 482 return cacheMissCount; 483 } 484 485 public void setCacheMissCount(long cacheMissCount) { 486 this.cacheMissCount = cacheMissCount; 487 } 488 489 public float getCurBlockCacheUsed() { 490 return curBlockCacheUsed; 491 } 492 493 public void setCurBlockCacheUsed(float curBlockCacheUsed) { 494 this.curBlockCacheUsed = curBlockCacheUsed; 495 } 496 497 public float getCurMemStoreUsed() { 498 return curMemStoreUsed; 499 } 500 501 public void setCurMemStoreUsed(float d) { 502 this.curMemStoreUsed = d; 503 } 504 505 public void setOffheapMemStore(boolean offheapMemstore) { 506 this.offheapMemstore = offheapMemstore; 507 } 508 509 public boolean isOffheapMemStore() { 510 return this.offheapMemstore; 511 } 512 } 513 514 /** 515 * POJO which holds the result of memory tuning done by HeapMemoryTuner implementation. It 516 * includes the new heap percentage for memstore and block cache. 517 */ 518 public static final class TunerResult { 519 private float memstoreSize; 520 private float blockCacheSize; 521 private final boolean needsTuning; 522 523 public TunerResult(boolean needsTuning) { 524 this.needsTuning = needsTuning; 525 } 526 527 public float getMemStoreSize() { 528 return memstoreSize; 529 } 530 531 public void setMemStoreSize(float memstoreSize) { 532 this.memstoreSize = memstoreSize; 533 } 534 535 public float getBlockCacheSize() { 536 return blockCacheSize; 537 } 538 539 public void setBlockCacheSize(float blockCacheSize) { 540 this.blockCacheSize = blockCacheSize; 541 } 542 543 public boolean needsTuning() { 544 return needsTuning; 545 } 546 } 547 548 /** 549 * Every class that wants to observe heap memory tune actions must implement this interface. 550 */ 551 public static interface HeapMemoryTuneObserver { 552 553 /** 554 * This method would be called by HeapMemoryManger when a heap memory tune action took place. 555 * @param newMemstoreSize The newly calculated global memstore size 556 * @param newBlockCacheSize The newly calculated global blockcache size 557 */ 558 void onHeapMemoryTune(long newMemstoreSize, long newBlockCacheSize); 559 } 560}