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

[12/33] incubator-geode git commit: GEODE-1101: rename SimpleMemoryAllocator to MemoryAllocator

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/82faa8af/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/LifecycleListenerJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/LifecycleListenerJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/LifecycleListenerJUnitTest.java
index a009661..e10ca0a 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/LifecycleListenerJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/LifecycleListenerJUnitTest.java
@@ -54,7 +54,7 @@ public class LifecycleListenerJUnitTest {
     this.afterCreateCallbacks.clear();
     this.afterReuseCallbacks.clear();
     this.beforeCloseCallbacks.clear();
-    SimpleMemoryAllocatorImpl.freeOffHeapMemory();
+    MemoryAllocatorImpl.freeOffHeapMemory();
   }
 
   @Test
@@ -63,7 +63,7 @@ public class LifecycleListenerJUnitTest {
     LifecycleListener.removeLifecycleListener(this.listener);
 
     SlabImpl slab = new SlabImpl(1024); // 1k
-    SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(),
+    MemoryAllocatorImpl ma = MemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(),
         new SlabImpl[] { slab });
 
     Assert.assertEquals(0, this.afterCreateCallbacks.size());
@@ -83,7 +83,7 @@ public class LifecycleListenerJUnitTest {
   public void testCallbacksAreCalledAfterCreate() {
     LifecycleListener.addLifecycleListener(this.listener);
     SlabImpl slab = new SlabImpl(1024); // 1k
-    SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(),
+    MemoryAllocatorImpl ma = MemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(),
         new SlabImpl[] { slab });
 
     Assert.assertEquals(1, this.afterCreateCallbacks.size());
@@ -104,10 +104,10 @@ public class LifecycleListenerJUnitTest {
 
     LifecycleListener.addLifecycleListener(this.listener);
 
-    System.setProperty(SimpleMemoryAllocatorImpl.FREE_OFF_HEAP_MEMORY_PROPERTY, "false");
+    System.setProperty(MemoryAllocatorImpl.FREE_OFF_HEAP_MEMORY_PROPERTY, "false");
 
     SlabImpl slab = new SlabImpl(1024); // 1k
-    SimpleMemoryAllocatorImpl ma = createAllocator(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new SlabImpl[] { slab });
+    MemoryAllocatorImpl ma = createAllocator(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new SlabImpl[] { slab });
 
     Assert.assertEquals(1, this.afterCreateCallbacks.size());
     Assert.assertEquals(0, this.afterReuseCallbacks.size());
@@ -125,7 +125,7 @@ public class LifecycleListenerJUnitTest {
     Assert.assertEquals(1, this.afterReuseCallbacks.size());
     Assert.assertEquals(1, this.beforeCloseCallbacks.size());
 
-    SimpleMemoryAllocatorImpl ma2 = createAllocator(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new SlabImpl[] { slab });
+    MemoryAllocatorImpl ma2 = createAllocator(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new SlabImpl[] { slab });
     assertEquals(null, ma2);
     
     Assert.assertEquals(1, this.afterCreateCallbacks.size());
@@ -139,20 +139,20 @@ public class LifecycleListenerJUnitTest {
     Assert.assertEquals(2, this.beforeCloseCallbacks.size());
   }
 
-  private SimpleMemoryAllocatorImpl createAllocator(OutOfOffHeapMemoryListener ooohml, OffHeapMemoryStats ohms, SlabImpl[] slab) {
+  private MemoryAllocatorImpl createAllocator(OutOfOffHeapMemoryListener ooohml, OffHeapMemoryStats ohms, SlabImpl[] slab) {
     try {
-       return SimpleMemoryAllocatorImpl.createForUnitTest(ooohml, ohms, slab);
+       return MemoryAllocatorImpl.createForUnitTest(ooohml, ohms, slab);
     } catch (IllegalStateException e) {
       return null;
     }
   }
   
-  private void closeAndFree(SimpleMemoryAllocatorImpl ma) {
-    System.setProperty(SimpleMemoryAllocatorImpl.FREE_OFF_HEAP_MEMORY_PROPERTY, "true");
+  private void closeAndFree(MemoryAllocatorImpl ma) {
+    System.setProperty(MemoryAllocatorImpl.FREE_OFF_HEAP_MEMORY_PROPERTY, "true");
     try {
       ma.close();
     } finally {
-      System.clearProperty(SimpleMemoryAllocatorImpl.FREE_OFF_HEAP_MEMORY_PROPERTY);
+      System.clearProperty(MemoryAllocatorImpl.FREE_OFF_HEAP_MEMORY_PROPERTY);
     }
   }
   
@@ -162,7 +162,7 @@ public class LifecycleListenerJUnitTest {
     LifecycleListener.addLifecycleListener(this.listener);
 
     SlabImpl slab = new SlabImpl(1024); // 1k
-    SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new SlabImpl[] { slab });
+    MemoryAllocatorImpl ma = MemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new SlabImpl[] { slab });
 
     Assert.assertEquals(1, this.afterCreateCallbacks.size());
     Assert.assertEquals(0, this.afterReuseCallbacks.size());
@@ -175,7 +175,7 @@ public class LifecycleListenerJUnitTest {
     Assert.assertEquals(1, this.beforeCloseCallbacks.size());
 
     slab = new SlabImpl(1024); // 1k
-    SimpleMemoryAllocatorImpl ma2 = SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new SlabImpl[] { slab });
+    MemoryAllocatorImpl ma2 = MemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new SlabImpl[] { slab });
 
     Assert.assertEquals(2, this.afterCreateCallbacks.size());
     Assert.assertEquals(0, this.afterReuseCallbacks.size());
@@ -189,11 +189,11 @@ public class LifecycleListenerJUnitTest {
   }
 
   static final class LifecycleListenerCallback {
-    private final SimpleMemoryAllocatorImpl allocator;
+    private final MemoryAllocatorImpl allocator;
     private final long timeStamp;
     private final Throwable creationTime;
 
-    LifecycleListenerCallback(SimpleMemoryAllocatorImpl allocator) {
+    LifecycleListenerCallback(MemoryAllocatorImpl allocator) {
       this.allocator = allocator;
       this.timeStamp = System.currentTimeMillis();
       this.creationTime = new Exception();
@@ -213,17 +213,17 @@ public class LifecycleListenerJUnitTest {
     }
 
     @Override
-    public void afterCreate(SimpleMemoryAllocatorImpl allocator) {
+    public void afterCreate(MemoryAllocatorImpl allocator) {
       this.afterCreateCallbacks.add(new LifecycleListenerCallback(allocator));
     }
 
     @Override
-    public void afterReuse(SimpleMemoryAllocatorImpl allocator) {
+    public void afterReuse(MemoryAllocatorImpl allocator) {
       this.afterReuseCallbacks.add(new LifecycleListenerCallback(allocator));
     }
 
     @Override
-    public void beforeClose(SimpleMemoryAllocatorImpl allocator) {
+    public void beforeClose(MemoryAllocatorImpl allocator) {
       this.beforeCloseCallbacks.add(new LifecycleListenerCallback(allocator));
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/82faa8af/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/MemoryAllocatorFillPatternIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/MemoryAllocatorFillPatternIntegrationTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/MemoryAllocatorFillPatternIntegrationTest.java
new file mode 100644
index 0000000..2f202f8
--- /dev/null
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/MemoryAllocatorFillPatternIntegrationTest.java
@@ -0,0 +1,246 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.gemstone.gemfire.internal.offheap;
+
+import static org.junit.Assert.*;
+
+import java.util.Collections;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.TimeUnit;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+
+/**
+ * Tests fill pattern validation for the {@link MemoryAllocatorImpl}.
+ */
+@Category(IntegrationTest.class)
+public class MemoryAllocatorFillPatternIntegrationTest {
+  private static Random random = ThreadLocalRandom.current();
+
+  /**
+   * Chunk operation types.
+   */
+  static enum Operation {
+    ALLOCATE,
+    FREE,
+    WRITE;
+    
+    // Holds all Operation values
+    private static Operation[] values = Operation.values(); 
+    
+    static Operation randomOperation() {
+      return values[random.nextInt(values.length)];
+    }
+  };
+  
+  /** Number of worker threads for advanced tests. */
+  private static final int WORKER_THREAD_COUNT = 5;
+  
+  /** Size of single test slab.*/
+  private static final int SLAB_SIZE = 1024 * 1024 * 50;
+  
+  /** Maximum number of bytes a worker thread can allocate during advanced tests.  */
+  private static final int MAX_WORKER_ALLOCATION_TOTAL_SIZE = SLAB_SIZE / WORKER_THREAD_COUNT / 2;
+  
+  /** Maximum allocation for a single Chunk.  */
+  private static final int MAX_WORKER_ALLOCATION_SIZE = 512;
+  
+  /** Canned data for write operations. */
+  private static final byte[] WRITE_BYTES = new String("Some string data.").getBytes();
+  
+  /** Minimum size for write operations. */
+  private static final int MIN_WORKER_ALLOCATION_SIZE = WRITE_BYTES.length;
+
+  /** Runtime for worker threads. */
+  private static final long RUN_TIME_IN_MILLIS = 1 * 1000 * 5;
+  
+  /** Chunk size for basic huge allocation test. */
+  private static final int HUGE_CHUNK_SIZE = 1024 * 200;
+  
+  /** Our test victim. */
+  private MemoryAllocatorImpl allocator = null;
+  
+  /** Our test victim's memory slab. */
+  private SlabImpl slab = null;
+
+  /**
+   * Enables fill validation and creates the test victim.
+   */
+  @Before
+  public void setUp() throws Exception {
+    System.setProperty("gemfire.validateOffHeapWithFill", "true");
+    this.slab = new SlabImpl(SLAB_SIZE);
+    this.allocator = MemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new SlabImpl[]{this.slab});
+  }
+
+  /**
+   * Frees off heap memory.
+   */
+  @After
+  public void tearDown() throws Exception {
+    MemoryAllocatorImpl.freeOffHeapMemory();
+    System.clearProperty("gemfire.validateOffHeapWithFill");
+  }
+  
+  /**
+   * This test hammers a MemoryAllocatorImpl with multiple threads exercising
+   * the fill validation of tiny Chunks for one minute.  This, of course, exercises many aspects of
+   * the MemoryAllocatorImpl and its helper classes.
+   * @throws Exception
+   */
+  @Test
+  public void testFillPatternAdvancedForTinyAllocations() throws Exception { 
+    doFillPatternAdvancedTest(new ChunkSizer() {
+      @Override
+      public int allocationSize() {
+        int allocation = random.nextInt(MAX_WORKER_ALLOCATION_SIZE+1);
+        
+        while(allocation < MIN_WORKER_ALLOCATION_SIZE) {
+          allocation = random.nextInt(MAX_WORKER_ALLOCATION_SIZE+1);
+        }
+        return allocation;
+      }
+    });
+  }
+
+  /**
+   * This test hammers a MemoryAllocatorImpl with multiple threads exercising
+   * the fill validation of huge Chunks for one minute.  This, of course, exercises many aspects of
+   * the MemoryAllocatorImpl and its helper classes.
+   * @throws Exception
+   */
+  @Test
+  public void testFillPatternAdvancedForHugeAllocations() throws Exception {
+    doFillPatternAdvancedTest(new ChunkSizer() {
+      @Override
+      public int allocationSize() {
+        return HUGE_CHUNK_SIZE;
+      }
+    });
+  }
+  
+  private interface ChunkSizer {
+    int allocationSize();
+  }
+  
+  private void doFillPatternAdvancedTest(final ChunkSizer chunkSizer) throws InterruptedException {
+    // Used to manage worker thread completion
+    final CountDownLatch latch = new CountDownLatch(WORKER_THREAD_COUNT);
+    
+    // Use to track any errors the worker threads will encounter
+    final List<Throwable> threadErrorList = Collections.synchronizedList(new LinkedList<Throwable>());
+    
+    /*
+     * Start up a number of worker threads.  These threads will randomly allocate, free,
+     * and write to Chunks.
+     */
+    for(int i = 0;i < WORKER_THREAD_COUNT;++i) {
+      new Thread(new Runnable() {
+        // Total allocation in bytes for this thread
+        private int totalAllocation = 0;
+        
+        // List of Chunks allocated by this thread
+        private List<OffHeapStoredObject> chunks = new LinkedList<OffHeapStoredObject>();
+        
+        // Time to end thread execution
+        private long endTime = System.currentTimeMillis() + RUN_TIME_IN_MILLIS;
+        
+        /**
+         * Allocates a chunk and adds it to the thread's Chunk list.
+         */
+        private void allocate() {          
+          int allocation = chunkSizer.allocationSize();
+          OffHeapStoredObject chunk = (OffHeapStoredObject) allocator.allocate(allocation);
+          
+          // This should always work just after allocation
+          chunk.validateFill();
+          
+          chunks.add(chunk);
+          totalAllocation += chunk.getSize();
+        }
+        
+        /**
+         * Frees a random chunk from the Chunk list.
+         */
+        private void free() {
+          OffHeapStoredObject chunk = chunks.remove(random.nextInt(chunks.size()));
+          totalAllocation -= chunk.getSize();
+          
+          /*
+           * Chunk is filled here but another thread may have already grabbed it so we
+           * cannot validate the fill.
+           */
+          chunk.release(); 
+        }
+        
+        /**
+         * Writes canned data to a random Chunk from the Chunk list.
+         */
+        private void write() {
+          OffHeapStoredObject chunk = chunks.get(random.nextInt(chunks.size()));
+          chunk.writeDataBytes(0, WRITE_BYTES);
+        }
+        
+        /**
+         * Randomly selects Chunk operations and executes them
+         * for a period of time.  Collects any error thrown during execution.
+         */
+        @Override
+        public void run() {
+          try {
+            for(long currentTime = System.currentTimeMillis();currentTime < endTime;currentTime = System.currentTimeMillis()) {
+              Operation op = (totalAllocation == 0 ? Operation.ALLOCATE : (totalAllocation >= MAX_WORKER_ALLOCATION_TOTAL_SIZE ? Operation.FREE : Operation.randomOperation()));
+              switch(op) {
+              case ALLOCATE:
+                allocate();
+                break;
+              case FREE:
+                free();
+                break;
+              case WRITE:
+                write();
+                break;
+              }
+            }
+          } catch (Throwable t) {
+            threadErrorList.add(t);
+          } finally {
+            latch.countDown();
+          }
+       }        
+      }).start();
+    }
+    
+    // Make sure each thread ended cleanly
+    assertTrue(latch.await(2, TimeUnit.MINUTES));
+    
+    // Fail on the first error we find
+    if(!threadErrorList.isEmpty()) {
+      fail(threadErrorList.get(0).getMessage());
+    }
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/82faa8af/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/MemoryAllocatorFillPatternJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/MemoryAllocatorFillPatternJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/MemoryAllocatorFillPatternJUnitTest.java
new file mode 100644
index 0000000..f1d223d
--- /dev/null
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/MemoryAllocatorFillPatternJUnitTest.java
@@ -0,0 +1,183 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.gemstone.gemfire.internal.offheap;
+
+import static org.junit.Assert.*;
+import static com.googlecode.catchexception.CatchException.*;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+
+import junit.framework.TestCase;
+
+/**
+ * Tests fill pattern validation for the {@link MemoryAllocatorImpl}.
+ * @author rholmes
+ */
+@Category(UnitTest.class)
+public class MemoryAllocatorFillPatternJUnitTest {
+  
+  /** Size of single test slab.*/
+  private static final int SLAB_SIZE = 1024 * 1024 * 50;
+  
+  /** Canned data for write operations. */
+  private static final byte[] WRITE_BYTES = new String("Some string data.").getBytes();
+  
+  /** Chunk size for basic huge allocation test. */
+  private static final int HUGE_CHUNK_SIZE = 1024 * 200;
+  
+  /** The number of chunks to allocate in order to force compaction. */
+  private static final int COMPACTION_CHUNKS = 3;
+  
+  /** Our slab size divided in three (with some padding for safety). */
+  private static final int COMPACTION_CHUNK_SIZE = (SLAB_SIZE / COMPACTION_CHUNKS) - 1024;
+  
+  /** This should force compaction when allocated. */
+  private static final int FORCE_COMPACTION_CHUNK_SIZE = COMPACTION_CHUNK_SIZE * 2;
+
+  /** Our test victim. */
+  private MemoryAllocatorImpl allocator = null;
+  
+  /** Our test victim's memory slab. */
+  private SlabImpl slab = null;
+
+  /**
+   * Enables fill validation and creates the test victim.
+   */
+  @Before
+  public void setUp() throws Exception {
+    System.setProperty("gemfire.validateOffHeapWithFill", "true");
+    this.slab = new SlabImpl(SLAB_SIZE);
+    this.allocator = MemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new SlabImpl[]{this.slab});
+  }
+
+  /**
+   * Frees off heap memory.
+   */
+  @After
+  public void tearDown() throws Exception {
+    MemoryAllocatorImpl.freeOffHeapMemory();
+    System.clearProperty("gemfire.validateOffHeapWithFill");
+  }
+
+  /**
+   * This tests the fill pattern for a single tiny Chunk allocation.
+   * @throws Exception
+   */
+  @Test
+  public void testFillPatternBasicForTinyAllocations() throws Exception {
+    doFillPatternBasic(1024);
+  }
+  
+  /**
+   * This tests the fill pattern for a single huge Chunk allocation.
+   * @throws Exception
+   */
+  @Test
+  public void testFillPatternBasicForHugeAllocations() throws Exception {
+    doFillPatternBasic(HUGE_CHUNK_SIZE);
+  }
+  
+  private void doFillPatternBasic(final int chunkSize) {
+    /*
+     * Pull a chunk off the fragment.  This will have no fill because
+     * it is a "fresh" chunk.
+     */
+    OffHeapStoredObject chunk = (OffHeapStoredObject) this.allocator.allocate(chunkSize);
+
+    /*
+     * Chunk should have valid fill from initial fragment allocation.
+     */
+    chunk.validateFill();
+         
+    // "Dirty" the chunk so the release has something to fill over
+    chunk.writeDataBytes(OffHeapStoredObject.MIN_CHUNK_SIZE + 1, WRITE_BYTES);
+
+    // This should free the Chunk (ref count == 1)
+    chunk.release();
+
+    /*
+     * This chunk should have a fill because it was reused from the
+     * free list (assuming no fragmentation at this point...)
+     */
+    chunk = (OffHeapStoredObject) this.allocator.allocate(chunkSize);
+    
+    // Make sure we have a fill this time
+    chunk.validateFill();
+    
+    // Give the fill code something to write over during the release
+    chunk.writeDataBytes(OffHeapStoredObject.MIN_CHUNK_SIZE + 1, WRITE_BYTES);
+    chunk.release();
+
+    // Again, make sure the release implemented the fill
+    chunk.validateFill();
+
+    // "Dirty up" the free chunk
+    chunk.writeDataBytes(OffHeapStoredObject.MIN_CHUNK_SIZE + 1, WRITE_BYTES);
+    
+    catchException(chunk).validateFill();
+    assertTrue(caughtException() instanceof IllegalStateException);
+    assertEquals("Fill pattern violated for chunk " + chunk.getAddress() + " with size " + chunk.getSize(), caughtException().getMessage());
+    
+  }
+
+  /**
+   * This tests that fill validation is working properly on newly created fragments after
+   * a compaction.
+   * @throws Exception
+   */
+  @Test
+  public void testFillPatternAfterCompaction() throws Exception {
+    /*
+     * Stores our allocated memory.
+     */
+    OffHeapStoredObject[] allocatedChunks = new OffHeapStoredObject[COMPACTION_CHUNKS];
+    
+    /*
+     * Use up most of our memory
+     * Our memory looks like [      ][      ][      ]
+     */
+    for(int i =0;i < allocatedChunks.length;++i) {
+      allocatedChunks[i] = (OffHeapStoredObject) this.allocator.allocate(COMPACTION_CHUNK_SIZE);
+      allocatedChunks[i].validateFill();
+    }
+
+    /*
+     * Release some of our allocated chunks.
+     */
+    for(int i=0;i < 2;++i) {
+      allocatedChunks[i].release();
+      allocatedChunks[i].validateFill();      
+    }
+    
+    /*
+     * Now, allocate another chunk that is slightly larger than one of
+     * our initial chunks.  This should force a compaction causing our
+     * memory to look like [            ][      ].
+     */
+    OffHeapStoredObject slightlyLargerChunk = (OffHeapStoredObject) this.allocator.allocate(FORCE_COMPACTION_CHUNK_SIZE);
+    
+    /*
+     * Make sure the compacted memory has the fill validation.
+     */
+    slightlyLargerChunk.validateFill();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/82faa8af/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/MemoryAllocatorJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/MemoryAllocatorJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/MemoryAllocatorJUnitTest.java
new file mode 100644
index 0000000..7639f8d
--- /dev/null
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/MemoryAllocatorJUnitTest.java
@@ -0,0 +1,594 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.gemstone.gemfire.internal.offheap;
+
+import static org.junit.Assert.*;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.concurrent.atomic.AtomicReference;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.contrib.java.lang.system.RestoreSystemProperties;
+import org.junit.experimental.categories.Category;
+
+import com.gemstone.gemfire.OutOfOffHeapMemoryException;
+import com.gemstone.gemfire.cache.CacheClosedException;
+import com.gemstone.gemfire.internal.logging.NullLogWriter;
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+
+@Category(UnitTest.class)
+public class MemoryAllocatorJUnitTest {
+  @Rule
+  public final RestoreSystemProperties restoreSystemProperties = new RestoreSystemProperties();
+
+  private static int round(int multiple, int v) {
+    return ((v+multiple-1)/multiple)*multiple;
+  }
+  @Test
+  public void testNullGetAllocator() {
+    try {
+      MemoryAllocatorImpl.getAllocator();
+      fail("expected CacheClosedException");
+    } catch (CacheClosedException expected) {
+    }
+  }
+  @Test
+  public void testConstructor() {
+    try {
+      MemoryAllocatorImpl.createForUnitTest(null, null, null);
+      fail("expected IllegalArgumentException");
+    } catch (IllegalArgumentException expected) {
+    }
+  }
+  @Test
+  public void testCreate() {
+    System.setProperty(MemoryAllocatorImpl.FREE_OFF_HEAP_MEMORY_PROPERTY, "false");
+    {
+      NullOutOfOffHeapMemoryListener listener = new NullOutOfOffHeapMemoryListener();
+      NullOffHeapMemoryStats stats = new NullOffHeapMemoryStats();
+      try {
+        MemoryAllocatorImpl.createForUnitTest(listener, stats, 10, 950, 100, new SlabFactory() {
+     @Override
+     public Slab create(int size) {
+        throw new OutOfMemoryError("expected");
+     }
+    });
+      } catch (OutOfMemoryError expected) {
+      }
+      assertTrue(listener.isClosed());
+      assertTrue(stats.isClosed());
+     }
+    {
+      NullOutOfOffHeapMemoryListener listener = new NullOutOfOffHeapMemoryListener();
+      NullOffHeapMemoryStats stats = new NullOffHeapMemoryStats();
+      int MAX_SLAB_SIZE = 100;
+      try {
+        SlabFactory factory = new SlabFactory() {
+          private int createCount = 0;
+          @Override
+          public Slab create(int size) {
+            createCount++;
+            if (createCount == 1) {
+              return new SlabImpl(size);
+            } else {
+              throw new OutOfMemoryError("expected");
+            }
+          }
+        };
+        MemoryAllocatorImpl.createForUnitTest(listener, stats, 10, 950, MAX_SLAB_SIZE, factory);
+      } catch (OutOfMemoryError expected) {
+      }
+      assertTrue(listener.isClosed());
+      assertTrue(stats.isClosed());
+    }
+    {
+      NullOutOfOffHeapMemoryListener listener = new NullOutOfOffHeapMemoryListener();
+      NullOffHeapMemoryStats stats = new NullOffHeapMemoryStats();
+      SlabFactory factory = new SlabFactory() {
+        @Override
+        public Slab create(int size) {
+          return new SlabImpl(size);
+        }
+      };
+      MemoryAllocator ma = 
+        MemoryAllocatorImpl.createForUnitTest(listener, stats, 10, 950, 100, factory);
+      try {
+        assertFalse(listener.isClosed());
+        assertFalse(stats.isClosed());
+        ma.close();
+        assertTrue(listener.isClosed());
+        assertFalse(stats.isClosed());
+        listener = new NullOutOfOffHeapMemoryListener();
+        NullOffHeapMemoryStats stats2 = new NullOffHeapMemoryStats();
+        {
+          SlabImpl slab = new SlabImpl(1024);
+          try {
+            MemoryAllocatorImpl.createForUnitTest(listener, stats2, new SlabImpl[]{slab});
+          } catch (IllegalStateException expected) {
+            assertTrue("unexpected message: " + expected.getMessage(), 
+                expected.getMessage().equals("attempted to reuse existing off-heap memory even though new off-heap memory was allocated"));
+          } finally {
+            slab.free();
+          }
+          assertFalse(stats.isClosed());
+          assertTrue(listener.isClosed());
+          assertTrue(stats2.isClosed());
+        }
+        listener = new NullOutOfOffHeapMemoryListener();
+        stats2 = new NullOffHeapMemoryStats();
+        MemoryAllocator ma2 = MemoryAllocatorImpl.createForUnitTest(listener, stats2, 10, 950, 100, factory);
+        assertSame(ma, ma2);
+        assertTrue(stats.isClosed());
+        assertFalse(listener.isClosed());
+        assertFalse(stats2.isClosed());
+        stats = stats2;
+        ma.close();
+        assertTrue(listener.isClosed());
+        assertFalse(stats.isClosed());
+      } finally {
+        MemoryAllocatorImpl.freeOffHeapMemory();
+      }
+      assertTrue(stats.isClosed());
+    }
+  }
+  @Test
+  public void testBasics() {
+    int BATCH_SIZE = 1;
+    int TINY_MULTIPLE = FreeListManager.TINY_MULTIPLE;
+    int HUGE_MULTIPLE = FreeListManager.HUGE_MULTIPLE;
+    int perObjectOverhead = OffHeapStoredObject.HEADER_SIZE;
+    int maxTiny = FreeListManager.MAX_TINY-perObjectOverhead;
+    int minHuge = maxTiny+1;
+    int TOTAL_MEM = (maxTiny+perObjectOverhead)*BATCH_SIZE /*+ (maxBig+perObjectOverhead)*BATCH_SIZE*/ + round(TINY_MULTIPLE, minHuge+1+perObjectOverhead)*BATCH_SIZE + (TINY_MULTIPLE+perObjectOverhead)*BATCH_SIZE /*+ (MIN_BIG_SIZE+perObjectOverhead)*BATCH_SIZE*/ + round(TINY_MULTIPLE, minHuge+perObjectOverhead+1);
+    SlabImpl slab = new SlabImpl(TOTAL_MEM);
+    try {
+      MemoryAllocatorImpl ma = MemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new SlabImpl[]{slab});
+      assertEquals(TOTAL_MEM, ma.getFreeMemory());
+      assertEquals(TOTAL_MEM, ma.freeList.getFreeFragmentMemory());
+      assertEquals(0, ma.freeList.getFreeTinyMemory());
+      assertEquals(0, ma.freeList.getFreeHugeMemory());
+      StoredObject tinymc = ma.allocate(maxTiny);
+      assertEquals(TOTAL_MEM-round(TINY_MULTIPLE, maxTiny+perObjectOverhead), ma.getFreeMemory());
+      assertEquals(round(TINY_MULTIPLE, maxTiny+perObjectOverhead)*(BATCH_SIZE-1), ma.freeList.getFreeTinyMemory());
+      StoredObject hugemc = ma.allocate(minHuge);
+      assertEquals(TOTAL_MEM-round(TINY_MULTIPLE, minHuge+perObjectOverhead)/*-round(BIG_MULTIPLE, maxBig+perObjectOverhead)*/-round(TINY_MULTIPLE, maxTiny+perObjectOverhead), ma.getFreeMemory());
+      long freeSlab = ma.freeList.getFreeFragmentMemory();
+      long oldFreeHugeMemory = ma.freeList.getFreeHugeMemory();
+      assertEquals(round(TINY_MULTIPLE, minHuge+perObjectOverhead)*(BATCH_SIZE-1), oldFreeHugeMemory);
+      hugemc.release();
+      assertEquals(round(TINY_MULTIPLE, minHuge+perObjectOverhead), ma.freeList.getFreeHugeMemory()-oldFreeHugeMemory);
+      assertEquals(TOTAL_MEM/*-round(BIG_MULTIPLE, maxBig+perObjectOverhead)*/-round(TINY_MULTIPLE, maxTiny+perObjectOverhead), ma.getFreeMemory());
+      assertEquals(TOTAL_MEM-round(TINY_MULTIPLE, maxTiny+perObjectOverhead), ma.getFreeMemory());
+      long oldFreeTinyMemory = ma.freeList.getFreeTinyMemory();
+      tinymc.release();
+      assertEquals(round(TINY_MULTIPLE, maxTiny+perObjectOverhead), ma.freeList.getFreeTinyMemory()-oldFreeTinyMemory);
+      assertEquals(TOTAL_MEM, ma.getFreeMemory());
+      // now lets reallocate from the free lists
+      tinymc = ma.allocate(maxTiny);
+      assertEquals(oldFreeTinyMemory, ma.freeList.getFreeTinyMemory());
+      assertEquals(TOTAL_MEM-round(TINY_MULTIPLE, maxTiny+perObjectOverhead), ma.getFreeMemory());
+      hugemc = ma.allocate(minHuge);
+      assertEquals(oldFreeHugeMemory, ma.freeList.getFreeHugeMemory());
+      assertEquals(TOTAL_MEM-round(TINY_MULTIPLE, minHuge+perObjectOverhead)/*-round(BIG_MULTIPLE, maxBig+perObjectOverhead)*/-round(TINY_MULTIPLE, maxTiny+perObjectOverhead), ma.getFreeMemory());
+      hugemc.release();
+      assertEquals(round(TINY_MULTIPLE, minHuge+perObjectOverhead), ma.freeList.getFreeHugeMemory()-oldFreeHugeMemory);
+      assertEquals(TOTAL_MEM/*-round(BIG_MULTIPLE, maxBig+perObjectOverhead)*/-round(TINY_MULTIPLE, maxTiny+perObjectOverhead), ma.getFreeMemory());
+      assertEquals(TOTAL_MEM-round(TINY_MULTIPLE, maxTiny+perObjectOverhead), ma.getFreeMemory());
+      tinymc.release();
+      assertEquals(round(TINY_MULTIPLE, maxTiny+perObjectOverhead), ma.freeList.getFreeTinyMemory()-oldFreeTinyMemory);
+      assertEquals(TOTAL_MEM, ma.getFreeMemory());
+      // None of the reallocates should have come from the slab.
+      assertEquals(freeSlab, ma.freeList.getFreeFragmentMemory());
+      tinymc = ma.allocate(1);
+      assertEquals(round(TINY_MULTIPLE, 1+perObjectOverhead), tinymc.getSize());
+      assertEquals(freeSlab-(round(TINY_MULTIPLE, 1+perObjectOverhead)*BATCH_SIZE), ma.freeList.getFreeFragmentMemory());
+      freeSlab = ma.freeList.getFreeFragmentMemory();
+      tinymc.release();
+      assertEquals(round(TINY_MULTIPLE, maxTiny+perObjectOverhead)+(round(TINY_MULTIPLE, 1+perObjectOverhead)*BATCH_SIZE), ma.freeList.getFreeTinyMemory()-oldFreeTinyMemory);
+      
+      hugemc = ma.allocate(minHuge+1);
+      assertEquals(round(TINY_MULTIPLE, minHuge+1+perObjectOverhead), hugemc.getSize());
+      assertEquals(round(TINY_MULTIPLE, minHuge+perObjectOverhead)*(BATCH_SIZE-1), ma.freeList.getFreeHugeMemory());
+      hugemc.release();
+      assertEquals(round(TINY_MULTIPLE, minHuge+perObjectOverhead)*BATCH_SIZE, ma.freeList.getFreeHugeMemory());
+      hugemc = ma.allocate(minHuge);
+      assertEquals(round(TINY_MULTIPLE, minHuge+perObjectOverhead)*(BATCH_SIZE-1), ma.freeList.getFreeHugeMemory());
+      if (BATCH_SIZE > 1) {
+        StoredObject hugemc2 = ma.allocate(minHuge);
+        assertEquals(round(TINY_MULTIPLE, minHuge+perObjectOverhead)*(BATCH_SIZE-2), ma.freeList.getFreeHugeMemory());
+        hugemc2.release();
+        assertEquals(round(TINY_MULTIPLE, minHuge+perObjectOverhead)*(BATCH_SIZE-1), ma.freeList.getFreeHugeMemory());
+      }
+      hugemc.release();
+      assertEquals(round(TINY_MULTIPLE, minHuge+perObjectOverhead)*BATCH_SIZE, ma.freeList.getFreeHugeMemory());
+      // now that we do compaction the following allocate works.
+      hugemc = ma.allocate(minHuge + HUGE_MULTIPLE + HUGE_MULTIPLE-1);
+    } finally {
+      MemoryAllocatorImpl.freeOffHeapMemory();
+    }
+  }
+  
+  @Test
+  public void testChunkCreateDirectByteBuffer() {
+    SlabImpl slab = new SlabImpl(1024*1024);
+    try {
+      MemoryAllocatorImpl ma = MemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new SlabImpl[]{slab});
+      ByteBuffer bb = ByteBuffer.allocate(1024);
+      for (int i=0; i < 1024; i++) {
+        bb.put((byte) i);
+      }
+      bb.position(0);
+      OffHeapStoredObject c = (OffHeapStoredObject) ma.allocateAndInitialize(bb.array(), false, false);
+      assertEquals(1024, c.getDataSize());
+      if (!Arrays.equals(bb.array(), c.getRawBytes())) {
+        fail("arrays are not equal. Expected " + Arrays.toString(bb.array()) + " but found: " + Arrays.toString(c.getRawBytes()));
+      }
+      ByteBuffer dbb = c.createDirectByteBuffer();
+      assertEquals(true, dbb.isDirect());
+      assertEquals(bb, dbb);
+    } finally {
+      MemoryAllocatorImpl.freeOffHeapMemory();
+    }
+  }
+  
+  @Test
+  public void testDebugLog() {
+    MemoryAllocatorImpl.debugLog("test debug log", false);
+    MemoryAllocatorImpl.debugLog("test debug log", true);
+  }
+  @Test
+  public void testGetLostChunks() {
+    SlabImpl slab = new SlabImpl(1024*1024);
+    try {
+      MemoryAllocatorImpl ma = MemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new SlabImpl[]{slab});
+      assertEquals(Collections.emptyList(), ma.getLostChunks());
+    } finally {
+      MemoryAllocatorImpl.freeOffHeapMemory();
+    }
+  }
+  @Test
+  public void testFindSlab() {
+    final int SLAB_SIZE = 1024*1024;
+    SlabImpl slab = new SlabImpl(SLAB_SIZE);
+    try {
+      MemoryAllocatorImpl ma = MemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new SlabImpl[]{slab});
+      assertEquals(0, ma.findSlab(slab.getMemoryAddress()));
+      assertEquals(0, ma.findSlab(slab.getMemoryAddress()+SLAB_SIZE-1));
+      try {
+        ma.findSlab(slab.getMemoryAddress()-1);
+        fail("expected IllegalStateException");
+      } catch (IllegalStateException expected) {
+      }
+      try {
+        ma.findSlab(slab.getMemoryAddress()+SLAB_SIZE);
+        fail("expected IllegalStateException");
+      } catch (IllegalStateException expected) {
+      }
+    } finally {
+      MemoryAllocatorImpl.freeOffHeapMemory();
+    }
+  }
+  @Test
+  public void testValidateAddressAndSize() {
+    final int SLAB_SIZE = 1024*1024;
+    SlabImpl slab = new SlabImpl(SLAB_SIZE);
+    try {
+      MemoryAllocatorImpl ma = MemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new SlabImpl[]{slab});
+      try {
+        MemoryAllocatorImpl.validateAddress(0L);
+        fail("expected IllegalStateException");
+      } catch (IllegalStateException expected) {
+        assertEquals("Unexpected exception message: " + expected.getMessage(), true, expected.getMessage().contains("addr was smaller than expected"));
+      }
+      try {
+        MemoryAllocatorImpl.validateAddress(1L);
+        fail("expected IllegalStateException");
+      } catch (IllegalStateException expected) {
+        assertEquals("Unexpected exception message: " + expected.getMessage(), true, expected.getMessage().contains("Valid addresses must be in one of the following ranges:"));
+      }
+      MemoryAllocatorImpl.validateAddressAndSizeWithinSlab(slab.getMemoryAddress(), SLAB_SIZE, false);
+      MemoryAllocatorImpl.validateAddressAndSizeWithinSlab(slab.getMemoryAddress(), SLAB_SIZE, true);
+      MemoryAllocatorImpl.validateAddressAndSizeWithinSlab(slab.getMemoryAddress(), -1, true);
+      try {
+        MemoryAllocatorImpl.validateAddressAndSizeWithinSlab(slab.getMemoryAddress()-1, SLAB_SIZE, true);
+        fail("expected IllegalStateException");
+      } catch (IllegalStateException expected) {
+        assertEquals("Unexpected exception message: " + expected.getMessage(), true, expected.getMessage().equals(" address 0x" + Long.toString(slab.getMemoryAddress()-1, 16) + " does not address the original slab memory"));
+      }
+      try {
+        MemoryAllocatorImpl.validateAddressAndSizeWithinSlab(slab.getMemoryAddress(), SLAB_SIZE+1, true);
+        fail("expected IllegalStateException");
+      } catch (IllegalStateException expected) {
+        assertEquals("Unexpected exception message: " + expected.getMessage(), true, expected.getMessage().equals(" address 0x" + Long.toString(slab.getMemoryAddress()+SLAB_SIZE, 16) + " does not address the original slab memory"));
+      }
+    } finally {
+      MemoryAllocatorImpl.freeOffHeapMemory();
+    }
+  }
+  @Test
+  public void testMemoryInspection() {
+    final int SLAB_SIZE = 1024*1024;
+    SlabImpl slab = new SlabImpl(SLAB_SIZE);
+    try {
+      MemoryAllocatorImpl ma = MemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new SlabImpl[]{slab});
+      MemoryInspector inspector = ma.getMemoryInspector();
+      assertNotNull(inspector);
+      assertEquals(null, inspector.getFirstBlock());
+      assertEquals(Collections.emptyList(), inspector.getSnapshot());
+      assertEquals(Collections.emptyList(), inspector.getAllocatedBlocks());
+      assertEquals(null, inspector.getBlockAfter(null));
+      inspector.createSnapshot();
+      // call this twice for code coverage
+      inspector.createSnapshot();
+      try {
+        assertEquals(inspector.getAllBlocks(), inspector.getSnapshot());
+        MemoryBlock firstBlock = inspector.getFirstBlock();
+        assertNotNull(firstBlock);
+        assertEquals(1024*1024, firstBlock.getBlockSize());
+        assertEquals("N/A", firstBlock.getDataType());
+        assertEquals(-1, firstBlock.getFreeListId());
+        assertTrue(firstBlock.getAddress() > 0);
+        assertNull(firstBlock.getNextBlock());
+        assertEquals(0, firstBlock.getRefCount());
+        assertEquals(0, firstBlock.getSlabId());
+        assertEquals(MemoryBlock.State.UNUSED, firstBlock.getState());
+        assertFalse(firstBlock.isCompressed());
+        assertFalse(firstBlock.isSerialized());
+        assertEquals(null, inspector.getBlockAfter(firstBlock));
+      } finally {
+        inspector.clearSnapshot();
+      }
+    } finally {
+      MemoryAllocatorImpl.freeOffHeapMemory();
+    }
+  }
+
+  @Test
+  public void testClose() {
+    System.setProperty(MemoryAllocatorImpl.FREE_OFF_HEAP_MEMORY_PROPERTY, "false");
+    SlabImpl slab = new SlabImpl(1024*1024);
+    boolean freeSlab = true;
+    SlabImpl[] slabs = new SlabImpl[]{slab};
+    try {
+      MemoryAllocatorImpl ma = MemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), slabs);
+      ma.close();
+      ma.close();
+      System.setProperty(MemoryAllocatorImpl.FREE_OFF_HEAP_MEMORY_PROPERTY, "true");
+      try {
+        ma = MemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), slabs);
+        ma.close();
+        freeSlab = false;
+        ma.close();
+      } finally {
+        System.clearProperty(MemoryAllocatorImpl.FREE_OFF_HEAP_MEMORY_PROPERTY);
+      }
+    } finally {
+      if (freeSlab) {
+        MemoryAllocatorImpl.freeOffHeapMemory();
+      }
+    }
+    
+  }
+  
+  @Test
+  public void testCompaction() {
+    final int perObjectOverhead = OffHeapStoredObject.HEADER_SIZE;
+    final int BIG_ALLOC_SIZE = 150000;
+    final int SMALL_ALLOC_SIZE = BIG_ALLOC_SIZE/2;
+    final int TOTAL_MEM = BIG_ALLOC_SIZE;
+    SlabImpl slab = new SlabImpl(TOTAL_MEM);
+    try {
+      MemoryAllocatorImpl ma = MemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new SlabImpl[]{slab});
+      StoredObject bmc = ma.allocate(BIG_ALLOC_SIZE-perObjectOverhead);
+      try {
+        StoredObject smc = ma.allocate(SMALL_ALLOC_SIZE-perObjectOverhead);
+        fail("Expected out of memory");
+      } catch (OutOfOffHeapMemoryException expected) {
+      }
+      bmc.release();
+      assertEquals(TOTAL_MEM, ma.freeList.getFreeMemory());
+      StoredObject smc1 = ma.allocate(SMALL_ALLOC_SIZE-perObjectOverhead);
+      StoredObject smc2 = ma.allocate(SMALL_ALLOC_SIZE-perObjectOverhead);
+      smc2.release();
+      assertEquals(TOTAL_MEM-SMALL_ALLOC_SIZE, ma.freeList.getFreeMemory());
+      try {
+        bmc = ma.allocate(BIG_ALLOC_SIZE-perObjectOverhead);
+        fail("Expected out of memory");
+      } catch (OutOfOffHeapMemoryException expected) {
+      }
+      smc1.release();
+      assertEquals(TOTAL_MEM, ma.freeList.getFreeMemory());
+      bmc = ma.allocate(BIG_ALLOC_SIZE-perObjectOverhead);
+      bmc.release();
+      assertEquals(TOTAL_MEM, ma.freeList.getFreeMemory());
+      ArrayList<StoredObject> mcs = new ArrayList<StoredObject>();
+      for (int i=0; i < BIG_ALLOC_SIZE/(8+perObjectOverhead); i++) {
+        mcs.add(ma.allocate(8));
+      }
+      checkMcs(mcs);
+      assertEquals(0, ma.freeList.getFreeMemory());
+      try {
+        ma.allocate(8);
+        fail("expected out of memory");
+      } catch (OutOfOffHeapMemoryException expected) {
+      }
+      mcs.remove(0).release(); // frees 8+perObjectOverhead
+      assertEquals(8+perObjectOverhead, ma.freeList.getFreeMemory());
+      mcs.remove(0).release(); // frees 8+perObjectOverhead
+      assertEquals((8+perObjectOverhead)*2, ma.freeList.getFreeMemory());
+      ma.allocate(16).release(); // allocates and frees 16+perObjectOverhead; still have perObjectOverhead
+      assertEquals((8+perObjectOverhead)*2, ma.freeList.getFreeMemory());
+      mcs.remove(0).release(); // frees 8+perObjectOverhead
+      assertEquals((8+perObjectOverhead)*3, ma.freeList.getFreeMemory());
+      mcs.remove(0).release(); // frees 8+perObjectOverhead
+      assertEquals((8+perObjectOverhead)*4, ma.freeList.getFreeMemory());
+      // At this point I should have 8*4 + perObjectOverhead*4 of free memory
+      ma.allocate(8*4+perObjectOverhead*3).release();
+      assertEquals((8+perObjectOverhead)*4, ma.freeList.getFreeMemory());
+      mcs.remove(0).release(); // frees 8+perObjectOverhead
+      assertEquals((8+perObjectOverhead)*5, ma.freeList.getFreeMemory());
+      // At this point I should have 8*5 + perObjectOverhead*5 of free memory
+      try {
+        ma.allocate((8*5+perObjectOverhead*4)+1);
+        fail("expected out of memory");
+      } catch (OutOfOffHeapMemoryException expected) {
+      }
+      mcs.remove(0).release(); // frees 8+perObjectOverhead
+      assertEquals((8+perObjectOverhead)*6, ma.freeList.getFreeMemory());
+      checkMcs(mcs);
+      // At this point I should have 8*6 + perObjectOverhead*6 of free memory
+      StoredObject mc24 = ma.allocate(24);
+      checkMcs(mcs);
+      assertEquals((8+perObjectOverhead)*6 - (24+perObjectOverhead), ma.freeList.getFreeMemory());
+      // At this point I should have 8*3 + perObjectOverhead*5 of free memory
+      StoredObject mc16 = ma.allocate(16);
+      checkMcs(mcs);
+      assertEquals((8+perObjectOverhead)*6 - (24+perObjectOverhead) - (16+perObjectOverhead), ma.freeList.getFreeMemory());
+      // At this point I should have 8*1 + perObjectOverhead*4 of free memory
+      mcs.add(ma.allocate(8));
+      checkMcs(mcs);
+      assertEquals((8+perObjectOverhead)*6 - (24+perObjectOverhead) - (16+perObjectOverhead) - (8+perObjectOverhead), ma.freeList.getFreeMemory());
+      // At this point I should have 8*0 + perObjectOverhead*3 of free memory
+      StoredObject mcDO = ma.allocate(perObjectOverhead*2);
+      checkMcs(mcs);
+      // At this point I should have 8*0 + perObjectOverhead*0 of free memory
+      assertEquals(0, ma.freeList.getFreeMemory());
+      try {
+        ma.allocate(1);
+        fail("expected out of memory");
+      } catch (OutOfOffHeapMemoryException expected) {
+      }
+      checkMcs(mcs);
+      assertEquals(0, ma.freeList.getFreeMemory());
+      mcDO.release();
+      assertEquals((perObjectOverhead*3), ma.freeList.getFreeMemory());
+      mcs.remove(mcs.size()-1).release();
+      assertEquals((perObjectOverhead*3)+(8+perObjectOverhead), ma.freeList.getFreeMemory());
+      mc16.release();
+      assertEquals((perObjectOverhead*3)+(8+perObjectOverhead)+(16+perObjectOverhead), ma.freeList.getFreeMemory());
+      mc24.release();
+      assertEquals((perObjectOverhead*3)+(8+perObjectOverhead)+(16+perObjectOverhead)+(24+perObjectOverhead), ma.freeList.getFreeMemory());
+      
+      long freeMem = ma.freeList.getFreeMemory();
+      for (StoredObject mc: mcs) {
+        mc.release();
+        assertEquals(freeMem+(8+perObjectOverhead), ma.freeList.getFreeMemory());
+        freeMem += (8+perObjectOverhead);
+      }
+      mcs.clear();
+      assertEquals(TOTAL_MEM, ma.freeList.getFreeMemory());
+      bmc = ma.allocate(BIG_ALLOC_SIZE-perObjectOverhead);
+      bmc.release();
+    } finally {
+      MemoryAllocatorImpl.freeOffHeapMemory();
+    }
+  }
+  
+  long expectedMemoryUsage;
+  boolean memoryUsageEventReceived;
+  @Test
+  public void testUsageEventListener() {
+    final int perObjectOverhead = OffHeapStoredObject.HEADER_SIZE;
+    final int SMALL_ALLOC_SIZE = 1000;
+    SlabImpl slab = new SlabImpl(3000);
+    try {
+      MemoryAllocatorImpl ma = MemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new SlabImpl[]{slab});
+      MemoryUsageListener listener = new MemoryUsageListener() {
+        @Override
+        public void updateMemoryUsed(final long bytesUsed) {
+          MemoryAllocatorJUnitTest.this.memoryUsageEventReceived = true;
+          assertEquals(MemoryAllocatorJUnitTest.this.expectedMemoryUsage, bytesUsed);
+        }
+      };
+      ma.addMemoryUsageListener(listener);
+      
+      this.expectedMemoryUsage = SMALL_ALLOC_SIZE;
+      this.memoryUsageEventReceived = false;
+      StoredObject smc = ma.allocate(SMALL_ALLOC_SIZE-perObjectOverhead);
+      assertEquals(true, this.memoryUsageEventReceived);
+      
+      this.expectedMemoryUsage = SMALL_ALLOC_SIZE * 2;
+      this.memoryUsageEventReceived = false;
+      smc = ma.allocate(SMALL_ALLOC_SIZE-perObjectOverhead);
+      assertEquals(true, this.memoryUsageEventReceived);
+      
+      MemoryUsageListener unaddedListener = new MemoryUsageListener() {
+        @Override
+        public void updateMemoryUsed(final long bytesUsed) {
+          throw new IllegalStateException("Should never be called");
+        }
+      };
+      ma.removeMemoryUsageListener(unaddedListener);
+      
+      ma.removeMemoryUsageListener(listener);
+      
+      ma.removeMemoryUsageListener(unaddedListener);
+
+      this.expectedMemoryUsage = SMALL_ALLOC_SIZE * 2;
+      this.memoryUsageEventReceived = false;
+      smc = ma.allocate(SMALL_ALLOC_SIZE-perObjectOverhead);
+      assertEquals(false, this.memoryUsageEventReceived);
+      
+    } finally {
+      MemoryAllocatorImpl.freeOffHeapMemory();
+    }
+  }
+  private void checkMcs(ArrayList<StoredObject> mcs) {
+    for (StoredObject mc: mcs) {
+      assertEquals(8+8, mc.getSize());
+    }
+  }
+  
+  @Test
+  public void testOutOfOffHeapMemory() {
+    final int perObjectOverhead = OffHeapStoredObject.HEADER_SIZE;
+    final int BIG_ALLOC_SIZE = 150000;
+    final int SMALL_ALLOC_SIZE = BIG_ALLOC_SIZE/2;
+    final int TOTAL_MEM = BIG_ALLOC_SIZE;
+    final SlabImpl slab = new SlabImpl(TOTAL_MEM);
+    final AtomicReference<OutOfOffHeapMemoryException> ooom = new AtomicReference<OutOfOffHeapMemoryException>();
+    final OutOfOffHeapMemoryListener oooml = new OutOfOffHeapMemoryListener() {
+      @Override
+      public void outOfOffHeapMemory(OutOfOffHeapMemoryException cause) {
+        ooom.set(cause);
+      }
+      @Override
+      public void close() {
+      }
+    };
+    try {
+      MemoryAllocatorImpl ma = MemoryAllocatorImpl.createForUnitTest(oooml, new NullOffHeapMemoryStats(), new SlabImpl[]{slab});
+      // make a big allocation
+      StoredObject bmc = ma.allocate(BIG_ALLOC_SIZE-perObjectOverhead);
+      assertNull(ooom.get());
+      // drive the ma to ooom with small allocations
+      try {
+        StoredObject smc = ma.allocate(SMALL_ALLOC_SIZE-perObjectOverhead);
+        fail("Expected out of memory");
+      } catch (OutOfOffHeapMemoryException expected) {
+      }
+      assertNotNull(ooom.get());
+      assertTrue(ooom.get().getMessage().contains("Out of off-heap memory. Could not allocate size of "));
+    } finally {
+      MemoryAllocatorImpl.freeOffHeapMemory();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/82faa8af/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/MemoryBlockNodeJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/MemoryBlockNodeJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/MemoryBlockNodeJUnitTest.java
index e1c3f4e..f0563ad 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/MemoryBlockNodeJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/MemoryBlockNodeJUnitTest.java
@@ -46,7 +46,7 @@ import com.gemstone.gemfire.test.junit.categories.UnitTest;
 @Category(UnitTest.class)
 public class MemoryBlockNodeJUnitTest {
 
-  private SimpleMemoryAllocatorImpl ma;
+  private MemoryAllocatorImpl ma;
   private OutOfOffHeapMemoryListener ooohml;
   private OffHeapMemoryStats stats;
   private Slab[] slabs = {
@@ -82,12 +82,12 @@ public class MemoryBlockNodeJUnitTest {
   public void setUp() {
     ooohml = mock(OutOfOffHeapMemoryListener.class);
     stats = mock(OffHeapMemoryStats.class);
-    ma = (SimpleMemoryAllocatorImpl) SimpleMemoryAllocatorImpl.createForUnitTest(ooohml, stats, slabs);
+    ma = (MemoryAllocatorImpl) MemoryAllocatorImpl.createForUnitTest(ooohml, stats, slabs);
   }
 
   @After
   public void tearDown() {
-    SimpleMemoryAllocatorImpl.freeOffHeapMemory();
+    MemoryAllocatorImpl.freeOffHeapMemory();
   }
   
   protected Object getValue() {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/82faa8af/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapHelperJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapHelperJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapHelperJUnitTest.java
index fa4e776..95b6869 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapHelperJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapHelperJUnitTest.java
@@ -47,7 +47,7 @@ public class OffHeapHelperJUnitTest extends AbstractStoredObjectTestBase {
     OutOfOffHeapMemoryListener ooohml = mock(OutOfOffHeapMemoryListener.class);
     OffHeapMemoryStats stats = mock(OffHeapMemoryStats.class);
 
-    ma = SimpleMemoryAllocatorImpl.create(ooohml, stats, 3, OffHeapStorage.MIN_SLAB_SIZE * 3, OffHeapStorage.MIN_SLAB_SIZE);
+    ma = MemoryAllocatorImpl.create(ooohml, stats, 3, OffHeapStorage.MIN_SLAB_SIZE * 3, OffHeapStorage.MIN_SLAB_SIZE);
 
   }
 
@@ -71,7 +71,7 @@ public class OffHeapHelperJUnitTest extends AbstractStoredObjectTestBase {
 
   @After
   public void tearDown() {
-    SimpleMemoryAllocatorImpl.freeOffHeapMemory();
+    MemoryAllocatorImpl.freeOffHeapMemory();
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/82faa8af/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapIndexJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapIndexJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapIndexJUnitTest.java
index ec494e8..571060b 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapIndexJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapIndexJUnitTest.java
@@ -57,7 +57,7 @@ public class OffHeapIndexJUnitTest {
   @After
   public void tearDown() {
     this.gfc.close();
-    SimpleMemoryAllocatorImpl.freeOffHeapMemory();
+    MemoryAllocatorImpl.freeOffHeapMemory();
     // TODO cleanup default disk store files
   }
   

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/82faa8af/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapRegionBase.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapRegionBase.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapRegionBase.java
index ae1b35d..fb1aa41 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapRegionBase.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapRegionBase.java
@@ -75,7 +75,7 @@ public abstract class OffHeapRegionBase {
   private void closeCache(GemFireCacheImpl gfc, boolean keepOffHeapAllocated) {
     gfc.close();
     if (!keepOffHeapAllocated) {
-      SimpleMemoryAllocatorImpl.freeOffHeapMemory();
+      MemoryAllocatorImpl.freeOffHeapMemory();
     }
     // TODO cleanup default disk store files
   }
@@ -100,7 +100,7 @@ public abstract class OffHeapRegionBase {
       assertEquals(0, ma.getUsedMemory());
       // do an allocation larger than the slab size
       // TODO: currently the compact will product slabs bigger than the max slab size
-      // (see the todo comment on compact() in SimpleMemoryAllocator).
+      // (see the todo comment on compact() in FreeListManager).
       // So we request 20m here since that it the total size.
       try {
         ma.allocate(1024*1024*20);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/82faa8af/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapRegionEntryHelperJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapRegionEntryHelperJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapRegionEntryHelperJUnitTest.java
index cf47a72..5cb6afb 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapRegionEntryHelperJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapRegionEntryHelperJUnitTest.java
@@ -65,12 +65,12 @@ public class OffHeapRegionEntryHelperJUnitTest {
     OutOfOffHeapMemoryListener ooohml = mock(OutOfOffHeapMemoryListener.class);
     OffHeapMemoryStats stats = mock(OffHeapMemoryStats.class);
 
-    ma = SimpleMemoryAllocatorImpl.create(ooohml, stats, 1, OffHeapStorage.MIN_SLAB_SIZE * 1, OffHeapStorage.MIN_SLAB_SIZE);
+    ma = MemoryAllocatorImpl.create(ooohml, stats, 1, OffHeapStorage.MIN_SLAB_SIZE * 1, OffHeapStorage.MIN_SLAB_SIZE);
   }
 
   @After
   public void tearDown() {
-    SimpleMemoryAllocatorImpl.freeOffHeapMemory();
+    MemoryAllocatorImpl.freeOffHeapMemory();
   }
 
   private OffHeapStoredObject createChunk(Object value) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/82faa8af/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapStorageJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapStorageJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapStorageJUnitTest.java
index d878358..d30d4c4 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapStorageJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapStorageJUnitTest.java
@@ -152,7 +152,7 @@ public class OffHeapStorageJUnitTest {
     StatisticsFactory localStatsFactory = new LocalStatisticsFactory(null);
     InternalDistributedSystem ids = mock(InternalDistributedSystem.class);
     MemoryAllocator ma = OffHeapStorage.createOffHeapStorage(localStatsFactory, OffHeapStorage.MIN_SLAB_SIZE, ids);
-    System.setProperty(SimpleMemoryAllocatorImpl.FREE_OFF_HEAP_MEMORY_PROPERTY, "true");
+    System.setProperty(MemoryAllocatorImpl.FREE_OFF_HEAP_MEMORY_PROPERTY, "true");
     ma.close();
   }
 
@@ -260,11 +260,11 @@ public class OffHeapStorageJUnitTest {
       verify(ooohml).outOfOffHeapMemory(ex);
 
     } finally {
-      System.setProperty(SimpleMemoryAllocatorImpl.FREE_OFF_HEAP_MEMORY_PROPERTY, "true");
+      System.setProperty(MemoryAllocatorImpl.FREE_OFF_HEAP_MEMORY_PROPERTY, "true");
       try {
         ma.close();
       } finally {
-        System.clearProperty(SimpleMemoryAllocatorImpl.FREE_OFF_HEAP_MEMORY_PROPERTY);
+        System.clearProperty(MemoryAllocatorImpl.FREE_OFF_HEAP_MEMORY_PROPERTY);
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/82faa8af/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapStoredObjectAddressStackJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapStoredObjectAddressStackJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapStoredObjectAddressStackJUnitTest.java
index 884787f..1d19854 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapStoredObjectAddressStackJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapStoredObjectAddressStackJUnitTest.java
@@ -107,13 +107,13 @@ public class OffHeapStoredObjectAddressStackJUnitTest {
   public void stackCreatedWithAddressIsNotEmpty() {
     SlabImpl slab = new SlabImpl(1024);
     try {
-      SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new SlabImpl[]{slab});
+      MemoryAllocatorImpl ma = MemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new SlabImpl[]{slab});
       OffHeapStoredObject chunk = (OffHeapStoredObject) ma.allocate(100);
 
       OffHeapStoredObjectAddressStack stack = new OffHeapStoredObjectAddressStack(chunk.getAddress());
       assertEquals(false, stack.isEmpty());
     } finally {
-      SimpleMemoryAllocatorImpl.freeOffHeapMemory();
+      MemoryAllocatorImpl.freeOffHeapMemory();
     }
   }
 
@@ -121,14 +121,14 @@ public class OffHeapStoredObjectAddressStackJUnitTest {
   public void stackWithChunkIsNotEmpty() {
     SlabImpl slab = new SlabImpl(1024);
     try {
-      SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new SlabImpl[]{slab});
+      MemoryAllocatorImpl ma = MemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new SlabImpl[]{slab});
       OffHeapStoredObject chunk = (OffHeapStoredObject) ma.allocate(100);
 
       OffHeapStoredObjectAddressStack stack = new OffHeapStoredObjectAddressStack();
       stack.offer(chunk.getAddress());
       assertEquals(false, stack.isEmpty());
     } finally {
-      SimpleMemoryAllocatorImpl.freeOffHeapMemory();
+      MemoryAllocatorImpl.freeOffHeapMemory();
     }
   }
 
@@ -136,7 +136,7 @@ public class OffHeapStoredObjectAddressStackJUnitTest {
   public void stackWithChunkTopEqualsAddress() {
     SlabImpl slab = new SlabImpl(1024);
     try {
-      SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new SlabImpl[]{slab});
+      MemoryAllocatorImpl ma = MemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new SlabImpl[]{slab});
       OffHeapStoredObject chunk = (OffHeapStoredObject) ma.allocate(100);
 
       long addr = chunk.getAddress();
@@ -144,7 +144,7 @@ public class OffHeapStoredObjectAddressStackJUnitTest {
       stack.offer(addr);
       assertEquals(addr, stack.getTopAddress());
     } finally {
-      SimpleMemoryAllocatorImpl.freeOffHeapMemory();
+      MemoryAllocatorImpl.freeOffHeapMemory();
     }
   }
 
@@ -163,7 +163,7 @@ public class OffHeapStoredObjectAddressStackJUnitTest {
   public void stackWithChunkClearReturnsAddressAndEmptiesStack() {
     SlabImpl slab = new SlabImpl(1024);
     try {
-      SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new SlabImpl[]{slab});
+      MemoryAllocatorImpl ma = MemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new SlabImpl[]{slab});
       OffHeapStoredObject chunk = (OffHeapStoredObject) ma.allocate(100);
 
       long addr = chunk.getAddress();
@@ -173,7 +173,7 @@ public class OffHeapStoredObjectAddressStackJUnitTest {
       assertEquals(addr, clearAddr);
       assertEquals(true, stack.isEmpty());
     } finally {
-      SimpleMemoryAllocatorImpl.freeOffHeapMemory();
+      MemoryAllocatorImpl.freeOffHeapMemory();
     }
   }
 
@@ -181,7 +181,7 @@ public class OffHeapStoredObjectAddressStackJUnitTest {
   public void stackWithChunkPollReturnsAddressAndEmptiesStack() {
     SlabImpl slab = new SlabImpl(1024);
     try {
-      SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new SlabImpl[]{slab});
+      MemoryAllocatorImpl ma = MemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new SlabImpl[]{slab});
       OffHeapStoredObject chunk = (OffHeapStoredObject) ma.allocate(100);
 
       long addr = chunk.getAddress();
@@ -191,7 +191,7 @@ public class OffHeapStoredObjectAddressStackJUnitTest {
       assertEquals(addr, pollAddr);
       assertEquals(true, stack.isEmpty());
     } finally {
-      SimpleMemoryAllocatorImpl.freeOffHeapMemory();
+      MemoryAllocatorImpl.freeOffHeapMemory();
     }
   }
 
@@ -199,7 +199,7 @@ public class OffHeapStoredObjectAddressStackJUnitTest {
   public void stackWithChunkTotalSizeIsChunkSize() {
     SlabImpl slab = new SlabImpl(1024);
     try {
-      SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new SlabImpl[]{slab});
+      MemoryAllocatorImpl ma = MemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new SlabImpl[]{slab});
       OffHeapStoredObject chunk = (OffHeapStoredObject) ma.allocate(100);
       int chunkSize = chunk.getSize();
 
@@ -208,7 +208,7 @@ public class OffHeapStoredObjectAddressStackJUnitTest {
       stack.offer(addr);
       assertEquals(chunkSize, stack.computeTotalSize());
     } finally {
-      SimpleMemoryAllocatorImpl.freeOffHeapMemory();
+      MemoryAllocatorImpl.freeOffHeapMemory();
     }
   }
 
@@ -217,7 +217,7 @@ public class OffHeapStoredObjectAddressStackJUnitTest {
   public void stackWithChunkLogShowsMsgAndSize() {
     SlabImpl slab = new SlabImpl(1024);
     try {
-      SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new SlabImpl[]{slab});
+      MemoryAllocatorImpl ma = MemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new SlabImpl[]{slab});
       OffHeapStoredObject chunk = (OffHeapStoredObject) ma.allocate(100);
       int chunkSize = chunk.getSize();
 
@@ -228,15 +228,15 @@ public class OffHeapStoredObjectAddressStackJUnitTest {
       stack.logSizes(lw, "foo");
       verify(lw).info("foo"+chunkSize);
     } finally {
-      SimpleMemoryAllocatorImpl.freeOffHeapMemory();
+      MemoryAllocatorImpl.freeOffHeapMemory();
     }
   }
   
   private class TestableSyncChunkStack extends OffHeapStoredObjectAddressStack {
     public boolean doConcurrentMod = true;
     public int chunk2Size;
-    private SimpleMemoryAllocatorImpl ma;
-    TestableSyncChunkStack(SimpleMemoryAllocatorImpl ma) {
+    private MemoryAllocatorImpl ma;
+    TestableSyncChunkStack(MemoryAllocatorImpl ma) {
       this.ma = ma;
     }
     @Override
@@ -253,7 +253,7 @@ public class OffHeapStoredObjectAddressStackJUnitTest {
   public void stackWithChunkTotalSizeIsChunkSizeWithConcurrentMod() {
     SlabImpl slab = new SlabImpl(1024);
     try {
-      SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new SlabImpl[]{slab});
+      MemoryAllocatorImpl ma = MemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new SlabImpl[]{slab});
       OffHeapStoredObject chunk = (OffHeapStoredObject) ma.allocate(100);
       int chunkSize = chunk.getSize();
 
@@ -263,7 +263,7 @@ public class OffHeapStoredObjectAddressStackJUnitTest {
       long totalSize = stack.computeTotalSize();
       assertEquals("chunkSize=" + chunkSize + " chunk2Size=" + stack.chunk2Size, chunkSize + stack.chunk2Size, totalSize);
     } finally {
-      SimpleMemoryAllocatorImpl.freeOffHeapMemory();
+      MemoryAllocatorImpl.freeOffHeapMemory();
     }
   }
 
@@ -272,7 +272,7 @@ public class OffHeapStoredObjectAddressStackJUnitTest {
   public void stackWithChunkLogShowsMsgAndSizeWithConcurrentMod() {
     SlabImpl slab = new SlabImpl(1024);
     try {
-      SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new SlabImpl[]{slab});
+      MemoryAllocatorImpl ma = MemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new SlabImpl[]{slab});
       OffHeapStoredObject chunk = (OffHeapStoredObject) ma.allocate(100);
       int chunkSize = chunk.getSize();
 
@@ -284,7 +284,7 @@ public class OffHeapStoredObjectAddressStackJUnitTest {
       verify(lw).info("foo"+chunkSize);
       verify(lw).info("foo"+stack.chunk2Size);
     } finally {
-      SimpleMemoryAllocatorImpl.freeOffHeapMemory();
+      MemoryAllocatorImpl.freeOffHeapMemory();
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/82faa8af/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapStoredObjectJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapStoredObjectJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapStoredObjectJUnitTest.java
index 956acb4..85c8d4c 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapStoredObjectJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapStoredObjectJUnitTest.java
@@ -63,12 +63,12 @@ public class OffHeapStoredObjectJUnitTest extends AbstractStoredObjectTestBase {
     OutOfOffHeapMemoryListener ooohml = mock(OutOfOffHeapMemoryListener.class);
     OffHeapMemoryStats stats = mock(OffHeapMemoryStats.class);
 
-    ma = SimpleMemoryAllocatorImpl.create(ooohml, stats, 3, OffHeapStorage.MIN_SLAB_SIZE * 3, OffHeapStorage.MIN_SLAB_SIZE);
+    ma = MemoryAllocatorImpl.create(ooohml, stats, 3, OffHeapStorage.MIN_SLAB_SIZE * 3, OffHeapStorage.MIN_SLAB_SIZE);
   }
 
   @After
   public void tearDown() {
-    SimpleMemoryAllocatorImpl.freeOffHeapMemory();
+    MemoryAllocatorImpl.freeOffHeapMemory();
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/82faa8af/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapValidationJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapValidationJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapValidationJUnitTest.java
index f8a5c8e..aa09449 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapValidationJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapValidationJUnitTest.java
@@ -184,7 +184,7 @@ public class OffHeapValidationJUnitTest {
     MemoryBlock firstBlock = inspector.getFirstBlock();
     assertEquals(MemoryBlock.State.UNUSED, firstBlock.getState());
     
-    //System.out.println(((SimpleMemoryAllocatorImpl)inspector).getSnapshot());
+    //System.out.println(((MemoryAllocatorImpl)inspector).getSnapshot());
     
     // sort the ExpectedValues into the same order as the MemberBlocks from inspector
     Collections.sort(expected, 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/82faa8af/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapWriteObjectAsByteArrayJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapWriteObjectAsByteArrayJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapWriteObjectAsByteArrayJUnitTest.java
index 7157eaa..5d79192 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapWriteObjectAsByteArrayJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapWriteObjectAsByteArrayJUnitTest.java
@@ -33,7 +33,7 @@ import com.gemstone.gemfire.internal.HeapDataOutputStream;
 import com.gemstone.gemfire.internal.cache.EntryEventImpl;
 import com.gemstone.gemfire.internal.offheap.NullOffHeapMemoryStats;
 import com.gemstone.gemfire.internal.offheap.NullOutOfOffHeapMemoryListener;
-import com.gemstone.gemfire.internal.offheap.SimpleMemoryAllocatorImpl;
+import com.gemstone.gemfire.internal.offheap.MemoryAllocatorImpl;
 import com.gemstone.gemfire.internal.offheap.StoredObject;
 import com.gemstone.gemfire.internal.offheap.SlabImpl;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
@@ -43,16 +43,16 @@ public class OffHeapWriteObjectAsByteArrayJUnitTest {
 
   @Before
   public void setUp() throws Exception {
-    SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new SlabImpl[]{new SlabImpl(1024*1024)});
+    MemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new SlabImpl[]{new SlabImpl(1024*1024)});
   }
 
   @After
   public void tearDown() throws Exception {
-    SimpleMemoryAllocatorImpl.freeOffHeapMemory();
+    MemoryAllocatorImpl.freeOffHeapMemory();
   }
   
   private StoredObject createStoredObject(byte[] bytes, boolean isSerialized, boolean isCompressed) {
-    return SimpleMemoryAllocatorImpl.getAllocator().allocateAndInitialize(bytes, isSerialized, isCompressed);
+    return MemoryAllocatorImpl.getAllocator().allocateAndInitialize(bytes, isSerialized, isCompressed);
   }
   
   private DataInputStream createInput(HeapDataOutputStream hdos) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/82faa8af/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OutOfOffHeapMemoryDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OutOfOffHeapMemoryDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OutOfOffHeapMemoryDUnitTest.java
index 25de4ea..9c2e5e8 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OutOfOffHeapMemoryDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OutOfOffHeapMemoryDUnitTest.java
@@ -91,7 +91,7 @@ public class OutOfOffHeapMemoryDUnitTest extends CacheTestCase {
   @SuppressWarnings("unused") // invoked by reflection from tearDown2()
   private static void cleanup() {
     disconnectFromDS();
-    SimpleMemoryAllocatorImpl.freeOffHeapMemory();
+    MemoryAllocatorImpl.freeOffHeapMemory();
     cache.set(null);
     system.set(null);
     isSmallerVM.set(false);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/82faa8af/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/SimpleMemoryAllocatorFillPatternIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/SimpleMemoryAllocatorFillPatternIntegrationTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/SimpleMemoryAllocatorFillPatternIntegrationTest.java
deleted file mode 100644
index 51bc0a2..0000000
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/SimpleMemoryAllocatorFillPatternIntegrationTest.java
+++ /dev/null
@@ -1,246 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package com.gemstone.gemfire.internal.offheap;
-
-import static org.junit.Assert.*;
-
-import java.util.Collections;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Random;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.ThreadLocalRandom;
-import java.util.concurrent.TimeUnit;
-
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
-
-/**
- * Tests fill pattern validation for the {@link SimpleMemoryAllocatorImpl}.
- */
-@Category(IntegrationTest.class)
-public class SimpleMemoryAllocatorFillPatternIntegrationTest {
-  private static Random random = ThreadLocalRandom.current();
-
-  /**
-   * Chunk operation types.
-   */
-  static enum Operation {
-    ALLOCATE,
-    FREE,
-    WRITE;
-    
-    // Holds all Operation values
-    private static Operation[] values = Operation.values(); 
-    
-    static Operation randomOperation() {
-      return values[random.nextInt(values.length)];
-    }
-  };
-  
-  /** Number of worker threads for advanced tests. */
-  private static final int WORKER_THREAD_COUNT = 5;
-  
-  /** Size of single test slab.*/
-  private static final int SLAB_SIZE = 1024 * 1024 * 50;
-  
-  /** Maximum number of bytes a worker thread can allocate during advanced tests.  */
-  private static final int MAX_WORKER_ALLOCATION_TOTAL_SIZE = SLAB_SIZE / WORKER_THREAD_COUNT / 2;
-  
-  /** Maximum allocation for a single Chunk.  */
-  private static final int MAX_WORKER_ALLOCATION_SIZE = 512;
-  
-  /** Canned data for write operations. */
-  private static final byte[] WRITE_BYTES = new String("Some string data.").getBytes();
-  
-  /** Minimum size for write operations. */
-  private static final int MIN_WORKER_ALLOCATION_SIZE = WRITE_BYTES.length;
-
-  /** Runtime for worker threads. */
-  private static final long RUN_TIME_IN_MILLIS = 1 * 1000 * 5;
-  
-  /** Chunk size for basic huge allocation test. */
-  private static final int HUGE_CHUNK_SIZE = 1024 * 200;
-  
-  /** Our test victim. */
-  private SimpleMemoryAllocatorImpl allocator = null;
-  
-  /** Our test victim's memory slab. */
-  private SlabImpl slab = null;
-
-  /**
-   * Enables fill validation and creates the test victim.
-   */
-  @Before
-  public void setUp() throws Exception {
-    System.setProperty("gemfire.validateOffHeapWithFill", "true");
-    this.slab = new SlabImpl(SLAB_SIZE);
-    this.allocator = SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new SlabImpl[]{this.slab});
-  }
-
-  /**
-   * Frees off heap memory.
-   */
-  @After
-  public void tearDown() throws Exception {
-    SimpleMemoryAllocatorImpl.freeOffHeapMemory();
-    System.clearProperty("gemfire.validateOffHeapWithFill");
-  }
-  
-  /**
-   * This test hammers a SimpleMemoryAllocatorImpl with multiple threads exercising
-   * the fill validation of tiny Chunks for one minute.  This, of course, exercises many aspects of
-   * the SimpleMemoryAllocatorImpl and its helper classes.
-   * @throws Exception
-   */
-  @Test
-  public void testFillPatternAdvancedForTinyAllocations() throws Exception { 
-    doFillPatternAdvancedTest(new ChunkSizer() {
-      @Override
-      public int allocationSize() {
-        int allocation = random.nextInt(MAX_WORKER_ALLOCATION_SIZE+1);
-        
-        while(allocation < MIN_WORKER_ALLOCATION_SIZE) {
-          allocation = random.nextInt(MAX_WORKER_ALLOCATION_SIZE+1);
-        }
-        return allocation;
-      }
-    });
-  }
-
-  /**
-   * This test hammers a SimpleMemoryAllocatorImpl with multiple threads exercising
-   * the fill validation of huge Chunks for one minute.  This, of course, exercises many aspects of
-   * the SimpleMemoryAllocatorImpl and its helper classes.
-   * @throws Exception
-   */
-  @Test
-  public void testFillPatternAdvancedForHugeAllocations() throws Exception {
-    doFillPatternAdvancedTest(new ChunkSizer() {
-      @Override
-      public int allocationSize() {
-        return HUGE_CHUNK_SIZE;
-      }
-    });
-  }
-  
-  private interface ChunkSizer {
-    int allocationSize();
-  }
-  
-  private void doFillPatternAdvancedTest(final ChunkSizer chunkSizer) throws InterruptedException {
-    // Used to manage worker thread completion
-    final CountDownLatch latch = new CountDownLatch(WORKER_THREAD_COUNT);
-    
-    // Use to track any errors the worker threads will encounter
-    final List<Throwable> threadErrorList = Collections.synchronizedList(new LinkedList<Throwable>());
-    
-    /*
-     * Start up a number of worker threads.  These threads will randomly allocate, free,
-     * and write to Chunks.
-     */
-    for(int i = 0;i < WORKER_THREAD_COUNT;++i) {
-      new Thread(new Runnable() {
-        // Total allocation in bytes for this thread
-        private int totalAllocation = 0;
-        
-        // List of Chunks allocated by this thread
-        private List<OffHeapStoredObject> chunks = new LinkedList<OffHeapStoredObject>();
-        
-        // Time to end thread execution
-        private long endTime = System.currentTimeMillis() + RUN_TIME_IN_MILLIS;
-        
-        /**
-         * Allocates a chunk and adds it to the thread's Chunk list.
-         */
-        private void allocate() {          
-          int allocation = chunkSizer.allocationSize();
-          OffHeapStoredObject chunk = (OffHeapStoredObject) allocator.allocate(allocation);
-          
-          // This should always work just after allocation
-          chunk.validateFill();
-          
-          chunks.add(chunk);
-          totalAllocation += chunk.getSize();
-        }
-        
-        /**
-         * Frees a random chunk from the Chunk list.
-         */
-        private void free() {
-          OffHeapStoredObject chunk = chunks.remove(random.nextInt(chunks.size()));
-          totalAllocation -= chunk.getSize();
-          
-          /*
-           * Chunk is filled here but another thread may have already grabbed it so we
-           * cannot validate the fill.
-           */
-          chunk.release(); 
-        }
-        
-        /**
-         * Writes canned data to a random Chunk from the Chunk list.
-         */
-        private void write() {
-          OffHeapStoredObject chunk = chunks.get(random.nextInt(chunks.size()));
-          chunk.writeDataBytes(0, WRITE_BYTES);
-        }
-        
-        /**
-         * Randomly selects Chunk operations and executes them
-         * for a period of time.  Collects any error thrown during execution.
-         */
-        @Override
-        public void run() {
-          try {
-            for(long currentTime = System.currentTimeMillis();currentTime < endTime;currentTime = System.currentTimeMillis()) {
-              Operation op = (totalAllocation == 0 ? Operation.ALLOCATE : (totalAllocation >= MAX_WORKER_ALLOCATION_TOTAL_SIZE ? Operation.FREE : Operation.randomOperation()));
-              switch(op) {
-              case ALLOCATE:
-                allocate();
-                break;
-              case FREE:
-                free();
-                break;
-              case WRITE:
-                write();
-                break;
-              }
-            }
-          } catch (Throwable t) {
-            threadErrorList.add(t);
-          } finally {
-            latch.countDown();
-          }
-       }        
-      }).start();
-    }
-    
-    // Make sure each thread ended cleanly
-    assertTrue(latch.await(2, TimeUnit.MINUTES));
-    
-    // Fail on the first error we find
-    if(!threadErrorList.isEmpty()) {
-      fail(threadErrorList.get(0).getMessage());
-    }
-  }
-  
-}