You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by zh...@apache.org on 2021/08/05 13:34:12 UTC

[hbase] branch branch-2.3 updated: HBASE-26142 NullPointerException when set 'hbase.hregion.memstore.mslab.indexchunksize.percent' to zero (#3531)

This is an automated email from the ASF dual-hosted git repository.

zhangduo pushed a commit to branch branch-2.3
in repository https://gitbox.apache.org/repos/asf/hbase.git


The following commit(s) were added to refs/heads/branch-2.3 by this push:
     new ef87474  HBASE-26142 NullPointerException when set 'hbase.hregion.memstore.mslab.indexchunksize.percent' to zero (#3531)
ef87474 is described below

commit ef87474917342a54882314648787822b6ca43e5b
Author: chenglei <ch...@apache.org>
AuthorDate: Thu Aug 5 20:48:18 2021 +0800

    HBASE-26142 NullPointerException when set 'hbase.hregion.memstore.mslab.indexchunksize.percent' to zero (#3531)
    
    Signed-off-by: Duo Zhang <zh...@apache.org>
---
 .../apache/hadoop/hbase/regionserver/Chunk.java    |  25 ++-
 .../hadoop/hbase/regionserver/ChunkCreator.java    |  77 ++++++---
 .../hadoop/hbase/regionserver/OffheapChunk.java    |   9 +-
 .../hadoop/hbase/regionserver/OnheapChunk.java     |   9 +-
 .../hadoop/hbase/regionserver/TestCellFlatSet.java |  10 +-
 .../hbase/regionserver/TestMemStoreChunkPool.java  | 178 +++++++++++++++++++--
 6 files changed, 255 insertions(+), 53 deletions(-)

diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Chunk.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Chunk.java
index 136efee..bf9b474 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Chunk.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Chunk.java
@@ -19,9 +19,9 @@ package org.apache.hadoop.hbase.regionserver;
 
 import java.nio.ByteBuffer;
 import java.util.concurrent.atomic.AtomicInteger;
-
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.regionserver.ChunkCreator.ChunkType;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
 
 import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
@@ -51,6 +51,8 @@ public abstract class Chunk {
   // The unique id associated with the chunk.
   private final int id;
 
+  private final ChunkType chunkType;
+
   // indicates if the chunk is formed by ChunkCreator#MemstorePool
   private final boolean fromPool;
 
@@ -60,8 +62,8 @@ public abstract class Chunk {
    * @param size in bytes
    * @param id the chunk id
    */
-  public Chunk(int size, int id) {
-    this(size, id, false);
+  public Chunk(int size, int id, ChunkType chunkType) {
+    this(size, id, chunkType, false);
   }
 
   /**
@@ -71,9 +73,10 @@ public abstract class Chunk {
    * @param id the chunk id
    * @param fromPool if the chunk is formed by pool
    */
-  public Chunk(int size, int id, boolean fromPool) {
+  public Chunk(int size, int id, ChunkType chunkType, boolean fromPool) {
     this.size = size;
     this.id = id;
+    this.chunkType = chunkType;
     this.fromPool = fromPool;
   }
 
@@ -81,16 +84,24 @@ public abstract class Chunk {
     return this.id;
   }
 
+  ChunkType getChunkType() {
+    return this.chunkType;
+  }
+
   boolean isFromPool() {
     return this.fromPool;
   }
 
   boolean isJumbo() {
-    return size > ChunkCreator.getInstance().getChunkSize();
+    return chunkType == ChunkCreator.ChunkType.JUMBO_CHUNK;
   }
 
   boolean isIndexChunk() {
-    return size == ChunkCreator.getInstance().getChunkSize(ChunkCreator.ChunkType.INDEX_CHUNK);
+    return chunkType == ChunkCreator.ChunkType.INDEX_CHUNK;
+  }
+
+  boolean isDataChunk() {
+    return chunkType == ChunkCreator.ChunkType.DATA_CHUNK;
   }
 
   /**
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ChunkCreator.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ChunkCreator.java
index d9f3271..211f782 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ChunkCreator.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ChunkCreator.java
@@ -81,6 +81,7 @@ public class ChunkCreator {
   static boolean chunkPoolDisabled = false;
   private MemStoreChunkPool dataChunksPool;
   private final int chunkSize;
+  private int indexChunkSize;
   private MemStoreChunkPool indexChunksPool;
 
   @VisibleForTesting
@@ -100,13 +101,14 @@ public class ChunkCreator {
                                HeapMemoryManager heapMemoryManager) {
     this.dataChunksPool = initializePool("data", globalMemStoreSize,
             (1 - indexChunkSizePercentage) * poolSizePercentage,
-            initialCountPercentage, chunkSize, heapMemoryManager);
+      initialCountPercentage, chunkSize, ChunkType.DATA_CHUNK, heapMemoryManager);
     // The index chunks pool is needed only when the index type is CCM.
     // Since the pools are not created at all when the index type isn't CCM,
     // we don't need to check it here.
+    this.indexChunkSize = (int) (indexChunkSizePercentage * chunkSize);
     this.indexChunksPool = initializePool("index", globalMemStoreSize,
             indexChunkSizePercentage * poolSizePercentage,
-            initialCountPercentage, (int) (indexChunkSizePercentage * chunkSize),
+      initialCountPercentage, this.indexChunkSize, ChunkType.INDEX_CHUNK,
             heapMemoryManager);
   }
 
@@ -171,14 +173,20 @@ public class ChunkCreator {
   Chunk getChunk(CompactingMemStore.IndexType chunkIndexType, ChunkType chunkType) {
     switch (chunkType) {
       case INDEX_CHUNK:
-        if (indexChunksPool != null) {
-          return getChunk(chunkIndexType, indexChunksPool.getChunkSize());
+        if (indexChunksPool == null) {
+          if (indexChunkSize <= 0) {
+            throw new IllegalArgumentException(
+                "chunkType is INDEX_CHUNK but indexChunkSize is:[" + this.indexChunkSize + "]");
+          }
+          return getChunk(chunkIndexType, chunkType, indexChunkSize);
+        } else {
+          return getChunk(chunkIndexType, chunkType, indexChunksPool.getChunkSize());
         }
       case DATA_CHUNK:
         if (dataChunksPool == null) {
-          return getChunk(chunkIndexType, chunkSize);
+          return getChunk(chunkIndexType, chunkType, chunkSize);
         } else {
-          return getChunk(chunkIndexType, dataChunksPool.getChunkSize());
+          return getChunk(chunkIndexType, chunkType, dataChunksPool.getChunkSize());
         }
       default:
         throw new IllegalArgumentException(
@@ -192,14 +200,14 @@ public class ChunkCreator {
    * @param chunkIndexType whether the requested chunk is going to be used with CellChunkMap index
    * @param size the size of the chunk to be allocated, in bytes
    */
-  Chunk getChunk(CompactingMemStore.IndexType chunkIndexType, int size) {
+  Chunk getChunk(CompactingMemStore.IndexType chunkIndexType, ChunkType chunkType, int size) {
     Chunk chunk = null;
     MemStoreChunkPool pool = null;
 
-    // if the size is suitable for one of the pools
-    if (dataChunksPool != null && size == dataChunksPool.getChunkSize()) {
+    // if it is one of the pools
+    if (dataChunksPool != null && chunkType == ChunkType.DATA_CHUNK) {
       pool = dataChunksPool;
-    } else if (indexChunksPool != null && size == indexChunksPool.getChunkSize()) {
+    } else if (indexChunksPool != null && chunkType == ChunkType.INDEX_CHUNK) {
       pool = indexChunksPool;
     }
 
@@ -219,7 +227,7 @@ public class ChunkCreator {
     if (chunk == null) {
       // the second parameter explains whether CellChunkMap index is requested,
       // in that case, put allocated on demand chunk mapping into chunkIdMap
-      chunk = createChunk(false, chunkIndexType, size);
+      chunk = createChunk(false, chunkIndexType, chunkType, size);
     }
 
     // now we need to actually do the expensive memory allocation step in case of a new chunk,
@@ -236,14 +244,15 @@ public class ChunkCreator {
    */
   Chunk getJumboChunk(int jumboSize) {
     int allocSize = jumboSize + SIZEOF_CHUNK_HEADER;
-    if (allocSize <= dataChunksPool.getChunkSize()) {
+
+    if (allocSize <= this.getChunkSize(ChunkType.DATA_CHUNK)) {
       LOG.warn("Jumbo chunk size " + jumboSize + " must be more than regular chunk size "
-              + dataChunksPool.getChunkSize() + ". Converting to regular chunk.");
+          + this.getChunkSize(ChunkType.DATA_CHUNK) + ". Converting to regular chunk.");
       return getChunk(CompactingMemStore.IndexType.CHUNK_MAP);
     }
     // the new chunk is going to hold the jumbo cell data and needs to be referenced by
     // a strong map. Therefore the CCM index type
-    return getChunk(CompactingMemStore.IndexType.CHUNK_MAP, allocSize);
+    return getChunk(CompactingMemStore.IndexType.CHUNK_MAP, ChunkType.JUMBO_CHUNK, allocSize);
   }
 
   /**
@@ -253,15 +262,16 @@ public class ChunkCreator {
    * @param size the size of the chunk to be allocated, in bytes
    * @return the chunk
    */
-  private Chunk createChunk(boolean pool, CompactingMemStore.IndexType chunkIndexType, int size) {
+  private Chunk createChunk(boolean pool, CompactingMemStore.IndexType chunkIndexType,
+      ChunkType chunkType, int size) {
     Chunk chunk = null;
     int id = chunkID.getAndIncrement();
     assert id > 0;
     // do not create offheap chunk on demand
     if (pool && this.offheap) {
-      chunk = new OffheapChunk(size, id, pool);
+      chunk = new OffheapChunk(size, id, chunkType, pool);
     } else {
-      chunk = new OnheapChunk(size, id, pool);
+      chunk = new OnheapChunk(size, id, chunkType, pool);
     }
     if (pool || (chunkIndexType == CompactingMemStore.IndexType.CHUNK_MAP)) {
       // put the pool chunk into the chunkIdMap so it is not GC-ed
@@ -272,12 +282,13 @@ public class ChunkCreator {
 
   // Chunks from pool are created covered with strong references anyway
   // TODO: change to CHUNK_MAP if it is generally defined
-  private Chunk createChunkForPool(CompactingMemStore.IndexType chunkIndexType, int chunkSize) {
+  private Chunk createChunkForPool(CompactingMemStore.IndexType chunkIndexType, ChunkType chunkType,
+      int chunkSize) {
     if (chunkSize != dataChunksPool.getChunkSize() &&
             chunkSize != indexChunksPool.getChunkSize()) {
       return null;
     }
-    return createChunk(true, chunkIndexType, chunkSize);
+    return createChunk(true, chunkIndexType, chunkType, chunkSize);
   }
 
   @VisibleForTesting
@@ -320,6 +331,7 @@ public class ChunkCreator {
    */
   private  class MemStoreChunkPool implements HeapMemoryTuneObserver {
     private final int chunkSize;
+    private final ChunkType chunkType;
     private int maxCount;
 
     // A queue of reclaimed chunks
@@ -334,15 +346,18 @@ public class ChunkCreator {
     private final LongAdder reusedChunkCount = new LongAdder();
     private final String label;
 
-    MemStoreChunkPool(String label, int chunkSize, int maxCount, int initialCount,
+    MemStoreChunkPool(String label, int chunkSize, ChunkType chunkType, int maxCount,
+        int initialCount,
         float poolSizePercentage) {
       this.label = label;
       this.chunkSize = chunkSize;
+      this.chunkType = chunkType;
       this.maxCount = maxCount;
       this.poolSizePercentage = poolSizePercentage;
       this.reclaimedChunks = new LinkedBlockingQueue<>();
       for (int i = 0; i < initialCount; i++) {
-        Chunk chunk = createChunk(true, CompactingMemStore.IndexType.ARRAY_MAP, chunkSize);
+        Chunk chunk =
+            createChunk(true, CompactingMemStore.IndexType.ARRAY_MAP, chunkType, chunkSize);
         chunk.init();
         reclaimedChunks.add(chunk);
       }
@@ -378,7 +393,7 @@ public class ChunkCreator {
           long created = this.chunkCount.get();
           if (created < this.maxCount) {
             if (this.chunkCount.compareAndSet(created, created + 1)) {
-              chunk = createChunkForPool(chunkIndexType, chunkSize);
+              chunk = createChunkForPool(chunkIndexType, chunkType, chunkSize);
               break;
             }
           } else {
@@ -477,7 +492,7 @@ public class ChunkCreator {
   }
 
   private MemStoreChunkPool initializePool(String label, long globalMemStoreSize,
-      float poolSizePercentage, float initialCountPercentage, int chunkSize,
+      float poolSizePercentage, float initialCountPercentage, int chunkSize, ChunkType chunkType,
       HeapMemoryManager heapMemoryManager) {
     if (poolSizePercentage <= 0) {
       LOG.info("{} poolSizePercentage is less than 0. So not using pool", label);
@@ -498,8 +513,8 @@ public class ChunkCreator {
     int initialCount = (int) (initialCountPercentage * maxCount);
     LOG.info("Allocating {} MemStoreChunkPool with chunk size {}, max count {}, initial count {}",
         label, StringUtils.byteDesc(chunkSize), maxCount, initialCount);
-    MemStoreChunkPool memStoreChunkPool = new MemStoreChunkPool(label, chunkSize, maxCount,
-            initialCount, poolSizePercentage);
+    MemStoreChunkPool memStoreChunkPool = new MemStoreChunkPool(label, chunkSize, chunkType,
+        maxCount, initialCount, poolSizePercentage);
     if (heapMemoryManager != null && memStoreChunkPool != null) {
       // Register with Heap Memory manager
       heapMemoryManager.registerTuneObserver(memStoreChunkPool);
@@ -596,6 +611,8 @@ public class ChunkCreator {
       case INDEX_CHUNK:
         if (indexChunksPool != null) {
           return indexChunksPool.getChunkSize();
+        } else {
+          return indexChunkSize;
         }
       case DATA_CHUNK:
         if (dataChunksPool != null) {
@@ -624,7 +641,7 @@ public class ChunkCreator {
       if (chunk != null) {
         if (chunk.isFromPool() && chunk.isIndexChunk()) {
           indexChunksPool.putbackChunks(chunk);
-        } else if (chunk.isFromPool() && chunk.size == dataChunksPool.getChunkSize()) {
+        } else if (chunk.isFromPool() && chunk.isDataChunk()) {
           dataChunksPool.putbackChunks(chunk);
         } else {
           // chunks which are not from one of the pools
@@ -639,5 +656,13 @@ public class ChunkCreator {
     return;
   }
 
+  MemStoreChunkPool getIndexChunksPool() {
+    return this.indexChunksPool;
+  }
+
+  MemStoreChunkPool getDataChunksPool() {
+    return this.dataChunksPool;
+  }
+
 }
 
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/OffheapChunk.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/OffheapChunk.java
index 255c3ad..d161397 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/OffheapChunk.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/OffheapChunk.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.regionserver;
 
 import java.nio.ByteBuffer;
 
+import org.apache.hadoop.hbase.regionserver.ChunkCreator.ChunkType;
 import org.apache.yetus.audience.InterfaceAudience;
 
 /**
@@ -27,13 +28,13 @@ import org.apache.yetus.audience.InterfaceAudience;
 @InterfaceAudience.Private
 public class OffheapChunk extends Chunk {
 
-  OffheapChunk(int size, int id) {
+  OffheapChunk(int size, int id, ChunkType chunkType) {
     // better if this is always created fromPool. This should not be called
-    super(size, id);
+    super(size, id, chunkType);
   }
 
-  OffheapChunk(int size, int id, boolean fromPool) {
-    super(size, id, fromPool);
+  OffheapChunk(int size, int id, ChunkType chunkType, boolean fromPool) {
+    super(size, id, chunkType, fromPool);
     assert fromPool == true;
   }
 
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/OnheapChunk.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/OnheapChunk.java
index 0522648..5ccb7b1 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/OnheapChunk.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/OnheapChunk.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.regionserver;
 
 import java.nio.ByteBuffer;
 
+import org.apache.hadoop.hbase.regionserver.ChunkCreator.ChunkType;
 import org.apache.yetus.audience.InterfaceAudience;
 
 /**
@@ -27,12 +28,12 @@ import org.apache.yetus.audience.InterfaceAudience;
 @InterfaceAudience.Private
 public class OnheapChunk extends Chunk {
 
-  OnheapChunk(int size, int id) {
-    super(size, id);
+  OnheapChunk(int size, int id, ChunkType chunkType) {
+    super(size, id, chunkType);
   }
 
-  OnheapChunk(int size, int id, boolean fromPool) {
-    super(size, id, fromPool);
+  OnheapChunk(int size, int id, ChunkType chunkType, boolean fromPool) {
+    super(size, id, chunkType, fromPool);
   }
 
   @Override
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCellFlatSet.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCellFlatSet.java
index 4c916cb..5d0dda9 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCellFlatSet.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCellFlatSet.java
@@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.io.util.MemorySizeUtil;
+import org.apache.hadoop.hbase.regionserver.ChunkCreator.ChunkType;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.apache.hadoop.hbase.util.ByteBufferUtils;
@@ -330,7 +331,9 @@ public class TestCellFlatSet {
     // allocate new chunks and use the data JUMBO chunk to hold the full data of the cells
     // and the normal index chunk to hold the cell-representations
     Chunk dataJumboChunk =
-        chunkCreator.getChunk(CompactingMemStore.IndexType.CHUNK_MAP, smallChunkSize);
+        chunkCreator.getChunk(CompactingMemStore.IndexType.CHUNK_MAP, ChunkType.JUMBO_CHUNK,
+          smallChunkSize);
+    assertTrue(dataJumboChunk.isJumbo());
     Chunk idxChunk  = chunkCreator.getChunk(CompactingMemStore.IndexType.CHUNK_MAP);
     // the array of index chunks to be used as a basis for CellChunkMap
     Chunk[] chunkArray = new Chunk[8];  // according to test currently written 8 is way enough
@@ -364,7 +367,10 @@ public class TestCellFlatSet {
 
       // Jumbo chunks are working only with one cell per chunk, thus always allocate a new jumbo
       // data chunk for next cell
-      dataJumboChunk = chunkCreator.getChunk(CompactingMemStore.IndexType.CHUNK_MAP,smallChunkSize);
+      dataJumboChunk =
+          chunkCreator.getChunk(CompactingMemStore.IndexType.CHUNK_MAP, ChunkType.JUMBO_CHUNK,
+            smallChunkSize);
+      assertTrue(dataJumboChunk.isJumbo());
       dataBuffer = dataJumboChunk.getData();
       dataOffset = ChunkCreator.SIZEOF_CHUNK_HEADER;
     }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreChunkPool.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreChunkPool.java
index e7f88ff..b2cd492 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreChunkPool.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreChunkPool.java
@@ -20,18 +20,23 @@ package org.apache.hadoop.hbase.regionserver;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 import java.io.IOException;
 import java.lang.management.ManagementFactory;
 import java.nio.ByteBuffer;
+import java.util.Collections;
 import java.util.List;
 import java.util.Random;
+import java.util.concurrent.atomic.AtomicReference;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.ByteBufferKeyValue;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.exceptions.UnexpectedStateException;
 import org.apache.hadoop.hbase.io.util.MemorySizeUtil;
+import org.apache.hadoop.hbase.regionserver.ChunkCreator.ChunkType;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -237,22 +242,30 @@ public class TestMemStoreChunkPool {
     ChunkCreator newCreator = new ChunkCreator(chunkSize, false, 400, 1, 0.5f, null, 0);
     assertEquals(initialCount, newCreator.getPoolSize());
     assertEquals(maxCount, newCreator.getMaxCount());
-    ChunkCreator.instance = newCreator;// Replace the global ref with the new one we created.
-                                             // Used it for the testing. Later in finally we put
-                                             // back the original
+    // Replace the global ref with the new one we created.
+    // Used it for the testing. Later in finally we put
+    // back the original
+    ChunkCreator.instance = newCreator;
+
     final KeyValue kv = new KeyValue(Bytes.toBytes("r"), Bytes.toBytes("f"), Bytes.toBytes("q"),
         new byte[valSize]);
+    final AtomicReference<Throwable> exceptionRef = new AtomicReference<Throwable>();
     try {
       Runnable r = new Runnable() {
         @Override
         public void run() {
-          MemStoreLAB memStoreLAB = new MemStoreLABImpl(conf);
-          for (int i = 0; i < maxCount; i++) {
-            memStoreLAB.copyCellInto(kv);// Try allocate size = chunkSize. Means every
-                                         // allocate call will result in a new chunk
+          try {
+            MemStoreLAB memStoreLAB = new MemStoreLABImpl(conf);
+            for (int i = 0; i < maxCount; i++) {
+              // Try allocate size = chunkSize. Means every
+              // allocate call will result in a new chunk
+              memStoreLAB.copyCellInto(kv);
+            }
+            // Close MemStoreLAB so that all chunks will be tried to be put back to pool
+            memStoreLAB.close();
+          } catch (Throwable execption) {
+            exceptionRef.set(execption);
           }
-          // Close MemStoreLAB so that all chunks will be tried to be put back to pool
-          memStoreLAB.close();
         }
       };
       Thread t1 = new Thread(r);
@@ -264,9 +277,154 @@ public class TestMemStoreChunkPool {
       t1.join();
       t2.join();
       t3.join();
-      assertTrue(newCreator.getPoolSize() <= maxCount);
+      assertTrue(exceptionRef.get() == null);
+      assertTrue(newCreator.getPoolSize() <= maxCount && newCreator.getPoolSize() > 0);
     } finally {
       ChunkCreator.instance = oldCreator;
     }
   }
+
+  // This test is for HBASE-26142, which throws NPE when indexChunksPool is null.
+  @Test
+  public void testNoIndexChunksPoolOrNoDataChunksPool() throws Exception {
+    final int maxCount = 10;
+    final int initialCount = 5;
+    final int newChunkSize = 40;
+    final int valSize = 7;
+
+    ChunkCreator oldCreator = ChunkCreator.getInstance();
+    try {
+      // Test dataChunksPool is not null and indexChunksPool is null
+      ChunkCreator newCreator = new ChunkCreator(newChunkSize, false, 400, 1, 0.5f, null, 0);
+      assertEquals(initialCount, newCreator.getPoolSize());
+      assertEquals(0, newCreator.getPoolSize(ChunkType.INDEX_CHUNK));
+      assertEquals(maxCount, newCreator.getMaxCount());
+      assertEquals(0, newCreator.getMaxCount(ChunkType.INDEX_CHUNK));
+      assertTrue(newCreator.getDataChunksPool() != null);
+      assertTrue(newCreator.getIndexChunksPool() == null);
+      ChunkCreator.instance = newCreator;
+      final KeyValue kv = new KeyValue(Bytes.toBytes("r"), Bytes.toBytes("f"), Bytes.toBytes("q"),
+          new byte[valSize]);
+
+      MemStoreLAB memStoreLAB = new MemStoreLABImpl(conf);
+      memStoreLAB.copyCellInto(kv);
+      memStoreLAB.close();
+      assertEquals(initialCount, newCreator.getPoolSize());
+      assertEquals(0, newCreator.getPoolSize(ChunkType.INDEX_CHUNK));
+
+      Chunk dataChunk = newCreator.getChunk(CompactingMemStore.IndexType.CHUNK_MAP);
+      assertTrue(dataChunk.isDataChunk());
+      assertTrue(dataChunk.isFromPool());
+      assertEquals(initialCount - 1, newCreator.getPoolSize());
+      assertEquals(0, newCreator.getPoolSize(ChunkType.INDEX_CHUNK));
+      newCreator.putbackChunks(Collections.singleton(dataChunk.getId()));
+      assertEquals(initialCount, newCreator.getPoolSize());
+      assertEquals(0, newCreator.getPoolSize(ChunkType.INDEX_CHUNK));
+
+      // We set ChunkCreator.indexChunkSize to 0, but we want to get a IndexChunk
+      try {
+        newCreator.getChunk(CompactingMemStore.IndexType.CHUNK_MAP, ChunkType.INDEX_CHUNK);
+        fail();
+      } catch (IllegalArgumentException e) {
+      }
+
+      Chunk jumboChunk = newCreator.getJumboChunk(newChunkSize + 10);
+      assertTrue(jumboChunk.isJumbo());
+      assertTrue(!jumboChunk.isFromPool());
+      assertEquals(initialCount, newCreator.getPoolSize());
+      assertEquals(0, newCreator.getPoolSize(ChunkType.INDEX_CHUNK));
+
+      // Test both dataChunksPool and indexChunksPool are null
+      newCreator = new ChunkCreator(newChunkSize, false, 400, 0, 0.5f, null, 0);
+      assertEquals(0, newCreator.getPoolSize());
+      assertEquals(0, newCreator.getPoolSize(ChunkType.INDEX_CHUNK));
+      assertEquals(0, newCreator.getMaxCount());
+      assertEquals(0, newCreator.getMaxCount(ChunkType.INDEX_CHUNK));
+      assertTrue(newCreator.getDataChunksPool() == null);
+      assertTrue(newCreator.getIndexChunksPool() == null);
+      ChunkCreator.instance = newCreator;
+
+      memStoreLAB = new MemStoreLABImpl(conf);
+      memStoreLAB.copyCellInto(kv);
+      memStoreLAB.close();
+      assertEquals(0, newCreator.getPoolSize());
+      assertEquals(0, newCreator.getPoolSize(ChunkType.INDEX_CHUNK));
+
+      dataChunk = newCreator.getChunk(CompactingMemStore.IndexType.CHUNK_MAP);
+      assertTrue(dataChunk.isDataChunk());
+      assertTrue(!dataChunk.isFromPool());
+      assertEquals(0, newCreator.getPoolSize());
+      assertEquals(0, newCreator.getPoolSize(ChunkType.INDEX_CHUNK));
+
+      try {
+        // We set ChunkCreator.indexChunkSize to 0, but we want to get a IndexChunk
+        newCreator.getChunk(CompactingMemStore.IndexType.CHUNK_MAP, ChunkType.INDEX_CHUNK);
+        fail();
+      } catch (IllegalArgumentException e) {
+      }
+
+      jumboChunk = newCreator.getJumboChunk(newChunkSize + 10);
+      assertTrue(jumboChunk.isJumbo());
+      assertTrue(!jumboChunk.isFromPool());
+      assertEquals(0, newCreator.getPoolSize());
+      assertEquals(0, newCreator.getPoolSize(ChunkType.INDEX_CHUNK));
+
+      // Test dataChunksPool is null and indexChunksPool is not null
+      newCreator = new ChunkCreator(newChunkSize, false, 400, 1, 0.5f, null, 1);
+      assertEquals(0, newCreator.getPoolSize());
+      assertEquals(initialCount, newCreator.getPoolSize(ChunkType.INDEX_CHUNK));
+      assertEquals(0, newCreator.getMaxCount());
+      assertEquals(maxCount, newCreator.getMaxCount(ChunkType.INDEX_CHUNK));
+      assertTrue(newCreator.getDataChunksPool() == null);
+      assertTrue(newCreator.getIndexChunksPool() != null);
+      assertEquals(newCreator.getChunkSize(ChunkType.DATA_CHUNK),
+        newCreator.getChunkSize(ChunkType.INDEX_CHUNK));
+      ChunkCreator.instance = newCreator;
+
+      memStoreLAB = new MemStoreLABImpl(conf);
+      memStoreLAB.copyCellInto(kv);
+      memStoreLAB.close();
+      assertEquals(0, newCreator.getPoolSize());
+      assertEquals(initialCount, newCreator.getPoolSize(ChunkType.INDEX_CHUNK));
+
+      dataChunk = newCreator.getChunk(CompactingMemStore.IndexType.CHUNK_MAP);
+      assertTrue(dataChunk.isDataChunk());
+      assertTrue(!dataChunk.isFromPool());
+      assertEquals(0, newCreator.getPoolSize());
+      assertEquals(initialCount, newCreator.getPoolSize(ChunkType.INDEX_CHUNK));
+
+      Chunk indexChunk =
+          newCreator.getChunk(CompactingMemStore.IndexType.CHUNK_MAP, ChunkType.INDEX_CHUNK);
+      assertEquals(0, newCreator.getPoolSize());
+      assertEquals(initialCount - 1, newCreator.getPoolSize(ChunkType.INDEX_CHUNK));
+      assertTrue(indexChunk.isIndexChunk());
+      assertTrue(indexChunk.isFromPool());
+      newCreator.putbackChunks(Collections.singleton(indexChunk.getId()));
+      assertEquals(0, newCreator.getPoolSize());
+      assertEquals(initialCount, newCreator.getPoolSize(ChunkType.INDEX_CHUNK));
+
+      jumboChunk = newCreator.getJumboChunk(newChunkSize + 10);
+      assertTrue(jumboChunk.isJumbo());
+      assertTrue(!jumboChunk.isFromPool());
+      assertEquals(0, newCreator.getPoolSize());
+      assertEquals(initialCount, newCreator.getPoolSize(ChunkType.INDEX_CHUNK));
+    } finally {
+      ChunkCreator.instance = oldCreator;
+    }
+
+    // Test both dataChunksPool and indexChunksPool are not null
+    assertTrue(ChunkCreator.getInstance().getDataChunksPool() != null);
+    assertTrue(ChunkCreator.getInstance().getIndexChunksPool() != null);
+    Chunk dataChunk = ChunkCreator.getInstance().getChunk(CompactingMemStore.IndexType.CHUNK_MAP);
+    assertTrue(dataChunk.isDataChunk());
+    assertTrue(dataChunk.isFromPool());
+    Chunk indexChunk = ChunkCreator.getInstance().getChunk(CompactingMemStore.IndexType.CHUNK_MAP,
+      ChunkType.INDEX_CHUNK);
+    assertTrue(indexChunk.isIndexChunk());
+    assertTrue(indexChunk.isFromPool());
+    Chunk jumboChunk =
+        ChunkCreator.getInstance().getJumboChunk(ChunkCreator.getInstance().getChunkSize() + 10);
+    assertTrue(jumboChunk.isJumbo());
+    assertTrue(!jumboChunk.isFromPool());
+  }
 }