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