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 /** Returns heap occupancy percentage, 0 <= n <= 1. or -0.0 for error asking JVM */ 234 public float getHeapOccupancyPercent() { 235 return this.heapOccupancyPercent == Float.MAX_VALUE 236 ? HEAP_OCCUPANCY_ERROR_VALUE 237 : this.heapOccupancyPercent; 238 } 239 240 private class HeapMemoryTunerChore extends ScheduledChore implements FlushRequestListener { 241 private HeapMemoryTuner heapMemTuner; 242 private AtomicLong blockedFlushCount = new AtomicLong(); 243 private AtomicLong unblockedFlushCount = new AtomicLong(); 244 private long evictCount = 0L; 245 private long cacheMissCount = 0L; 246 private TunerContext tunerContext = new TunerContext(); 247 private boolean alarming = false; 248 249 public HeapMemoryTunerChore() { 250 super(server.getServerName() + "-HeapMemoryTunerChore", server, defaultChorePeriod); 251 Class<? extends HeapMemoryTuner> tunerKlass = server.getConfiguration().getClass( 252 HBASE_RS_HEAP_MEMORY_TUNER_CLASS, DefaultHeapMemoryTuner.class, HeapMemoryTuner.class); 253 heapMemTuner = ReflectionUtils.newInstance(tunerKlass, server.getConfiguration()); 254 tunerContext.setOffheapMemStore(regionServerAccounting.isOffheap()); 255 } 256 257 @Override 258 protected void chore() { 259 // Sample heap occupancy 260 final MemoryUsage usage = MemorySizeUtil.safeGetHeapMemoryUsage(); 261 if (usage != null) { 262 heapOccupancyPercent = (float) usage.getUsed() / (float) usage.getCommitted(); 263 } else { 264 // previously, an exception would have meant death for the tuning chore 265 // so switch to alarming so that we similarly stop tuning until we get 266 // heap usage information again. 267 heapOccupancyPercent = Float.MAX_VALUE; 268 } 269 // If we are above the heap occupancy alarm low watermark, switch to short 270 // sleeps for close monitoring. Stop autotuning, we are in a danger zone. 271 if (heapOccupancyPercent >= heapOccupancyLowWatermark) { 272 if (!alarming) { 273 LOG.warn("heapOccupancyPercent " + heapOccupancyPercent 274 + " is above heap occupancy alarm watermark (" + heapOccupancyLowWatermark + ")"); 275 alarming = true; 276 } 277 metricsHeapMemoryManager.increaseAboveHeapOccupancyLowWatermarkCounter(); 278 triggerNow(); 279 try { 280 // Need to sleep ourselves since we've told the chore's sleeper 281 // to skip the next sleep cycle. 282 Thread.sleep(1000); 283 } catch (InterruptedException e) { 284 // Interrupted, propagate 285 Thread.currentThread().interrupt(); 286 } 287 } else { 288 if (alarming) { 289 LOG.info("heapOccupancyPercent " + heapOccupancyPercent 290 + " is now below the heap occupancy alarm watermark (" + heapOccupancyLowWatermark 291 + ")"); 292 alarming = false; 293 } 294 } 295 // Autotune if tuning is enabled and allowed 296 if (tunerOn && !alarming) { 297 tune(); 298 } 299 } 300 301 private void tune() { 302 // TODO check if we can increase the memory boundaries 303 // while remaining in the limits 304 long curEvictCount; 305 long curCacheMisCount; 306 long blockedFlushCnt; 307 long unblockedFlushCnt; 308 curEvictCount = blockCache.getStats().getEvictedCount(); 309 tunerContext.setEvictCount(curEvictCount - evictCount); 310 evictCount = curEvictCount; 311 curCacheMisCount = blockCache.getStats().getMissCachingCount(); 312 tunerContext.setCacheMissCount(curCacheMisCount - cacheMissCount); 313 cacheMissCount = curCacheMisCount; 314 blockedFlushCnt = blockedFlushCount.getAndSet(0); 315 tunerContext.setBlockedFlushCount(blockedFlushCnt); 316 metricsHeapMemoryManager.updateBlockedFlushCount(blockedFlushCnt); 317 unblockedFlushCnt = unblockedFlushCount.getAndSet(0); 318 tunerContext.setUnblockedFlushCount(unblockedFlushCnt); 319 metricsHeapMemoryManager.updateUnblockedFlushCount(unblockedFlushCnt); 320 // TODO : add support for offheap metrics 321 tunerContext.setCurBlockCacheUsed((float) blockCache.getCurrentSize() / maxHeapSize); 322 metricsHeapMemoryManager.setCurBlockCacheSizeGauge(blockCache.getCurrentSize()); 323 long globalMemstoreHeapSize = regionServerAccounting.getGlobalMemStoreHeapSize(); 324 tunerContext.setCurMemStoreUsed((float) globalMemstoreHeapSize / maxHeapSize); 325 metricsHeapMemoryManager.setCurMemStoreSizeGauge(globalMemstoreHeapSize); 326 tunerContext.setCurBlockCacheSize(blockCachePercent); 327 tunerContext.setCurMemStoreSize(globalMemStorePercent); 328 TunerResult result = null; 329 try { 330 result = this.heapMemTuner.tune(tunerContext); 331 } catch (Throwable t) { 332 LOG.error("Exception thrown from the HeapMemoryTuner implementation", t); 333 } 334 if (result != null && result.needsTuning()) { 335 float memstoreSize = result.getMemStoreSize(); 336 float blockCacheSize = result.getBlockCacheSize(); 337 LOG.debug("From HeapMemoryTuner new memstoreSize: " + memstoreSize 338 + ". new blockCacheSize: " + blockCacheSize); 339 if (memstoreSize < globalMemStorePercentMinRange) { 340 LOG.info("New memstoreSize from HeapMemoryTuner " + memstoreSize + " is below min level " 341 + globalMemStorePercentMinRange + ". Resetting memstoreSize to min size"); 342 memstoreSize = globalMemStorePercentMinRange; 343 } else if (memstoreSize > globalMemStorePercentMaxRange) { 344 LOG.info("New memstoreSize from HeapMemoryTuner " + memstoreSize + " is above max level " 345 + globalMemStorePercentMaxRange + ". Resetting memstoreSize to max size"); 346 memstoreSize = globalMemStorePercentMaxRange; 347 } 348 if (blockCacheSize < blockCachePercentMinRange) { 349 LOG.info( 350 "New blockCacheSize from HeapMemoryTuner " + blockCacheSize + " is below min level " 351 + blockCachePercentMinRange + ". Resetting blockCacheSize to min size"); 352 blockCacheSize = blockCachePercentMinRange; 353 } else if (blockCacheSize > blockCachePercentMaxRange) { 354 LOG.info( 355 "New blockCacheSize from HeapMemoryTuner " + blockCacheSize + " is above max level " 356 + blockCachePercentMaxRange + ". Resetting blockCacheSize to min size"); 357 blockCacheSize = blockCachePercentMaxRange; 358 } 359 int gml = (int) (memstoreSize * CONVERT_TO_PERCENTAGE); 360 int bcul = (int) ((blockCacheSize) * CONVERT_TO_PERCENTAGE); 361 if (CONVERT_TO_PERCENTAGE - (gml + bcul) < CLUSTER_MINIMUM_MEMORY_THRESHOLD) { 362 LOG.info("Current heap configuration from HeapMemoryTuner exceeds " 363 + "the threshold required for successful cluster operation. " 364 + "The combined value cannot exceed 0.8. " + MemorySizeUtil.MEMSTORE_SIZE_KEY + " is " 365 + memstoreSize + " and " + HFILE_BLOCK_CACHE_SIZE_KEY + " is " + blockCacheSize); 366 // TODO can adjust the value so as not exceed 80%. Is that correct? may be. 367 } else { 368 int memStoreDeltaSize = 369 (int) ((memstoreSize - globalMemStorePercent) * CONVERT_TO_PERCENTAGE); 370 int blockCacheDeltaSize = 371 (int) ((blockCacheSize - blockCachePercent) * CONVERT_TO_PERCENTAGE); 372 metricsHeapMemoryManager.updateMemStoreDeltaSizeHistogram(memStoreDeltaSize); 373 metricsHeapMemoryManager.updateBlockCacheDeltaSizeHistogram(blockCacheDeltaSize); 374 long newBlockCacheSize = (long) (maxHeapSize * blockCacheSize); 375 // we could have got an increase or decrease in size for the offheap memstore 376 // also if the flush had happened due to heap overhead. In that case it is ok 377 // to adjust the onheap memstore limit configs 378 long newMemstoreSize = (long) (maxHeapSize * memstoreSize); 379 LOG.info("Setting block cache heap size to " + newBlockCacheSize 380 + " and memstore heap size to " + newMemstoreSize); 381 blockCachePercent = blockCacheSize; 382 blockCache.setMaxSize(newBlockCacheSize); 383 globalMemStorePercent = memstoreSize; 384 // Internally sets it to RegionServerAccounting 385 // TODO : Set directly on RSAccounting?? 386 memStoreFlusher.setGlobalMemStoreLimit(newMemstoreSize); 387 for (HeapMemoryTuneObserver observer : tuneObservers) { 388 // Risky.. If this newMemstoreSize decreases we reduce the count in offheap chunk pool 389 observer.onHeapMemoryTune(newMemstoreSize, newBlockCacheSize); 390 } 391 } 392 } else { 393 metricsHeapMemoryManager.increaseTunerDoNothingCounter(); 394 if (LOG.isDebugEnabled()) { 395 LOG.debug("No changes made by HeapMemoryTuner."); 396 } 397 } 398 } 399 400 @Override 401 public void flushRequested(FlushType type, Region region) { 402 switch (type) { 403 case ABOVE_ONHEAP_HIGHER_MARK: 404 blockedFlushCount.incrementAndGet(); 405 break; 406 case ABOVE_ONHEAP_LOWER_MARK: 407 unblockedFlushCount.incrementAndGet(); 408 break; 409 // Removed the counting of the offheap related flushes (after reviews). Will add later if 410 // needed 411 default: 412 // In case of any other flush don't do any action. 413 break; 414 } 415 } 416 } 417 418 /** 419 * POJO to pass all the relevant information required to do the heap memory tuning. It holds the 420 * flush counts and block cache evictions happened within the interval. Also holds the current 421 * heap percentage allocated for memstore and block cache. 422 */ 423 public static final class TunerContext { 424 private long blockedFlushCount; 425 private long unblockedFlushCount; 426 private long evictCount; 427 private long cacheMissCount; 428 private float curBlockCacheUsed; 429 private float curMemStoreUsed; 430 private float curMemStoreSize; 431 private float curBlockCacheSize; 432 private boolean offheapMemstore; 433 434 public long getBlockedFlushCount() { 435 return blockedFlushCount; 436 } 437 438 public void setBlockedFlushCount(long blockedFlushCount) { 439 this.blockedFlushCount = blockedFlushCount; 440 } 441 442 public long getUnblockedFlushCount() { 443 return unblockedFlushCount; 444 } 445 446 public void setUnblockedFlushCount(long unblockedFlushCount) { 447 this.unblockedFlushCount = unblockedFlushCount; 448 } 449 450 public long getEvictCount() { 451 return evictCount; 452 } 453 454 public void setEvictCount(long evictCount) { 455 this.evictCount = evictCount; 456 } 457 458 public float getCurMemStoreSize() { 459 return curMemStoreSize; 460 } 461 462 public void setCurMemStoreSize(float curMemStoreSize) { 463 this.curMemStoreSize = curMemStoreSize; 464 } 465 466 public float getCurBlockCacheSize() { 467 return curBlockCacheSize; 468 } 469 470 public void setCurBlockCacheSize(float curBlockCacheSize) { 471 this.curBlockCacheSize = curBlockCacheSize; 472 } 473 474 public long getCacheMissCount() { 475 return cacheMissCount; 476 } 477 478 public void setCacheMissCount(long cacheMissCount) { 479 this.cacheMissCount = cacheMissCount; 480 } 481 482 public float getCurBlockCacheUsed() { 483 return curBlockCacheUsed; 484 } 485 486 public void setCurBlockCacheUsed(float curBlockCacheUsed) { 487 this.curBlockCacheUsed = curBlockCacheUsed; 488 } 489 490 public float getCurMemStoreUsed() { 491 return curMemStoreUsed; 492 } 493 494 public void setCurMemStoreUsed(float d) { 495 this.curMemStoreUsed = d; 496 } 497 498 public void setOffheapMemStore(boolean offheapMemstore) { 499 this.offheapMemstore = offheapMemstore; 500 } 501 502 public boolean isOffheapMemStore() { 503 return this.offheapMemstore; 504 } 505 } 506 507 /** 508 * POJO which holds the result of memory tuning done by HeapMemoryTuner implementation. It 509 * includes the new heap percentage for memstore and block cache. 510 */ 511 public static final class TunerResult { 512 private float memstoreSize; 513 private float blockCacheSize; 514 private final boolean needsTuning; 515 516 public TunerResult(boolean needsTuning) { 517 this.needsTuning = needsTuning; 518 } 519 520 public float getMemStoreSize() { 521 return memstoreSize; 522 } 523 524 public void setMemStoreSize(float memstoreSize) { 525 this.memstoreSize = memstoreSize; 526 } 527 528 public float getBlockCacheSize() { 529 return blockCacheSize; 530 } 531 532 public void setBlockCacheSize(float blockCacheSize) { 533 this.blockCacheSize = blockCacheSize; 534 } 535 536 public boolean needsTuning() { 537 return needsTuning; 538 } 539 } 540 541 /** 542 * Every class that wants to observe heap memory tune actions must implement this interface. 543 */ 544 public static interface HeapMemoryTuneObserver { 545 546 /** 547 * This method would be called by HeapMemoryManger when a heap memory tune action took place. 548 * @param newMemstoreSize The newly calculated global memstore size 549 * @param newBlockCacheSize The newly calculated global blockcache size 550 */ 551 void onHeapMemoryTune(long newMemstoreSize, long newBlockCacheSize); 552 } 553}