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