You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by ds...@apache.org on 2016/05/17 23:16:21 UTC

incubator-geode git commit: refactored for unit testing

Repository: incubator-geode
Updated Branches:
  refs/heads/feature/GEODE-1292 3e62e809a -> ebd98303c


refactored for unit testing


Project: http://git-wip-us.apache.org/repos/asf/incubator-geode/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-geode/commit/ebd98303
Tree: http://git-wip-us.apache.org/repos/asf/incubator-geode/tree/ebd98303
Diff: http://git-wip-us.apache.org/repos/asf/incubator-geode/diff/ebd98303

Branch: refs/heads/feature/GEODE-1292
Commit: ebd98303c3419b625fc5684dc98914f16fc4a5fe
Parents: 3e62e80
Author: Darrel Schneider <ds...@pivotal.io>
Authored: Tue May 17 15:44:46 2016 -0700
Committer: Darrel Schneider <ds...@pivotal.io>
Committed: Tue May 17 15:44:46 2016 -0700

----------------------------------------------------------------------
 .../internal/offheap/FreeListManager.java       | 257 ++++++++++++-------
 .../OffHeapStoredObjectAddressStack.java        |   5 +-
 .../gemfire/internal/offheap/SlabImpl.java      |  11 +-
 .../internal/offheap/FreeListManagerTest.java   |  96 ++++++-
 4 files changed, 268 insertions(+), 101 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ebd98303/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/FreeListManager.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/FreeListManager.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/FreeListManager.java
