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/03/05 01:51:06 UTC

[06/38] incubator-geode git commit: Added Slab, SlabFactory, SlabImpl, and AddressableMemoryManager.

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/e2a126d4/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/MemoryChunkJUnitTestBase.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/MemoryChunkJUnitTestBase.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/MemoryChunkJUnitTestBase.java
deleted file mode 100644
index 1b1d300..0000000
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/MemoryChunkJUnitTestBase.java
+++ /dev/null
@@ -1,290 +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 java.util.ArrayList;
-import java.util.Arrays;
-import java.util.concurrent.atomic.AtomicIntegerArray;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReadWriteLock;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-
-import org.junit.Test;
-
-import static org.junit.Assert.*;
-
-public abstract class MemoryChunkJUnitTestBase {
-  static {
-    ClassLoader.getSystemClassLoader().setDefaultAssertionStatus(true);
-  }
-
-  
-  protected abstract MemoryChunk createChunk(int size);
-  
-  @Test
-  public void testByteReadWrite() {
-    int CHUNK_SIZE = 1024;
-    MemoryChunk mc = createChunk(CHUNK_SIZE);
-    try {
-    for (int i=0; i<CHUNK_SIZE; i++) {
-      mc.writeByte(i, (byte)(i%128));
-    }
-    for (int i=0; i<CHUNK_SIZE; i++) {
-      assertEquals(i%128, mc.readByte(i));
-    }
-    } finally {
-      mc.release();
-    }
-  }
-  
-  @Test
-  public void testGetSize() {
-    MemoryChunk mc = createChunk(5);
-    try {
-      assertEquals(5, mc.getSize());
-    } finally {
-      mc.release();
-    }
-    mc = createChunk(0);
-    try {
-      assertEquals(0, mc.getSize());
-    } finally {
-      mc.release();
-    }
-    mc = createChunk(1024);
-    try {
-      assertEquals(1024, mc.getSize());
-    } finally {
-      mc.release();
-    }
-  }
-
-  @Test
-  public void testToString() {
-    int CHUNK_SIZE = 1024;
-    MemoryChunk mc = createChunk(CHUNK_SIZE);
-    try {
-      mc.toString();
-    } finally {
-      mc.release();
-    }
-  }
- 
-  @Test
-  public void testCopyBytes() {
-    int CHUNK_SIZE = 1024;
-    MemoryChunk mc = createChunk(CHUNK_SIZE*2);
-    try {
-      for (int i=0; i<CHUNK_SIZE; i++) {
-        mc.writeByte(i, (byte)(i%128));
-      }
-      for (int i=0; i<CHUNK_SIZE; i++) {
-        assertEquals(i%128, mc.readByte(i));
-      }
-      mc.copyBytes(0, CHUNK_SIZE, CHUNK_SIZE);
-      for (int i=0; i<CHUNK_SIZE; i++) {
-        assertEquals(i%128, mc.readByte(CHUNK_SIZE+i));
-      }
-      mc.copyBytes(0, 1, CHUNK_SIZE);
-      for (int i=0; i<CHUNK_SIZE; i++) {
-        assertEquals(i%128, mc.readByte(1+i));
-      }
-    } finally {
-      mc.release();
-    }
-  }
- 
-  
-  @Test
-  public void testByteArrayReadWrite() {
-    byte[] writeBytes = new byte[256];
-    int v = Byte.MIN_VALUE;
-    for (int i=0; i < writeBytes.length; i++) {
-      writeBytes[i] = (byte)v++;
-    }
-    int ARRAYS_PER_CHUNK = 100;
-    int CHUNK_SIZE = ARRAYS_PER_CHUNK * writeBytes.length;
-    MemoryChunk mc = createChunk(CHUNK_SIZE);
-    try {
-    for (int i=0; i<CHUNK_SIZE; i+=writeBytes.length) {
-      mc.writeBytes(i, writeBytes);
-    }
-    byte[] readBytes = new byte[writeBytes.length];
-    for (int i=0; i<CHUNK_SIZE; i+=writeBytes.length) {
-      mc.readBytes(i, readBytes);
-      assertTrue("expected " + Arrays.toString(writeBytes) + " but found " + Arrays.toString(readBytes), Arrays.equals(writeBytes, readBytes));
-    }
-    } finally {
-      mc.release();
-    }
-  }
-  public void DISABLEtestBytePerf() throws InterruptedException {
-    final int ITEM_SIZE = 1;
-    final int ITEMS_PER_CHUNK = 100000;
-    final int CHUNK_SIZE = ITEMS_PER_CHUNK * ITEM_SIZE;
-    final MemoryChunk mc = createChunk(CHUNK_SIZE);
-    try {
-      final int WRITE_ITERATIONS = 90000;
-      final Runnable writeRun = new Runnable() {
-        public void run() {
-          for (int j=0; j<WRITE_ITERATIONS; j++) {
-            for (int i=0; i<CHUNK_SIZE; i+=ITEM_SIZE) {
-              mc.writeByte(i, (byte)1);
-            }
-          }
-        }
-      };
-      long startWrite = System.nanoTime();
-      writeRun.run();
-      long endWrite = System.nanoTime();
-      final int READ_ITERATIONS = 90000/10;
-      final AtomicIntegerArray readTimes = new AtomicIntegerArray(READ_ITERATIONS);
-      final int THREAD_COUNT = 3;
-      final Thread[] threads = new Thread[THREAD_COUNT];
-      final ReadWriteLock rwl = new ReentrantReadWriteLock();
-      final Lock rl = rwl.readLock();
-      final AtomicLong longHolder = new AtomicLong();
-      final Runnable r = new Runnable() {
-        public void run() {
-          long c = 0;
-          long lastTs = System.nanoTime();
-          long startTs;
-          for (int j=0; j<READ_ITERATIONS; j++) {
-            startTs = lastTs;
-            for (int i=0; i<CHUNK_SIZE; i+=ITEM_SIZE) {
-//              c += mc.readByte(i);
-              rl.lock();
-              try {
-                c+= mc.readByte(i);
-              } finally {
-                rl.unlock();
-              }
-            }
-            lastTs = System.nanoTime();
-            readTimes.addAndGet(j, (int) (lastTs-startTs));
-          }
-          longHolder.addAndGet(c);
-          //System.out.println("c="+c);
-        }
-      };
-      for (int t=0; t < THREAD_COUNT; t++) {
-        threads[t] = new Thread(r);
-      }
-      long start = System.nanoTime();
-      for (int t=0; t < THREAD_COUNT; t++) {
-        threads[t].start();
-      }
-      for (int t=0; t < THREAD_COUNT; t++) {
-        threads[t].join();
-      }
-//      long start = System.nanoTime();
-//      r.run();
-      long end = System.nanoTime();
-      System.out.println("longHolder=" + longHolder.get());
-      System.out.println(computeHistogram(readTimes, 1000000));
-//      for (int i=0; i < 30; i++) {
-//        System.out.print(readTimes[i]);
-//        System.out.print(' ');
-//      }
-//      System.out.println();
-//      for (int i=readTimes.length-30; i < readTimes.length; i++) {
-//        System.out.print(readTimes[i]);
-//        System.out.print(' ');
-//      }
-//      System.out.println();
-      System.out.println((end-start) / READ_ITERATIONS);
-      System.out.println("BytePerfReads:  " + (double)((long)CHUNK_SIZE*READ_ITERATIONS*THREAD_COUNT)/(double)((end-start)/1000000) + " bytes/ms");
-      System.out.println("BytePerfWrites: " + (double)((long)CHUNK_SIZE*WRITE_ITERATIONS)/(double)((endWrite-startWrite)/1000000) + " bytes/ms");
-    } finally {
-      mc.release();
-    }
-  }
-  static private ArrayList<Bucket> computeHistogram(AtomicIntegerArray originalValues, final int granualarity) {
-    int[] values = new int[originalValues.length()];
-    for (int i=0; i < values.length; i++) {
-      values[i] = originalValues.get(i);
-    }
-    Arrays.sort(values);
-    ArrayList<Bucket> result = new ArrayList<Bucket>();
-    Bucket curBucket = new Bucket(values[0]);
-    result.add(curBucket);
-    for (int i=1; i < values.length; i++) {
-      int curVal = values[i];
-      if (!curBucket.addValue(curVal, granualarity)) {
-        curBucket = new Bucket(curVal);
-        result.add(curBucket);
-      }
-    }
-    return result;
-  }
-  static private class Bucket {
-    public Bucket(long l) {
-      base = l;
-      total = l;
-      count = 1;
-    }
-    public boolean addValue(long curVal, int granualarity) {
-      if (curVal < base || (curVal-base) > granualarity) {
-        return false;
-      }
-      total += curVal;
-      count++;
-      return true;
-    }
-    private final long base;
-    private long total;
-    private int count;
-    
-    @Override
-    public String toString() {
-      return "" + (total/count) + ":" + count;
-    }
-  }
-  public void DISABLEtest256ByteArrayPerf() {
-    byte[] writeBytes = new byte[256];
-    for (int i=0; i < writeBytes.length; i++) {
-      writeBytes[i] = 1;
-    }
-    int ARRAYS_PER_CHUNK = 100000;
-    int CHUNK_SIZE = ARRAYS_PER_CHUNK * writeBytes.length;
-    MemoryChunk mc = createChunk(CHUNK_SIZE);
-    try {
-      int WRITE_ITERATIONS = 2000;
-      long startWrite = System.nanoTime();
-      for (int j=0; j<WRITE_ITERATIONS; j++) {
-        for (int i=0; i<CHUNK_SIZE; i+=writeBytes.length) {
-          mc.writeBytes(i, writeBytes);
-        }
-      }
-      long endWrite = System.nanoTime();
-      byte[] readBytes = new byte[writeBytes.length];
-      int READ_ITERATIONS = 2000;
-      long start = System.nanoTime();
-      for (int j=0; j<READ_ITERATIONS; j++) {
-        for (int i=0; i<CHUNK_SIZE; i+=writeBytes.length) {
-          mc.readBytes(i, readBytes);
-        }
-      }
-      long end = System.nanoTime();
-      System.out.println("ByteArray("+writeBytes.length+")PerfReads: " + (double)((long)CHUNK_SIZE*(long)READ_ITERATIONS)/(double)((end-start)/1000000) + " bytes/ms");
-      System.out.println("ByteArray("+writeBytes.length+")PerfWrites: " + (double)((long)CHUNK_SIZE*(long)WRITE_ITERATIONS)/(double)((endWrite-startWrite)/1000000) + " bytes/ms");
-    } finally {
-      mc.release();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/e2a126d4/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/ObjectChunkJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/ObjectChunkJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/ObjectChunkJUnitTest.java
index 9271b53..f66ef2c 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/ObjectChunkJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/ObjectChunkJUnitTest.java
@@ -255,7 +255,7 @@ public class ObjectChunkJUnitTest extends AbstractStoredObjectTestBase {
 
     ObjectChunk chunk = (ObjectChunk) ma.allocateAndInitialize(regionEntryValueAsBytes, isSerialized, isCompressed);
 
-    int headerBeforeSerializedBitSet = UnsafeMemoryChunk.readAbsoluteIntVolatile(chunk.getMemoryAddress() + ObjectChunk.REF_COUNT_OFFSET);
+    int headerBeforeSerializedBitSet = AddressableMemoryManager.readIntVolatile(chunk.getMemoryAddress() + ObjectChunk.REF_COUNT_OFFSET);
 
     assertThat(chunk.isSerialized()).isFalse();
 
@@ -263,7 +263,7 @@ public class ObjectChunkJUnitTest extends AbstractStoredObjectTestBase {
 
     assertThat(chunk.isSerialized()).isTrue();
 
-    int headerAfterSerializedBitSet = UnsafeMemoryChunk.readAbsoluteIntVolatile(chunk.getMemoryAddress() + ObjectChunk.REF_COUNT_OFFSET);
+    int headerAfterSerializedBitSet = AddressableMemoryManager.readIntVolatile(chunk.getMemoryAddress() + ObjectChunk.REF_COUNT_OFFSET);
 
     assertThat(headerAfterSerializedBitSet).isEqualTo(headerBeforeSerializedBitSet | ObjectChunk.IS_SERIALIZED_BIT);
 
@@ -289,7 +289,7 @@ public class ObjectChunkJUnitTest extends AbstractStoredObjectTestBase {
 
     ObjectChunk chunk = (ObjectChunk) ma.allocateAndInitialize(regionEntryValueAsBytes, isSerialized, isCompressed);
 
-    int headerBeforeCompressedBitSet = UnsafeMemoryChunk.readAbsoluteIntVolatile(chunk.getMemoryAddress() + ObjectChunk.REF_COUNT_OFFSET);
+    int headerBeforeCompressedBitSet = AddressableMemoryManager.readIntVolatile(chunk.getMemoryAddress() + ObjectChunk.REF_COUNT_OFFSET);
 
     assertThat(chunk.isCompressed()).isFalse();
 
@@ -297,7 +297,7 @@ public class ObjectChunkJUnitTest extends AbstractStoredObjectTestBase {
 
     assertThat(chunk.isCompressed()).isTrue();
 
-    int headerAfterCompressedBitSet = UnsafeMemoryChunk.readAbsoluteIntVolatile(chunk.getMemoryAddress() + ObjectChunk.REF_COUNT_OFFSET);
+    int headerAfterCompressedBitSet = AddressableMemoryManager.readIntVolatile(chunk.getMemoryAddress() + ObjectChunk.REF_COUNT_OFFSET);
 
     assertThat(headerAfterCompressedBitSet).isEqualTo(headerBeforeCompressedBitSet | ObjectChunk.IS_COMPRESSED_BIT);
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/e2a126d4/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 b1e3af0..bb8aca6 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
@@ -38,7 +38,7 @@ import com.gemstone.gemfire.test.junit.categories.UnitTest;
 @Category(UnitTest.class)
 public class OffHeapHelperJUnitTest extends AbstractStoredObjectTestBase {
 
-  private MemoryChunkWithRefCount storedObject                 = null;
+  private StoredObject storedObject                 = null;
   private Object                  deserializedRegionEntryValue = null;
   private byte[]                  serializedRegionEntryValue   = null;
   private MemoryAllocator         ma;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/e2a126d4/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 8de0406..3412afb 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
@@ -92,7 +92,7 @@ public abstract class OffHeapRegionBase {
       assertNotNull(ma);
       final long offHeapSize = ma.getFreeMemory();
       assertEquals(0, ma.getUsedMemory());
-      MemoryChunk mc1 = ma.allocate(64);
+      StoredObject mc1 = ma.allocate(64);
       assertEquals(64+perObjectOverhead(), ma.getUsedMemory());
       assertEquals(offHeapSize-(64+perObjectOverhead()), ma.getFreeMemory());
       mc1.release();
@@ -123,7 +123,7 @@ public abstract class OffHeapRegionBase {
       assertNotNull(ma);
       final long offHeapSize = ma.getFreeMemory();
       assertEquals(0, ma.getUsedMemory());
-      MemoryChunk mc1 = ma.allocate(64);
+      StoredObject mc1 = ma.allocate(64);
       assertEquals(64+perObjectOverhead(), ma.getUsedMemory());
       assertEquals(offHeapSize-(64+perObjectOverhead()), ma.getFreeMemory());
       mc1.release();
@@ -163,7 +163,7 @@ public abstract class OffHeapRegionBase {
       final long offHeapSize = ma.getFreeMemory();
       assertEquals(0, ma.getUsedMemory());
       byte[] data = new byte[] {1,2,3,4,5,6,7,8};
-      MemoryChunk mc1 = (MemoryChunk)ma.allocateAndInitialize(data, false, false);
+      MemoryChunkWithRefCount mc1 = (MemoryChunkWithRefCount)ma.allocateAndInitialize(data, false, false);
       assertEquals(data.length+perObjectOverhead(), ma.getUsedMemory());
       assertEquals(offHeapSize-(data.length+perObjectOverhead()), ma.getFreeMemory());
       byte[] data2 = new byte[data.length];

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/e2a126d4/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 9c83f5b..8ef2d42 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
@@ -35,7 +35,7 @@ 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.StoredObject;
-import com.gemstone.gemfire.internal.offheap.UnsafeMemoryChunk;
+import com.gemstone.gemfire.internal.offheap.SlabImpl;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
 @Category(UnitTest.class)
@@ -43,7 +43,7 @@ public class OffHeapWriteObjectAsByteArrayJUnitTest {
 
   @Before
   public void setUp() throws Exception {
-    SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new UnsafeMemoryChunk[]{new UnsafeMemoryChunk(1024*1024)});
+    SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new SlabImpl[]{new SlabImpl(1024*1024)});
   }
 
   @After

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/e2a126d4/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
index 51f46a1..a744fe2 100644
--- 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
@@ -84,7 +84,7 @@ public class SimpleMemoryAllocatorFillPatternIntegrationTest {
   private SimpleMemoryAllocatorImpl allocator = null;
   
   /** Our test victim's memory slab. */
-  private UnsafeMemoryChunk slab = null;
+  private SlabImpl slab = null;
 
   /**
    * Enables fill validation and creates the test victim.
@@ -92,8 +92,8 @@ public class SimpleMemoryAllocatorFillPatternIntegrationTest {
   @Before
   public void setUp() throws Exception {
     System.setProperty("gemfire.validateOffHeapWithFill", "true");
-    this.slab = new UnsafeMemoryChunk(SLAB_SIZE);
-    this.allocator = SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new UnsafeMemoryChunk[]{this.slab});
+    this.slab = new SlabImpl(SLAB_SIZE);
+    this.allocator = SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new SlabImpl[]{this.slab});
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/e2a126d4/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/SimpleMemoryAllocatorFillPatternJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/SimpleMemoryAllocatorFillPatternJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/SimpleMemoryAllocatorFillPatternJUnitTest.java
index 7c26f86..ceef791 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/SimpleMemoryAllocatorFillPatternJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/SimpleMemoryAllocatorFillPatternJUnitTest.java
@@ -57,7 +57,7 @@ public class SimpleMemoryAllocatorFillPatternJUnitTest {
   private SimpleMemoryAllocatorImpl allocator = null;
   
   /** Our test victim's memory slab. */
-  private UnsafeMemoryChunk slab = null;
+  private SlabImpl slab = null;
 
   /**
    * Enables fill validation and creates the test victim.
@@ -65,8 +65,8 @@ public class SimpleMemoryAllocatorFillPatternJUnitTest {
   @Before
   public void setUp() throws Exception {
     System.setProperty("gemfire.validateOffHeapWithFill", "true");
-    this.slab = new UnsafeMemoryChunk(SLAB_SIZE);
-    this.allocator = SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new UnsafeMemoryChunk[]{this.slab});
+    this.slab = new SlabImpl(SLAB_SIZE);
+    this.allocator = SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new SlabImpl[]{this.slab});
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/e2a126d4/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/SimpleMemoryAllocatorJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/SimpleMemoryAllocatorJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/SimpleMemoryAllocatorJUnitTest.java
index 1f17f9b..1cbb19b 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/SimpleMemoryAllocatorJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/SimpleMemoryAllocatorJUnitTest.java
@@ -97,9 +97,9 @@ public class SimpleMemoryAllocatorJUnitTest {
       LastSevereLogger logger = new LastSevereLogger();
       try {
         SimpleMemoryAllocatorImpl.createForUnitTest(listener, stats, logger, 10, 950, 100,
-            new AddressableMemoryChunkFactory() {
+            new SlabFactory() {
           @Override
-          public AddressableMemoryChunk create(int size) {
+          public Slab create(int size) {
             throw new OutOfMemoryError("expected");
           }
         });
@@ -116,13 +116,13 @@ public class SimpleMemoryAllocatorJUnitTest {
       LastSevereLogger logger = new LastSevereLogger();
       int MAX_SLAB_SIZE = 100;
       try {
-        AddressableMemoryChunkFactory factory = new AddressableMemoryChunkFactory() {
+        SlabFactory factory = new SlabFactory() {
           private int createCount = 0;
           @Override
-          public AddressableMemoryChunk create(int size) {
+          public Slab create(int size) {
             createCount++;
             if (createCount == 1) {
-              return new UnsafeMemoryChunk(size);
+              return new SlabImpl(size);
             } else {
               throw new OutOfMemoryError("expected");
             }
@@ -139,10 +139,10 @@ public class SimpleMemoryAllocatorJUnitTest {
     {
       NullOutOfOffHeapMemoryListener listener = new NullOutOfOffHeapMemoryListener();
       NullOffHeapMemoryStats stats = new NullOffHeapMemoryStats();
-      AddressableMemoryChunkFactory factory = new AddressableMemoryChunkFactory() {
+      SlabFactory factory = new SlabFactory() {
         @Override
-        public AddressableMemoryChunk create(int size) {
-          return new UnsafeMemoryChunk(size);
+        public Slab create(int size) {
+          return new SlabImpl(size);
         }
       };
       MemoryAllocator ma = 
@@ -156,14 +156,14 @@ public class SimpleMemoryAllocatorJUnitTest {
         listener = new NullOutOfOffHeapMemoryListener();
         NullOffHeapMemoryStats stats2 = new NullOffHeapMemoryStats();
         {
-          UnsafeMemoryChunk slab = new UnsafeMemoryChunk(1024);
+          SlabImpl slab = new SlabImpl(1024);
           try {
-            SimpleMemoryAllocatorImpl.createForUnitTest(listener, stats2, new UnsafeMemoryChunk[]{slab});
+            SimpleMemoryAllocatorImpl.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.release();
+            slab.free();
           }
           assertFalse(stats.isClosed());
           assertTrue(listener.isClosed());
@@ -195,17 +195,17 @@ public class SimpleMemoryAllocatorJUnitTest {
     int maxTiny = com.gemstone.gemfire.internal.offheap.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);
-    UnsafeMemoryChunk slab = new UnsafeMemoryChunk(TOTAL_MEM);
+    SlabImpl slab = new SlabImpl(TOTAL_MEM);
     try {
-      SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new UnsafeMemoryChunk[]{slab});
+      SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.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());
-      MemoryChunk tinymc = ma.allocate(maxTiny);
+      MemoryChunkWithRefCount 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());
-      MemoryChunk hugemc = ma.allocate(minHuge);
+      MemoryChunkWithRefCount 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();
@@ -249,7 +249,7 @@ public class SimpleMemoryAllocatorJUnitTest {
       hugemc = ma.allocate(minHuge);
       assertEquals(round(TINY_MULTIPLE, minHuge+perObjectOverhead)*(BATCH_SIZE-1), ma.freeList.getFreeHugeMemory());
       if (BATCH_SIZE > 1) {
-        MemoryChunk hugemc2 = ma.allocate(minHuge);
+        MemoryChunkWithRefCount 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());
@@ -265,9 +265,9 @@ public class SimpleMemoryAllocatorJUnitTest {
   
   @Test
   public void testChunkCreateDirectByteBuffer() {
-    UnsafeMemoryChunk slab = new UnsafeMemoryChunk(1024*1024);
+    SlabImpl slab = new SlabImpl(1024*1024);
     try {
-      SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new UnsafeMemoryChunk[]{slab});
+      SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new SlabImpl[]{slab});
       ByteBuffer bb = ByteBuffer.allocate(1024);
       for (int i=0; i < 1024; i++) {
         bb.put((byte) i);
@@ -293,9 +293,9 @@ public class SimpleMemoryAllocatorJUnitTest {
   }
   @Test
   public void testGetLostChunks() {
-    UnsafeMemoryChunk slab = new UnsafeMemoryChunk(1024*1024);
+    SlabImpl slab = new SlabImpl(1024*1024);
     try {
-      SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new UnsafeMemoryChunk[]{slab});
+      SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new SlabImpl[]{slab});
       assertEquals(Collections.emptyList(), ma.getLostChunks());
     } finally {
       SimpleMemoryAllocatorImpl.freeOffHeapMemory();
@@ -304,9 +304,9 @@ public class SimpleMemoryAllocatorJUnitTest {
   @Test
   public void testFindSlab() {
     final int SLAB_SIZE = 1024*1024;
-    UnsafeMemoryChunk slab = new UnsafeMemoryChunk(SLAB_SIZE);
+    SlabImpl slab = new SlabImpl(SLAB_SIZE);
     try {
-      SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new UnsafeMemoryChunk[]{slab});
+      SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new SlabImpl[]{slab});
       assertEquals(0, ma.findSlab(slab.getMemoryAddress()));
       assertEquals(0, ma.findSlab(slab.getMemoryAddress()+SLAB_SIZE-1));
       try {
@@ -326,9 +326,9 @@ public class SimpleMemoryAllocatorJUnitTest {
   @Test
   public void testValidateAddressAndSize() {
     final int SLAB_SIZE = 1024*1024;
-    UnsafeMemoryChunk slab = new UnsafeMemoryChunk(SLAB_SIZE);
+    SlabImpl slab = new SlabImpl(SLAB_SIZE);
     try {
-      SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new UnsafeMemoryChunk[]{slab});
+      SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new SlabImpl[]{slab});
       try {
         SimpleMemoryAllocatorImpl.validateAddress(0L);
         fail("expected IllegalStateException");
@@ -363,9 +363,9 @@ public class SimpleMemoryAllocatorJUnitTest {
   @Test
   public void testMemoryInspection() {
     final int SLAB_SIZE = 1024*1024;
-    UnsafeMemoryChunk slab = new UnsafeMemoryChunk(SLAB_SIZE);
+    SlabImpl slab = new SlabImpl(SLAB_SIZE);
     try {
-      SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new UnsafeMemoryChunk[]{slab});
+      SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new SlabImpl[]{slab});
       MemoryInspector inspector = ma.getMemoryInspector();
       assertNotNull(inspector);
       assertEquals(null, inspector.getFirstBlock());
@@ -401,9 +401,9 @@ public class SimpleMemoryAllocatorJUnitTest {
   @Test
   public void testClose() {
     System.setProperty(SimpleMemoryAllocatorImpl.FREE_OFF_HEAP_MEMORY_PROPERTY, "false");
-    UnsafeMemoryChunk slab = new UnsafeMemoryChunk(1024*1024);
+    SlabImpl slab = new SlabImpl(1024*1024);
     boolean freeSlab = true;
-    UnsafeMemoryChunk[] slabs = new UnsafeMemoryChunk[]{slab};
+    SlabImpl[] slabs = new SlabImpl[]{slab};
     try {
       SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), slabs);
       ma.close();
@@ -431,19 +431,19 @@ public class SimpleMemoryAllocatorJUnitTest {
     final int BIG_ALLOC_SIZE = 150000;
     final int SMALL_ALLOC_SIZE = BIG_ALLOC_SIZE/2;
     final int TOTAL_MEM = BIG_ALLOC_SIZE;
-    UnsafeMemoryChunk slab = new UnsafeMemoryChunk(TOTAL_MEM);
+    SlabImpl slab = new SlabImpl(TOTAL_MEM);
     try {
-      SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new UnsafeMemoryChunk[]{slab});
-      MemoryChunk bmc = ma.allocate(BIG_ALLOC_SIZE-perObjectOverhead);
+      SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new SlabImpl[]{slab});
+      MemoryChunkWithRefCount bmc = ma.allocate(BIG_ALLOC_SIZE-perObjectOverhead);
       try {
-        MemoryChunk smc = ma.allocate(SMALL_ALLOC_SIZE-perObjectOverhead);
+        MemoryChunkWithRefCount smc = ma.allocate(SMALL_ALLOC_SIZE-perObjectOverhead);
         fail("Expected out of memory");
       } catch (OutOfOffHeapMemoryException expected) {
       }
       bmc.release();
       assertEquals(TOTAL_MEM, ma.freeList.getFreeMemory());
-      MemoryChunk smc1 = ma.allocate(SMALL_ALLOC_SIZE-perObjectOverhead);
-      MemoryChunk smc2 = ma.allocate(SMALL_ALLOC_SIZE-perObjectOverhead);
+      MemoryChunkWithRefCount smc1 = ma.allocate(SMALL_ALLOC_SIZE-perObjectOverhead);
+      MemoryChunkWithRefCount smc2 = ma.allocate(SMALL_ALLOC_SIZE-perObjectOverhead);
       smc2.release();
       assertEquals(TOTAL_MEM-SMALL_ALLOC_SIZE, ma.freeList.getFreeMemory());
       try {
@@ -456,7 +456,7 @@ public class SimpleMemoryAllocatorJUnitTest {
       bmc = ma.allocate(BIG_ALLOC_SIZE-perObjectOverhead);
       bmc.release();
       assertEquals(TOTAL_MEM, ma.freeList.getFreeMemory());
-      ArrayList<MemoryChunk> mcs = new ArrayList<MemoryChunk>();
+      ArrayList<MemoryChunkWithRefCount> mcs = new ArrayList<MemoryChunkWithRefCount>();
       for (int i=0; i < BIG_ALLOC_SIZE/(8+perObjectOverhead); i++) {
         mcs.add(ma.allocate(8));
       }
@@ -492,11 +492,11 @@ public class SimpleMemoryAllocatorJUnitTest {
       assertEquals((8+perObjectOverhead)*6, ma.freeList.getFreeMemory());
       checkMcs(mcs);
       // At this point I should have 8*6 + perObjectOverhead*6 of free memory
-      MemoryChunk mc24 = ma.allocate(24);
+      MemoryChunkWithRefCount 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
-      MemoryChunk mc16 = ma.allocate(16);
+      MemoryChunkWithRefCount 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
@@ -504,7 +504,7 @@ public class SimpleMemoryAllocatorJUnitTest {
       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
-      MemoryChunk mcDO = ma.allocate(perObjectOverhead*2);
+      MemoryChunkWithRefCount 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());
@@ -525,7 +525,7 @@ public class SimpleMemoryAllocatorJUnitTest {
       assertEquals((perObjectOverhead*3)+(8+perObjectOverhead)+(16+perObjectOverhead)+(24+perObjectOverhead), ma.freeList.getFreeMemory());
       
       long freeMem = ma.freeList.getFreeMemory();
-      for (MemoryChunk mc: mcs) {
+      for (MemoryChunkWithRefCount mc: mcs) {
         mc.release();
         assertEquals(freeMem+(8+perObjectOverhead), ma.freeList.getFreeMemory());
         freeMem += (8+perObjectOverhead);
@@ -545,9 +545,9 @@ public class SimpleMemoryAllocatorJUnitTest {
   public void testUsageEventListener() {
     final int perObjectOverhead = com.gemstone.gemfire.internal.offheap.ObjectChunk.OFF_HEAP_HEADER_SIZE;
     final int SMALL_ALLOC_SIZE = 1000;
-    UnsafeMemoryChunk slab = new UnsafeMemoryChunk(3000);
+    SlabImpl slab = new SlabImpl(3000);
     try {
-      SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new UnsafeMemoryChunk[]{slab});
+      SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new SlabImpl[]{slab});
       MemoryUsageListener listener = new MemoryUsageListener() {
         @Override
         public void updateMemoryUsed(final long bytesUsed) {
@@ -559,7 +559,7 @@ public class SimpleMemoryAllocatorJUnitTest {
       
       this.expectedMemoryUsage = SMALL_ALLOC_SIZE;
       this.memoryUsageEventReceived = false;
-      MemoryChunk smc = ma.allocate(SMALL_ALLOC_SIZE-perObjectOverhead);
+      MemoryChunkWithRefCount smc = ma.allocate(SMALL_ALLOC_SIZE-perObjectOverhead);
       assertEquals(true, this.memoryUsageEventReceived);
       
       this.expectedMemoryUsage = SMALL_ALLOC_SIZE * 2;
@@ -588,8 +588,8 @@ public class SimpleMemoryAllocatorJUnitTest {
       SimpleMemoryAllocatorImpl.freeOffHeapMemory();
     }
   }
-  private void checkMcs(ArrayList<MemoryChunk> mcs) {
-    for (MemoryChunk mc: mcs) {
+  private void checkMcs(ArrayList<MemoryChunkWithRefCount> mcs) {
+    for (MemoryChunkWithRefCount mc: mcs) {
       assertEquals(8+8, mc.getSize());
     }
   }
@@ -600,7 +600,7 @@ public class SimpleMemoryAllocatorJUnitTest {
     final int BIG_ALLOC_SIZE = 150000;
     final int SMALL_ALLOC_SIZE = BIG_ALLOC_SIZE/2;
     final int TOTAL_MEM = BIG_ALLOC_SIZE;
-    final UnsafeMemoryChunk slab = new UnsafeMemoryChunk(TOTAL_MEM);
+    final SlabImpl slab = new SlabImpl(TOTAL_MEM);
     final AtomicReference<OutOfOffHeapMemoryException> ooom = new AtomicReference<OutOfOffHeapMemoryException>();
     final OutOfOffHeapMemoryListener oooml = new OutOfOffHeapMemoryListener() {
       @Override
@@ -612,13 +612,13 @@ public class SimpleMemoryAllocatorJUnitTest {
       }
     };
     try {
-      SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.createForUnitTest(oooml, new NullOffHeapMemoryStats(), new UnsafeMemoryChunk[]{slab});
+      SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.createForUnitTest(oooml, new NullOffHeapMemoryStats(), new SlabImpl[]{slab});
       // make a big allocation
-      MemoryChunk bmc = ma.allocate(BIG_ALLOC_SIZE-perObjectOverhead);
+      MemoryChunkWithRefCount bmc = ma.allocate(BIG_ALLOC_SIZE-perObjectOverhead);
       assertNull(ooom.get());
       // drive the ma to ooom with small allocations
       try {
-        MemoryChunk smc = ma.allocate(SMALL_ALLOC_SIZE-perObjectOverhead);
+        MemoryChunkWithRefCount smc = ma.allocate(SMALL_ALLOC_SIZE-perObjectOverhead);
         fail("Expected out of memory");
       } catch (OutOfOffHeapMemoryException expected) {
       }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/e2a126d4/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/SyncChunkStackJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/SyncChunkStackJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/SyncChunkStackJUnitTest.java
index 3ae6159..f101688 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/SyncChunkStackJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/SyncChunkStackJUnitTest.java
@@ -104,9 +104,9 @@ public class SyncChunkStackJUnitTest {
   
   @Test
   public void stackCreatedWithAddressIsNotEmpty() {
-    UnsafeMemoryChunk slab = new UnsafeMemoryChunk(1024);
+    SlabImpl slab = new SlabImpl(1024);
     try {
-      SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new UnsafeMemoryChunk[]{slab});
+      SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new SlabImpl[]{slab});
       ObjectChunk chunk = (ObjectChunk) ma.allocate(100);
 
       SyncChunkStack stack = new SyncChunkStack(chunk.getMemoryAddress());
@@ -118,9 +118,9 @@ public class SyncChunkStackJUnitTest {
 
   @Test
   public void stackWithChunkIsNotEmpty() {
-    UnsafeMemoryChunk slab = new UnsafeMemoryChunk(1024);
+    SlabImpl slab = new SlabImpl(1024);
     try {
-      SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new UnsafeMemoryChunk[]{slab});
+      SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new SlabImpl[]{slab});
       ObjectChunk chunk = (ObjectChunk) ma.allocate(100);
 
       SyncChunkStack stack = new SyncChunkStack();
@@ -133,9 +133,9 @@ public class SyncChunkStackJUnitTest {
 
   @Test
   public void stackWithChunkTopEqualsAddress() {
-    UnsafeMemoryChunk slab = new UnsafeMemoryChunk(1024);
+    SlabImpl slab = new SlabImpl(1024);
     try {
-      SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new UnsafeMemoryChunk[]{slab});
+      SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new SlabImpl[]{slab});
       ObjectChunk chunk = (ObjectChunk) ma.allocate(100);
 
       long addr = chunk.getMemoryAddress();
@@ -160,9 +160,9 @@ public class SyncChunkStackJUnitTest {
 
   @Test
   public void stackWithChunkClearReturnsAddressAndEmptiesStack() {
-    UnsafeMemoryChunk slab = new UnsafeMemoryChunk(1024);
+    SlabImpl slab = new SlabImpl(1024);
     try {
-      SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new UnsafeMemoryChunk[]{slab});
+      SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new SlabImpl[]{slab});
       ObjectChunk chunk = (ObjectChunk) ma.allocate(100);
 
       long addr = chunk.getMemoryAddress();
@@ -178,9 +178,9 @@ public class SyncChunkStackJUnitTest {
 
   @Test
   public void stackWithChunkPollReturnsAddressAndEmptiesStack() {
-    UnsafeMemoryChunk slab = new UnsafeMemoryChunk(1024);
+    SlabImpl slab = new SlabImpl(1024);
     try {
-      SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new UnsafeMemoryChunk[]{slab});
+      SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new SlabImpl[]{slab});
       ObjectChunk chunk = (ObjectChunk) ma.allocate(100);
 
       long addr = chunk.getMemoryAddress();
@@ -196,9 +196,9 @@ public class SyncChunkStackJUnitTest {
 
   @Test
   public void stackWithChunkTotalSizeIsChunkSize() {
-    UnsafeMemoryChunk slab = new UnsafeMemoryChunk(1024);
+    SlabImpl slab = new SlabImpl(1024);
     try {
-      SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new UnsafeMemoryChunk[]{slab});
+      SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new SlabImpl[]{slab});
       ObjectChunk chunk = (ObjectChunk) ma.allocate(100);
       int chunkSize = chunk.getSize();
 
@@ -214,9 +214,9 @@ public class SyncChunkStackJUnitTest {
 
   @Test
   public void stackWithChunkLogShowsMsgAndSize() {
-    UnsafeMemoryChunk slab = new UnsafeMemoryChunk(1024);
+    SlabImpl slab = new SlabImpl(1024);
     try {
-      SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new UnsafeMemoryChunk[]{slab});
+      SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new SlabImpl[]{slab});
       ObjectChunk chunk = (ObjectChunk) ma.allocate(100);
       int chunkSize = chunk.getSize();
 
@@ -250,9 +250,9 @@ public class SyncChunkStackJUnitTest {
   }
   @Test
   public void stackWithChunkTotalSizeIsChunkSizeWithConcurrentMod() {
-    UnsafeMemoryChunk slab = new UnsafeMemoryChunk(1024);
+    SlabImpl slab = new SlabImpl(1024);
     try {
-      SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new UnsafeMemoryChunk[]{slab});
+      SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new SlabImpl[]{slab});
       ObjectChunk chunk = (ObjectChunk) ma.allocate(100);
       int chunkSize = chunk.getSize();
 
@@ -269,9 +269,9 @@ public class SyncChunkStackJUnitTest {
 
   @Test
   public void stackWithChunkLogShowsMsgAndSizeWithConcurrentMod() {
-    UnsafeMemoryChunk slab = new UnsafeMemoryChunk(1024);
+    SlabImpl slab = new SlabImpl(1024);
     try {
-      SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new UnsafeMemoryChunk[]{slab});
+      SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new SlabImpl[]{slab});
       ObjectChunk chunk = (ObjectChunk) ma.allocate(100);
       int chunkSize = chunk.getSize();
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/e2a126d4/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/UnsafeMemoryChunkJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/UnsafeMemoryChunkJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/UnsafeMemoryChunkJUnitTest.java
deleted file mode 100644
index d7168a7..0000000
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/UnsafeMemoryChunkJUnitTest.java
+++ /dev/null
@@ -1,87 +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 org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
-
-@Category(UnitTest.class)
-public class UnsafeMemoryChunkJUnitTest extends MemoryChunkJUnitTestBase {
-
-  @Override
-  protected MemoryChunk createChunk(int size) {
-    return new UnsafeMemoryChunk(size);
-  }
-
-  @Test
-  public void testGetAddress() {
-    MemoryChunk mc = createChunk(1024);
-    try {
-      AddressableMemoryChunk umc = (AddressableMemoryChunk) mc;
-      assertNotEquals(0, umc.getMemoryAddress());
-    } finally {
-      mc.release();
-    }
-  }
-  
-  @Test(expected=AssertionError.class)
-  public void readAbsoluteBytesFailsIfSizeLessThanZero() {
-    UnsafeMemoryChunk.readAbsoluteBytes(0L, null, 0, -1);
-  }
-  @Test
-  public void readAbsoluteBytesDoesNothingIfSizeIsZero() {
-    UnsafeMemoryChunk.readAbsoluteBytes(0L, new byte[0], 0, 0);
-  }
-  @Test(expected=AssertionError.class)
-  public void readAbsoluteBytesFailsIfSizeGreaterThanArrayLength() {
-    UnsafeMemoryChunk.readAbsoluteBytes(0L, new byte[0], 0, 1);
-  }
-  @Test(expected=AssertionError.class)
-  public void readAbsoluteBytesFailsIfByteOffsetNegative() {
-    UnsafeMemoryChunk.readAbsoluteBytes(0L, new byte[0], -1, 0);
-  }
-  @Test(expected=AssertionError.class)
-  public void readAbsoluteBytesFailsIfByteOffsetGreaterThanArrayLength() {
-    UnsafeMemoryChunk.readAbsoluteBytes(0L, new byte[0], 1, 0);
-  }
-  
-  @Test(expected=AssertionError.class)
-  public void writeAbsoluteBytesFailsIfSizeLessThanZero() {
-    UnsafeMemoryChunk.writeAbsoluteBytes(0L, null, 0, -1);
-  }
-  @Test
-  public void writeAbsoluteBytesDoesNothingIfSizeIsZero() {
-    UnsafeMemoryChunk.writeAbsoluteBytes(0L, new byte[0], 0, 0);
-  }
-  @Test(expected=AssertionError.class)
-  public void writeAbsoluteBytesFailsIfSizeGreaterThanArrayLength() {
-    UnsafeMemoryChunk.writeAbsoluteBytes(0L, new byte[0], 0, 1);
-  }
-  @Test(expected=AssertionError.class)
-  public void writeAbsoluteBytesFailsIfByteOffsetNegative() {
-    UnsafeMemoryChunk.writeAbsoluteBytes(0L, new byte[0], -1, 0);
-  }
-  @Test(expected=AssertionError.class)
-  public void writeAbsoluteBytesFailsIfByteOffsetGreaterThanArrayLength() {
-    UnsafeMemoryChunk.writeAbsoluteBytes(0L, new byte[0], 1, 0);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/e2a126d4/geode-core/src/test/java/com/gemstone/gemfire/pdx/OffHeapByteSourceJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/pdx/OffHeapByteSourceJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/pdx/OffHeapByteSourceJUnitTest.java
index 6457425..2c5e834 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/pdx/OffHeapByteSourceJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/pdx/OffHeapByteSourceJUnitTest.java
@@ -25,7 +25,7 @@ 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.StoredObject;
-import com.gemstone.gemfire.internal.offheap.UnsafeMemoryChunk;
+import com.gemstone.gemfire.internal.offheap.SlabImpl;
 import com.gemstone.gemfire.internal.tcp.ByteBufferInputStream.ByteSource;
 import com.gemstone.gemfire.internal.tcp.ByteBufferInputStream.ByteSourceFactory;
 import com.gemstone.gemfire.internal.tcp.ByteBufferInputStream.OffHeapByteSource;
@@ -36,7 +36,7 @@ public class OffHeapByteSourceJUnitTest extends ByteSourceJUnitTest {
 
   @Before
   public void setUp() throws Exception {
-    SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new UnsafeMemoryChunk[]{new UnsafeMemoryChunk(1024*1024)});
+    SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new SlabImpl[]{new SlabImpl(1024*1024)});
   }
 
   @After