001/** 002 * 003 * Licensed to the Apache Software Foundation (ASF) under one 004 * or more contributor license agreements. See the NOTICE file 005 * distributed with this work for additional information 006 * regarding copyright ownership. The ASF licenses this file 007 * to you under the Apache License, Version 2.0 (the 008 * "License"); you may not use this file except in compliance 009 * with the License. You may obtain a copy of the License at 010 * 011 * http://www.apache.org/licenses/LICENSE-2.0 012 * 013 * Unless required by applicable law or agreed to in writing, software 014 * distributed under the License is distributed on an "AS IS" BASIS, 015 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 016 * See the License for the specific language governing permissions and 017 * limitations under the License. 018 */ 019package org.apache.hadoop.hbase.regionserver; 020 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; 027 028import org.apache.hadoop.conf.Configuration; 029import org.apache.hadoop.hbase.ChoreService; 030import org.apache.hadoop.hbase.HConstants; 031import org.apache.hadoop.hbase.ScheduledChore; 032import org.apache.hadoop.hbase.Server; 033import org.apache.hadoop.hbase.io.hfile.BlockCache; 034import org.apache.hadoop.hbase.io.hfile.CombinedBlockCache; 035import org.apache.hadoop.hbase.io.hfile.ResizableBlockCache; 036import org.apache.hadoop.hbase.io.util.MemorySizeUtil; 037import org.apache.hadoop.util.ReflectionUtils; 038import org.apache.yetus.audience.InterfaceAudience; 039import org.slf4j.Logger; 040import org.slf4j.LoggerFactory; 041 042import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; 043 044/** 045 * Manages tuning of Heap memory using <code>HeapMemoryTuner</code>. Most part of the heap memory is 046 * split between Memstores and BlockCache. This manager helps in tuning sizes of both these 047 * dynamically, as per the R/W load on the servers. 048 */ 049@InterfaceAudience.Private 050public class HeapMemoryManager { 051 private static final Logger LOG = LoggerFactory.getLogger(HeapMemoryManager.class); 052 private static final int CONVERT_TO_PERCENTAGE = 100; 053 private static final int CLUSTER_MINIMUM_MEMORY_THRESHOLD = 054 (int) (CONVERT_TO_PERCENTAGE * HConstants.HBASE_CLUSTER_MINIMUM_MEMORY_THRESHOLD); 055 056 public static final String BLOCK_CACHE_SIZE_MAX_RANGE_KEY = "hfile.block.cache.size.max.range"; 057 public static final String BLOCK_CACHE_SIZE_MIN_RANGE_KEY = "hfile.block.cache.size.min.range"; 058 public static final String MEMSTORE_SIZE_MAX_RANGE_KEY = 059 "hbase.regionserver.global.memstore.size.max.range"; 060 public static final String MEMSTORE_SIZE_MIN_RANGE_KEY = 061 "hbase.regionserver.global.memstore.size.min.range"; 062 public static final String HBASE_RS_HEAP_MEMORY_TUNER_PERIOD = 063 "hbase.regionserver.heapmemory.tuner.period"; 064 public static final int HBASE_RS_HEAP_MEMORY_TUNER_DEFAULT_PERIOD = 60 * 1000; 065 public static final String HBASE_RS_HEAP_MEMORY_TUNER_CLASS = 066 "hbase.regionserver.heapmemory.tuner.class"; 067 068 public static final float HEAP_OCCUPANCY_ERROR_VALUE = -0.0f; 069 070 private float globalMemStorePercent; 071 private float globalMemStorePercentMinRange; 072 private float globalMemStorePercentMaxRange; 073 074 private float blockCachePercent; 075 private float blockCachePercentMinRange; 076 private float blockCachePercentMaxRange; 077 078 private float heapOccupancyPercent; 079 080 private final ResizableBlockCache blockCache; 081 // TODO : remove this and mark regionServerAccounting as the observer directly 082 private final FlushRequester memStoreFlusher; 083 private final Server server; 084 private final RegionServerAccounting regionServerAccounting; 085 086 private HeapMemoryTunerChore heapMemTunerChore = null; 087 private final boolean tunerOn; 088 private final int defaultChorePeriod; 089 private final float heapOccupancyLowWatermark; 090 091 private final long maxHeapSize; 092 { 093 // note that this initialization still isn't threadsafe, because updating a long isn't atomic. 094 long tempMaxHeap = -1L; 095 try { 096 final MemoryUsage usage = MemorySizeUtil.safeGetHeapMemoryUsage(); 097 if (usage != null) { 098 tempMaxHeap = usage.getMax(); 099 } 100 } finally { 101 maxHeapSize = tempMaxHeap; 102 } 103 } 104 105 private MetricsHeapMemoryManager metricsHeapMemoryManager; 106 107 private List<HeapMemoryTuneObserver> tuneObservers = new ArrayList<>(); 108 109 @VisibleForTesting 110 HeapMemoryManager(BlockCache blockCache, FlushRequester memStoreFlusher, 111 Server server, RegionServerAccounting regionServerAccounting) { 112 Configuration conf = server.getConfiguration(); 113 this.blockCache = toResizableBlockCache(blockCache); 114 this.memStoreFlusher = memStoreFlusher; 115 this.server = server; 116 this.regionServerAccounting = regionServerAccounting; 117 this.tunerOn = doInit(conf); 118 this.defaultChorePeriod = conf.getInt(HBASE_RS_HEAP_MEMORY_TUNER_PERIOD, 119 HBASE_RS_HEAP_MEMORY_TUNER_DEFAULT_PERIOD); 120 this.heapOccupancyLowWatermark = conf.getFloat(HConstants.HEAP_OCCUPANCY_LOW_WATERMARK_KEY, 121 HConstants.DEFAULT_HEAP_OCCUPANCY_LOW_WATERMARK); 122 metricsHeapMemoryManager = new MetricsHeapMemoryManager(); 123 } 124 125 private ResizableBlockCache toResizableBlockCache(BlockCache blockCache) { 126 if (blockCache instanceof CombinedBlockCache) { 127 return (ResizableBlockCache) ((CombinedBlockCache) blockCache).getOnHeapCache(); 128 } else { 129 return (ResizableBlockCache) blockCache; 130 } 131 } 132 133 private boolean doInit(Configuration conf) { 134 boolean tuningEnabled = true; 135 globalMemStorePercent = MemorySizeUtil.getGlobalMemStoreHeapPercent(conf, false); 136 blockCachePercent = conf.getFloat(HFILE_BLOCK_CACHE_SIZE_KEY, 137 HConstants.HFILE_BLOCK_CACHE_SIZE_DEFAULT); 138 MemorySizeUtil.checkForClusterFreeHeapMemoryLimit(conf); 139 // Initialize max and min range for memstore heap space 140 globalMemStorePercentMinRange = conf.getFloat(MEMSTORE_SIZE_MIN_RANGE_KEY, 141 globalMemStorePercent); 142 globalMemStorePercentMaxRange = conf.getFloat(MEMSTORE_SIZE_MAX_RANGE_KEY, 143 globalMemStorePercent); 144 if (globalMemStorePercent < globalMemStorePercentMinRange) { 145 LOG.warn("Setting " + MEMSTORE_SIZE_MIN_RANGE_KEY + " to " + globalMemStorePercent 146 + ", same value as " + MemorySizeUtil.MEMSTORE_SIZE_KEY 147 + " because supplied value greater than initial memstore size value."); 148 globalMemStorePercentMinRange = globalMemStorePercent; 149 conf.setFloat(MEMSTORE_SIZE_MIN_RANGE_KEY, globalMemStorePercentMinRange); 150 } 151 if (globalMemStorePercent > globalMemStorePercentMaxRange) { 152 LOG.warn("Setting " + MEMSTORE_SIZE_MAX_RANGE_KEY + " to " + globalMemStorePercent 153 + ", same value as " + MemorySizeUtil.MEMSTORE_SIZE_KEY 154 + " because supplied value less than initial memstore size value."); 155 globalMemStorePercentMaxRange = globalMemStorePercent; 156 conf.setFloat(MEMSTORE_SIZE_MAX_RANGE_KEY, globalMemStorePercentMaxRange); 157 } 158 if (globalMemStorePercent == globalMemStorePercentMinRange 159 && globalMemStorePercent == globalMemStorePercentMaxRange) { 160 tuningEnabled = false; 161 } 162 // Initialize max and min range for block cache 163 blockCachePercentMinRange = conf.getFloat(BLOCK_CACHE_SIZE_MIN_RANGE_KEY, blockCachePercent); 164 blockCachePercentMaxRange = conf.getFloat(BLOCK_CACHE_SIZE_MAX_RANGE_KEY, blockCachePercent); 165 if (blockCachePercent < blockCachePercentMinRange) { 166 LOG.warn("Setting " + BLOCK_CACHE_SIZE_MIN_RANGE_KEY + " to " + blockCachePercent 167 + ", same value as " + HFILE_BLOCK_CACHE_SIZE_KEY 168 + " because supplied value greater than initial block cache size."); 169 blockCachePercentMinRange = blockCachePercent; 170 conf.setFloat(BLOCK_CACHE_SIZE_MIN_RANGE_KEY, blockCachePercentMinRange); 171 } 172 if (blockCachePercent > blockCachePercentMaxRange) { 173 LOG.warn("Setting " + BLOCK_CACHE_SIZE_MAX_RANGE_KEY + " to " + blockCachePercent 174 + ", same value as " + HFILE_BLOCK_CACHE_SIZE_KEY 175 + " because supplied value less than initial block cache size."); 176 blockCachePercentMaxRange = blockCachePercent; 177 conf.setFloat(BLOCK_CACHE_SIZE_MAX_RANGE_KEY, blockCachePercentMaxRange); 178 } 179 if (tuningEnabled && blockCachePercent == blockCachePercentMinRange 180 && blockCachePercent == blockCachePercentMaxRange) { 181 tuningEnabled = false; 182 } 183 184 int gml = (int) (globalMemStorePercentMaxRange * CONVERT_TO_PERCENTAGE); 185 int bcul = (int) ((blockCachePercentMinRange) * CONVERT_TO_PERCENTAGE); 186 if (CONVERT_TO_PERCENTAGE - (gml + bcul) < CLUSTER_MINIMUM_MEMORY_THRESHOLD) { 187 throw new RuntimeException("Current heap configuration for MemStore and BlockCache exceeds " 188 + "the threshold required for successful cluster operation. " 189 + "The combined value cannot exceed 0.8. Please check the settings for " 190 + MEMSTORE_SIZE_MAX_RANGE_KEY + " and " + BLOCK_CACHE_SIZE_MIN_RANGE_KEY 191 + " in your configuration. " + MEMSTORE_SIZE_MAX_RANGE_KEY + " is " 192 + globalMemStorePercentMaxRange + " and " + BLOCK_CACHE_SIZE_MIN_RANGE_KEY + " is " 193 + blockCachePercentMinRange); 194 } 195 gml = (int) (globalMemStorePercentMinRange * CONVERT_TO_PERCENTAGE); 196 bcul = (int) ((blockCachePercentMaxRange) * CONVERT_TO_PERCENTAGE); 197 if (CONVERT_TO_PERCENTAGE - (gml + bcul) < CLUSTER_MINIMUM_MEMORY_THRESHOLD) { 198 throw new RuntimeException("Current heap configuration for MemStore and BlockCache exceeds " 199 + "the threshold required for successful cluster operation. " 200 + "The combined value cannot exceed 0.8. Please check the settings for " 201 + MEMSTORE_SIZE_MIN_RANGE_KEY + " and " + BLOCK_CACHE_SIZE_MAX_RANGE_KEY 202 + " in your configuration. " + MEMSTORE_SIZE_MIN_RANGE_KEY + " is " 203 + globalMemStorePercentMinRange + " and " + BLOCK_CACHE_SIZE_MAX_RANGE_KEY + " is " 204 + blockCachePercentMaxRange); 205 } 206 return tuningEnabled; 207 } 208 209 public void start(ChoreService service) { 210 LOG.info("Starting, tuneOn={}", this.tunerOn); 211 this.heapMemTunerChore = new HeapMemoryTunerChore(); 212 service.scheduleChore(heapMemTunerChore); 213 if (tunerOn) { 214 // Register HeapMemoryTuner as a memstore flush listener 215 memStoreFlusher.registerFlushRequestListener(heapMemTunerChore); 216 } 217 } 218 219 public void stop() { 220 // The thread is Daemon. Just interrupting the ongoing process. 221 LOG.info("Stopping"); 222 this.heapMemTunerChore.cancel(true); 223 } 224 225 public void registerTuneObserver(HeapMemoryTuneObserver observer) { 226 this.tuneObservers.add(observer); 227 } 228 229 // Used by the test cases. 230 boolean isTunerOn() { 231 return this.tunerOn; 232 } 233 234 /** 235 * @return heap occupancy percentage, 0 <= n <= 1. or -0.0 for error asking JVM 236 */ 237 public float getHeapOccupancyPercent() { 238 return this.heapOccupancyPercent == Float.MAX_VALUE ? HEAP_OCCUPANCY_ERROR_VALUE : this.heapOccupancyPercent; 239 } 240 241 private class HeapMemoryTunerChore extends ScheduledChore implements FlushRequestListener { 242 private HeapMemoryTuner heapMemTuner; 243 private AtomicLong blockedFlushCount = new AtomicLong(); 244 private AtomicLong unblockedFlushCount = new AtomicLong(); 245 private long evictCount = 0L; 246 private long cacheMissCount = 0L; 247 private TunerContext tunerContext = new TunerContext(); 248 private boolean alarming = false; 249 250 public HeapMemoryTunerChore() { 251 super(server.getServerName() + "-HeapMemoryTunerChore", server, defaultChorePeriod); 252 Class<? extends HeapMemoryTuner> tunerKlass = server.getConfiguration().getClass( 253 HBASE_RS_HEAP_MEMORY_TUNER_CLASS, DefaultHeapMemoryTuner.class, HeapMemoryTuner.class); 254 heapMemTuner = ReflectionUtils.newInstance(tunerKlass, server.getConfiguration()); 255 tunerContext 256 .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 (" + 293 heapOccupancyLowWatermark + ")"); 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("New blockCacheSize from HeapMemoryTuner " + blockCacheSize 352 + " is below min level " + blockCachePercentMinRange 353 + ". Resetting blockCacheSize to min size"); 354 blockCacheSize = blockCachePercentMinRange; 355 } else if (blockCacheSize > blockCachePercentMaxRange) { 356 LOG.info("New blockCacheSize from HeapMemoryTuner " + blockCacheSize 357 + " is above max level " + blockCachePercentMaxRange 358 + ". 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 367 + " is " + memstoreSize + " and " + HFILE_BLOCK_CACHE_SIZE_KEY + " is " 368 + blockCacheSize); 369 // TODO can adjust the value so as not exceed 80%. Is that correct? may be. 370 } else { 371 int memStoreDeltaSize = 372 (int) ((memstoreSize - globalMemStorePercent) * CONVERT_TO_PERCENTAGE); 373 int blockCacheDeltaSize = 374 (int) ((blockCacheSize - blockCachePercent) * CONVERT_TO_PERCENTAGE); 375 metricsHeapMemoryManager.updateMemStoreDeltaSizeHistogram(memStoreDeltaSize); 376 metricsHeapMemoryManager.updateBlockCacheDeltaSizeHistogram(blockCacheDeltaSize); 377 long newBlockCacheSize = (long) (maxHeapSize * blockCacheSize); 378 // we could have got an increase or decrease in size for the offheap memstore 379 // also if the flush had happened due to heap overhead. In that case it is ok 380 // to adjust the onheap memstore limit configs 381 long newMemstoreSize = (long) (maxHeapSize * memstoreSize); 382 LOG.info("Setting block cache heap size to " + newBlockCacheSize 383 + " and memstore heap size to " + newMemstoreSize); 384 blockCachePercent = blockCacheSize; 385 blockCache.setMaxSize(newBlockCacheSize); 386 globalMemStorePercent = memstoreSize; 387 // Internally sets it to RegionServerAccounting 388 // TODO : Set directly on RSAccounting?? 389 memStoreFlusher.setGlobalMemStoreLimit(newMemstoreSize); 390 for (HeapMemoryTuneObserver observer : tuneObservers) { 391 // Risky.. If this newMemstoreSize decreases we reduce the count in offheap chunk pool 392 observer.onHeapMemoryTune(newMemstoreSize, newBlockCacheSize); 393 } 394 } 395 } else { 396 metricsHeapMemoryManager.increaseTunerDoNothingCounter(); 397 if (LOG.isDebugEnabled()) { 398 LOG.debug("No changes made by HeapMemoryTuner."); 399 } 400 } 401 } 402 403 @Override 404 public void flushRequested(FlushType type, Region region) { 405 switch (type) { 406 case ABOVE_ONHEAP_HIGHER_MARK: 407 blockedFlushCount.incrementAndGet(); 408 break; 409 case ABOVE_ONHEAP_LOWER_MARK: 410 unblockedFlushCount.incrementAndGet(); 411 break; 412 // Removed the counting of the offheap related flushes (after reviews). Will add later if 413 // needed 414 default: 415 // In case of any other flush don't do any action. 416 break; 417 } 418 } 419 } 420 421 /** 422 * POJO to pass all the relevant information required to do the heap memory tuning. It holds the 423 * flush counts and block cache evictions happened within the interval. Also holds the current 424 * heap percentage allocated for memstore and block cache. 425 */ 426 public static final class TunerContext { 427 private long blockedFlushCount; 428 private long unblockedFlushCount; 429 private long evictCount; 430 private long cacheMissCount; 431 private float curBlockCacheUsed; 432 private float curMemStoreUsed; 433 private float curMemStoreSize; 434 private float curBlockCacheSize; 435 private boolean offheapMemstore; 436 437 public long getBlockedFlushCount() { 438 return blockedFlushCount; 439 } 440 441 public void setBlockedFlushCount(long blockedFlushCount) { 442 this.blockedFlushCount = blockedFlushCount; 443 } 444 445 public long getUnblockedFlushCount() { 446 return unblockedFlushCount; 447 } 448 449 public void setUnblockedFlushCount(long unblockedFlushCount) { 450 this.unblockedFlushCount = unblockedFlushCount; 451 } 452 453 public long getEvictCount() { 454 return evictCount; 455 } 456 457 public void setEvictCount(long evictCount) { 458 this.evictCount = evictCount; 459 } 460 461 public float getCurMemStoreSize() { 462 return curMemStoreSize; 463 } 464 465 public void setCurMemStoreSize(float curMemStoreSize) { 466 this.curMemStoreSize = curMemStoreSize; 467 } 468 469 public float getCurBlockCacheSize() { 470 return curBlockCacheSize; 471 } 472 473 public void setCurBlockCacheSize(float curBlockCacheSize) { 474 this.curBlockCacheSize = curBlockCacheSize; 475 } 476 477 public long getCacheMissCount() { 478 return cacheMissCount; 479 } 480 481 public void setCacheMissCount(long cacheMissCount) { 482 this.cacheMissCount = cacheMissCount; 483 } 484 485 public float getCurBlockCacheUsed() { 486 return curBlockCacheUsed; 487 } 488 489 public void setCurBlockCacheUsed(float curBlockCacheUsed) { 490 this.curBlockCacheUsed = curBlockCacheUsed; 491 } 492 493 public float getCurMemStoreUsed() { 494 return curMemStoreUsed; 495 } 496 497 public void setCurMemStoreUsed(float d) { 498 this.curMemStoreUsed = d; 499 } 500 501 public void setOffheapMemStore(boolean offheapMemstore) { 502 this.offheapMemstore = offheapMemstore; 503 } 504 505 public boolean isOffheapMemStore() { 506 return this.offheapMemstore; 507 } 508 } 509 510 /** 511 * POJO which holds the result of memory tuning done by HeapMemoryTuner implementation. 512 * It includes the new heap percentage for memstore and block cache. 513 */ 514 public static final class TunerResult { 515 private float memstoreSize; 516 private float blockCacheSize; 517 private final boolean needsTuning; 518 519 public TunerResult(boolean needsTuning) { 520 this.needsTuning = needsTuning; 521 } 522 523 public float getMemStoreSize() { 524 return memstoreSize; 525 } 526 527 public void setMemStoreSize(float memstoreSize) { 528 this.memstoreSize = memstoreSize; 529 } 530 531 public float getBlockCacheSize() { 532 return blockCacheSize; 533 } 534 535 public void setBlockCacheSize(float blockCacheSize) { 536 this.blockCacheSize = blockCacheSize; 537 } 538 539 public boolean needsTuning() { 540 return needsTuning; 541 } 542 } 543 544 /** 545 * Every class that wants to observe heap memory tune actions must implement this interface. 546 */ 547 public static interface HeapMemoryTuneObserver { 548 549 /** 550 * This method would be called by HeapMemoryManger when a heap memory tune action took place. 551 * @param newMemstoreSize The newly calculated global memstore size 552 * @param newBlockCacheSize The newly calculated global blockcache size 553 */ 554 void onHeapMemoryTune(long newMemstoreSize, long newBlockCacheSize); 555 } 556}