index 2e086ee..9940b7d 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/FreeListManager.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/FreeListManager.java
@@ -304,21 +304,40 @@ public class FreeListManager {
   
   /**
    * Return true if the two chunks have been combined into one.
-   * If low and high are adjacent to each other then low's size
-   * will be increased by the size of high and true will be returned.
+   * If low and high are adjacent to each other
+   * and the combined size is small enough (see isSmallEnough)
+   * then low's size will be increased by the size of high
+   * and true will be returned.
    */
-  private boolean combineIfAdjacent(long lowAddr, long highAddr) {
+  boolean combineIfAdjacentAndSmallEnough(long lowAddr, long highAddr) {
+    assert lowAddr <= highAddr;
     int lowSize = OffHeapStoredObject.getSize(lowAddr);
-    if (lowAddr + lowSize == highAddr) {
-      long newSize = lowSize + OffHeapStoredObject.getSize(highAddr);
-      if (newSize <= Integer.MAX_VALUE) {
+    if (isAdjacent(lowAddr, lowSize, highAddr)) {
+      int highSize = OffHeapStoredObject.getSize(highAddr);
+      int combinedSize = lowSize + highSize;
+      if (isSmallEnough(combinedSize)) {
         // append the highAddr chunk to lowAddr
-        OffHeapStoredObject.setSize(lowAddr, (int)newSize);
+        OffHeapStoredObject.setSize(lowAddr, (int)combinedSize);
         return true;
       }
     }
     return false;
   }
+
+  /**
+   * Returns true if the area if memory (starting at lowAddr and extending to
+   * lowAddr+lowSize) is right before (i.e. adjacent) to highAddr.
+   */
+  boolean isAdjacent(long lowAddr, int lowSize, long highAddr) {
+    return (lowAddr + lowSize) == highAddr;
+  }
+  /**
+   * Return true if size is small enough to be set as the size
+   * of a OffHeapStoredObject.
+   */
+  boolean isSmallEnough(long size) {
+    return size <= Integer.MAX_VALUE;
+  }
   
   /**
    * Defragments memory and returns true if enough memory to allocate chunkSize
@@ -335,98 +354,150 @@ public class FreeListManager {
           // So just return true causing the caller to retry the allocation.
           return true;
         }
-        ArrayList<OffHeapStoredObjectAddressStack> freeChunks = new ArrayList<OffHeapStoredObjectAddressStack>();
-        collectFreeChunks(freeChunks);
-        final int SORT_ARRAY_BLOCK_SIZE = 128;
-        long[] sorted = new long[SORT_ARRAY_BLOCK_SIZE];
-        int sortedSize = 0;
-        boolean result = false;
-        int largestFragment = 0;
-        for (OffHeapStoredObjectAddressStack l: freeChunks) {
-          long addr = l.poll();
-          while (addr != 0) {
-            int idx = Arrays.binarySearch(sorted, 0, sortedSize, addr);
-            idx = -idx;
-            idx--;
-            if (idx == sortedSize) {
-              // addr is > everything in the array
-              if (sortedSize == 0) {
-                // nothing was in the array
-                sorted[0] = addr;
-                sortedSize++;
-              } else {
-                if (!combineIfAdjacent(sorted[idx-1], addr)) {
-                  if (sortedSize >= sorted.length) {
-                    long[] newSorted = new long[sorted.length+SORT_ARRAY_BLOCK_SIZE];
-                    System.arraycopy(sorted, 0, newSorted, 0, sorted.length);
-                    sorted = newSorted;
-                  }
-                  sortedSize++;
-                  sorted[idx] = addr;
-                }
-              }
-            } else {
-              if (combineIfAdjacent(addr, sorted[idx])) {
-                sorted[idx] = addr;
-              } else {
-                if (idx == 0 || !combineIfAdjacent(sorted[idx-1], addr)) {
-                  if (sortedSize >= sorted.length) {
-                    long[] newSorted = new long[sorted.length+SORT_ARRAY_BLOCK_SIZE];
-                    System.arraycopy(sorted, 0, newSorted, 0, idx);
-                    newSorted[idx] = addr;
-                    System.arraycopy(sorted, idx, newSorted, idx+1, sortedSize-idx);
-                    sorted = newSorted;
-                  } else {
-                    System.arraycopy(sorted, idx, sorted, idx+1, sortedSize-idx);
-                    sorted[idx] = addr;
-                  }
-                  sortedSize++;
-                }
-              }
-            }
-            addr = l.poll();
-          }
-        }
-        for (int i=sortedSize-1; i > 0; i--) {
-          if (combineIfAdjacent(sorted[i-1], sorted[i])) {
-            sorted[i] = 0L;
-          }
-        }
-        this.lastFragmentAllocation.set(0);
-        ArrayList<Fragment> tmp = new ArrayList<Fragment>();
-        for (int i=sortedSize-1; i >= 0; i--) {
-          long addr = sorted[i];
-          if (addr == 0L) continue;
-          int addrSize = OffHeapStoredObject.getSize(addr);
-          Fragment f = createFragment(addr, addrSize);
-          if (addrSize >= chunkSize) {
-            result = true;
-          }
-          if (addrSize > largestFragment) {
-            largestFragment = addrSize;
-            // TODO it might be better to sort them biggest first
-            tmp.add(0, f);
-          } else {
-            tmp.add(f);
-          }
-        }
-        this.fragmentList.addAll(tmp);
-
-        fillFragments();
+        boolean result = doDefragment(chunkSize);
 
         // Signal any waiters that a defragmentation happened.
         this.defragmentationCount.incrementAndGet();
 
-        this.ma.getStats().setLargestFragment(largestFragment);
-        this.ma.getStats().setFragments(tmp.size());        
-        this.ma.getStats().setFragmentation(getFragmentation());
-
         return result;
       } // sync
     } finally {
       this.ma.getStats().endDefragmentation(startDefragmentationTime);
     }
   }
+  
+  /**
+   * Simple interface the represents a "stack" of primitive longs.
+   * Currently this interface only allows supports poll but more
+   * could be added if needed in the future.
+   * This interface was introduced to aid unit testing.
+   * The only implementation of it is OffHeapStoredObjectAddressStack.
+   */
+  public interface LongStack {
+    /**
+     * Retrieves and removes the top of this stack,
+     * or returns {@code 0L} if this stack is empty.
+     */
+    public long poll();
+  }
+  /**
+   * Manages an array of primitive longs. The array can grow.
+   */
+  public static class ResizableLongArray {
+    private static final int SORT_ARRAY_BLOCK_SIZE = 128;
+    long[] data = new long[SORT_ARRAY_BLOCK_SIZE];
+    int size = 0;
+    
+    public int binarySearch(long l) {
+      return Arrays.binarySearch(data, 0, size, l);
+    }
+    public int size() {
+      return size;
+    }
+    public long get(int idx) {
+      return data[idx];
+    }
+    public void set(int idx, long l) {
+      data[idx] = l;
+    }
+    public void add(long l) {
+      if (size >= data.length) {
+        long[] newData = new long[data.length+SORT_ARRAY_BLOCK_SIZE];
+        System.arraycopy(data, 0, newData, 0, data.length);
+        data = newData;
+      }
+      data[size] = l;
+      size++;
+    }
+    public void insert(int idx, long l) {
+      if (size >= data.length) {
+        long[] newData = new long[data.length+SORT_ARRAY_BLOCK_SIZE];
+        System.arraycopy(data, 0, newData, 0, idx);
+        newData[idx] = l;
+        System.arraycopy(data, idx, newData, idx+1, size-idx);
+        data = newData;
+      } else {
+        System.arraycopy(data, idx, data, idx+1, size-idx);
+        data[idx] = l;
+      }
+      size++;
+    }
+  }
+  /**
+   * Defragments memory and returns true if enough memory to allocate chunkSize
+   * is freed. Otherwise returns false;
+   * Unlike the defragment method this method is not thread safe and does not check
+   * for a concurrent defragment. It should only be called by defragment and unit tests.
+   */
+  boolean doDefragment(int chunkSize) {
+    boolean result = false;
+    ArrayList<LongStack> freeChunks = new ArrayList<LongStack>();
+    collectFreeChunks(freeChunks);
+    ResizableLongArray sorted = new ResizableLongArray();
+    for (LongStack l: freeChunks) {
+      long addr = l.poll();
+      while (addr != 0) {
+        int idx = sorted.binarySearch(addr);
+        idx = -idx;
+        idx--;
+        int sortedSize = sorted.size();
+        if (idx == sortedSize) {
+          // addr is > everything in the array
+          if (sortedSize == 0) {
+            // nothing was in the array
+            sorted.add(addr);
+          } else {
+            if (!combineIfAdjacentAndSmallEnough(sorted.get(idx-1), addr)) {
+              sorted.add(addr);
+            }
+          }
+        } else {
+          if (combineIfAdjacentAndSmallEnough(addr, sorted.get(idx))) {
+            sorted.set(idx, addr);
+          } else {
+            if (idx == 0 || !combineIfAdjacentAndSmallEnough(sorted.get(idx-1), addr)) {
+              sorted.insert(idx, addr);
+            }
+          }
+        }
+        addr = l.poll();
+      }
+    }
+    for (int i=sorted.size()-1; i > 0; i--) {
+      if (combineIfAdjacentAndSmallEnough(sorted.get(i-1), sorted.get(i))) {
+        sorted.set(i, 0L);
+      }
+    }
+    
+    int largestFragment = 0;
+    this.lastFragmentAllocation.set(0);
+    ArrayList<Fragment> tmp = new ArrayList<Fragment>();
+    for (int i=sorted.size()-1; i >= 0; i--) {
+      long addr = sorted.get(i);
+      if (addr == 0L) continue;
+      int addrSize = OffHeapStoredObject.getSize(addr);
+      Fragment f = createFragment(addr, addrSize);
+      if (addrSize >= chunkSize) {
+        result = true;
+      }
+      if (addrSize > largestFragment) {
+        largestFragment = addrSize;
+        // TODO it might be better to sort them biggest first
+        tmp.add(0, f);
+      } else {
+        tmp.add(f);
+      }
+    }
+    this.fragmentList.addAll(tmp);
+
+    fillFragments();
+
+    this.ma.getStats().setLargestFragment(largestFragment);
+    this.ma.getStats().setFragments(tmp.size());        
+    this.ma.getStats().setFragmentation(getFragmentation());
+
+    return result;
+  }
 
   /**
    * Unit tests override this method to get better test coverage
@@ -482,7 +553,7 @@ public class FreeListManager {
     }
   }
 
-  private void collectFreeChunks(List<OffHeapStoredObjectAddressStack> l) {
+  private void collectFreeChunks(List<LongStack> l) {
     collectFreeFragmentChunks(l);
     collectFreeHugeChunks(l);
     collectFreeTinyChunks(l);
@@ -490,7 +561,7 @@ public class FreeListManager {
   List<Fragment> getFragmentList() {
     return this.fragmentList;
   }
-  private void collectFreeFragmentChunks(List<OffHeapStoredObjectAddressStack> l) {
+  private void collectFreeFragmentChunks(List<LongStack> l) {
     if (this.fragmentList.size() == 0) return;
     OffHeapStoredObjectAddressStack result = new OffHeapStoredObjectAddressStack();
     for (Fragment f: this.fragmentList) {
@@ -519,7 +590,7 @@ public class FreeListManager {
       l.add(result);
     }
   }
-  private void collectFreeTinyChunks(List<OffHeapStoredObjectAddressStack> l) {
+  private void collectFreeTinyChunks(List<LongStack> l) {
     for (int i=0; i < this.tinyFreeLists.length(); i++) {
       OffHeapStoredObjectAddressStack cl = this.tinyFreeLists.get(i);
       if (cl != null) {
@@ -530,7 +601,7 @@ public class FreeListManager {
       }
     }
   }
-  private void collectFreeHugeChunks(List<OffHeapStoredObjectAddressStack> l) {
+  private void collectFreeHugeChunks(List<LongStack> l) {
     OffHeapStoredObject c = this.hugeChunkSet.pollFirst();
     OffHeapStoredObjectAddressStack result = null;
     while (c != null) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ebd98303/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapStoredObjectAddressStack.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapStoredObjectAddressStack.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapStoredObjectAddressStack.java
index b69d3a6..1719b1f 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapStoredObjectAddressStack.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapStoredObjectAddressStack.java
@@ -18,13 +18,15 @@ package com.gemstone.gemfire.internal.offheap;
 
 import org.apache.logging.log4j.Logger;
 
+import com.gemstone.gemfire.internal.offheap.FreeListManager.LongStack;
+
 /**
  * A "stack" of addresses of OffHeapStoredObject instances. The stored objects are not kept
  * in java object form but instead each one is just an off-heap address.
  * This class is used for each "tiny" free-list of the FreeListManager.
  * This class is thread safe.
  */
-public class OffHeapStoredObjectAddressStack {
+public class OffHeapStoredObjectAddressStack implements LongStack {
   // Ok to read without sync but must be synced on write
   private volatile long topAddr;
   
@@ -46,6 +48,7 @@ public class OffHeapStoredObjectAddressStack {
       this.topAddr = e;
     }
   }
+  @Override
   public long poll() {
     long result;
     synchronized (this) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ebd98303/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/SlabImpl.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/SlabImpl.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/SlabImpl.java
index 1c88bde..3169352 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/SlabImpl.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/SlabImpl.java
@@ -26,7 +26,16 @@ public class SlabImpl implements Slab {
   private final int size;
   
   public SlabImpl(int size) {
-    this(AddressableMemoryManager.allocate(size), size);
+    this(size, false);
+  }
+
+  /**
+   * This constructor is used by unit test to add padding
+   * to prevent intermittent combining of fragments during
+   * defragmentation.
+   */
+  public SlabImpl(int size, boolean withPadding) {
+    this(AddressableMemoryManager.allocate(size + (withPadding ? 4 : 0)), size);
   }
 
   public SlabImpl(long addr, int size) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ebd98303/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/FreeListManagerTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/FreeListManagerTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/FreeListManagerTest.java
index 4cf1df0..9c6a3db 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/FreeListManagerTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/FreeListManagerTest.java
@@ -57,6 +57,10 @@ public class FreeListManagerTest {
     return new TestableFreeListManager(ma, slabs);
   }
   
+  private static TestableFreeListManager createFreeListManager(MemoryAllocatorImpl ma, Slab[] slabs, int maxCombine) {
+    return new TestableFreeListManager(ma, slabs, maxCombine);
+  }
+  
   private void setUpSingleSlabManager() {
     setUpSingleSlabManager(DEFAULT_SLAB_SIZE);
   }
@@ -285,14 +289,22 @@ public class FreeListManagerTest {
   }
   
   @Test
+  public void testSlabImplToString() {
+    Slab slab = new SlabImpl(DEFAULT_SLAB_SIZE);
+    String slabAsString = slab.toString();
+    assertThat(slabAsString.contains("MemoryAddress="+slab.getMemoryAddress()));
+    assertThat(slabAsString.contains("Size="+DEFAULT_SLAB_SIZE));
+  }
+  
+  @Test
   public void defragmentWithChunkSizeOfMaxSlabReturnsTrue() {
     int SMALL_SLAB = 16;
     int MEDIUM_SLAB = 128;
-    Slab slab = new SlabImpl(DEFAULT_SLAB_SIZE);
+    Slab slab = new SlabImpl(DEFAULT_SLAB_SIZE, true);
     this.freeListManager = createFreeListManager(ma, new Slab[] {
-        new SlabImpl(SMALL_SLAB), 
-        new SlabImpl(SMALL_SLAB), 
-        new SlabImpl(MEDIUM_SLAB), 
+        new SlabImpl(SMALL_SLAB, true), 
+        new SlabImpl(SMALL_SLAB, true), 
+        new SlabImpl(MEDIUM_SLAB, true), 
         slab});
     ArrayList<OffHeapStoredObject> chunks = new ArrayList<>();
     chunks.add(this.freeListManager.allocate(SMALL_SLAB-8+1));
@@ -303,8 +315,9 @@ public class FreeListManagerTest {
       OffHeapStoredObject.release(c.getAddress(), this.freeListManager);
     }
     
+    this.freeListManager.firstDefragmentation = false;
     assertThat(this.freeListManager.defragment(DEFAULT_SLAB_SIZE)).isTrue();
-    //assertThat(this.freeListManager.getFragmentList()).hasSize(4); // TODO intermittently fails because Fragments may be merged
+    assertThat(this.freeListManager.getFragmentList()).hasSize(4);
   }
   
   @Test
@@ -326,6 +339,31 @@ public class FreeListManagerTest {
       OffHeapStoredObject.release(c.getAddress(), this.freeListManager);
     }
     
+    this.freeListManager.firstDefragmentation = false;
+    assertThat(this.freeListManager.defragment(DEFAULT_SLAB_SIZE/2)).isTrue();
+  }
+  
+  @Test
+  public void defragmentWhenDisallowingCombine() {
+    int SMALL_SLAB = 16;
+    int MEDIUM_SLAB = 128;
+    Slab slab = new SlabImpl(DEFAULT_SLAB_SIZE);
+    this.freeListManager = createFreeListManager(ma, new Slab[] {
+        new SlabImpl(SMALL_SLAB), 
+        new SlabImpl(SMALL_SLAB), 
+        new SlabImpl(MEDIUM_SLAB), 
+        slab}, DEFAULT_SLAB_SIZE/2);
+    ArrayList<OffHeapStoredObject> chunks = new ArrayList<>();
+    chunks.add(this.freeListManager.allocate(SMALL_SLAB-8+1));
+    chunks.add(this.freeListManager.allocate(DEFAULT_SLAB_SIZE/2-8));
+    chunks.add(this.freeListManager.allocate(DEFAULT_SLAB_SIZE/2-8));
+    this.freeListManager.allocate(SMALL_SLAB-8+1);
+    for (OffHeapStoredObject c: chunks) {
+      OffHeapStoredObject.release(c.getAddress(), this.freeListManager);
+    }
+    
+    this.freeListManager.firstDefragmentation = false;
+    assertThat(this.freeListManager.defragment((DEFAULT_SLAB_SIZE/2)+1)).isFalse();
     assertThat(this.freeListManager.defragment(DEFAULT_SLAB_SIZE/2)).isTrue();
   }
   
@@ -335,7 +373,7 @@ public class FreeListManagerTest {
     OffHeapStoredObject c = freeListManager.allocate(DEFAULT_SLAB_SIZE-8);
     this.freeListManager.firstDefragmentation = false;
     assertThat(this.freeListManager.defragment(1)).isFalse();
-    // call defragmen twice for extra code coverage
+    // call defragment twice for extra code coverage
     assertThat(this.freeListManager.defragment(1)).isFalse();
     assertThat(this.freeListManager.getFragmentList()).isEmpty();
   }
@@ -430,6 +468,7 @@ public class FreeListManagerTest {
     this.freeListManager.allocate(DEFAULT_SLAB_SIZE-8-(OffHeapStoredObject.MIN_CHUNK_SIZE-1));
     this.freeListManager.allocate(MEDIUM_SLAB-8-(OffHeapStoredObject.MIN_CHUNK_SIZE-1));
     
+    this.freeListManager.firstDefragmentation = false;
     assertThat(this.freeListManager.defragment(SMALL_SLAB)).isTrue();
   }
  @Test
@@ -806,6 +845,34 @@ public class FreeListManagerTest {
     assertThat(spy.getFragmentation()).isEqualTo(67); //Math.rint(66.66)
   }
   
+  @Test
+  public void isAdjacentBoundaryConditions() {
+    SlabImpl chunk = new SlabImpl(10);
+    this.freeListManager = createFreeListManager(ma, new Slab[] {chunk});
+    
+    assertThat(!this.freeListManager.isAdjacent(Long.MAX_VALUE-4, 4, Long.MAX_VALUE+1));
+    assertThat(this.freeListManager.isAdjacent(Long.MAX_VALUE-4, 4, Long.MAX_VALUE));
+    assertThat(this.freeListManager.isAdjacent(-8L, 4, -4L));
+    long lowAddr = Long.MAX_VALUE;
+    long highAddr = lowAddr + 4;
+    assertThat(this.freeListManager.isAdjacent(lowAddr, 4, highAddr));
+    assertThat(!this.freeListManager.isAdjacent(lowAddr, 4, highAddr-1));
+    assertThat(!this.freeListManager.isAdjacent(lowAddr, 4, highAddr+1));
+    lowAddr = highAddr;
+    highAddr = lowAddr + 4;
+    assertThat(this.freeListManager.isAdjacent(lowAddr, 4, highAddr));
+    assertThat(!this.freeListManager.isAdjacent(highAddr, 4, lowAddr));
+  }
+  @Test
+  public void isSmallEnoughBoundaryConditions() {
+    SlabImpl chunk = new SlabImpl(10);
+    this.freeListManager = createFreeListManager(ma, new Slab[] {chunk});
+    
+    assertThat(this.freeListManager.isSmallEnough(Integer.MAX_VALUE));
+    assertThat(this.freeListManager.isSmallEnough(Integer.MAX_VALUE-1));
+    assertThat(!this.freeListManager.isSmallEnough(Integer.MAX_VALUE+1L));
+    assertThat(!this.freeListManager.isSmallEnough(Long.MAX_VALUE));
+  }
   /**
    * Just like Fragment except that the first time allocate is called
    * it returns false indicating that the allocate failed.
@@ -832,9 +899,15 @@ public class FreeListManagerTest {
   private static class TestableFreeListManager extends FreeListManager {
     private boolean firstTime = true;
     private boolean firstDefragmentation = true;
+    private final int maxCombine;
 
     public TestableFreeListManager(MemoryAllocatorImpl ma, Slab[] slabs) {
+      this(ma, slabs, 0);
+    }
+
+    public TestableFreeListManager(MemoryAllocatorImpl ma, Slab[] slabs, int maxCombine) {
       super(ma, slabs);
+      this.maxCombine = maxCombine;
     }
 
     @Override
@@ -860,8 +933,19 @@ public class FreeListManagerTest {
         this.firstDefragmentation = false;
         // Force defragmentation into thinking a concurrent defragmentation happened.
         this.defragmentationCount.incrementAndGet();
+      } else {
+        super.afterDefragmentationCountFetched();
+      }
+    }
+    @Override
+    boolean isSmallEnough(long size) {
+      if (this.maxCombine != 0) {
+        return size <= this.maxCombine;
+      } else {
+        return super.isSmallEnough(size);
       }
     }
+
     
   }
 }