HBASE-28401 Introduce a close method for memstore for release active segment (#5705)

Signed-off-by: Bryan Beaudreault <bbeaudreault@apache.org>
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AbstractMemStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AbstractMemStore.java
index 62ff6f9..9a88cab 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AbstractMemStore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AbstractMemStore.java
@@ -376,6 +376,15 @@
     return snapshot;
   }
 
+  @Override
+  public void close() {
+    // active should never be null
+    active.close();
+    // for snapshot, either it is empty, where we do not reference any real segment which contains a
+    // memstore lab, or it is during snapshot, where we will clear it when calling clearSnapshot, so
+    // we do not need to close it here
+  }
+
   /** Returns an ordered list of segments from most recent to oldest in memstore */
   protected abstract List<Segment> getSegments() throws IOException;
 
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
index dccfd0c..43a6335 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
@@ -726,6 +726,7 @@
   }
 
   private ImmutableCollection<HStoreFile> closeWithoutLock() throws IOException {
+    memstore.close();
     // Clear so metrics doesn't find them.
     ImmutableCollection<HStoreFile> result = storeEngine.getStoreFileManager().clearFiles();
     Collection<HStoreFile> compactedfiles = storeEngine.getStoreFileManager().clearCompactedFiles();
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStore.java
index 947944b..cd8eecd 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStore.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hbase.regionserver;
 
+import java.io.Closeable;
 import java.io.IOException;
 import java.util.List;
 import org.apache.hadoop.hbase.Cell;
@@ -31,7 +32,7 @@
  * </p>
  */
 @InterfaceAudience.Private
-public interface MemStore {
+public interface MemStore extends Closeable {
 
   /**
    * Creates a snapshot of the current memstore. Snapshot must be cleared by call to
@@ -131,4 +132,15 @@
   default void stopReplayingFromWAL() {
     return;
   }
+
+  /**
+   * Close the memstore.
+   * <p>
+   * Usually this should only be called when there is nothing in the memstore, unless we are going
+   * to abort ourselves.
+   * <p>
+   * For normal cases, this method is only used to fix the reference counting, see HBASE-27941.
+   */
+  @Override
+  void close();
 }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtil.java
index 8f78161..79b041f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtil.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtil.java
@@ -1699,6 +1699,8 @@
    */
   public HRegion createLocalHRegion(RegionInfo info, Configuration conf, TableDescriptor desc,
     WAL wal) throws IOException {
+    ChunkCreator.initialize(MemStoreLAB.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null,
+      MemStoreLAB.INDEX_CHUNK_SIZE_PERCENTAGE_DEFAULT);
     return HRegion.createHRegion(info, getDataTestDir(), conf, desc, wal);
   }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoreRegionCoprocessor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoreRegionCoprocessor.java
index 6b52ce4..e569457 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoreRegionCoprocessor.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoreRegionCoprocessor.java
@@ -32,7 +32,9 @@
 import org.apache.hadoop.hbase.client.RegionInfoBuilder;
 import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
+import org.apache.hadoop.hbase.regionserver.ChunkCreator;
 import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.regionserver.MemStoreLAB;
 import org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost;
 import org.apache.hadoop.hbase.regionserver.RegionServerServices;
 import org.apache.hadoop.hbase.testclassification.CoprocessorTests;
@@ -72,6 +74,8 @@
     TableDescriptor td = TableDescriptorBuilder.newBuilder(tn).setColumnFamily(cfd).build();
     RegionInfo ri = RegionInfoBuilder.newBuilder(tn).build();
     this.rss = new MockRegionServerServices(HTU.getConfiguration());
+    ChunkCreator.initialize(MemStoreLAB.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null,
+      MemStoreLAB.INDEX_CHUNK_SIZE_PERCENTAGE_DEFAULT);
     this.region = HRegion.openHRegion(ri, td, null, HTU.getConfiguration(), this.rss, null);
   }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java
index 0379520..85cee07 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java
@@ -53,7 +53,6 @@
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
-import org.apache.hadoop.hbase.wal.WALFactory;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.ClassRule;
@@ -134,7 +133,6 @@
   private final String testDescription;
   private HRegion region;
   private HStore store;
-  private WALFactory walFactory;
   private FileSystem fs;
 
   public TestCacheOnWriteInSchema(CacheOnWriteType cowType) {
@@ -179,24 +177,17 @@
     fs.delete(logdir, true);
 
     RegionInfo info = RegionInfoBuilder.newBuilder(htd.getTableName()).build();
-    walFactory = new WALFactory(conf, id);
 
-    region = TEST_UTIL.createLocalHRegion(info, conf, htd, walFactory.getWAL(info));
-    region.setBlockCache(BlockCacheFactory.createBlockCache(conf));
-    store = new HStore(region, hcd, conf, false);
+    region = HBaseTestingUtil.createRegionAndWAL(info, logdir, conf, htd,
+      BlockCacheFactory.createBlockCache(conf));
+    store = region.getStore(hcd.getName());
   }
 
   @After
   public void tearDown() throws IOException {
     IOException ex = null;
     try {
-      region.close();
-    } catch (IOException e) {
-      LOG.warn("Caught Exception", e);
-      ex = e;
-    }
-    try {
-      walFactory.close();
+      HBaseTestingUtil.closeRegionAndWAL(region);
     } catch (IOException e) {
       LOG.warn("Caught Exception", e);
       ex = e;
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
index d244ca7..abeec8a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
@@ -447,32 +447,24 @@
 
   /**
    * A test case of HBASE-21041
-   * @throws Exception Exception
    */
   @Test
   public void testFlushAndMemstoreSizeCounting() throws Exception {
     byte[] family = Bytes.toBytes("family");
     this.region = initHRegion(tableName, method, CONF, family);
-    final WALFactory wals = new WALFactory(CONF, method);
-    try {
-      for (byte[] row : HBaseTestingUtil.ROWS) {
-        Put put = new Put(row);
-        put.addColumn(family, family, row);
-        region.put(put);
-      }
-      region.flush(true);
-      // After flush, data size should be zero
-      assertEquals(0, region.getMemStoreDataSize());
-      // After flush, a new active mutable segment is created, so the heap size
-      // should equal to MutableSegment.DEEP_OVERHEAD
-      assertEquals(MutableSegment.DEEP_OVERHEAD, region.getMemStoreHeapSize());
-      // After flush, offheap should be zero
-      assertEquals(0, region.getMemStoreOffHeapSize());
-    } finally {
-      HBaseTestingUtil.closeRegionAndWAL(this.region);
-      this.region = null;
-      wals.close();
+    for (byte[] row : HBaseTestingUtil.ROWS) {
+      Put put = new Put(row);
+      put.addColumn(family, family, row);
+      region.put(put);
     }
+    region.flush(true);
+    // After flush, data size should be zero
+    assertEquals(0, region.getMemStoreDataSize());
+    // After flush, a new active mutable segment is created, so the heap size
+    // should equal to MutableSegment.DEEP_OVERHEAD
+    assertEquals(MutableSegment.DEEP_OVERHEAD, region.getMemStoreHeapSize());
+    // After flush, offheap should be zero
+    assertEquals(0, region.getMemStoreOffHeapSize());
   }
 
   /**
@@ -1283,6 +1275,12 @@
     // throwing a DroppedSnapshotException to force an abort. Just clean up the mess.
     region.close(true);
     wal.close();
+    // release the snapshot and active segment, so netty will not report memory leak
+    for (HStore store : region.getStores()) {
+      AbstractMemStore memstore = (AbstractMemStore) store.memstore;
+      memstore.doClearSnapShot();
+      memstore.close();
+    }
 
     // 2. Test case where START_FLUSH succeeds but COMMIT_FLUSH will throw exception
     wal.flushActions = new FlushAction[] { FlushAction.COMMIT_FLUSH };
@@ -1297,15 +1295,18 @@
     // DroppedSnapshotException. Below COMMIT_FLUSH will cause flush to abort
     wal.flushActions = new FlushAction[] { FlushAction.COMMIT_FLUSH, FlushAction.ABORT_FLUSH };
 
-    try {
-      region.flush(true);
-      fail("This should have thrown exception");
-    } catch (DroppedSnapshotException expected) {
-      // we expect this exception, since we were able to write the snapshot, but failed to
-      // write the flush marker to WAL
-    } catch (IOException unexpected) {
-      throw unexpected;
+    // we expect this exception, since we were able to write the snapshot, but failed to
+    // write the flush marker to WAL
+    assertThrows(DroppedSnapshotException.class, () -> region.flush(true));
+
+    region.close(true);
+    // release the snapshot and active segment, so netty will not report memory leak
+    for (HStore store : region.getStores()) {
+      AbstractMemStore memstore = (AbstractMemStore) store.memstore;
+      memstore.doClearSnapShot();
+      memstore.close();
     }
+    region = null;
   }
 
   @Test
@@ -3735,14 +3736,14 @@
     byte[][] families = { fam1, fam2 };
 
     // Setting up region
-    try {
-      this.region = initHRegion(tableName, method, CONF, families);
-    } catch (IOException e) {
-      e.printStackTrace();
-      fail("Got IOException during initHRegion, " + e.getMessage());
-    }
+    region = initHRegion(tableName, method, CONF, families);
     region.closed.set(true);
-    assertThrows(NotServingRegionException.class, () -> region.getScanner(null));
+    try {
+      assertThrows(NotServingRegionException.class, () -> region.getScanner(null));
+    } finally {
+      // so we can close the region in tearDown
+      region.closed.set(false);
+    }
   }
 
   @Test
@@ -4543,14 +4544,14 @@
   /**
    * So can be overridden in subclasses.
    */
-  int getNumQualifiersForTestWritesWhileScanning() {
+  protected int getNumQualifiersForTestWritesWhileScanning() {
     return 100;
   }
 
   /**
    * So can be overridden in subclasses.
    */
-  int getTestCountForTestWritesWhileScanning() {
+  protected int getTestCountForTestWritesWhileScanning() {
     return 100;
   }
 
@@ -5829,12 +5830,12 @@
    * @return A region on which you must call {@link HBaseTestingUtil#closeRegionAndWAL(HRegion)}
    *         when done.
    */
-  protected HRegion initHRegion(TableName tableName, String callingMethod, Configuration conf,
+  private HRegion initHRegion(TableName tableName, String callingMethod, Configuration conf,
     boolean isReadOnly, byte[]... families) throws IOException {
     return initHRegion(tableName, null, null, callingMethod, conf, isReadOnly, families);
   }
 
-  protected HRegion initHRegion(TableName tableName, byte[] startKey, byte[] stopKey,
+  private HRegion initHRegion(TableName tableName, byte[] startKey, byte[] stopKey,
     String callingMethod, Configuration conf, boolean isReadOnly, byte[]... families)
     throws IOException {
     Path logDir = TEST_UTIL.getDataTestDirOnTestFS(callingMethod + ".log");
@@ -5849,7 +5850,7 @@
    * @return A region on which you must call {@link HBaseTestingUtil#closeRegionAndWAL(HRegion)}
    *         when done.
    */
-  public HRegion initHRegion(TableName tableName, byte[] startKey, byte[] stopKey,
+  protected HRegion initHRegion(TableName tableName, byte[] startKey, byte[] stopKey,
     Configuration conf, boolean isReadOnly, Durability durability, WAL wal, byte[]... families)
     throws IOException {
     ChunkCreator.initialize(MemStoreLAB.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null,
@@ -6708,14 +6709,12 @@
     WAL wal = mockWAL();
     when(rss.getWAL(any(RegionInfo.class))).thenReturn(wal);
 
-    // create and then open a region first so that it can be closed later
-    region =
-      HRegion.createHRegion(hri, rootDir, TEST_UTIL.getConfiguration(), htd, rss.getWAL(hri));
-    region =
-      HRegion.openHRegion(hri, htd, rss.getWAL(hri), TEST_UTIL.getConfiguration(), rss, null);
-
-    // close the region
-    region.close(false);
+    // create the region
+    region = HBaseTestingUtil.createRegionAndWAL(hri, rootDir, CONF, htd);
+    HBaseTestingUtil.closeRegionAndWAL(region);
+    region = null;
+    // open the region first and then close it
+    HRegion.openHRegion(hri, htd, rss.getWAL(hri), TEST_UTIL.getConfiguration(), rss, null).close();
 
     // 2 times, one for region open, the other close region
     verify(wal, times(2)).appendMarker(any(RegionInfo.class), (WALKeyImpl) any(WALKeyImpl.class),
@@ -7249,7 +7248,7 @@
       qual2.length));
   }
 
-  HRegion initHRegion(TableName tableName, String callingMethod, byte[]... families)
+  private HRegion initHRegion(TableName tableName, String callingMethod, byte[]... families)
     throws IOException {
     return initHRegion(tableName, callingMethod, HBaseConfiguration.create(), families);
   }
@@ -7727,12 +7726,7 @@
 
     holder.start();
     latch.await();
-    try {
-      region.close();
-    } catch (IOException e) {
-      LOG.info("Caught expected exception", e);
-    }
-    region = null;
+    assertThrows(IOException.class, () -> region.close());
     holder.join();
 
     // Verify the region tried to abort the server
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionWithInMemoryFlush.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionWithInMemoryFlush.java
index f43d265..27bdae4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionWithInMemoryFlush.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionWithInMemoryFlush.java
@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.hbase.regionserver;
 
+import static org.junit.Assert.assertEquals;
+
 import java.io.IOException;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
@@ -28,8 +30,6 @@
 import org.apache.hadoop.hbase.testclassification.VerySlowRegionServerTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.wal.WAL;
-import org.apache.hadoop.hbase.wal.WALFactory;
-import org.junit.Assert;
 import org.junit.ClassRule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -50,7 +50,7 @@
    *         when done.
    */
   @Override
-  public HRegion initHRegion(TableName tableName, byte[] startKey, byte[] stopKey,
+  protected HRegion initHRegion(TableName tableName, byte[] startKey, byte[] stopKey,
     Configuration conf, boolean isReadOnly, Durability durability, WAL wal, byte[]... families)
     throws IOException {
     boolean[] inMemory = new boolean[families.length];
@@ -64,7 +64,7 @@
   }
 
   @Override
-  int getTestCountForTestWritesWhileScanning() {
+  protected int getTestCountForTestWritesWhileScanning() {
     return 10;
   }
 
@@ -73,44 +73,35 @@
    * easy on it. See if that helps.
    */
   @Override
-  int getNumQualifiersForTestWritesWhileScanning() {
+  protected int getNumQualifiersForTestWritesWhileScanning() {
     return 10;
   }
 
   /**
    * A test case of HBASE-21041
-   * @throws Exception Exception
    */
   @Override
   @Test
   public void testFlushAndMemstoreSizeCounting() throws Exception {
     byte[] family = Bytes.toBytes("family");
     this.region = initHRegion(tableName, method, CONF, family);
-    final WALFactory wals = new WALFactory(CONF, method);
     int count = 0;
-    try {
-      for (byte[] row : HBaseTestingUtil.ROWS) {
-        Put put = new Put(row);
-        put.addColumn(family, family, row);
-        region.put(put);
-        // In memory flush every 1000 puts
-        if (count++ % 1000 == 0) {
-          ((CompactingMemStore) (region.getStore(family).memstore)).flushInMemory();
-        }
+    for (byte[] row : HBaseTestingUtil.ROWS) {
+      Put put = new Put(row);
+      put.addColumn(family, family, row);
+      region.put(put);
+      // In memory flush every 1000 puts
+      if (count++ % 1000 == 0) {
+        ((CompactingMemStore) (region.getStore(family).memstore)).flushInMemory();
       }
-      region.flush(true);
-      // After flush, data size should be zero
-      Assert.assertEquals(0, region.getMemStoreDataSize());
-      // After flush, a new active mutable segment is created, so the heap size
-      // should equal to MutableSegment.DEEP_OVERHEAD
-      Assert.assertEquals(MutableSegment.DEEP_OVERHEAD, region.getMemStoreHeapSize());
-      // After flush, offheap size should be zero
-      Assert.assertEquals(0, region.getMemStoreOffHeapSize());
-
-    } finally {
-      HBaseTestingUtil.closeRegionAndWAL(this.region);
-      this.region = null;
-      wals.close();
     }
+    region.flush(true);
+    // After flush, data size should be zero
+    assertEquals(0, region.getMemStoreDataSize());
+    // After flush, a new active mutable segment is created, so the heap size
+    // should equal to MutableSegment.DEEP_OVERHEAD
+    assertEquals(MutableSegment.DEEP_OVERHEAD, region.getMemStoreHeapSize());
+    // After flush, offheap size should be zero
+    assertEquals(0, region.getMemStoreOffHeapSize());
   }
 }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreSegmentsIterator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreSegmentsIterator.java
index e75dc5e..6f5ef2c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreSegmentsIterator.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreSegmentsIterator.java
@@ -42,7 +42,6 @@
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManagerTestHelper;
-import org.apache.hadoop.hbase.wal.WAL;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.ClassRule;
@@ -60,18 +59,17 @@
   public static final HBaseClassTestRule CLASS_RULE =
     HBaseClassTestRule.forClass(TestMemStoreSegmentsIterator.class);
 
-  protected static String TABLE = "test_mscsi";
-  protected static String FAMILY = "f";
-  protected static String COLUMN = "c";
-  protected static String ROOT_SUB_PATH = "testMemStoreSegmentsIterator";
-  protected static long LESS_THAN_INTEGER_MAX_VALUE_SEQ_ID = Long.valueOf(Integer.MAX_VALUE) - 1;
-  protected static long GREATER_THAN_INTEGER_MAX_VALUE_SEQ_ID = Long.valueOf(Integer.MAX_VALUE) + 1;
+  private static String TABLE = "test_mscsi";
+  private static String FAMILY = "f";
+  private static String COLUMN = "c";
+  private static String ROOT_SUB_PATH = "testMemStoreSegmentsIterator";
+  private static long LESS_THAN_INTEGER_MAX_VALUE_SEQ_ID = Long.valueOf(Integer.MAX_VALUE) - 1;
+  private static long GREATER_THAN_INTEGER_MAX_VALUE_SEQ_ID = Long.valueOf(Integer.MAX_VALUE) + 1;
 
-  protected CellComparator comparator;
-  protected int compactionKVMax;
-  protected WAL wal;
-  protected HRegion region;
-  protected HStore store;
+  private CellComparator comparator;
+  private int compactionKVMax;
+  private HRegion region;
+  private HStore store;
 
   @Before
   public void setup() throws IOException {
@@ -85,10 +83,9 @@
 
     RegionInfo info = RegionInfoBuilder.newBuilder(TableName.valueOf(TABLE)).build();
     Path rootPath = hbaseUtility.getDataTestDir(ROOT_SUB_PATH);
-    this.wal = HBaseTestingUtil.createWal(conf, rootPath, info);
     this.region =
-      HRegion.createHRegion(info, rootPath, conf, tableDescriptorBuilder.build(), this.wal, true);
-    this.store = new HStore(this.region, columnFamilyDescriptor, conf, false);
+      HBaseTestingUtil.createRegionAndWAL(info, rootPath, conf, tableDescriptorBuilder.build());
+    this.store = region.getStore(columnFamilyDescriptor.getName());
     this.comparator = CellComparator.getInstance();
     this.compactionKVMax = HConstants.COMPACTION_KV_MAX_DEFAULT;
   }
@@ -150,21 +147,8 @@
   @After
   public void tearDown() throws Exception {
     EnvironmentEdgeManagerTestHelper.reset();
-    if (store != null) {
-      try {
-        store.close();
-      } catch (IOException e) {
-      }
-      store = null;
-    }
     if (region != null) {
-      region.close();
-      region = null;
-    }
-
-    if (wal != null) {
-      wal.close();
-      wal = null;
+      HBaseTestingUtil.closeRegionAndWAL(region);
     }
   }
 }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/WALPerformanceEvaluation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/WALPerformanceEvaluation.java
index fcbbb3c..c462e2e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/WALPerformanceEvaluation.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/WALPerformanceEvaluation.java
@@ -56,6 +56,7 @@
 import org.apache.hadoop.hbase.io.crypto.KeyProviderForTesting;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.LogRoller;
+import org.apache.hadoop.hbase.regionserver.MemStoreLAB;
 import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl;
 import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
 import org.apache.hadoop.hbase.trace.TraceUtil;
@@ -266,6 +267,9 @@
     // Internal config. goes off number of threads; if more threads than handlers, stuff breaks.
     // In regionserver, number of handlers == number of threads.
     getConf().setInt(HConstants.REGION_SERVER_HANDLER_COUNT, numThreads);
+    // We do not need memstore here, so disable memstore lab, otherwise we need to initialize
+    // ChunkCreator
+    getConf().setBoolean(MemStoreLAB.USEMSLAB_KEY, false);
 
     if (rootRegionDir == null) {
       TEST_UTIL = new HBaseTestingUtil(getConf());