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 &lt;= n &lt;= 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}