You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ap...@apache.org on 2019/01/25 18:16:44 UTC

[hbase] branch branch-1.4 updated: HBASE-21748 Port HBASE-21738 (Remove all the CLSM#size operation in our memstore because it's an quite time consuming) to branch-1

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

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


The following commit(s) were added to refs/heads/branch-1.4 by this push:
     new e022b78  HBASE-21748 Port HBASE-21738 (Remove all the CLSM#size operation in our memstore because it's an quite time consuming) to branch-1
e022b78 is described below

commit e022b78bada8de779246c5d5079f27eaa4c17cda
Author: Andrew Purtell <ap...@apache.org>
AuthorDate: Wed Jan 23 18:02:02 2019 -0800

    HBASE-21748 Port HBASE-21738 (Remove all the CLSM#size operation in our memstore because it's an quite time consuming) to branch-1
---
 .../hadoop/hbase/regionserver/CellSkipListSet.java | 10 +++++
 .../hadoop/hbase/regionserver/DefaultMemStore.java | 21 ++++++++--
 .../org/apache/hadoop/hbase/io/TestHeapSize.java   |  4 +-
 .../hbase/regionserver/TestCellSkipListSet.java    |  4 +-
 .../hbase/regionserver/TestDefaultMemStore.java    | 49 +++++++++++-----------
 .../hbase/regionserver/TestMemStoreChunkPool.java  | 12 +++---
 .../hadoop/hbase/regionserver/TestStore.java       |  9 ++--
 7 files changed, 69 insertions(+), 40 deletions(-)

diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellSkipListSet.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellSkipListSet.java
index 916a428..f02294d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellSkipListSet.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellSkipListSet.java
@@ -18,6 +18,8 @@
  */
 package org.apache.hadoop.hbase.regionserver;
 
+import com.google.common.annotations.VisibleForTesting;
+
 import java.util.Collection;
 import java.util.Comparator;
 import java.util.Iterator;
@@ -172,6 +174,9 @@ public class CellSkipListSet implements NavigableSet<Cell> {
   }
 
   public int size() {
+    if (delegatee instanceof ConcurrentSkipListMap) {
+      throw new UnsupportedOperationException("ConcurrentSkipListMap.size() is time-consuming");
+    }
     return this.delegatee.size();
   }
 
@@ -182,4 +187,9 @@ public class CellSkipListSet implements NavigableSet<Cell> {
   public <T> T[] toArray(T[] a) {
     throw new UnsupportedOperationException("Not implemented");
   }
+
+  @VisibleForTesting
+  int sizeForTests() {
+    return this.delegatee.size();
+  }
 }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultMemStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultMemStore.java
index 299c998..32bfbd5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultMemStore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultMemStore.java
@@ -27,6 +27,7 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.NavigableSet;
 import java.util.SortedSet;
+import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.commons.logging.Log;
@@ -132,7 +133,7 @@ public class DefaultMemStore implements MemStore {
       }
     }
     MemStoreSnapshot memStoreSnapshot = new MemStoreSnapshot(this.snapshotId,
-        snapshotSection.getCellSkipListSet().size(), snapshotSection.getHeapSize().get(),
+        snapshotSection.getCellsCount().get(), snapshotSection.getHeapSize().get(),
         snapshotSection.getTimeRangeTracker(),
         new CollectionBackedScanner(snapshotSection.getCellSkipListSet(), this.comparator),
         this.tagsPresent);
@@ -236,6 +237,9 @@ public class DefaultMemStore implements MemStore {
   private long internalAdd(final Cell toAdd, boolean mslabUsed) {
     boolean notPresent = addToCellSet(toAdd);
     long s = heapSizeChange(toAdd, notPresent);
+    if (notPresent) {
+      activeSection.getCellsCount().incrementAndGet();
+    }
     // If there's already a same cell in the CellSet and we are using MSLAB, we must count in the
     // MSLAB allocation size as well, or else there will be memory leak (occupied heap size larger
     // than the counted number)
@@ -294,6 +298,7 @@ public class DefaultMemStore implements MemStore {
       snapshotSection.getCellSkipListSet().remove(cell);
       long sz = heapSizeChange(cell, true);
       snapshotSection.getHeapSize().addAndGet(-sz);
+      snapshotSection.getCellsCount().decrementAndGet();
     }
 
     // If the key is in the memstore, delete it. Update this.size.
@@ -302,6 +307,7 @@ public class DefaultMemStore implements MemStore {
       removeFromCellSet(found);
       long sz = heapSizeChange(found, true);
       activeSection.getHeapSize().addAndGet(-sz);
+      activeSection.getCellsCount().decrementAndGet();
     }
   }
 
@@ -589,6 +595,7 @@ public class DefaultMemStore implements MemStore {
             long delta = heapSizeChange(cur, true);
             addedSize -= delta;
             activeSection.getHeapSize().addAndGet(-delta);
+            activeSection.getCellsCount().decrementAndGet();
             if (removedCells != null) {
               removedCells.add(cur);
             }
@@ -1013,7 +1020,7 @@ public class DefaultMemStore implements MemStore {
     @Override
     public synchronized boolean seekToLastRow() {
       Cell first = activeAtCreation.getCellSkipListSet().isEmpty() ? null
-        : activeAtCreation.getCellSkipListSet().last();
+          : activeAtCreation.getCellSkipListSet().last();
       Cell second = snapshotAtCreation.getCellSkipListSet().isEmpty() ? null
           : snapshotAtCreation.getCellSkipListSet().last();
       Cell higherCell = getHighest(first, second);
@@ -1036,7 +1043,8 @@ public class DefaultMemStore implements MemStore {
 
   public final static long DEEP_OVERHEAD = ClassSize.align(FIXED_OVERHEAD +
       (2 * ClassSize.ATOMIC_LONG) + (2 * ClassSize.TIMERANGE_TRACKER) +
-      (2 * ClassSize.CELL_SKIPLIST_SET) + (2 * ClassSize.CONCURRENT_SKIPLISTMAP));
+      (2 * ClassSize.CELL_SKIPLIST_SET) + (2 * ClassSize.CONCURRENT_SKIPLISTMAP) +
+      ClassSize.ATOMIC_INTEGER);
 
   /*
    * Calculate how the MemStore size has changed.  Includes overhead of the
@@ -1128,6 +1136,7 @@ public class DefaultMemStore implements MemStore {
      * Used to track own heapSize.
      */
     private final AtomicLong heapSize;
+    private final AtomicInteger cellCount;
     private final MemStoreLAB allocator;
 
     static Section newSnapshotSection(final KeyValue.KVComparator c) {
@@ -1143,6 +1152,7 @@ public class DefaultMemStore implements MemStore {
             final Configuration conf, long initHeapSize) {
       this.cellSet = new CellSkipListSet(c);
       this.heapSize = new AtomicLong(initHeapSize);
+      this.cellCount = new AtomicInteger(0);
       if (conf != null && conf.getBoolean(USEMSLAB_KEY, USEMSLAB_DEFAULT)) {
         String className = conf.get(MSLAB_CLASS_NAME, HeapMemStoreLAB.class.getName());
         this.allocator = ReflectionUtils.instantiateWithCustomCtor(className,
@@ -1164,9 +1174,12 @@ public class DefaultMemStore implements MemStore {
       return heapSize;
     }
 
+    AtomicInteger getCellsCount() {
+      return cellCount;
+    }
+
     MemStoreLAB getMemStoreLAB() {
       return allocator;
     }
-
   }
 }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestHeapSize.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestHeapSize.java
index 12559e7..170633e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestHeapSize.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestHeapSize.java
@@ -305,13 +305,15 @@ public class TestHeapSize  {
     actual = DefaultMemStore.DEEP_OVERHEAD;
     expected = ClassSize.estimateBase(cl, false);
     expected += (2 * ClassSize.estimateBase(AtomicLong.class, false));
+    expected += ClassSize.estimateBase(AtomicInteger.class, false);
     expected += (2 * ClassSize.estimateBase(CellSkipListSet.class, false));
     expected += (2 * ClassSize.estimateBase(ConcurrentSkipListMap.class, false));
     expected += (2 * ClassSize.estimateBase(TimeRangeTracker.class, false));
-    if(expected != actual) {
+    if (expected != actual) {
       ClassSize.estimateBase(cl, true);
       ClassSize.estimateBase(AtomicLong.class, true);
       ClassSize.estimateBase(AtomicLong.class, true);
+      ClassSize.estimateBase(AtomicInteger.class, true);
       ClassSize.estimateBase(CellSkipListSet.class, true);
       ClassSize.estimateBase(CellSkipListSet.class, true);
       ClassSize.estimateBase(ConcurrentSkipListMap.class, true);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCellSkipListSet.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCellSkipListSet.java
index e487c03..b8c8c3d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCellSkipListSet.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCellSkipListSet.java
@@ -44,7 +44,7 @@ public class TestCellSkipListSet extends TestCase {
     KeyValue kv = new KeyValue(bytes, bytes, bytes, bytes);
     this.csls.add(kv);
     assertTrue(this.csls.contains(kv));
-    assertEquals(1, this.csls.size());
+    assertEquals(1, this.csls.sizeForTests());
     Cell first = this.csls.first();
     assertTrue(kv.equals(first));
     assertTrue(Bytes.equals(kv.getValue(), first.getValue()));
@@ -52,7 +52,7 @@ public class TestCellSkipListSet extends TestCase {
     byte [] overwriteValue = Bytes.toBytes("overwrite");
     KeyValue overwrite = new KeyValue(bytes, bytes, bytes, overwriteValue);
     this.csls.add(overwrite);
-    assertEquals(1, this.csls.size());
+    assertEquals(1, this.csls.sizeForTests());
     first = this.csls.first();
     assertTrue(Bytes.equals(overwrite.getValue(), first.getValue()));
     assertFalse(Bytes.equals(overwrite.getValue(), kv.getValue()));
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java
index d0ee86b..f6d026c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java
@@ -88,7 +88,7 @@ public class TestDefaultMemStore extends TestCase {
     KeyValue samekey = new KeyValue(bytes, bytes, bytes, other);
     this.memstore.add(samekey);
     Cell found = this.memstore.activeSection.getCellSkipListSet().first();
-    assertEquals(1, this.memstore.activeSection.getCellSkipListSet().size());
+    assertEquals(1, this.memstore.activeSection.getCellSkipListSet().sizeForTests());
     assertTrue(Bytes.toString(found.getValue()), CellUtil.matchingValue(samekey, found));
   }
 
@@ -105,7 +105,8 @@ public class TestDefaultMemStore extends TestCase {
       // make sure chunk size increased even when writing the same cell, if using MSLAB
       if (this.memstore.activeSection.getMemStoreLAB() instanceof HeapMemStoreLAB) {
         assertEquals(2 * memstore.getCellLength(kv),
-          ((HeapMemStoreLAB) this.memstore.activeSection.getMemStoreLAB()).getCurrentChunk().getNextFreeOffset());
+          ((HeapMemStoreLAB) this.memstore.activeSection.getMemStoreLAB())
+            .getCurrentChunk().getNextFreeOffset());
       }
     } else {
       // make sure no memstore size change w/o MSLAB
@@ -492,7 +493,8 @@ public class TestDefaultMemStore extends TestCase {
     for (int i = 0; i < snapshotCount; i++) {
       addRows(this.memstore);
       runSnapshot(this.memstore);
-      assertEquals("History not being cleared", 0, this.memstore.snapshotSection.getCellSkipListSet().size());
+      assertEquals("History not being cleared", 0,
+        this.memstore.snapshotSection.getCellSkipListSet().sizeForTests());
     }
   }
 
@@ -513,7 +515,8 @@ public class TestDefaultMemStore extends TestCase {
     m.add(key2);
 
     assertTrue("Expected memstore to hold 3 values, actually has " +
-        m.activeSection.getCellSkipListSet().size(), m.activeSection.getCellSkipListSet().size() == 3);
+        m.activeSection.getCellSkipListSet().sizeForTests(),
+        m.activeSection.getCellSkipListSet().sizeForTests() == 3);
   }
 
   //////////////////////////////////////////////////////////////////////////////
@@ -587,12 +590,12 @@ public class TestDefaultMemStore extends TestCase {
     memstore.add(new KeyValue(row, fam ,qf3, val));
     //Creating a snapshot
     memstore.snapshot();
-    assertEquals(3, memstore.snapshotSection.getCellSkipListSet().size());
+    assertEquals(3, memstore.snapshotSection.getCellSkipListSet().sizeForTests());
     //Adding value to "new" memstore
-    assertEquals(0, memstore.activeSection.getCellSkipListSet().size());
+    assertEquals(0, memstore.activeSection.getCellSkipListSet().sizeForTests());
     memstore.add(new KeyValue(row, fam ,qf4, val));
     memstore.add(new KeyValue(row, fam ,qf5, val));
-    assertEquals(2, memstore.activeSection.getCellSkipListSet().size());
+    assertEquals(2, memstore.activeSection.getCellSkipListSet().sizeForTests());
   }
 
   //////////////////////////////////////////////////////////////////////////////
@@ -614,7 +617,7 @@ public class TestDefaultMemStore extends TestCase {
     memstore.add(put2);
     memstore.add(put3);
 
-    assertEquals(3, memstore.activeSection.getCellSkipListSet().size());
+    assertEquals(3, memstore.activeSection.getCellSkipListSet().sizeForTests());
 
     KeyValue del2 = new KeyValue(row, fam, qf1, ts2, KeyValue.Type.Delete, val);
     memstore.delete(del2);
@@ -625,7 +628,7 @@ public class TestDefaultMemStore extends TestCase {
     expected.add(put2);
     expected.add(put1);
 
-    assertEquals(4, memstore.activeSection.getCellSkipListSet().size());
+    assertEquals(4, memstore.activeSection.getCellSkipListSet().sizeForTests());
     int i = 0;
     for(Cell cell : memstore.activeSection.getCellSkipListSet()) {
       assertEquals(expected.get(i++), cell);
@@ -648,7 +651,7 @@ public class TestDefaultMemStore extends TestCase {
     memstore.add(put2);
     memstore.add(put3);
 
-    assertEquals(3, memstore.activeSection.getCellSkipListSet().size());
+    assertEquals(3, memstore.activeSection.getCellSkipListSet().sizeForTests());
 
     KeyValue del2 =
       new KeyValue(row, fam, qf1, ts2, KeyValue.Type.DeleteColumn, val);
@@ -660,15 +663,13 @@ public class TestDefaultMemStore extends TestCase {
     expected.add(put2);
     expected.add(put1);
 
-
-    assertEquals(4, memstore.activeSection.getCellSkipListSet().size());
+    assertEquals(4, memstore.activeSection.getCellSkipListSet().sizeForTests());
     int i = 0;
     for (Cell cell: memstore.activeSection.getCellSkipListSet()) {
       assertEquals(expected.get(i++), cell);
     }
   }
 
-
   public void testGetWithDeleteFamily() throws IOException {
     byte [] row = Bytes.toBytes("testrow");
     byte [] fam = Bytes.toBytes("testfamily");
@@ -699,9 +700,7 @@ public class TestDefaultMemStore extends TestCase {
     expected.add(put4);
     expected.add(put3);
 
-
-
-    assertEquals(5, memstore.activeSection.getCellSkipListSet().size());
+    assertEquals(5, memstore.activeSection.getCellSkipListSet().sizeForTests());
     int i = 0;
     for (Cell cell: memstore.activeSection.getCellSkipListSet()) {
       assertEquals(expected.get(i++), cell);
@@ -717,7 +716,7 @@ public class TestDefaultMemStore extends TestCase {
     memstore.add(new KeyValue(row, fam, qf, ts, val));
     KeyValue delete = new KeyValue(row, fam, qf, ts, KeyValue.Type.Delete, val);
     memstore.delete(delete);
-    assertEquals(2, memstore.activeSection.getCellSkipListSet().size());
+    assertEquals(2, memstore.activeSection.getCellSkipListSet().sizeForTests());
     assertEquals(delete, memstore.activeSection.getCellSkipListSet().first());
   }
 
@@ -730,7 +729,7 @@ public class TestDefaultMemStore extends TestCase {
         "row1", "fam", "a", 100, KeyValue.Type.Delete, "dont-care");
     memstore.delete(delete);
 
-    assertEquals(2, memstore.activeSection.getCellSkipListSet().size());
+    assertEquals(2, memstore.activeSection.getCellSkipListSet().sizeForTests());
     assertEquals(delete, memstore.activeSection.getCellSkipListSet().first());
   }
   public void testRetainsDeleteColumn() throws IOException {
@@ -742,9 +741,10 @@ public class TestDefaultMemStore extends TestCase {
         KeyValue.Type.DeleteColumn, "dont-care");
     memstore.delete(delete);
 
-    assertEquals(2, memstore.activeSection.getCellSkipListSet().size());
+    assertEquals(2, memstore.activeSection.getCellSkipListSet().sizeForTests());
     assertEquals(delete, memstore.activeSection.getCellSkipListSet().first());
   }
+
   public void testRetainsDeleteFamily() throws IOException {
     // add a put to memstore
     memstore.add(KeyValueTestUtil.create("row1", "fam", "a", 100, "dont-care"));
@@ -754,7 +754,7 @@ public class TestDefaultMemStore extends TestCase {
         KeyValue.Type.DeleteFamily, "dont-care");
     memstore.delete(delete);
 
-    assertEquals(2, memstore.activeSection.getCellSkipListSet().size());
+    assertEquals(2, memstore.activeSection.getCellSkipListSet().sizeForTests());
     assertEquals(delete, memstore.activeSection.getCellSkipListSet().first());
   }
 
@@ -869,7 +869,7 @@ public class TestDefaultMemStore extends TestCase {
     long newSize = this.memstore.activeSection.getHeapSize().get();
     assert(newSize > oldSize);
     //The kv1 should be removed.
-    assert(memstore.activeSection.getCellSkipListSet().size() == 2);
+    assert(memstore.activeSection.getCellSkipListSet().sizeForTests() == 2);
 
     KeyValue kv4 = KeyValueTestUtil.create("r", "f", "q", 104, "v");
     kv4.setSequenceId(1);
@@ -877,7 +877,7 @@ public class TestDefaultMemStore extends TestCase {
     this.memstore.upsert(l, 3, null);
     assertEquals(newSize, this.memstore.activeSection.getHeapSize().get());
     //The kv2 should be removed.
-    assert(memstore.activeSection.getCellSkipListSet().size() == 2);
+    assert(memstore.activeSection.getCellSkipListSet().sizeForTests() == 2);
     //this.memstore = null;
   }
 
@@ -1038,10 +1038,11 @@ public class TestDefaultMemStore extends TestCase {
 
   private long runSnapshot(final DefaultMemStore hmc) throws UnexpectedStateException {
     // Save off old state.
-    int oldHistorySize = hmc.snapshotSection.getCellSkipListSet().size();
+    int oldHistorySize = hmc.snapshotSection.getCellSkipListSet().sizeForTests();
     MemStoreSnapshot snapshot = hmc.snapshot();
     // Make some assertions about what just happened.
-    assertTrue("History size has not increased", oldHistorySize < hmc.snapshotSection.getCellSkipListSet().size());
+    assertTrue("History size has not increased", oldHistorySize <
+      hmc.snapshotSection.getCellSkipListSet().sizeForTests());
     long t = memstore.timeOfOldestEdit();
     assertTrue("Time of oldest edit is not Long.MAX_VALUE", t == Long.MAX_VALUE);
     hmc.clearSnapshot(snapshot.getId());
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 d1ad68c..b132781 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
@@ -116,13 +116,13 @@ public class TestMemStoreChunkPool {
 
     // Creating a snapshot
     MemStoreSnapshot snapshot = memstore.snapshot();
-    assertEquals(3, memstore.snapshotSection.getCellSkipListSet().size());
+    assertEquals(3, memstore.snapshotSection.getCellSkipListSet().sizeForTests());
 
     // Adding value to "new" memstore
-    assertEquals(0, memstore.activeSection.getCellSkipListSet().size());
+    assertEquals(0, memstore.activeSection.getCellSkipListSet().sizeForTests());
     memstore.add(new KeyValue(row, fam, qf4, val));
     memstore.add(new KeyValue(row, fam, qf5, val));
-    assertEquals(2, memstore.activeSection.getCellSkipListSet().size());
+    assertEquals(2, memstore.activeSection.getCellSkipListSet().sizeForTests());
     memstore.clearSnapshot(snapshot.getId());
 
     int chunkCount = chunkPool.getPoolSize();
@@ -153,13 +153,13 @@ public class TestMemStoreChunkPool {
 
     // Creating a snapshot
     MemStoreSnapshot snapshot = memstore.snapshot();
-    assertEquals(3, memstore.snapshotSection.getCellSkipListSet().size());
+    assertEquals(3, memstore.snapshotSection.getCellSkipListSet().sizeForTests());
 
     // Adding value to "new" memstore
-    assertEquals(0, memstore.activeSection.getCellSkipListSet().size());
+    assertEquals(0, memstore.activeSection.getCellSkipListSet().sizeForTests());
     memstore.add(new KeyValue(row, fam, qf4, val));
     memstore.add(new KeyValue(row, fam, qf5, val));
-    assertEquals(2, memstore.activeSection.getCellSkipListSet().size());
+    assertEquals(2, memstore.activeSection.getCellSkipListSet().sizeForTests());
 
     // opening scanner before clear the snapshot
     List<KeyValueScanner> scanners = memstore.getScanners(0);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java
index cf9485a..bad7a48 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java
@@ -596,7 +596,8 @@ public class TestStore {
     this.store.snapshot();
     flushStore(store, id++);
     Assert.assertEquals(storeFilessize, this.store.getStorefiles().size());
-    Assert.assertEquals(0, ((DefaultMemStore)this.store.memstore).activeSection.getCellSkipListSet().size());
+    Assert.assertEquals(0, ((DefaultMemStore)this.store.memstore)
+      .activeSection.getCellSkipListSet().sizeForTests());
   }
 
   private void assertCheck() {
@@ -641,7 +642,8 @@ public class TestStore {
     flushStore(store, id++);
     Assert.assertEquals(1, this.store.getStorefiles().size());
     // from the one we inserted up there, and a new one
-    Assert.assertEquals(2, ((DefaultMemStore)this.store.memstore).activeSection.getCellSkipListSet().size());
+    Assert.assertEquals(2, ((DefaultMemStore)this.store.memstore)
+      .activeSection.getCellSkipListSet().sizeForTests());
 
     // how many key/values for this row are there?
     Get get = new Get(row);
@@ -742,7 +744,8 @@ public class TestStore {
     // then flush.
     flushStore(store, id++);
     Assert.assertEquals(1, this.store.getStorefiles().size());
-    Assert.assertEquals(1, ((DefaultMemStore)this.store.memstore).activeSection.getCellSkipListSet().size());
+    Assert.assertEquals(1, ((DefaultMemStore)this.store.memstore)
+      .activeSection.getCellSkipListSet().sizeForTests());
 
     // now increment again:
     newValue += 1;