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      tunerContext.setCurBlockCacheUsed((float) blockCache.getCurrentSize() / maxHeapSize);
321      metricsHeapMemoryManager.setCurBlockCacheSizeGauge(blockCache.getCurrentSize());
322      long globalMemstoreDataSize = regionServerAccounting.getGlobalMemStoreDataSize();
323      long globalMemstoreHeapSize = regionServerAccounting.getGlobalMemStoreHeapSize();
324      long globalMemStoreOffHeapSize = regionServerAccounting.getGlobalMemStoreOffHeapSize();
325      tunerContext.setCurMemStoreUsed((float) globalMemstoreHeapSize / maxHeapSize);
326      metricsHeapMemoryManager.setCurMemStoreSizeGauge(globalMemstoreDataSize);
327      metricsHeapMemoryManager.setCurMemStoreOnHeapSizeGauge(globalMemstoreHeapSize);
328      metricsHeapMemoryManager.setCurMemStoreOffHeapSizeGauge(globalMemStoreOffHeapSize);
329      tunerContext.setCurBlockCacheSize(blockCachePercent);
330      tunerContext.setCurMemStoreSize(globalMemStorePercent);
331      TunerResult result = null;
332      try {
333        result = this.heapMemTuner.tune(tunerContext);
334      } catch (Throwable t) {
335        LOG.error("Exception thrown from the HeapMemoryTuner implementation", t);
336      }
337      if (result != null && result.needsTuning()) {
338        float memstoreSize = result.getMemStoreSize();
339        float blockCacheSize = result.getBlockCacheSize();
340        LOG.debug("From HeapMemoryTuner new memstoreSize: " + memstoreSize
341          + ". new blockCacheSize: " + blockCacheSize);
342        if (memstoreSize < globalMemStorePercentMinRange) {
343          LOG.info("New memstoreSize from HeapMemoryTuner " + memstoreSize + " is below min level "
344            + globalMemStorePercentMinRange + ". Resetting memstoreSize to min size");
345          memstoreSize = globalMemStorePercentMinRange;
346        } else if (memstoreSize > globalMemStorePercentMaxRange) {
347          LOG.info("New memstoreSize from HeapMemoryTuner " + memstoreSize + " is above max level "
348            + globalMemStorePercentMaxRange + ". Resetting memstoreSize to max size");
349          memstoreSize = globalMemStorePercentMaxRange;
350        }
351        if (blockCacheSize < blockCachePercentMinRange) {
352          LOG.info(
353            "New blockCacheSize from HeapMemoryTuner " + blockCacheSize + " is below min level "
354              + blockCachePercentMinRange + ". Resetting blockCacheSize to min size");
355          blockCacheSize = blockCachePercentMinRange;
356        } else if (blockCacheSize > blockCachePercentMaxRange) {
357          LOG.info(
358            "New blockCacheSize from HeapMemoryTuner " + blockCacheSize + " is above max level "
359              + blockCachePercentMaxRange + ". Resetting blockCacheSize to min size");
360          blockCacheSize = blockCachePercentMaxRange;
361        }
362        int gml = (int) (memstoreSize * CONVERT_TO_PERCENTAGE);
363        int bcul = (int) ((blockCacheSize) * CONVERT_TO_PERCENTAGE);
364        if (CONVERT_TO_PERCENTAGE - (gml + bcul) < CLUSTER_MINIMUM_MEMORY_THRESHOLD) {
365          LOG.info("Current heap configuration from HeapMemoryTuner exceeds "
366            + "the threshold required for successful cluster operation. "
367            + "The combined value cannot exceed 0.8. " + MemorySizeUtil.MEMSTORE_SIZE_KEY + " is "
368            + memstoreSize + " and " + HFILE_BLOCK_CACHE_SIZE_KEY + " is " + blockCacheSize);
369          // TODO can adjust the value so as not exceed 80%. Is that correct? may be.
370        } else {
371          int memStoreDeltaSize =
372            (int) ((memstoreSize - globalMemStorePercent) * CONVERT_TO_PERCENTAGE);
373          int blockCacheDeltaSize =
374            (int) ((blockCacheSize - blockCachePercent) * CONVERT_TO_PERCENTAGE);
375          metricsHeapMemoryManager.updateMemStoreDeltaSizeHistogram(memStoreDeltaSize);
376          metricsHeapMemoryManager.updateBlockCacheDeltaSizeHistogram(blockCacheDeltaSize);
377          long newBlockCacheSize = (long) (maxHeapSize * blockCacheSize);
378          // we could have got an increase or decrease in size for the offheap memstore
379          // also if the flush had happened due to heap overhead. In that case it is ok
380          // to adjust the onheap memstore limit configs
381          long newMemstoreSize = (long) (maxHeapSize * memstoreSize);
382          LOG.info("Setting block cache heap size to " + newBlockCacheSize
383            + " and memstore heap size to " + newMemstoreSize);
384          blockCachePercent = blockCacheSize;
385          blockCache.setMaxSize(newBlockCacheSize);
386          globalMemStorePercent = memstoreSize;
387          // Internally sets it to RegionServerAccounting
388          // TODO : Set directly on RSAccounting??
389          memStoreFlusher.setGlobalMemStoreLimit(newMemstoreSize);
390          for (HeapMemoryTuneObserver observer : tuneObservers) {
391            // Risky.. If this newMemstoreSize decreases we reduce the count in offheap chunk pool
392            observer.onHeapMemoryTune(newMemstoreSize, newBlockCacheSize);
393          }
394        }
395      } else {
396        metricsHeapMemoryManager.increaseTunerDoNothingCounter();
397        if (LOG.isDebugEnabled()) {
398          LOG.debug("No changes made by HeapMemoryTuner.");
399        }
400      }
401    }
402
403    @Override
404    public void flushRequested(FlushType type, Region region) {
405      switch (type) {
406        case ABOVE_ONHEAP_HIGHER_MARK:
407          blockedFlushCount.incrementAndGet();
408          break;
409        case ABOVE_ONHEAP_LOWER_MARK:
410          unblockedFlushCount.incrementAndGet();
411          break;
412        // Removed the counting of the offheap related flushes (after reviews). Will add later if
413        // needed
414        default:
415          // In case of any other flush don't do any action.
416          break;
417      }
418    }
419  }
420
421  /**
422   * POJO to pass all the relevant information required to do the heap memory tuning. It holds the
423   * flush counts and block cache evictions happened within the interval. Also holds the current
424   * heap percentage allocated for memstore and block cache.
425   */
426  public static final class TunerContext {
427    private long blockedFlushCount;
428    private long unblockedFlushCount;
429    private long evictCount;
430    private long cacheMissCount;
431    private float curBlockCacheUsed;
432    private float curMemStoreUsed;
433    private float curMemStoreSize;
434    private float curBlockCacheSize;
435    private boolean offheapMemstore;
436
437    public long getBlockedFlushCount() {
438      return blockedFlushCount;
439    }
440
441    public void setBlockedFlushCount(long blockedFlushCount) {
442      this.blockedFlushCount = blockedFlushCount;
443    }
444
445    public long getUnblockedFlushCount() {
446      return unblockedFlushCount;
447    }
448
449    public void setUnblockedFlushCount(long unblockedFlushCount) {
450      this.unblockedFlushCount = unblockedFlushCount;
451    }
452
453    public long getEvictCount() {
454      return evictCount;
455    }
456
457    public void setEvictCount(long evictCount) {
458      this.evictCount = evictCount;
459    }
460
461    public float getCurMemStoreSize() {
462      return curMemStoreSize;
463    }
464
465    public void setCurMemStoreSize(float curMemStoreSize) {
466      this.curMemStoreSize = curMemStoreSize;
467    }
468
469    public float getCurBlockCacheSize() {
470      return curBlockCacheSize;
471    }
472
473    public void setCurBlockCacheSize(float curBlockCacheSize) {
474      this.curBlockCacheSize = curBlockCacheSize;
475    }
476
477    public long getCacheMissCount() {
478      return cacheMissCount;
479    }
480
481    public void setCacheMissCount(long cacheMissCount) {
482      this.cacheMissCount = cacheMissCount;
483    }
484
485    public float getCurBlockCacheUsed() {
486      return curBlockCacheUsed;
487    }
488
489    public void setCurBlockCacheUsed(float curBlockCacheUsed) {
490      this.curBlockCacheUsed = curBlockCacheUsed;
491    }
492
493    public float getCurMemStoreUsed() {
494      return curMemStoreUsed;
495    }
496
497    public void setCurMemStoreUsed(float d) {
498      this.curMemStoreUsed = d;
499    }
500
501    public void setOffheapMemStore(boolean offheapMemstore) {
502      this.offheapMemstore = offheapMemstore;
503    }
504
505    public boolean isOffheapMemStore() {
506      return this.offheapMemstore;
507    }
508  }
509
510  /**
511   * POJO which holds the result of memory tuning done by HeapMemoryTuner implementation. It
512   * includes the new heap percentage for memstore and block cache.
513   */
514  public static final class TunerResult {
515    private float memstoreSize;
516    private float blockCacheSize;
517    private final boolean needsTuning;
518
519    public TunerResult(boolean needsTuning) {
520      this.needsTuning = needsTuning;
521    }
522
523    public float getMemStoreSize() {
524      return memstoreSize;
525    }
526
527    public void setMemStoreSize(float memstoreSize) {
528      this.memstoreSize = memstoreSize;
529    }
530
531    public float getBlockCacheSize() {
532      return blockCacheSize;
533    }
534
535    public void setBlockCacheSize(float blockCacheSize) {
536      this.blockCacheSize = blockCacheSize;
537    }
538
539    public boolean needsTuning() {
540      return needsTuning;
541    }
542  }
543
544  /**
545   * Every class that wants to observe heap memory tune actions must implement this interface.
546   */
547  public static interface HeapMemoryTuneObserver {
548
549    /**
550     * This method would be called by HeapMemoryManger when a heap memory tune action took place.
551     * @param newMemstoreSize   The newly calculated global memstore size
552     * @param newBlockCacheSize The newly calculated global blockcache size
553     */
554    void onHeapMemoryTune(long newMemstoreSize, long newBlockCacheSize);
555  }
556}