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
042/**
043 * Manages tuning of Heap memory using <code>HeapMemoryTuner</code>. Most part of the heap memory is
044 * split between Memstores and BlockCache. This manager helps in tuning sizes of both these
045 * dynamically, as per the R/W load on the servers.
046 */
047@InterfaceAudience.Private
048public class HeapMemoryManager {
049  private static final Logger LOG = LoggerFactory.getLogger(HeapMemoryManager.class);
050  private static final int CONVERT_TO_PERCENTAGE = 100;
051  private static final int CLUSTER_MINIMUM_MEMORY_THRESHOLD =
052    (int) (CONVERT_TO_PERCENTAGE * HConstants.HBASE_CLUSTER_MINIMUM_MEMORY_THRESHOLD);
053
054  public static final String BLOCK_CACHE_SIZE_MAX_RANGE_KEY = "hfile.block.cache.size.max.range";
055  public static final String BLOCK_CACHE_SIZE_MIN_RANGE_KEY = "hfile.block.cache.size.min.range";
056  public static final String MEMSTORE_SIZE_MAX_RANGE_KEY =
057      "hbase.regionserver.global.memstore.size.max.range";
058  public static final String MEMSTORE_SIZE_MIN_RANGE_KEY =
059      "hbase.regionserver.global.memstore.size.min.range";
060  public static final String HBASE_RS_HEAP_MEMORY_TUNER_PERIOD =
061      "hbase.regionserver.heapmemory.tuner.period";
062  public static final int HBASE_RS_HEAP_MEMORY_TUNER_DEFAULT_PERIOD = 60 * 1000;
063  public static final String HBASE_RS_HEAP_MEMORY_TUNER_CLASS =
064      "hbase.regionserver.heapmemory.tuner.class";
065
066  public static final float HEAP_OCCUPANCY_ERROR_VALUE = -0.0f;
067
068  private float globalMemStorePercent;
069  private float globalMemStorePercentMinRange;
070  private float globalMemStorePercentMaxRange;
071
072  private float blockCachePercent;
073  private float blockCachePercentMinRange;
074  private float blockCachePercentMaxRange;
075
076  private float heapOccupancyPercent;
077
078  private final ResizableBlockCache blockCache;
079  // TODO : remove this and mark regionServerAccounting as the observer directly
080  private final FlushRequester memStoreFlusher;
081  private final Server server;
082  private final RegionServerAccounting regionServerAccounting;
083
084  private HeapMemoryTunerChore heapMemTunerChore = null;
085  private final boolean tunerOn;
086  private final int defaultChorePeriod;
087  private final float heapOccupancyLowWatermark;
088
089  private final long maxHeapSize;
090  {
091    // note that this initialization still isn't threadsafe, because updating a long isn't atomic.
092    long tempMaxHeap = -1L;
093    try {
094      final MemoryUsage usage = MemorySizeUtil.safeGetHeapMemoryUsage();
095      if (usage != null) {
096        tempMaxHeap = usage.getMax();
097      }
098    } finally {
099      maxHeapSize = tempMaxHeap;
100    }
101  }
102
103  private MetricsHeapMemoryManager metricsHeapMemoryManager;
104
105  private List<HeapMemoryTuneObserver> tuneObservers = new ArrayList<>();
106
107  HeapMemoryManager(BlockCache blockCache, FlushRequester memStoreFlusher,
108                Server server, RegionServerAccounting regionServerAccounting) {
109    Configuration conf = server.getConfiguration();
110    this.blockCache = toResizableBlockCache(blockCache);
111    this.memStoreFlusher = memStoreFlusher;
112    this.server = server;
113    this.regionServerAccounting = regionServerAccounting;
114    this.tunerOn = doInit(conf);
115    this.defaultChorePeriod = conf.getInt(HBASE_RS_HEAP_MEMORY_TUNER_PERIOD,
116      HBASE_RS_HEAP_MEMORY_TUNER_DEFAULT_PERIOD);
117    this.heapOccupancyLowWatermark = conf.getFloat(HConstants.HEAP_OCCUPANCY_LOW_WATERMARK_KEY,
118      HConstants.DEFAULT_HEAP_OCCUPANCY_LOW_WATERMARK);
119    metricsHeapMemoryManager = new MetricsHeapMemoryManager();
120  }
121
122  private ResizableBlockCache toResizableBlockCache(BlockCache blockCache) {
123    if (blockCache instanceof CombinedBlockCache) {
124      return (ResizableBlockCache) ((CombinedBlockCache) blockCache).getFirstLevelCache();
125    } else {
126      return (ResizableBlockCache) blockCache;
127    }
128  }
129
130  private boolean doInit(Configuration conf) {
131    boolean tuningEnabled = true;
132    globalMemStorePercent = MemorySizeUtil.getGlobalMemStoreHeapPercent(conf, false);
133    blockCachePercent = conf.getFloat(HFILE_BLOCK_CACHE_SIZE_KEY,
134        HConstants.HFILE_BLOCK_CACHE_SIZE_DEFAULT);
135    MemorySizeUtil.checkForClusterFreeHeapMemoryLimit(conf);
136    // Initialize max and min range for memstore heap space
137    globalMemStorePercentMinRange = conf.getFloat(MEMSTORE_SIZE_MIN_RANGE_KEY,
138        globalMemStorePercent);
139    globalMemStorePercentMaxRange = conf.getFloat(MEMSTORE_SIZE_MAX_RANGE_KEY,
140        globalMemStorePercent);
141    if (globalMemStorePercent < globalMemStorePercentMinRange) {
142      LOG.warn("Setting " + MEMSTORE_SIZE_MIN_RANGE_KEY + " to " + globalMemStorePercent
143          + ", same value as " + MemorySizeUtil.MEMSTORE_SIZE_KEY
144          + " because supplied value greater than initial memstore size value.");
145      globalMemStorePercentMinRange = globalMemStorePercent;
146      conf.setFloat(MEMSTORE_SIZE_MIN_RANGE_KEY, globalMemStorePercentMinRange);
147    }
148    if (globalMemStorePercent > globalMemStorePercentMaxRange) {
149      LOG.warn("Setting " + MEMSTORE_SIZE_MAX_RANGE_KEY + " to " + globalMemStorePercent
150          + ", same value as " + MemorySizeUtil.MEMSTORE_SIZE_KEY
151          + " because supplied value less than initial memstore size value.");
152      globalMemStorePercentMaxRange = globalMemStorePercent;
153      conf.setFloat(MEMSTORE_SIZE_MAX_RANGE_KEY, globalMemStorePercentMaxRange);
154    }
155    if (globalMemStorePercent == globalMemStorePercentMinRange
156        && globalMemStorePercent == globalMemStorePercentMaxRange) {
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 (tuningEnabled && blockCachePercent == blockCachePercentMinRange
177        && blockCachePercent == blockCachePercentMaxRange) {
178      tuningEnabled = false;
179    }
180
181    int gml = (int) (globalMemStorePercentMaxRange * CONVERT_TO_PERCENTAGE);
182    int bcul = (int) ((blockCachePercentMinRange) * CONVERT_TO_PERCENTAGE);
183    if (CONVERT_TO_PERCENTAGE - (gml + bcul) < CLUSTER_MINIMUM_MEMORY_THRESHOLD) {
184      throw new RuntimeException("Current heap configuration for MemStore and BlockCache exceeds "
185          + "the threshold required for successful cluster operation. "
186          + "The combined value cannot exceed 0.8. Please check the settings for "
187          + MEMSTORE_SIZE_MAX_RANGE_KEY + " and " + BLOCK_CACHE_SIZE_MIN_RANGE_KEY
188          + " in your configuration. " + MEMSTORE_SIZE_MAX_RANGE_KEY + " is "
189          + globalMemStorePercentMaxRange + " and " + BLOCK_CACHE_SIZE_MIN_RANGE_KEY + " is "
190          + blockCachePercentMinRange);
191    }
192    gml = (int) (globalMemStorePercentMinRange * CONVERT_TO_PERCENTAGE);
193    bcul = (int) ((blockCachePercentMaxRange) * CONVERT_TO_PERCENTAGE);
194    if (CONVERT_TO_PERCENTAGE - (gml + bcul) < CLUSTER_MINIMUM_MEMORY_THRESHOLD) {
195      throw new RuntimeException("Current heap configuration for MemStore and BlockCache exceeds "
196          + "the threshold required for successful cluster operation. "
197          + "The combined value cannot exceed 0.8. Please check the settings for "
198          + MEMSTORE_SIZE_MIN_RANGE_KEY + " and " + BLOCK_CACHE_SIZE_MAX_RANGE_KEY
199          + " in your configuration. " + MEMSTORE_SIZE_MIN_RANGE_KEY + " is "
200          + globalMemStorePercentMinRange + " and " + BLOCK_CACHE_SIZE_MAX_RANGE_KEY + " is "
201          + blockCachePercentMaxRange);
202    }
203    return tuningEnabled;
204  }
205
206  public void start(ChoreService service) {
207    LOG.info("Starting, tuneOn={}", this.tunerOn);
208    this.heapMemTunerChore = new HeapMemoryTunerChore();
209    service.scheduleChore(heapMemTunerChore);
210    if (tunerOn) {
211      // Register HeapMemoryTuner as a memstore flush listener
212      memStoreFlusher.registerFlushRequestListener(heapMemTunerChore);
213    }
214  }
215
216  public void stop() {
217    // The thread is Daemon. Just interrupting the ongoing process.
218    LOG.info("Stopping");
219    this.heapMemTunerChore.cancel(true);
220  }
221
222  public void registerTuneObserver(HeapMemoryTuneObserver observer) {
223    this.tuneObservers.add(observer);
224  }
225
226  // Used by the test cases.
227  boolean isTunerOn() {
228    return this.tunerOn;
229  }
230
231  /**
232   * @return heap occupancy percentage, 0 &lt;= n &lt;= 1. or -0.0 for error asking JVM
233   */
234  public float getHeapOccupancyPercent() {
235    return this.heapOccupancyPercent == Float.MAX_VALUE ? HEAP_OCCUPANCY_ERROR_VALUE : this.heapOccupancyPercent;
236  }
237
238  private class HeapMemoryTunerChore extends ScheduledChore implements FlushRequestListener {
239    private HeapMemoryTuner heapMemTuner;
240    private AtomicLong blockedFlushCount = new AtomicLong();
241    private AtomicLong unblockedFlushCount = new AtomicLong();
242    private long evictCount = 0L;
243    private long cacheMissCount = 0L;
244    private TunerContext tunerContext = new TunerContext();
245    private boolean alarming = false;
246
247    public HeapMemoryTunerChore() {
248      super(server.getServerName() + "-HeapMemoryTunerChore", server, defaultChorePeriod);
249      Class<? extends HeapMemoryTuner> tunerKlass = server.getConfiguration().getClass(
250          HBASE_RS_HEAP_MEMORY_TUNER_CLASS, DefaultHeapMemoryTuner.class, HeapMemoryTuner.class);
251      heapMemTuner = ReflectionUtils.newInstance(tunerKlass, server.getConfiguration());
252      tunerContext
253          .setOffheapMemStore(regionServerAccounting.isOffheap());
254    }
255
256    @Override
257    protected void chore() {
258      // Sample heap occupancy
259      final MemoryUsage usage = MemorySizeUtil.safeGetHeapMemoryUsage();
260      if (usage != null) {
261        heapOccupancyPercent = (float)usage.getUsed() / (float)usage.getCommitted();
262      } else {
263        // previously, an exception would have meant death for the tuning chore
264        // so switch to alarming so that we similarly stop tuning until we get
265        // heap usage information again.
266        heapOccupancyPercent = Float.MAX_VALUE;
267      }
268      // If we are above the heap occupancy alarm low watermark, switch to short
269      // sleeps for close monitoring. Stop autotuning, we are in a danger zone.
270      if (heapOccupancyPercent >= heapOccupancyLowWatermark) {
271        if (!alarming) {
272          LOG.warn("heapOccupancyPercent " + heapOccupancyPercent +
273            " is above heap occupancy alarm watermark (" + heapOccupancyLowWatermark + ")");
274          alarming = true;
275        }
276        metricsHeapMemoryManager.increaseAboveHeapOccupancyLowWatermarkCounter();
277        triggerNow();
278        try {
279          // Need to sleep ourselves since we've told the chore's sleeper
280          // to skip the next sleep cycle.
281          Thread.sleep(1000);
282        } catch (InterruptedException e) {
283          // Interrupted, propagate
284          Thread.currentThread().interrupt();
285        }
286      } else {
287        if (alarming) {
288          LOG.info("heapOccupancyPercent " + heapOccupancyPercent +
289            " is now below the heap occupancy alarm watermark (" +
290            heapOccupancyLowWatermark + ")");
291          alarming = false;
292        }
293      }
294      // Autotune if tuning is enabled and allowed
295      if (tunerOn && !alarming) {
296        tune();
297      }
298    }
299
300    private void tune() {
301      // TODO check if we can increase the memory boundaries
302      // while remaining in the limits
303      long curEvictCount;
304      long curCacheMisCount;
305      long blockedFlushCnt;
306      long unblockedFlushCnt;
307      curEvictCount = blockCache.getStats().getEvictedCount();
308      tunerContext.setEvictCount(curEvictCount - evictCount);
309      evictCount = curEvictCount;
310      curCacheMisCount = blockCache.getStats().getMissCachingCount();
311      tunerContext.setCacheMissCount(curCacheMisCount-cacheMissCount);
312      cacheMissCount = curCacheMisCount;
313      blockedFlushCnt = blockedFlushCount.getAndSet(0);
314      tunerContext.setBlockedFlushCount(blockedFlushCnt);
315      metricsHeapMemoryManager.updateBlockedFlushCount(blockedFlushCnt);
316      unblockedFlushCnt = unblockedFlushCount.getAndSet(0);
317      tunerContext.setUnblockedFlushCount(unblockedFlushCnt);
318      metricsHeapMemoryManager.updateUnblockedFlushCount(unblockedFlushCnt);
319      // TODO : add support for offheap metrics
320      tunerContext.setCurBlockCacheUsed((float) blockCache.getCurrentSize() / maxHeapSize);
321      metricsHeapMemoryManager.setCurBlockCacheSizeGauge(blockCache.getCurrentSize());
322      long globalMemstoreHeapSize = regionServerAccounting.getGlobalMemStoreHeapSize();
323      tunerContext.setCurMemStoreUsed((float) globalMemstoreHeapSize / maxHeapSize);
324      metricsHeapMemoryManager.setCurMemStoreSizeGauge(globalMemstoreHeapSize);
325      tunerContext.setCurBlockCacheSize(blockCachePercent);
326      tunerContext.setCurMemStoreSize(globalMemStorePercent);
327      TunerResult result = null;
328      try {
329        result = this.heapMemTuner.tune(tunerContext);
330      } catch (Throwable t) {
331        LOG.error("Exception thrown from the HeapMemoryTuner implementation", t);
332      }
333      if (result != null && result.needsTuning()) {
334        float memstoreSize = result.getMemStoreSize();
335        float blockCacheSize = result.getBlockCacheSize();
336        LOG.debug("From HeapMemoryTuner new memstoreSize: " + memstoreSize
337            + ". new blockCacheSize: " + blockCacheSize);
338        if (memstoreSize < globalMemStorePercentMinRange) {
339          LOG.info("New memstoreSize from HeapMemoryTuner " + memstoreSize + " is below min level "
340              + globalMemStorePercentMinRange + ". Resetting memstoreSize to min size");
341          memstoreSize = globalMemStorePercentMinRange;
342        } else if (memstoreSize > globalMemStorePercentMaxRange) {
343          LOG.info("New memstoreSize from HeapMemoryTuner " + memstoreSize + " is above max level "
344              + globalMemStorePercentMaxRange + ". Resetting memstoreSize to max size");
345          memstoreSize = globalMemStorePercentMaxRange;
346        }
347        if (blockCacheSize < blockCachePercentMinRange) {
348          LOG.info("New blockCacheSize from HeapMemoryTuner " + blockCacheSize
349              + " is below min level " + blockCachePercentMinRange
350              + ". Resetting blockCacheSize to min size");
351          blockCacheSize = blockCachePercentMinRange;
352        } else if (blockCacheSize > blockCachePercentMaxRange) {
353          LOG.info("New blockCacheSize from HeapMemoryTuner " + blockCacheSize
354              + " is above max level " + blockCachePercentMaxRange
355              + ". Resetting blockCacheSize to min size");
356          blockCacheSize = blockCachePercentMaxRange;
357        }
358        int gml = (int) (memstoreSize * CONVERT_TO_PERCENTAGE);
359        int bcul = (int) ((blockCacheSize) * CONVERT_TO_PERCENTAGE);
360        if (CONVERT_TO_PERCENTAGE - (gml + bcul) < CLUSTER_MINIMUM_MEMORY_THRESHOLD) {
361          LOG.info("Current heap configuration from HeapMemoryTuner exceeds "
362              + "the threshold required for successful cluster operation. "
363              + "The combined value cannot exceed 0.8. " + MemorySizeUtil.MEMSTORE_SIZE_KEY
364              + " is " + memstoreSize + " and " + HFILE_BLOCK_CACHE_SIZE_KEY + " is "
365              + 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.
509   * It 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}