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 java.util.Map;
021import java.util.Set;
022import java.util.concurrent.BlockingQueue;
023import java.util.concurrent.ConcurrentHashMap;
024import java.util.concurrent.Executors;
025import java.util.concurrent.LinkedBlockingQueue;
026import java.util.concurrent.ScheduledExecutorService;
027import java.util.concurrent.TimeUnit;
028import java.util.concurrent.atomic.AtomicInteger;
029import java.util.concurrent.atomic.AtomicLong;
030import java.util.concurrent.atomic.LongAdder;
031import org.apache.hadoop.hbase.regionserver.HeapMemoryManager.HeapMemoryTuneObserver;
032import org.apache.hadoop.hbase.util.Bytes;
033import org.apache.hadoop.util.StringUtils;
034import org.apache.yetus.audience.InterfaceAudience;
035import org.slf4j.Logger;
036import org.slf4j.LoggerFactory;
037
038import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
039
040/**
041 * Does the management of memstoreLAB chunk creations. A monotonically incrementing id is associated
042 * with every chunk
043 */
044@InterfaceAudience.Private
045public class ChunkCreator {
046  private static final Logger LOG = LoggerFactory.getLogger(ChunkCreator.class);
047  // monotonically increasing chunkid. Starts at 1.
048  private AtomicInteger chunkID = new AtomicInteger(1);
049  // maps the chunk against the monotonically increasing chunk id. We need to preserve the
050  // natural ordering of the key
051  // CellChunkMap creation should convert the weak ref to hard reference
052
053  // chunk id of each chunk is the first integer written on each chunk,
054  // the header size need to be changed in case chunk id size is changed
055  public static final int SIZEOF_CHUNK_HEADER = Bytes.SIZEOF_INT;
056
057  /**
058   * Types of chunks, based on their sizes
059   */
060  public enum ChunkType {
061    // An index chunk is a small chunk, allocated from the index chunks pool.
062    // Its size is fixed and is 10% of the size of a data chunk.
063    INDEX_CHUNK,
064    // A data chunk is a regular chunk, allocated from the data chunks pool.
065    // Its size is fixed and given as input to the ChunkCreator c'tor.
066    DATA_CHUNK,
067    // A jumbo chunk isn't allocated from pool. Its size is bigger than the size of a
068    // data chunk, and is determined per chunk (meaning, there is no fixed jumbo size).
069    JUMBO_CHUNK
070  }
071
072  // mapping from chunk IDs to chunks
073  private Map<Integer, Chunk> chunkIdMap = new ConcurrentHashMap<Integer, Chunk>();
074
075  private final boolean offheap;
076  static ChunkCreator instance;
077  static boolean chunkPoolDisabled = false;
078  private MemStoreChunkPool dataChunksPool;
079  private final int chunkSize;
080  private int indexChunkSize;
081  private MemStoreChunkPool indexChunksPool;
082
083  ChunkCreator(int chunkSize, boolean offheap, long globalMemStoreSize, float poolSizePercentage,
084    float initialCountPercentage, HeapMemoryManager heapMemoryManager,
085    float indexChunkSizePercentage) {
086    this.offheap = offheap;
087    this.chunkSize = chunkSize; // in case pools are not allocated
088    initializePools(chunkSize, globalMemStoreSize, poolSizePercentage, indexChunkSizePercentage,
089      initialCountPercentage, heapMemoryManager);
090  }
091
092  private void initializePools(int chunkSize, long globalMemStoreSize, float poolSizePercentage,
093    float indexChunkSizePercentage, float initialCountPercentage,
094    HeapMemoryManager heapMemoryManager) {
095    this.dataChunksPool = initializePool("data", globalMemStoreSize,
096      (1 - indexChunkSizePercentage) * poolSizePercentage, initialCountPercentage, chunkSize,
097      ChunkType.DATA_CHUNK, heapMemoryManager);
098    // The index chunks pool is needed only when the index type is CCM.
099    // Since the pools are not created at all when the index type isn't CCM,
100    // we don't need to check it here.
101    this.indexChunkSize = (int) (indexChunkSizePercentage * chunkSize);
102    this.indexChunksPool =
103      initializePool("index", globalMemStoreSize, indexChunkSizePercentage * poolSizePercentage,
104        initialCountPercentage, this.indexChunkSize, ChunkType.INDEX_CHUNK, heapMemoryManager);
105  }
106
107  /**
108   * Initializes the instance of ChunkCreator
109   * @param chunkSize              the chunkSize
110   * @param offheap                indicates if the chunk is to be created offheap or not
111   * @param globalMemStoreSize     the global memstore size
112   * @param poolSizePercentage     pool size percentage
113   * @param initialCountPercentage the initial count of the chunk pool if any
114   * @param heapMemoryManager      the heapmemory manager
115   * @return singleton MSLABChunkCreator
116   */
117  @edu.umd.cs.findbugs.annotations.SuppressWarnings(
118      value = { "LI_LAZY_INIT_STATIC", "MS_EXPOSE_REP" },
119      justification = "Method is called by single thread at the starting of RS")
120  public static ChunkCreator initialize(int chunkSize, boolean offheap, long globalMemStoreSize,
121    float poolSizePercentage, float initialCountPercentage, HeapMemoryManager heapMemoryManager,
122    float indexChunkSizePercent) {
123    if (instance != null) {
124      return instance;
125    }
126    instance = new ChunkCreator(chunkSize, offheap, globalMemStoreSize, poolSizePercentage,
127      initialCountPercentage, heapMemoryManager, indexChunkSizePercent);
128    return instance;
129  }
130
131  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "MS_EXPOSE_REP",
132      justification = "singleton pattern")
133  public static ChunkCreator getInstance() {
134    return instance;
135  }
136
137  /**
138   * Creates and inits a data chunk. The default implementation.
139   * @return the chunk that was initialized
140   */
141  Chunk getChunk() {
142    return getChunk(ChunkType.DATA_CHUNK);
143  }
144
145  /**
146   * Creates and inits a chunk with specific type.
147   * @return the chunk that was initialized
148   */
149  Chunk getChunk(ChunkType chunkType) {
150    switch (chunkType) {
151      case INDEX_CHUNK:
152        if (indexChunksPool == null) {
153          if (indexChunkSize <= 0) {
154            throw new IllegalArgumentException(
155              "chunkType is INDEX_CHUNK but indexChunkSize is:[" + this.indexChunkSize + "]");
156          }
157          return getChunk(chunkType, indexChunkSize);
158        } else {
159          return getChunk(chunkType, indexChunksPool.getChunkSize());
160        }
161      case DATA_CHUNK:
162        if (dataChunksPool == null) {
163          return getChunk(chunkType, chunkSize);
164        } else {
165          return getChunk(chunkType, dataChunksPool.getChunkSize());
166        }
167      default:
168        throw new IllegalArgumentException("chunkType must either be INDEX_CHUNK or DATA_CHUNK");
169    }
170  }
171
172  /**
173   * Creates and inits a chunk.
174   * @return the chunk that was initialized
175   * @param size the size of the chunk to be allocated, in bytes
176   */
177  Chunk getChunk(ChunkType chunkType, int size) {
178    Chunk chunk = null;
179    MemStoreChunkPool pool = null;
180
181    // if it is one of the pools
182    if (dataChunksPool != null && chunkType == ChunkType.DATA_CHUNK) {
183      pool = dataChunksPool;
184    } else if (indexChunksPool != null && chunkType == ChunkType.INDEX_CHUNK) {
185      pool = indexChunksPool;
186    }
187
188    // if we have a pool
189    if (pool != null) {
190      // the pool creates the chunk internally. The chunk#init() call happens here
191      chunk = pool.getChunk();
192      // the pool has run out of maxCount
193      if (chunk == null) {
194        if (LOG.isTraceEnabled()) {
195          LOG.trace("The chunk pool is full. Reached maxCount= " + pool.getMaxCount()
196            + ". Creating chunk onheap.");
197        }
198      }
199    }
200
201    if (chunk == null) {
202      chunk = createChunk(false, chunkType, size);
203    }
204
205    // now we need to actually do the expensive memory allocation step in case of a new chunk,
206    // else only the offset is set to the beginning of the chunk to accept allocations
207    chunk.init();
208    return chunk;
209  }
210
211  /**
212   * Creates and inits a chunk of a special size, bigger than a regular chunk size. Such a chunk
213   * will never come from pool and will always be on demand allocated.
214   * @return the chunk that was initialized
215   * @param jumboSize the special size to be used
216   */
217  Chunk getJumboChunk(int jumboSize) {
218    int allocSize = jumboSize + SIZEOF_CHUNK_HEADER;
219
220    if (allocSize <= this.getChunkSize(ChunkType.DATA_CHUNK)) {
221      LOG.warn("Jumbo chunk size " + jumboSize + " must be more than regular chunk size "
222        + this.getChunkSize(ChunkType.DATA_CHUNK) + ". Converting to regular chunk.");
223      return getChunk();
224    }
225    // the new chunk is going to hold the jumbo cell data and needs to be referenced by
226    // a strong map.
227    return getChunk(ChunkType.JUMBO_CHUNK, allocSize);
228  }
229
230  /**
231   * Creates the chunk either onheap or offheap
232   * @param pool      indicates if the chunks have to be created which will be used by the Pool
233   * @param chunkType whether the requested chunk is data chunk or index chunk.
234   * @param size      the size of the chunk to be allocated, in bytes
235   * @return the chunk
236   */
237  private Chunk createChunk(boolean pool, ChunkType chunkType, int size) {
238    Chunk chunk = null;
239    int id = chunkID.getAndIncrement();
240    assert id > 0;
241    // do not create offheap chunk on demand
242    if (pool && this.offheap) {
243      chunk = new OffheapChunk(size, id, chunkType, pool);
244    } else {
245      chunk = new OnheapChunk(size, id, chunkType, pool);
246    }
247
248    /**
249     * Here we always put the chunk into the {@link ChunkCreator#chunkIdMap} no matter whether the
250     * chunk is pooled or not. <br/>
251     * For {@link CompactingMemStore},because the chunk could only be acquired from
252     * {@link ChunkCreator} through {@link MemStoreLABImpl}, and
253     * {@link CompactingMemStore#indexType} could only be {@link IndexType.CHUNK_MAP} when using
254     * {@link MemStoreLABImpl}, so we must put chunk into this {@link ChunkCreator#chunkIdMap} to
255     * make sure the chunk could be got by chunkId.
256     * <p>
257     * For {@link DefaultMemStore},it is also reasonable to put the chunk in
258     * {@link ChunkCreator#chunkIdMap} because: <br/>
259     * 1.When the {@link MemStoreLAB} which created the chunk is not closed, this chunk is used by
260     * the {@link Segment} which references this {@link MemStoreLAB}, so this chunk certainly should
261     * not be GC-ed, putting the chunk in {@link ChunkCreator#chunkIdMap} does not prevent useless
262     * chunk to be GC-ed. <br/>
263     * 2.When the {@link MemStoreLAB} which created the chunk is closed, and if the chunk is not
264     * pooled, {@link ChunkCreator#removeChunk} is invoked to remove the chunk from this
265     * {@link ChunkCreator#chunkIdMap}, so there is no memory leak.
266     */
267    this.chunkIdMap.put(chunk.getId(), chunk);
268
269    return chunk;
270  }
271
272  // Chunks from pool are created covered with strong references anyway.
273  private Chunk createChunkForPool(ChunkType chunkType, int chunkSize) {
274    if (chunkSize != dataChunksPool.getChunkSize() && chunkSize != indexChunksPool.getChunkSize()) {
275      return null;
276    }
277    return createChunk(true, chunkType, chunkSize);
278  }
279
280  // Used to translate the ChunkID into a chunk ref
281  Chunk getChunk(int id) {
282    // can return null if chunk was never mapped
283    return chunkIdMap.get(id);
284  }
285
286  boolean isOffheap() {
287    return this.offheap;
288  }
289
290  private void removeChunks(Set<Integer> chunkIDs) {
291    this.chunkIdMap.keySet().removeAll(chunkIDs);
292  }
293
294  Chunk removeChunk(int chunkId) {
295    return this.chunkIdMap.remove(chunkId);
296  }
297
298  // the chunks in the chunkIdMap may already be released so we shouldn't relay
299  // on this counting for strong correctness. This method is used only in testing.
300  int numberOfMappedChunks() {
301    return this.chunkIdMap.size();
302  }
303
304  void clearChunkIds() {
305    this.chunkIdMap.clear();
306  }
307
308  /**
309   * A pool of {@link Chunk} instances. MemStoreChunkPool caches a number of retired chunks for
310   * reusing, it could decrease allocating bytes when writing, thereby optimizing the garbage
311   * collection on JVM.
312   */
313  private class MemStoreChunkPool implements HeapMemoryTuneObserver {
314    private final int chunkSize;
315    private final ChunkType chunkType;
316    private int maxCount;
317
318    // A queue of reclaimed chunks
319    private final BlockingQueue<Chunk> reclaimedChunks;
320    private final float poolSizePercentage;
321
322    /** Statistics thread schedule pool */
323    private final ScheduledExecutorService scheduleThreadPool;
324    /** Statistics thread */
325    private static final int statThreadPeriod = 60 * 5;
326    private final AtomicLong chunkCount = new AtomicLong();
327    private final LongAdder reusedChunkCount = new LongAdder();
328    private final String label;
329
330    MemStoreChunkPool(String label, int chunkSize, ChunkType chunkType, int maxCount,
331      int initialCount, float poolSizePercentage) {
332      this.label = label;
333      this.chunkSize = chunkSize;
334      this.chunkType = chunkType;
335      this.maxCount = maxCount;
336      this.poolSizePercentage = poolSizePercentage;
337      this.reclaimedChunks = new LinkedBlockingQueue<>();
338      for (int i = 0; i < initialCount; i++) {
339        Chunk chunk = createChunk(true, chunkType, chunkSize);
340        chunk.init();
341        reclaimedChunks.add(chunk);
342      }
343      chunkCount.set(initialCount);
344      final String n = Thread.currentThread().getName();
345      scheduleThreadPool = Executors.newScheduledThreadPool(1, new ThreadFactoryBuilder()
346        .setNameFormat(n + "-MemStoreChunkPool Statistics").setDaemon(true).build());
347      this.scheduleThreadPool.scheduleAtFixedRate(new StatisticsThread(), statThreadPeriod,
348        statThreadPeriod, TimeUnit.SECONDS);
349    }
350
351    /**
352     * Poll a chunk from the pool, reset it if not null, else create a new chunk to return if we
353     * have not yet created max allowed chunks count. When we have already created max allowed
354     * chunks and no free chunks as of now, return null. It is the responsibility of the caller to
355     * make a chunk then. Note: Chunks returned by this pool must be put back to the pool after its
356     * use.
357     * @return a chunk
358     * @see #putbackChunks(Chunk)
359     */
360    Chunk getChunk() {
361      Chunk chunk = reclaimedChunks.poll();
362      if (chunk != null) {
363        chunk.reset();
364        reusedChunkCount.increment();
365      } else {
366        // Make a chunk iff we have not yet created the maxCount chunks
367        while (true) {
368          long created = this.chunkCount.get();
369          if (created < this.maxCount) {
370            if (this.chunkCount.compareAndSet(created, created + 1)) {
371              chunk = createChunkForPool(chunkType, chunkSize);
372              break;
373            }
374          } else {
375            break;
376          }
377        }
378      }
379      return chunk;
380    }
381
382    int getChunkSize() {
383      return chunkSize;
384    }
385
386    /**
387     * Add the chunks to the pool, when the pool achieves the max size, it will skip the remaining
388     * chunks
389     */
390    private void putbackChunks(Chunk c) {
391      int toAdd = this.maxCount - reclaimedChunks.size();
392      if (c.isFromPool() && c.size == chunkSize && toAdd > 0) {
393        reclaimedChunks.add(c);
394      } else {
395        // remove the chunk (that is not going to pool)
396        // though it is initially from the pool or not
397        ChunkCreator.this.removeChunk(c.getId());
398      }
399    }
400
401    private class StatisticsThread extends Thread {
402      StatisticsThread() {
403        super("MemStoreChunkPool.StatisticsThread");
404        setDaemon(true);
405      }
406
407      @Override
408      public void run() {
409        logStats();
410      }
411
412      private void logStats() {
413        if (!LOG.isDebugEnabled()) return;
414        long created = chunkCount.get();
415        long reused = reusedChunkCount.sum();
416        long total = created + reused;
417        LOG.debug(
418          "{} stats (chunk size={}): current pool size={}, created chunk count={}, "
419            + "reused chunk count={}, reuseRatio={}",
420          label, chunkSize, reclaimedChunks.size(), created, reused,
421          (total == 0 ? "0" : StringUtils.formatPercent((float) reused / (float) total, 2)));
422      }
423    }
424
425    private int getMaxCount() {
426      return this.maxCount;
427    }
428
429    @Override
430    public void onHeapMemoryTune(long newMemstoreSize, long newBlockCacheSize) {
431      // don't do any tuning in case of offheap memstore
432      if (isOffheap()) {
433        LOG.warn("{} not tuning the chunk pool as it is offheap", label);
434        return;
435      }
436      int newMaxCount = (int) (newMemstoreSize * poolSizePercentage / getChunkSize());
437      if (newMaxCount != this.maxCount) {
438        // We need an adjustment in the chunks numbers
439        if (newMaxCount > this.maxCount) {
440          // Max chunks getting increased. Just change the variable. Later calls to getChunk() would
441          // create and add them to Q
442          LOG.info("{} max count for chunks increased from {} to {}", this.label, this.maxCount,
443            newMaxCount);
444          this.maxCount = newMaxCount;
445        } else {
446          // Max chunks getting decreased. We may need to clear off some of the pooled chunks now
447          // itself. If the extra chunks are serving already, do not pool those when we get them
448          // back
449          LOG.info("{} max count for chunks decreased from {} to {}", this.label, this.maxCount,
450            newMaxCount);
451          this.maxCount = newMaxCount;
452          if (this.reclaimedChunks.size() > newMaxCount) {
453            synchronized (this) {
454              while (this.reclaimedChunks.size() > newMaxCount) {
455                this.reclaimedChunks.poll();
456              }
457            }
458          }
459        }
460      }
461    }
462  }
463
464  static void clearDisableFlag() {
465    chunkPoolDisabled = false;
466  }
467
468  private MemStoreChunkPool initializePool(String label, long globalMemStoreSize,
469    float poolSizePercentage, float initialCountPercentage, int chunkSize, ChunkType chunkType,
470    HeapMemoryManager heapMemoryManager) {
471    if (poolSizePercentage <= 0) {
472      LOG.info("{} poolSizePercentage is less than 0. So not using pool", label);
473      return null;
474    }
475    if (chunkPoolDisabled) {
476      return null;
477    }
478    if (poolSizePercentage > 1.0) {
479      throw new IllegalArgumentException(
480        MemStoreLAB.CHUNK_POOL_MAXSIZE_KEY + " must be between 0.0 and 1.0");
481    }
482    int maxCount = (int) (globalMemStoreSize * poolSizePercentage / chunkSize);
483    if (initialCountPercentage > 1.0 || initialCountPercentage < 0) {
484      throw new IllegalArgumentException(
485        label + " " + MemStoreLAB.CHUNK_POOL_INITIALSIZE_KEY + " must be between 0.0 and 1.0");
486    }
487    int initialCount = (int) (initialCountPercentage * maxCount);
488    LOG.info("Allocating {} MemStoreChunkPool with chunk size {}, max count {}, initial count {}",
489      label, StringUtils.byteDesc(chunkSize), maxCount, initialCount);
490    MemStoreChunkPool memStoreChunkPool = new MemStoreChunkPool(label, chunkSize, chunkType,
491      maxCount, initialCount, poolSizePercentage);
492    if (heapMemoryManager != null && memStoreChunkPool != null) {
493      // Register with Heap Memory manager
494      heapMemoryManager.registerTuneObserver(memStoreChunkPool);
495    }
496    return memStoreChunkPool;
497  }
498
499  int getMaxCount() {
500    return getMaxCount(ChunkType.DATA_CHUNK);
501  }
502
503  int getMaxCount(ChunkType chunkType) {
504    switch (chunkType) {
505      case INDEX_CHUNK:
506        if (indexChunksPool != null) {
507          return indexChunksPool.getMaxCount();
508        }
509        break;
510      case DATA_CHUNK:
511        if (dataChunksPool != null) {
512          return dataChunksPool.getMaxCount();
513        }
514        break;
515      default:
516        throw new IllegalArgumentException("chunkType must either be INDEX_CHUNK or DATA_CHUNK");
517    }
518
519    return 0;
520  }
521
522  int getPoolSize() {
523    return getPoolSize(ChunkType.DATA_CHUNK);
524  }
525
526  int getPoolSize(ChunkType chunkType) {
527    switch (chunkType) {
528      case INDEX_CHUNK:
529        if (indexChunksPool != null) {
530          return indexChunksPool.reclaimedChunks.size();
531        }
532        break;
533      case DATA_CHUNK:
534        if (dataChunksPool != null) {
535          return dataChunksPool.reclaimedChunks.size();
536        }
537        break;
538      default:
539        throw new IllegalArgumentException("chunkType must either be INDEX_CHUNK or DATA_CHUNK");
540    }
541    return 0;
542  }
543
544  boolean isChunkInPool(int chunkId) {
545    Chunk c = getChunk(chunkId);
546    if (c == null) {
547      return false;
548    }
549
550    // chunks that are from pool will return true chunk reference not null
551    if (dataChunksPool != null && dataChunksPool.reclaimedChunks.contains(c)) {
552      return true;
553    } else if (indexChunksPool != null && indexChunksPool.reclaimedChunks.contains(c)) {
554      return true;
555    }
556    return false;
557  }
558
559  /*
560   * Only used in testing
561   */
562  void clearChunksInPool() {
563    if (dataChunksPool != null) {
564      dataChunksPool.reclaimedChunks.clear();
565    }
566    if (indexChunksPool != null) {
567      indexChunksPool.reclaimedChunks.clear();
568    }
569  }
570
571  int getChunkSize() {
572    return getChunkSize(ChunkType.DATA_CHUNK);
573  }
574
575  int getChunkSize(ChunkType chunkType) {
576    switch (chunkType) {
577      case INDEX_CHUNK:
578        if (indexChunksPool != null) {
579          return indexChunksPool.getChunkSize();
580        } else {
581          return indexChunkSize;
582        }
583      case DATA_CHUNK:
584        if (dataChunksPool != null) {
585          return dataChunksPool.getChunkSize();
586        } else { // When pools are empty
587          return chunkSize;
588        }
589      default:
590        throw new IllegalArgumentException("chunkType must either be INDEX_CHUNK or DATA_CHUNK");
591    }
592  }
593
594  synchronized void putbackChunks(Set<Integer> chunks) {
595    // if there is no pool just try to clear the chunkIdMap in case there is something
596    if (dataChunksPool == null && indexChunksPool == null) {
597      this.removeChunks(chunks);
598      return;
599    }
600
601    // if there is a pool, go over all chunk IDs that came back, the chunks may be from pool or not
602    for (int chunkID : chunks) {
603      // translate chunk ID to chunk, if chunk initially wasn't in pool
604      // this translation will (most likely) return null
605      Chunk chunk = ChunkCreator.this.getChunk(chunkID);
606      if (chunk != null) {
607        if (chunk.isFromPool() && chunk.isIndexChunk()) {
608          indexChunksPool.putbackChunks(chunk);
609        } else if (chunk.isFromPool() && chunk.isDataChunk()) {
610          dataChunksPool.putbackChunks(chunk);
611        } else {
612          // chunks which are not from one of the pools
613          // should be released without going to the pools.
614          // Removing them from chunkIdMap will cause their removal by the GC.
615          this.removeChunk(chunkID);
616        }
617      }
618      // if chunk is null, it was never covered by the chunkIdMap (and so wasn't in pool also),
619      // so we have nothing to do on its release
620    }
621    return;
622  }
623
624  MemStoreChunkPool getIndexChunksPool() {
625    return this.indexChunksPool;
626  }
627
628  MemStoreChunkPool getDataChunksPool() {
629    return this.dataChunksPool;
630  }
631
632}