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