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