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