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/15 01:23:51 UTC

[08/44] incubator-geode git commit: GEODE-982: refactor off-heap

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/DataAsAddressJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/DataAsAddressJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/DataAsAddressJUnitTest.java
deleted file mode 100644
index 8a251fd..0000000
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/DataAsAddressJUnitTest.java
+++ /dev/null
@@ -1,368 +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 com.gemstone.gemfire.compression.Compressor;
-import com.gemstone.gemfire.internal.cache.BytesAndBitsForCompactor;
-import com.gemstone.gemfire.internal.cache.CachePerfStats;
-import com.gemstone.gemfire.internal.cache.EntryEventImpl;
-import com.gemstone.gemfire.internal.cache.RegionEntryContext;
-import com.gemstone.gemfire.internal.offheap.DataAsAddress;
-
-import com.gemstone.gemfire.internal.offheap.OffHeapRegionEntryHelper;
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.mockito.Mock;
-
-import java.nio.ByteBuffer;
-
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.mockito.Mockito.*;
-
-@Category(UnitTest.class)
-public class DataAsAddressJUnitTest extends AbstractStoredObjectTestBase {
-
-    @Override
-    public Object getValue() {
-        return Integer.valueOf(123456789);
-    }
-
-    @Override
-    public byte[] getValueAsByteArray() {
-        return convertValueToByteArray(getValue());
-    }
-
-    private byte[] convertValueToByteArray(Object value) {
-        return ByteBuffer.allocate(Integer.SIZE / Byte.SIZE).putInt((Integer) value).array();
-    }
-
-    @Override
-    public Object convertByteArrayToObject(byte[] valueInByteArray) {
-        return ByteBuffer.wrap(valueInByteArray).getInt();
-    }
-
-    @Override
-    public Object convertSerializedByteArrayToObject(byte[] valueInSerializedByteArray) {
-       return EntryEventImpl.deserialize(valueInSerializedByteArray);
-    }
-
-    @Override
-    public DataAsAddress createValueAsUnserializedStoredObject(Object value) {
-        byte[] valueInByteArray;
-        if(value instanceof Integer) {
-            valueInByteArray = convertValueToByteArray(value);
-        } else {
-            valueInByteArray = (byte[]) value;
-        }
-        //encode a non-serialized entry value to address
-        long encodedAddress = OffHeapRegionEntryHelper.encodeDataAsAddress(valueInByteArray, false, false);
-        return new DataAsAddress(encodedAddress);
-    }
-
-    @Override
-    public DataAsAddress createValueAsSerializedStoredObject(Object value) {
-        byte[] valueInSerializedByteArray = EntryEventImpl.serialize(value);
-        //encode a serialized entry value to address
-        long encodedAddress = OffHeapRegionEntryHelper.encodeDataAsAddress(valueInSerializedByteArray, true, false);
-        return new DataAsAddress(encodedAddress);
-    }
-
-    public DataAsAddress createValueAsCompressedStoredObject(Object value) {
-        byte[] valueInSerializedByteArray = EntryEventImpl.serialize(value);
-        //encode a serialized, compressed entry value to address
-        long encodedAddress = OffHeapRegionEntryHelper.encodeDataAsAddress(valueInSerializedByteArray, true, true);
-        return new DataAsAddress(encodedAddress);
-    }
-
-    public DataAsAddress createValueAsUncompressedStoredObject(Object value) {
-        byte[] valueInSerializedByteArray = EntryEventImpl.serialize(value);
-        //encode a serialized, uncompressed entry value to address
-        long encodedAddress = OffHeapRegionEntryHelper.encodeDataAsAddress(valueInSerializedByteArray, true, false);
-        return new DataAsAddress(encodedAddress);
-    }
-
-    @Test
-    public void shouldReturnCorrectEncodingAddress() {
-
-        DataAsAddress address1 = new DataAsAddress(10001L);
-        assertNotNull(address1);
-        assertEquals("Encoding address should be:", 10001, address1.getEncodedAddress());
-
-        DataAsAddress address2 = new DataAsAddress(10002L);
-        assertNotNull(address2);
-        assertEquals("Returning always 10001 expected 10002", 10002, address2.getEncodedAddress());
-    }
-
-    @Test
-    public void twoAddressesShouldBeEqualIfEncodingAddressIsSame() {
-        DataAsAddress address1 = new DataAsAddress(10001L);
-        DataAsAddress address2 = new DataAsAddress(10001L);
-
-        assertEquals("Two addresses are equal if encoding address is same", true, address1.equals(address2));
-    }
-
-    @Test
-    public void twoAddressesShouldNotBeEqualIfEncodingAddressIsNotSame() {
-        DataAsAddress address1 = new DataAsAddress(10001L);
-        DataAsAddress address2 = new DataAsAddress(10002L);
-
-        assertEquals("Two addresses are not equal if encoding address is not same", false, address1.equals(address2));
-    }
-
-    @Test
-    public void twoAddressesAreNotEqualIfTheyAreNotTypeDataAsAddress() {
-        DataAsAddress address1 = new DataAsAddress(10001L);
-        Long address2 = new Long(10002L);
-
-        assertEquals("Two addresses are not equal if encoding address is not same", false, address1.equals(address2));
-    }
-
-    @Test
-    public void addressHashCodeShouldBe() {
-        DataAsAddress address1 = new DataAsAddress(10001L);
-        assertEquals("", 10001, address1.hashCode());
-    }
-
-    @Test
-    public void getSizeInBytesAlwaysReturnsZero() {
-        DataAsAddress address1 = new DataAsAddress(10001L);
-        DataAsAddress address2 = new DataAsAddress(10002L);
-
-        assertEquals("getSizeInBytes", 0, address1.getSizeInBytes());
-        assertEquals("getSizeInBytes", 0, address2.getSizeInBytes());
-    }
-
-    @Test
-    public void getValueSizeInBytesAlwaysReturnsZero() {
-        DataAsAddress address1 = new DataAsAddress(10001L);
-        DataAsAddress address2 = new DataAsAddress(10002L);
-
-        assertEquals("getSizeInBytes", 0, address1.getValueSizeInBytes());
-        assertEquals("getSizeInBytes", 0, address2.getValueSizeInBytes());
-    }
-
-    @Test
-    public void retainShouldAlwaysBeTrue() {
-        DataAsAddress address1 = new DataAsAddress(10001L);
-        DataAsAddress address2 = new DataAsAddress(10002L);
-
-        assertEquals("retain", true, address1.retain());
-        assertEquals("retain", true, address2.retain());
-    }
-
-    @Test
-    public void dataAsAddressShouldImplementReleaseToAdhereToStoredObject() {
-        DataAsAddress address = new DataAsAddress(10001L);
-        address.release();
-    }
-
-    @Test
-    public void isCompressedShouldReturnTrueIfCompressed() {
-        Object regionEntryValue = getValue();
-
-        DataAsAddress offheapAddress = createValueAsCompressedStoredObject(regionEntryValue);
-
-        assertEquals("Should return true as it is compressed", true, offheapAddress.isCompressed());
-    }
-
-    @Test
-    public void isCompressedShouldReturnFalseIfNotCompressed() {
-        Object regionEntryValue = getValue();
-
-        DataAsAddress offheapAddress = createValueAsUncompressedStoredObject(regionEntryValue);
-
-        assertEquals("Should return false as it is compressed", false, offheapAddress.isCompressed());
-    }
-
-    @Test
-    public void isSerializedShouldReturnTrueIfSeriazlied() {
-        Object regionEntryValue = getValue();
-
-        DataAsAddress offheapAddress = createValueAsSerializedStoredObject(regionEntryValue);
-
-        assertEquals("Should return true as it is serialized", true, offheapAddress.isSerialized());
-    }
-
-    @Test
-    public void isSerializedShouldReturnFalseIfNotSeriazlied() {
-        Object regionEntryValue = getValue();
-
-        DataAsAddress offheapAddress = createValueAsUnserializedStoredObject(regionEntryValue);
-
-        assertEquals("Should return false as it is serialized", false, offheapAddress.isSerialized());
-    }
-
-    @Test
-    public void getDecompressedBytesShouldReturnDecompressedBytesIfCompressed() {
-        Object regionEntryValue = getValue();
-        byte[] regionEntryValueAsBytes =  convertValueToByteArray(regionEntryValue);
-
-        //encode a non-serialized and compressed entry value to address - last argument is to let that it is compressed
-        long encodedAddress = OffHeapRegionEntryHelper.encodeDataAsAddress(regionEntryValueAsBytes, false, true);
-        DataAsAddress offheapAddress = new DataAsAddress(encodedAddress);
-
-        RegionEntryContext regionContext = mock(RegionEntryContext.class);
-        CachePerfStats cacheStats = mock(CachePerfStats.class);
-        Compressor compressor = mock(Compressor.class);
-
-        long startTime = 10000L;
-
-        //mock required things
-        when(regionContext.getCompressor()).thenReturn(compressor);
-        when(compressor.decompress(regionEntryValueAsBytes)).thenReturn(regionEntryValueAsBytes);
-        when(regionContext.getCachePerfStats()).thenReturn(cacheStats);
-        when(cacheStats.startDecompression()).thenReturn(startTime);
-
-        //invoke the thing
-        byte[] bytes = offheapAddress.getDecompressedBytes(regionContext);
-
-        //verify the thing happened
-        verify(cacheStats, atLeastOnce()).startDecompression();
-        verify(compressor, times(1)).decompress(regionEntryValueAsBytes);
-        verify(cacheStats, atLeastOnce()).endDecompression(startTime);
-
-        assertArrayEquals(regionEntryValueAsBytes, bytes);
-    }
-
-    @Test
-    public void getDecompressedBytesShouldNotTryToDecompressIfNotCompressed() {
-        Object regionEntryValue = getValue();
-
-        DataAsAddress offheapAddress = createValueAsUncompressedStoredObject(regionEntryValue);
-
-        //mock the thing
-        RegionEntryContext regionContext = mock(RegionEntryContext.class);
-        Compressor compressor = mock(Compressor.class);
-        when(regionContext.getCompressor()).thenReturn(compressor);
-
-        //invoke the thing
-        byte[] actualValueInBytes = offheapAddress.getDecompressedBytes(regionContext);
-
-        //createValueAsUncompressedStoredObject does uses a serialized value - so convert it to object
-        Object actualRegionValue = convertSerializedByteArrayToObject(actualValueInBytes);
-
-        //verify the thing happened
-        verify(regionContext, never()).getCompressor();
-        assertEquals(regionEntryValue, actualRegionValue);
-    }
-
-    @Test
-    public void getRawBytesShouldReturnAByteArray() {
-        byte[] regionEntryValueAsBytes = getValueAsByteArray();
-
-        DataAsAddress offheapAddress = createValueAsUnserializedStoredObject(regionEntryValueAsBytes);
-        byte[] actual = offheapAddress.getRawBytes();
-
-        assertArrayEquals(regionEntryValueAsBytes, actual);
-    }
-
-    @Test
-    public void getSerializedValueShouldReturnASerializedByteArray() {
-        Object regionEntryValue = getValue();
-
-        DataAsAddress offheapAddress = createValueAsSerializedStoredObject(regionEntryValue);
-
-        byte[] actualSerializedValue = offheapAddress.getSerializedValue();
-
-        Object actualRegionEntryValue = convertSerializedByteArrayToObject(actualSerializedValue);
-
-        assertEquals(regionEntryValue, actualRegionEntryValue);
-    }
-
-    @Test
-    public void getDeserializedObjectShouldReturnADeserializedObject() {
-        Object regionEntryValue = getValue();
-
-        DataAsAddress offheapAddress = createValueAsSerializedStoredObject(regionEntryValue);
-
-        Integer actualRegionEntryValue = (Integer) offheapAddress.getDeserializedValue(null, null);
-
-        assertEquals(regionEntryValue, actualRegionEntryValue);
-    }
-
-    @Test
-    public void getDeserializedObjectShouldReturnAByteArrayAsIsIfNotSerialized() {
-        byte[] regionEntryValueAsBytes = getValueAsByteArray();
-
-        DataAsAddress offheapAddress = createValueAsUnserializedStoredObject(regionEntryValueAsBytes);
-
-        byte[] deserializeValue = (byte[]) offheapAddress.getDeserializedValue(null, null);
-
-        assertArrayEquals(regionEntryValueAsBytes, deserializeValue);
-    }
-
-    @Test
-    public void fillSerializedValueShouldFillWrapperWithSerializedValueIfValueIsSerialized() {
-        Object regionEntryValue = getValue();
-        byte[] serializedRegionEntryValue = EntryEventImpl.serialize(regionEntryValue);
-
-        //encode a serialized entry value to address
-        long encodedAddress = OffHeapRegionEntryHelper.encodeDataAsAddress(serializedRegionEntryValue, true, false);
-
-        DataAsAddress offheapAddress = new DataAsAddress(encodedAddress);
-
-        //mock the things
-        BytesAndBitsForCompactor wrapper = mock(BytesAndBitsForCompactor.class);
-
-        byte userBits = 1;
-        offheapAddress.fillSerializedValue(wrapper, userBits);
-
-        verify(wrapper, times(1)).setData(serializedRegionEntryValue, userBits, serializedRegionEntryValue.length, true);
-    }
-
-    @Test
-    public void fillSerializedValueShouldFillWrapperWithDeserializedValueIfValueIsNotSerialized() {
-        Object regionEntryValue = getValue();
-        byte[] regionEntryValueAsBytes =  convertValueToByteArray(regionEntryValue);
-
-        //encode a un serialized entry value to address
-        long encodedAddress = OffHeapRegionEntryHelper.encodeDataAsAddress(regionEntryValueAsBytes, false, false);
-
-        DataAsAddress offheapAddress = new DataAsAddress(encodedAddress);
-
-        //mock the things
-        BytesAndBitsForCompactor wrapper = mock(BytesAndBitsForCompactor.class);
-
-        byte userBits = 1;
-        offheapAddress.fillSerializedValue(wrapper, userBits);
-
-        verify(wrapper, times(1)).setData(regionEntryValueAsBytes, userBits, regionEntryValueAsBytes.length, true);
-    }
-
-    @Test
-    public void getStringFormShouldCatchExceptionAndReturnErrorMessageAsString() {
-        Object regionEntryValueAsBytes = getValue();
-
-        byte[] serializedValue = EntryEventImpl.serialize(regionEntryValueAsBytes);
-
-        //store -127 (DSCODE.ILLEGAL) - in order the deserialize to throw exception
-        serializedValue[0] = -127;
-
-        //encode a un serialized entry value to address
-        long encodedAddress = OffHeapRegionEntryHelper.encodeDataAsAddress(serializedValue, true, false);
-
-        DataAsAddress offheapAddress = new DataAsAddress(encodedAddress);
-
-        String errorMessage = offheapAddress.getStringForm();
-
-        assertEquals(true, errorMessage.contains("Could not convert object to string because "));
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/DirectByteBufferMemoryChunkJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/DirectByteBufferMemoryChunkJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/DirectByteBufferMemoryChunkJUnitTest.java
deleted file mode 100644
index 97b261c..0000000
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/DirectByteBufferMemoryChunkJUnitTest.java
+++ /dev/null
@@ -1,33 +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.nio.ByteBuffer;
-
-import org.junit.experimental.categories.Category;
-
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
-
-@Category(UnitTest.class)
-public class DirectByteBufferMemoryChunkJUnitTest extends MemoryChunkJUnitTestBase {
-
-  @Override
-  protected MemoryChunk createChunk(int size) {
-    return new ByteBufferMemoryChunk(ByteBuffer.allocateDirect(size));
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/FragmentJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/FragmentJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/FragmentJUnitTest.java
index 54eac9e..569b003 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/FragmentJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/FragmentJUnitTest.java
@@ -37,7 +37,7 @@ import com.gemstone.gemfire.test.junit.categories.UnitTest;
 @Category(UnitTest.class)
 public class FragmentJUnitTest {
 
-  private UnsafeMemoryChunk[] slabs;
+  private SlabImpl[] slabs;
 
   static {
     ClassLoader.getSystemClassLoader().setDefaultAssertionStatus(true);
@@ -60,15 +60,15 @@ public class FragmentJUnitTest {
 
   @Before
   public void setUp() throws Exception {
-    UnsafeMemoryChunk slab1 = new UnsafeMemoryChunk((int)OffHeapStorage.MIN_SLAB_SIZE);
-    UnsafeMemoryChunk slab2 = new UnsafeMemoryChunk((int)OffHeapStorage.MIN_SLAB_SIZE);
-    slabs = new UnsafeMemoryChunk[]{slab1, slab2};
+    SlabImpl slab1 = new SlabImpl((int)OffHeapStorage.MIN_SLAB_SIZE);
+    SlabImpl slab2 = new SlabImpl((int)OffHeapStorage.MIN_SLAB_SIZE);
+    slabs = new SlabImpl[]{slab1, slab2};
   }
 
   @After
   public void tearDown() throws Exception {
     for (int i=0; i < slabs.length; i++) {
-      slabs[i].release();
+      slabs[i].free();
     }
   }
   
@@ -120,9 +120,9 @@ public class FragmentJUnitTest {
   @Test
   public void getMemoryAdressIsAlwaysFragmentBaseAddress() {
     Fragment fragment = new Fragment(slabs[0].getMemoryAddress(), slabs[0].getSize());
-    softly.assertThat(fragment.getMemoryAddress()).isEqualTo(slabs[0].getMemoryAddress());
+    softly.assertThat(fragment.getAddress()).isEqualTo(slabs[0].getMemoryAddress());
     fragment.allocate(fragment.getFreeIndex(), fragment.getFreeIndex() + 256);
-    softly.assertThat(fragment.getMemoryAddress()).isEqualTo(slabs[0].getMemoryAddress());
+    softly.assertThat(fragment.getAddress()).isEqualTo(slabs[0].getMemoryAddress());
   }
   
   @Test
@@ -200,7 +200,7 @@ public class FragmentJUnitTest {
   public void fragmentHashCodeIsHashCodeOfItsMemoryAddress() {
     Fragment fragment0 = new Fragment(slabs[0].getMemoryAddress(), slabs[0].getSize());
     Fragment fragment1 = new Fragment(slabs[1].getMemoryAddress(), slabs[1].getSize());
-    Long fragmentAddress = fragment0.getMemoryAddress();
+    Long fragmentAddress = fragment0.getAddress();
     softly.assertThat(fragment0.hashCode()).isEqualTo(fragmentAddress.hashCode())
                                            .isNotEqualTo(fragment1.hashCode());
   }
@@ -208,12 +208,12 @@ public class FragmentJUnitTest {
   @Test
   public void fragmentFillSetsAllBytesToTheSameConstantValue() {
     Fragment fragment = new Fragment(slabs[0].getMemoryAddress(), slabs[0].getSize());
-    Long fragmentAddress = fragment.getMemoryAddress();
+    Long fragmentAddress = fragment.getAddress();
     byte[] bytes = new byte[(int)OffHeapStorage.MIN_SLAB_SIZE];
     byte[] expectedBytes = new byte[(int)OffHeapStorage.MIN_SLAB_SIZE];
-    Arrays.fill(expectedBytes, ObjectChunk.FILL_BYTE);;
+    Arrays.fill(expectedBytes, OffHeapStoredObject.FILL_BYTE);;
     fragment.fill();
-    UnsafeMemoryChunk.readAbsoluteBytes(fragmentAddress, bytes, 0, (int)OffHeapStorage.MIN_SLAB_SIZE);
+    AddressableMemoryManager.readBytes(fragmentAddress, bytes, 0, (int)OffHeapStorage.MIN_SLAB_SIZE);
     assertThat(bytes, is(equalTo(expectedBytes)));
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/FreeListManagerTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/FreeListManagerTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/FreeListManagerTest.java
index 9bfb2eb..3787129 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/FreeListManagerTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/FreeListManagerTest.java
@@ -69,7 +69,7 @@ public class FreeListManagerTest {
     }
   }
   
-  private static TestableFreeListManager createFreeListManager(SimpleMemoryAllocatorImpl ma, AddressableMemoryChunk[] slabs) {
+  private static TestableFreeListManager createFreeListManager(SimpleMemoryAllocatorImpl ma, Slab[] slabs) {
     return new TestableFreeListManager(ma, slabs);
   }
   
@@ -77,8 +77,8 @@ public class FreeListManagerTest {
     setUpSingleSlabManager(DEFAULT_SLAB_SIZE);
   }
   private void setUpSingleSlabManager(int slabSize) {
-    UnsafeMemoryChunk slab = new UnsafeMemoryChunk(slabSize);
-    this.freeListManager = createFreeListManager(ma, new UnsafeMemoryChunk[] {slab});
+    Slab slab = new SlabImpl(slabSize);
+    this.freeListManager = createFreeListManager(ma, new Slab[] {slab});
   }
 
   @Test
@@ -104,12 +104,12 @@ public class FreeListManagerTest {
     setUpSingleSlabManager();
     int tinySize = 10;
 
-    ObjectChunk c = this.freeListManager.allocate(tinySize);
+    OffHeapStoredObject c = this.freeListManager.allocate(tinySize);
     
     validateChunkSizes(c, tinySize);
   }
   
-  private void validateChunkSizes(ObjectChunk c, int dataSize) {
+  private void validateChunkSizes(OffHeapStoredObject c, int dataSize) {
     assertThat(c).isNotNull();
     assertThat(c.getDataSize()).isEqualTo(dataSize);
     assertThat(c.getSize()).isEqualTo(computeExpectedSize(dataSize));
@@ -120,8 +120,8 @@ public class FreeListManagerTest {
     setUpSingleSlabManager();
     int tinySize = 10;
     
-    ObjectChunk c = this.freeListManager.allocate(tinySize);
-    ObjectChunk.release(c.getMemoryAddress(), this.freeListManager);
+    OffHeapStoredObject c = this.freeListManager.allocate(tinySize);
+    OffHeapStoredObject.release(c.getAddress(), this.freeListManager);
     c = this.freeListManager.allocate(tinySize);
 
     validateChunkSizes(c, tinySize);
@@ -132,8 +132,8 @@ public class FreeListManagerTest {
     setUpSingleSlabManager();
     int dataSize = 10;
     
-    ObjectChunk c = this.freeListManager.allocate(dataSize);
-    ObjectChunk.release(c.getMemoryAddress(), this.freeListManager);
+    OffHeapStoredObject c = this.freeListManager.allocate(dataSize);
+    OffHeapStoredObject.release(c.getAddress(), this.freeListManager);
     this.freeListManager.allocate(dataSize);
     // free list will now be empty
     c = this.freeListManager.allocate(dataSize);
@@ -146,7 +146,7 @@ public class FreeListManagerTest {
     setUpSingleSlabManager();
     int hugeSize = FreeListManager.MAX_TINY+1;
 
-    ObjectChunk c = this.freeListManager.allocate(hugeSize);
+    OffHeapStoredObject c = this.freeListManager.allocate(hugeSize);
 
     validateChunkSizes(c, hugeSize);
   }
@@ -156,8 +156,8 @@ public class FreeListManagerTest {
     setUpSingleSlabManager();
     int dataSize = FreeListManager.MAX_TINY+1;
     
-    ObjectChunk c = this.freeListManager.allocate(dataSize);
-    ObjectChunk.release(c.getMemoryAddress(), this.freeListManager);
+    OffHeapStoredObject c = this.freeListManager.allocate(dataSize);
+    OffHeapStoredObject.release(c.getAddress(), this.freeListManager);
     this.freeListManager.allocate(dataSize);
     // free list will now be empty
     c = this.freeListManager.allocate(dataSize);
@@ -170,8 +170,8 @@ public class FreeListManagerTest {
     setUpSingleSlabManager();
     int dataSize = FreeListManager.MAX_TINY+1+1024;
     
-    ObjectChunk c = this.freeListManager.allocate(dataSize);
-    ObjectChunk.release(c.getMemoryAddress(), this.freeListManager);
+    OffHeapStoredObject c = this.freeListManager.allocate(dataSize);
+    OffHeapStoredObject.release(c.getAddress(), this.freeListManager);
     dataSize = FreeListManager.MAX_TINY+1;
     c = this.freeListManager.allocate(dataSize);
     
@@ -188,8 +188,8 @@ public class FreeListManagerTest {
     setUpSingleSlabManager();
     int dataSize = 10;
     
-    ObjectChunk c = this.freeListManager.allocate(dataSize);
-    ObjectChunk.release(c.getMemoryAddress(), this.freeListManager);
+    OffHeapStoredObject c = this.freeListManager.allocate(dataSize);
+    OffHeapStoredObject.release(c.getAddress(), this.freeListManager);
     
     assertThat(this.freeListManager.getFreeTinyMemory()).isEqualTo(computeExpectedSize(dataSize));
   }
@@ -199,13 +199,13 @@ public class FreeListManagerTest {
     setUpSingleSlabManager();
     int dataSize = 10;
     
-    ObjectChunk c = this.freeListManager.allocate(dataSize);
-    ObjectChunk.release(c.getMemoryAddress(), this.freeListManager);
+    OffHeapStoredObject c = this.freeListManager.allocate(dataSize);
+    OffHeapStoredObject.release(c.getAddress(), this.freeListManager);
     
     int dataSize2 = 100;
     
-    ObjectChunk c2 = this.freeListManager.allocate(dataSize2);
-    ObjectChunk.release(c2.getMemoryAddress(), this.freeListManager);
+    OffHeapStoredObject c2 = this.freeListManager.allocate(dataSize2);
+    OffHeapStoredObject.release(c2.getAddress(), this.freeListManager);
     
     assertThat(this.freeListManager.getFreeTinyMemory()).isEqualTo(computeExpectedSize(dataSize)+computeExpectedSize(dataSize2));
   }
@@ -221,8 +221,8 @@ public class FreeListManagerTest {
     setUpSingleSlabManager();
     int dataSize = FreeListManager.MAX_TINY+1+1024;
     
-    ObjectChunk c = this.freeListManager.allocate(dataSize);
-    ObjectChunk.release(c.getMemoryAddress(), this.freeListManager);
+    OffHeapStoredObject c = this.freeListManager.allocate(dataSize);
+    OffHeapStoredObject.release(c.getAddress(), this.freeListManager);
     
     assertThat(this.freeListManager.getFreeHugeMemory()).isEqualTo(computeExpectedSize(dataSize));
   }
@@ -237,7 +237,7 @@ public class FreeListManagerTest {
   @Test
   public void freeFragmentMemorySomeOfFragmentAllocated() {
     setUpSingleSlabManager();
-    ObjectChunk c = this.freeListManager.allocate(DEFAULT_SLAB_SIZE/4-8);
+    OffHeapStoredObject c = this.freeListManager.allocate(DEFAULT_SLAB_SIZE/4-8);
     
     assertThat(this.freeListManager.getFreeFragmentMemory()).isEqualTo((DEFAULT_SLAB_SIZE/4)*3);
   }
@@ -245,13 +245,13 @@ public class FreeListManagerTest {
   @Test
   public void freeFragmentMemoryMostOfFragmentAllocated() {
     setUpSingleSlabManager();
-    ObjectChunk c = this.freeListManager.allocate(DEFAULT_SLAB_SIZE-8-8);
+    OffHeapStoredObject c = this.freeListManager.allocate(DEFAULT_SLAB_SIZE-8-8);
     
     assertThat(this.freeListManager.getFreeFragmentMemory()).isZero();
   }
   
   private int computeExpectedSize(int dataSize) {
-    return ((dataSize + ObjectChunk.OFF_HEAP_HEADER_SIZE + 7) / 8) * 8;
+    return ((dataSize + OffHeapStoredObject.HEADER_SIZE + 7) / 8) * 8;
   }
 
   @Test(expected = AssertionError.class)
@@ -264,11 +264,11 @@ public class FreeListManagerTest {
   public void allocateFromMultipleSlabs() {
     int SMALL_SLAB = 16;
     int MEDIUM_SLAB = 128;
-    UnsafeMemoryChunk slab = new UnsafeMemoryChunk(DEFAULT_SLAB_SIZE);
-    this.freeListManager = createFreeListManager(ma, new UnsafeMemoryChunk[] {
-        new UnsafeMemoryChunk(SMALL_SLAB), 
-        new UnsafeMemoryChunk(SMALL_SLAB), 
-        new UnsafeMemoryChunk(MEDIUM_SLAB), 
+    Slab slab = new SlabImpl(DEFAULT_SLAB_SIZE);
+    this.freeListManager = createFreeListManager(ma, new Slab[] {
+        new SlabImpl(SMALL_SLAB), 
+        new SlabImpl(SMALL_SLAB), 
+        new SlabImpl(MEDIUM_SLAB), 
         slab});
     this.freeListManager.allocate(SMALL_SLAB-8+1);
     this.freeListManager.allocate(DEFAULT_SLAB_SIZE-8);
@@ -282,19 +282,19 @@ public class FreeListManagerTest {
   public void compactWithLargeChunkSizeReturnsFalse() {
     int SMALL_SLAB = 16;
     int MEDIUM_SLAB = 128;
-    UnsafeMemoryChunk slab = new UnsafeMemoryChunk(DEFAULT_SLAB_SIZE);
-    this.freeListManager = createFreeListManager(ma, new UnsafeMemoryChunk[] {
-        new UnsafeMemoryChunk(SMALL_SLAB), 
-        new UnsafeMemoryChunk(SMALL_SLAB), 
-        new UnsafeMemoryChunk(MEDIUM_SLAB), 
+    Slab slab = new SlabImpl(DEFAULT_SLAB_SIZE);
+    this.freeListManager = createFreeListManager(ma, new Slab[] {
+        new SlabImpl(SMALL_SLAB), 
+        new SlabImpl(SMALL_SLAB), 
+        new SlabImpl(MEDIUM_SLAB), 
         slab});
-    ArrayList<ObjectChunk> chunks = new ArrayList<>();
+    ArrayList<OffHeapStoredObject> chunks = new ArrayList<>();
     chunks.add(this.freeListManager.allocate(SMALL_SLAB-8+1));
     chunks.add(this.freeListManager.allocate(DEFAULT_SLAB_SIZE/2-8));
     chunks.add(this.freeListManager.allocate(DEFAULT_SLAB_SIZE/2-8));
     chunks.add(this.freeListManager.allocate(SMALL_SLAB-8+1));
-    for (ObjectChunk c: chunks) {
-      ObjectChunk.release(c.getMemoryAddress(), this.freeListManager);
+    for (OffHeapStoredObject c: chunks) {
+      OffHeapStoredObject.release(c.getAddress(), this.freeListManager);
     }
     this.freeListManager.firstCompact = false;
     assertThat(this.freeListManager.compact(DEFAULT_SLAB_SIZE+1)).isFalse();
@@ -304,19 +304,19 @@ public class FreeListManagerTest {
   public void compactWithChunkSizeOfMaxSlabReturnsTrue() {
     int SMALL_SLAB = 16;
     int MEDIUM_SLAB = 128;
-    UnsafeMemoryChunk slab = new UnsafeMemoryChunk(DEFAULT_SLAB_SIZE);
-    this.freeListManager = createFreeListManager(ma, new UnsafeMemoryChunk[] {
-        new UnsafeMemoryChunk(SMALL_SLAB), 
-        new UnsafeMemoryChunk(SMALL_SLAB), 
-        new UnsafeMemoryChunk(MEDIUM_SLAB), 
+    Slab slab = new SlabImpl(DEFAULT_SLAB_SIZE);
+    this.freeListManager = createFreeListManager(ma, new Slab[] {
+        new SlabImpl(SMALL_SLAB), 
+        new SlabImpl(SMALL_SLAB), 
+        new SlabImpl(MEDIUM_SLAB), 
         slab});
-    ArrayList<ObjectChunk> chunks = new ArrayList<>();
+    ArrayList<OffHeapStoredObject> chunks = new ArrayList<>();
     chunks.add(this.freeListManager.allocate(SMALL_SLAB-8+1));
     chunks.add(this.freeListManager.allocate(DEFAULT_SLAB_SIZE/2-8));
     chunks.add(this.freeListManager.allocate(DEFAULT_SLAB_SIZE/2-8));
     chunks.add(this.freeListManager.allocate(SMALL_SLAB-8+1));
-    for (ObjectChunk c: chunks) {
-      ObjectChunk.release(c.getMemoryAddress(), this.freeListManager);
+    for (OffHeapStoredObject c: chunks) {
+      OffHeapStoredObject.release(c.getAddress(), this.freeListManager);
     }
     
     assertThat(this.freeListManager.compact(DEFAULT_SLAB_SIZE)).isTrue();
@@ -327,19 +327,19 @@ public class FreeListManagerTest {
   public void compactWithLiveChunks() {
     int SMALL_SLAB = 16;
     int MEDIUM_SLAB = 128;
-    UnsafeMemoryChunk slab = new UnsafeMemoryChunk(DEFAULT_SLAB_SIZE);
-    this.freeListManager = createFreeListManager(ma, new UnsafeMemoryChunk[] {
-        new UnsafeMemoryChunk(SMALL_SLAB), 
-        new UnsafeMemoryChunk(SMALL_SLAB), 
-        new UnsafeMemoryChunk(MEDIUM_SLAB), 
+    Slab slab = new SlabImpl(DEFAULT_SLAB_SIZE);
+    this.freeListManager = createFreeListManager(ma, new Slab[] {
+        new SlabImpl(SMALL_SLAB), 
+        new SlabImpl(SMALL_SLAB), 
+        new SlabImpl(MEDIUM_SLAB), 
         slab});
-    ArrayList<ObjectChunk> chunks = new ArrayList<>();
+    ArrayList<OffHeapStoredObject> chunks = new ArrayList<>();
     chunks.add(this.freeListManager.allocate(SMALL_SLAB-8+1));
     this.freeListManager.allocate(DEFAULT_SLAB_SIZE/2-8);
     chunks.add(this.freeListManager.allocate(DEFAULT_SLAB_SIZE/2-8));
     this.freeListManager.allocate(SMALL_SLAB-8+1);
-    for (ObjectChunk c: chunks) {
-      ObjectChunk.release(c.getMemoryAddress(), this.freeListManager);
+    for (OffHeapStoredObject c: chunks) {
+      OffHeapStoredObject.release(c.getAddress(), this.freeListManager);
     }
     
     assertThat(this.freeListManager.compact(DEFAULT_SLAB_SIZE/2)).isTrue();
@@ -348,7 +348,7 @@ public class FreeListManagerTest {
   @Test
   public void compactAfterAllocatingAll() {
     setUpSingleSlabManager();
-    ObjectChunk c = freeListManager.allocate(DEFAULT_SLAB_SIZE-8);
+    OffHeapStoredObject c = freeListManager.allocate(DEFAULT_SLAB_SIZE-8);
     this.freeListManager.firstCompact = false;
     assertThat(this.freeListManager.compact(1)).isFalse();
     // call compact twice for extra code coverage
@@ -359,46 +359,46 @@ public class FreeListManagerTest {
   @Test
   public void afterAllocatingAllOneSizeCompactToAllocateDifferentSize() {
     setUpSingleSlabManager();
-    ArrayList<ObjectChunk> chunksToFree = new ArrayList<>();
-    ArrayList<ObjectChunk> chunksToFreeLater = new ArrayList<>();
+    ArrayList<OffHeapStoredObject> chunksToFree = new ArrayList<>();
+    ArrayList<OffHeapStoredObject> chunksToFreeLater = new ArrayList<>();
     int ALLOCATE_COUNT = 1000;
-    ObjectChunk bigChunk = freeListManager.allocate(DEFAULT_SLAB_SIZE-8-(ALLOCATE_COUNT*32)-256-256);
+    OffHeapStoredObject bigChunk = freeListManager.allocate(DEFAULT_SLAB_SIZE-8-(ALLOCATE_COUNT*32)-256-256);
     for (int i=0; i < ALLOCATE_COUNT; i++) {
-      ObjectChunk c = freeListManager.allocate(24);
+      OffHeapStoredObject c = freeListManager.allocate(24);
       if (i%3 != 2) {
         chunksToFree.add(c);
       } else {
         chunksToFreeLater.add(c);
       }
     }
-    ObjectChunk c1 = freeListManager.allocate(64-8);
-    ObjectChunk c2 = freeListManager.allocate(64-8);
-    ObjectChunk c3 = freeListManager.allocate(64-8);
-    ObjectChunk c4 = freeListManager.allocate(64-8);
+    OffHeapStoredObject c1 = freeListManager.allocate(64-8);
+    OffHeapStoredObject c2 = freeListManager.allocate(64-8);
+    OffHeapStoredObject c3 = freeListManager.allocate(64-8);
+    OffHeapStoredObject c4 = freeListManager.allocate(64-8);
 
-    ObjectChunk mediumChunk1 = freeListManager.allocate(128-8);
-    ObjectChunk mediumChunk2 = freeListManager.allocate(128-8);
+    OffHeapStoredObject mediumChunk1 = freeListManager.allocate(128-8);
+    OffHeapStoredObject mediumChunk2 = freeListManager.allocate(128-8);
 
-    ObjectChunk.release(bigChunk.getMemoryAddress(), freeListManager);
+    OffHeapStoredObject.release(bigChunk.getAddress(), freeListManager);
     int s = chunksToFree.size();
     for (int i=s/2; i >=0; i--) {
-      ObjectChunk c = chunksToFree.get(i);
-      ObjectChunk.release(c.getMemoryAddress(), freeListManager);
+      OffHeapStoredObject c = chunksToFree.get(i);
+      OffHeapStoredObject.release(c.getAddress(), freeListManager);
     }
     for (int i=(s/2)+1; i < s; i++) {
-      ObjectChunk c = chunksToFree.get(i);
-      ObjectChunk.release(c.getMemoryAddress(), freeListManager);
+      OffHeapStoredObject c = chunksToFree.get(i);
+      OffHeapStoredObject.release(c.getAddress(), freeListManager);
     }
-    ObjectChunk.release(c3.getMemoryAddress(), freeListManager);
-    ObjectChunk.release(c1.getMemoryAddress(), freeListManager);
-    ObjectChunk.release(c2.getMemoryAddress(), freeListManager);
-    ObjectChunk.release(c4.getMemoryAddress(), freeListManager);
-    ObjectChunk.release(mediumChunk1.getMemoryAddress(), freeListManager);
-    ObjectChunk.release(mediumChunk2.getMemoryAddress(), freeListManager);
+    OffHeapStoredObject.release(c3.getAddress(), freeListManager);
+    OffHeapStoredObject.release(c1.getAddress(), freeListManager);
+    OffHeapStoredObject.release(c2.getAddress(), freeListManager);
+    OffHeapStoredObject.release(c4.getAddress(), freeListManager);
+    OffHeapStoredObject.release(mediumChunk1.getAddress(), freeListManager);
+    OffHeapStoredObject.release(mediumChunk2.getAddress(), freeListManager);
     this.freeListManager.firstCompact = false;
     assertThat(freeListManager.compact(DEFAULT_SLAB_SIZE-(ALLOCATE_COUNT*32))).isFalse();
     for (int i=0; i < ((256*2)/96); i++) {
-      ObjectChunk.release(chunksToFreeLater.get(i).getMemoryAddress(), freeListManager);
+      OffHeapStoredObject.release(chunksToFreeLater.get(i).getAddress(), freeListManager);
     }
     assertThat(freeListManager.compact(DEFAULT_SLAB_SIZE-(ALLOCATE_COUNT*32))).isTrue();
   }
@@ -407,14 +407,14 @@ public class FreeListManagerTest {
   public void afterAllocatingAndFreeingCompact() {
     int slabSize = 1024*3;
     setUpSingleSlabManager(slabSize);
-    ObjectChunk bigChunk1 = freeListManager.allocate(slabSize/3-8);
-    ObjectChunk bigChunk2 = freeListManager.allocate(slabSize/3-8);
-    ObjectChunk bigChunk3 = freeListManager.allocate(slabSize/3-8);
+    OffHeapStoredObject bigChunk1 = freeListManager.allocate(slabSize/3-8);
+    OffHeapStoredObject bigChunk2 = freeListManager.allocate(slabSize/3-8);
+    OffHeapStoredObject bigChunk3 = freeListManager.allocate(slabSize/3-8);
     this.freeListManager.firstCompact = false;
     assertThat(freeListManager.compact(1)).isFalse();
-    ObjectChunk.release(bigChunk3.getMemoryAddress(), freeListManager);
-    ObjectChunk.release(bigChunk2.getMemoryAddress(), freeListManager);
-    ObjectChunk.release(bigChunk1.getMemoryAddress(), freeListManager);
+    OffHeapStoredObject.release(bigChunk3.getAddress(), freeListManager);
+    OffHeapStoredObject.release(bigChunk2.getAddress(), freeListManager);
+    OffHeapStoredObject.release(bigChunk1.getAddress(), freeListManager);
     assertThat(freeListManager.compact(slabSize)).isTrue();
   }
   
@@ -422,29 +422,29 @@ public class FreeListManagerTest {
   public void compactWithEmptyTinyFreeList() {
     setUpSingleSlabManager();
     Fragment originalFragment = this.freeListManager.getFragmentList().get(0);
-    ObjectChunk c = freeListManager.allocate(16);
-    ObjectChunk.release(c.getMemoryAddress(), this.freeListManager);
+    OffHeapStoredObject c = freeListManager.allocate(16);
+    OffHeapStoredObject.release(c.getAddress(), this.freeListManager);
     c = freeListManager.allocate(16);
     this.freeListManager.firstCompact = false;
     assertThat(this.freeListManager.compact(1)).isTrue();
     assertThat(this.freeListManager.getFragmentList()).hasSize(1);
     Fragment compactedFragment = this.freeListManager.getFragmentList().get(0);
     assertThat(compactedFragment.getSize()).isEqualTo(originalFragment.getSize()-(16+8));
-    assertThat(compactedFragment.getMemoryAddress()).isEqualTo(originalFragment.getMemoryAddress()+(16+8));
+    assertThat(compactedFragment.getAddress()).isEqualTo(originalFragment.getAddress()+(16+8));
   }
   
   @Test
   public void allocationsThatLeaveLessThanMinChunkSizeFreeInAFragment() {
     int SMALL_SLAB = 16;
     int MEDIUM_SLAB = 128;
-    UnsafeMemoryChunk slab = new UnsafeMemoryChunk(DEFAULT_SLAB_SIZE);
-    this.freeListManager = createFreeListManager(ma, new UnsafeMemoryChunk[] {
-        new UnsafeMemoryChunk(SMALL_SLAB), 
-        new UnsafeMemoryChunk(SMALL_SLAB), 
-        new UnsafeMemoryChunk(MEDIUM_SLAB), 
+    Slab slab = new SlabImpl(DEFAULT_SLAB_SIZE);
+    this.freeListManager = createFreeListManager(ma, new Slab[] {
+        new SlabImpl(SMALL_SLAB), 
+        new SlabImpl(SMALL_SLAB), 
+        new SlabImpl(MEDIUM_SLAB), 
         slab});
-    this.freeListManager.allocate(DEFAULT_SLAB_SIZE-8-(ObjectChunk.MIN_CHUNK_SIZE-1));
-    this.freeListManager.allocate(MEDIUM_SLAB-8-(ObjectChunk.MIN_CHUNK_SIZE-1));
+    this.freeListManager.allocate(DEFAULT_SLAB_SIZE-8-(OffHeapStoredObject.MIN_CHUNK_SIZE-1));
+    this.freeListManager.allocate(MEDIUM_SLAB-8-(OffHeapStoredObject.MIN_CHUNK_SIZE-1));
     
     assertThat(this.freeListManager.compact(SMALL_SLAB)).isTrue();
   }
@@ -561,32 +561,32 @@ public class FreeListManagerTest {
   
   @Test
   public void okToReuseSameSlabs() {
-    UnsafeMemoryChunk slab = new UnsafeMemoryChunk(DEFAULT_SLAB_SIZE);
-    UnsafeMemoryChunk[] slabs = new UnsafeMemoryChunk[] {slab};
+    Slab slab = new SlabImpl(DEFAULT_SLAB_SIZE);
+    Slab[] slabs = new Slab[] {slab};
     this.freeListManager = createFreeListManager(ma, slabs);
     assertThat(this.freeListManager.okToReuse(slabs)).isTrue();
   }
   @Test
   public void notOkToReuseDifferentSlabs() {
-    UnsafeMemoryChunk slab = new UnsafeMemoryChunk(DEFAULT_SLAB_SIZE);
-    UnsafeMemoryChunk[] slabs = new UnsafeMemoryChunk[] {slab};
+    Slab slab = new SlabImpl(DEFAULT_SLAB_SIZE);
+    Slab[] slabs = new Slab[] {slab};
     this.freeListManager = createFreeListManager(ma, slabs);
-    UnsafeMemoryChunk[] slabs2 = new UnsafeMemoryChunk[] {slab};
+    Slab[] slabs2 = new Slab[] {slab};
     assertThat(this.freeListManager.okToReuse(slabs2)).isFalse();
   }
   @Test
   public void firstSlabAlwaysLargest() {
-    this.freeListManager = createFreeListManager(ma, new UnsafeMemoryChunk[] {
-        new UnsafeMemoryChunk(10), 
-        new UnsafeMemoryChunk(100)});
+    this.freeListManager = createFreeListManager(ma, new Slab[] {
+        new SlabImpl(10), 
+        new SlabImpl(100)});
     assertThat(this.freeListManager.getLargestSlabSize()).isEqualTo(10);
   }
 
   @Test
   public void findSlab() {
-    UnsafeMemoryChunk chunk = new UnsafeMemoryChunk(10);
+    Slab chunk = new SlabImpl(10);
     long address = chunk.getMemoryAddress();
-    this.freeListManager = createFreeListManager(ma, new UnsafeMemoryChunk[] {chunk});
+    this.freeListManager = createFreeListManager(ma, new Slab[] {chunk});
     assertThat(this.freeListManager.findSlab(address)).isEqualTo(0);
     assertThat(this.freeListManager.findSlab(address+9)).isEqualTo(0);
     catchException(this.freeListManager).findSlab(address-1);
@@ -601,10 +601,10 @@ public class FreeListManagerTest {
   
   @Test
   public void findSecondSlab() {
-    UnsafeMemoryChunk chunk = new UnsafeMemoryChunk(10);
+    Slab chunk = new SlabImpl(10);
     long address = chunk.getMemoryAddress();
-    UnsafeMemoryChunk slab = new UnsafeMemoryChunk(DEFAULT_SLAB_SIZE);
-    this.freeListManager = createFreeListManager(ma, new UnsafeMemoryChunk[] {slab, chunk});
+    Slab slab = new SlabImpl(DEFAULT_SLAB_SIZE);
+    this.freeListManager = createFreeListManager(ma, new Slab[] {slab, chunk});
     assertThat(this.freeListManager.findSlab(address)).isEqualTo(1);
     assertThat(this.freeListManager.findSlab(address+9)).isEqualTo(1);
     catchException(this.freeListManager).findSlab(address-1);
@@ -619,9 +619,9 @@ public class FreeListManagerTest {
   
   @Test
   public void validateAddressWithinSlab() {
-    UnsafeMemoryChunk chunk = new UnsafeMemoryChunk(10);
+    Slab chunk = new SlabImpl(10);
     long address = chunk.getMemoryAddress();
-    this.freeListManager = createFreeListManager(ma, new UnsafeMemoryChunk[] {chunk});
+    this.freeListManager = createFreeListManager(ma, new Slab[] {chunk});
     assertThat(this.freeListManager.validateAddressAndSizeWithinSlab(address, -1)).isTrue();
     assertThat(this.freeListManager.validateAddressAndSizeWithinSlab(address+9, -1)).isTrue();
     assertThat(this.freeListManager.validateAddressAndSizeWithinSlab(address-1, -1)).isFalse();
@@ -630,9 +630,9 @@ public class FreeListManagerTest {
   
   @Test
   public void validateAddressAndSizeWithinSlab() {
-    UnsafeMemoryChunk chunk = new UnsafeMemoryChunk(10);
+    Slab chunk = new SlabImpl(10);
     long address = chunk.getMemoryAddress();
-    this.freeListManager = createFreeListManager(ma, new UnsafeMemoryChunk[] {chunk});
+    this.freeListManager = createFreeListManager(ma, new Slab[] {chunk});
     assertThat(this.freeListManager.validateAddressAndSizeWithinSlab(address, 1)).isTrue();
     assertThat(this.freeListManager.validateAddressAndSizeWithinSlab(address, 10)).isTrue();
     catchException(this.freeListManager).validateAddressAndSizeWithinSlab(address, 0);
@@ -647,10 +647,10 @@ public class FreeListManagerTest {
   
   @Test
   public void descriptionOfOneSlab() {
-    UnsafeMemoryChunk chunk = new UnsafeMemoryChunk(10);
+    Slab chunk = new SlabImpl(10);
     long address = chunk.getMemoryAddress();
     long endAddress = address+10;
-    this.freeListManager = createFreeListManager(ma, new UnsafeMemoryChunk[] {chunk});
+    this.freeListManager = createFreeListManager(ma, new Slab[] {chunk});
     StringBuilder sb = new StringBuilder();
     this.freeListManager.getSlabDescriptions(sb);
     assertThat(sb.toString()).isEqualTo("[" + Long.toString(address, 16) + ".." + Long.toString(endAddress, 16) + "] ");
@@ -658,23 +658,23 @@ public class FreeListManagerTest {
 
   @Test
   public void orderBlocksContainsFragment() {
-    UnsafeMemoryChunk chunk = new UnsafeMemoryChunk(10);
+    Slab chunk = new SlabImpl(10);
     long address = chunk.getMemoryAddress();
-    this.freeListManager = createFreeListManager(ma, new UnsafeMemoryChunk[] {chunk});
+    this.freeListManager = createFreeListManager(ma, new Slab[] {chunk});
     List<MemoryBlock> ob = this.freeListManager.getOrderedBlocks();
     assertThat(ob).hasSize(1);
-    assertThat(ob.get(0).getMemoryAddress()).isEqualTo(address);
+    assertThat(ob.get(0).getAddress()).isEqualTo(address);
     assertThat(ob.get(0).getBlockSize()).isEqualTo(10);
   }
   
   @Test
   public void orderBlocksContainsTinyFree() {
-    UnsafeMemoryChunk chunk = new UnsafeMemoryChunk(96);
+    Slab chunk = new SlabImpl(96);
     long address = chunk.getMemoryAddress();
-    this.freeListManager = createFreeListManager(ma, new UnsafeMemoryChunk[] {chunk});
-    ObjectChunk c = this.freeListManager.allocate(24);
-    ObjectChunk c2 = this.freeListManager.allocate(24);
-    ObjectChunk.release(c.getMemoryAddress(), this.freeListManager);
+    this.freeListManager = createFreeListManager(ma, new Slab[] {chunk});
+    OffHeapStoredObject c = this.freeListManager.allocate(24);
+    OffHeapStoredObject c2 = this.freeListManager.allocate(24);
+    OffHeapStoredObject.release(c.getAddress(), this.freeListManager);
 
     List<MemoryBlock> ob = this.freeListManager.getOrderedBlocks();
     assertThat(ob).hasSize(3);
@@ -682,59 +682,59 @@ public class FreeListManagerTest {
 
   @Test
   public void allocatedBlocksEmptyIfNoAllocations() {
-    UnsafeMemoryChunk chunk = new UnsafeMemoryChunk(10);
-    this.freeListManager = createFreeListManager(ma, new UnsafeMemoryChunk[] {chunk});
+    Slab chunk = new SlabImpl(10);
+    this.freeListManager = createFreeListManager(ma, new Slab[] {chunk});
     List<MemoryBlock> ob = this.freeListManager.getAllocatedBlocks();
     assertThat(ob).hasSize(0);
   }
 
   @Test
   public void allocatedBlocksEmptyAfterFree() {
-    UnsafeMemoryChunk chunk = new UnsafeMemoryChunk(96);
-    this.freeListManager = createFreeListManager(ma, new UnsafeMemoryChunk[] {chunk});
-    ObjectChunk c = this.freeListManager.allocate(24);
-    ObjectChunk.release(c.getMemoryAddress(), this.freeListManager);
+    Slab chunk = new SlabImpl(96);
+    this.freeListManager = createFreeListManager(ma, new Slab[] {chunk});
+    OffHeapStoredObject c = this.freeListManager.allocate(24);
+    OffHeapStoredObject.release(c.getAddress(), this.freeListManager);
     List<MemoryBlock> ob = this.freeListManager.getAllocatedBlocks();
     assertThat(ob).hasSize(0);
   }
 
   @Test
   public void allocatedBlocksHasAllocatedChunk() {
-    UnsafeMemoryChunk chunk = new UnsafeMemoryChunk(96);
-    this.freeListManager = createFreeListManager(ma, new UnsafeMemoryChunk[] {chunk});
-    ObjectChunk c = this.freeListManager.allocate(24);
+    Slab chunk = new SlabImpl(96);
+    this.freeListManager = createFreeListManager(ma, new Slab[] {chunk});
+    OffHeapStoredObject c = this.freeListManager.allocate(24);
     List<MemoryBlock> ob = this.freeListManager.getAllocatedBlocks();
     assertThat(ob).hasSize(1);
-    assertThat(ob.get(0).getMemoryAddress()).isEqualTo(c.getMemoryAddress());
+    assertThat(ob.get(0).getAddress()).isEqualTo(c.getAddress());
   }
   
   @Test
   public void allocatedBlocksHasBothAllocatedChunks() {
-    UnsafeMemoryChunk chunk = new UnsafeMemoryChunk(96);
-    this.freeListManager = createFreeListManager(ma, new UnsafeMemoryChunk[] {chunk});
-    ObjectChunk c = this.freeListManager.allocate(24);
-    ObjectChunk c2 = this.freeListManager.allocate(33);
+    Slab chunk = new SlabImpl(96);
+    this.freeListManager = createFreeListManager(ma, new Slab[] {chunk});
+    OffHeapStoredObject c = this.freeListManager.allocate(24);
+    OffHeapStoredObject c2 = this.freeListManager.allocate(33);
     List<MemoryBlock> ob = this.freeListManager.getAllocatedBlocks();
     assertThat(ob).hasSize(2);
   }
   
   @Test
   public void allocateFromFragmentWithBadIndexesReturnsNull() {
-    UnsafeMemoryChunk chunk = new UnsafeMemoryChunk(96);
-    this.freeListManager = createFreeListManager(ma, new UnsafeMemoryChunk[] {chunk});
+    Slab chunk = new SlabImpl(96);
+    this.freeListManager = createFreeListManager(ma, new Slab[] {chunk});
     assertThat(this.freeListManager.allocateFromFragment(-1, 32)).isNull();
     assertThat(this.freeListManager.allocateFromFragment(1, 32)).isNull();
   }
 
   @Test
   public void testLogging() {
-    UnsafeMemoryChunk chunk = new UnsafeMemoryChunk(32);
-    UnsafeMemoryChunk chunk2 = new UnsafeMemoryChunk(1024*1024*5);
-    this.freeListManager = createFreeListManager(ma, new UnsafeMemoryChunk[] {chunk, chunk2});
-    ObjectChunk c = this.freeListManager.allocate(24);
-    ObjectChunk c2 = this.freeListManager.allocate(1024*1024);
-    ObjectChunk.release(c.getMemoryAddress(), this.freeListManager);
-    ObjectChunk.release(c2.getMemoryAddress(), this.freeListManager);
+    Slab chunk = new SlabImpl(32);
+    Slab chunk2 = new SlabImpl(1024*1024*5);
+    this.freeListManager = createFreeListManager(ma, new Slab[] {chunk, chunk2});
+    OffHeapStoredObject c = this.freeListManager.allocate(24);
+    OffHeapStoredObject c2 = this.freeListManager.allocate(1024*1024);
+    OffHeapStoredObject.release(c.getAddress(), this.freeListManager);
+    OffHeapStoredObject.release(c2.getAddress(), this.freeListManager);
     
     LogWriter lw = mock(LogWriter.class);
     this.freeListManager.logOffHeapState(lw, 1024);
@@ -742,8 +742,8 @@ public class FreeListManagerTest {
   
   @Test
   public void fragmentationShouldBeZeroIfNumberOfFragmentsIsZero() {
-    UnsafeMemoryChunk chunk = new UnsafeMemoryChunk(10);
-    this.freeListManager = createFreeListManager(ma, new UnsafeMemoryChunk[] {chunk});
+    SlabImpl chunk = new SlabImpl(10);
+    this.freeListManager = createFreeListManager(ma, new Slab[] {chunk});
     
     FreeListManager spy = spy(this.freeListManager);
     
@@ -754,8 +754,8 @@ public class FreeListManagerTest {
   
   @Test
   public void fragmentationShouldBeZeroIfNumberOfFragmentsIsOne() {
-    UnsafeMemoryChunk chunk = new UnsafeMemoryChunk(10);
-    this.freeListManager = createFreeListManager(ma, new UnsafeMemoryChunk[] {chunk});
+    SlabImpl chunk = new SlabImpl(10);
+    this.freeListManager = createFreeListManager(ma, new Slab[] {chunk});
     
     FreeListManager spy = spy(this.freeListManager);
     
@@ -766,8 +766,8 @@ public class FreeListManagerTest {
   
   @Test
   public void fragmentationShouldBeZeroIfUsedMemoryIsZero() {
-    UnsafeMemoryChunk chunk = new UnsafeMemoryChunk(10);
-    this.freeListManager = createFreeListManager(ma, new UnsafeMemoryChunk[] {chunk});
+    SlabImpl chunk = new SlabImpl(10);
+    this.freeListManager = createFreeListManager(ma, new Slab[] {chunk});
     
     FreeListManager spy = spy(this.freeListManager);
     
@@ -778,46 +778,46 @@ public class FreeListManagerTest {
   
   @Test
   public void fragmentationShouldBe100IfAllFreeMemoryIsFragmentedAsMinChunks() {
-    UnsafeMemoryChunk chunk = new UnsafeMemoryChunk(10);
-    this.freeListManager = createFreeListManager(ma, new UnsafeMemoryChunk[] {chunk});
+    SlabImpl chunk = new SlabImpl(10);
+    this.freeListManager = createFreeListManager(ma, new Slab[] {chunk});
     
     FreeListManager spy = spy(this.freeListManager);
     
     when(spy.getUsedMemory()).thenReturn(1L);
     when(spy.getFragmentCount()).thenReturn(2);
-    when(spy.getFreeMemory()).thenReturn((long)ObjectChunk.MIN_CHUNK_SIZE * 2);
+    when(spy.getFreeMemory()).thenReturn((long)OffHeapStoredObject.MIN_CHUNK_SIZE * 2);
     
     assertThat(spy.getFragmentation()).isEqualTo(100);
   }
   
   @Test
   public void fragmentationShouldBeRoundedToNearestInteger() {
-    UnsafeMemoryChunk chunk = new UnsafeMemoryChunk(10);
-    this.freeListManager = createFreeListManager(ma, new UnsafeMemoryChunk[] {chunk});
+    SlabImpl chunk = new SlabImpl(10);
+    this.freeListManager = createFreeListManager(ma, new Slab[] {chunk});
     
     FreeListManager spy = spy(this.freeListManager);
     
     when(spy.getUsedMemory()).thenReturn(1L);
     when(spy.getFragmentCount()).thenReturn(4);
-    when(spy.getFreeMemory()).thenReturn((long)ObjectChunk.MIN_CHUNK_SIZE * 8);
+    when(spy.getFreeMemory()).thenReturn((long)OffHeapStoredObject.MIN_CHUNK_SIZE * 8);
     
     assertThat(spy.getFragmentation()).isEqualTo(50); //Math.rint(50.0)
     
     when(spy.getUsedMemory()).thenReturn(1L);
     when(spy.getFragmentCount()).thenReturn(3);
-    when(spy.getFreeMemory()).thenReturn((long)ObjectChunk.MIN_CHUNK_SIZE * 8);
+    when(spy.getFreeMemory()).thenReturn((long)OffHeapStoredObject.MIN_CHUNK_SIZE * 8);
     
     assertThat(spy.getFragmentation()).isEqualTo(38); //Math.rint(37.5)
     
     when(spy.getUsedMemory()).thenReturn(1L);
     when(spy.getFragmentCount()).thenReturn(6);
-    when(spy.getFreeMemory()).thenReturn((long)ObjectChunk.MIN_CHUNK_SIZE * 17);
+    when(spy.getFreeMemory()).thenReturn((long)OffHeapStoredObject.MIN_CHUNK_SIZE * 17);
     
     assertThat(spy.getFragmentation()).isEqualTo(35); //Math.rint(35.29)
     
     when(spy.getUsedMemory()).thenReturn(1L);
     when(spy.getFragmentCount()).thenReturn(6);
-    when(spy.getFreeMemory()).thenReturn((long)ObjectChunk.MIN_CHUNK_SIZE * 9);
+    when(spy.getFreeMemory()).thenReturn((long)OffHeapStoredObject.MIN_CHUNK_SIZE * 9);
     
     assertThat(spy.getFragmentation()).isEqualTo(67); //Math.rint(66.66)
   }
@@ -851,10 +851,10 @@ public class FreeListManagerTest {
     }
 
     @Override
-    protected SyncChunkStack createFreeListForEmptySlot(AtomicReferenceArray<SyncChunkStack> freeLists, int idx) {
+    protected OffHeapStoredObjectAddressStack createFreeListForEmptySlot(AtomicReferenceArray<OffHeapStoredObjectAddressStack> freeLists, int idx) {
       if (this.firstTime) {
         this.firstTime = false;
-        SyncChunkStack clq = super.createFreeListForEmptySlot(freeLists, idx);
+        OffHeapStoredObjectAddressStack clq = super.createFreeListForEmptySlot(freeLists, idx);
         if (!freeLists.compareAndSet(idx, null, clq)) {
           fail("this should never happen. Indicates a concurrent modification");
         }
@@ -872,7 +872,7 @@ public class FreeListManagerTest {
       }
     }
     
-    public TestableFreeListManager(SimpleMemoryAllocatorImpl ma, AddressableMemoryChunk[] slabs) {
+    public TestableFreeListManager(SimpleMemoryAllocatorImpl ma, Slab[] slabs) {
       super(ma, slabs);
     }
     

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/FreeListOffHeapRegionJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/FreeListOffHeapRegionJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/FreeListOffHeapRegionJUnitTest.java
index 6790f6a..0272e38 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/FreeListOffHeapRegionJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/FreeListOffHeapRegionJUnitTest.java
@@ -40,7 +40,7 @@ public class FreeListOffHeapRegionJUnitTest extends OffHeapRegionBase {
 
   @Override
   public int perObjectOverhead() {
-    return ObjectChunk.OFF_HEAP_HEADER_SIZE;
+    return OffHeapStoredObject.HEADER_SIZE;
   }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/HeapByteBufferMemoryChunkJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/HeapByteBufferMemoryChunkJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/HeapByteBufferMemoryChunkJUnitTest.java
deleted file mode 100644
index 77cb5ad..0000000
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/HeapByteBufferMemoryChunkJUnitTest.java
+++ /dev/null
@@ -1,33 +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.nio.ByteBuffer;
-
-import org.junit.experimental.categories.Category;
-
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
-
-@Category(UnitTest.class)
-public class HeapByteBufferMemoryChunkJUnitTest extends MemoryChunkJUnitTestBase {
-
-  @Override
-  protected MemoryChunk createChunk(int size) {
-    return new ByteBufferMemoryChunk(ByteBuffer.allocate(size));
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/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 97ae486..a009661 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
@@ -62,9 +62,9 @@ public class LifecycleListenerJUnitTest {
     LifecycleListener.addLifecycleListener(this.listener);
     LifecycleListener.removeLifecycleListener(this.listener);
 
-    UnsafeMemoryChunk slab = new UnsafeMemoryChunk(1024); // 1k
+    SlabImpl slab = new SlabImpl(1024); // 1k
     SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(),
-        new UnsafeMemoryChunk[] { slab });
+        new SlabImpl[] { slab });
 
     Assert.assertEquals(0, this.afterCreateCallbacks.size());
     Assert.assertEquals(0, this.afterReuseCallbacks.size());
@@ -82,9 +82,9 @@ public class LifecycleListenerJUnitTest {
   @Test
   public void testCallbacksAreCalledAfterCreate() {
     LifecycleListener.addLifecycleListener(this.listener);
-    UnsafeMemoryChunk slab = new UnsafeMemoryChunk(1024); // 1k
+    SlabImpl slab = new SlabImpl(1024); // 1k
     SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(),
-        new UnsafeMemoryChunk[] { slab });
+        new SlabImpl[] { slab });
 
     Assert.assertEquals(1, this.afterCreateCallbacks.size());
     Assert.assertEquals(0, this.afterReuseCallbacks.size());
@@ -106,8 +106,8 @@ public class LifecycleListenerJUnitTest {
 
     System.setProperty(SimpleMemoryAllocatorImpl.FREE_OFF_HEAP_MEMORY_PROPERTY, "false");
 
-    UnsafeMemoryChunk slab = new UnsafeMemoryChunk(1024); // 1k
-    SimpleMemoryAllocatorImpl ma = createAllocator(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new UnsafeMemoryChunk[] { slab });
+    SlabImpl slab = new SlabImpl(1024); // 1k
+    SimpleMemoryAllocatorImpl 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 UnsafeMemoryChunk[] { slab });
+    SimpleMemoryAllocatorImpl ma2 = createAllocator(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new SlabImpl[] { slab });
     assertEquals(null, ma2);
     
     Assert.assertEquals(1, this.afterCreateCallbacks.size());
@@ -139,7 +139,7 @@ public class LifecycleListenerJUnitTest {
     Assert.assertEquals(2, this.beforeCloseCallbacks.size());
   }
 
-  private SimpleMemoryAllocatorImpl createAllocator(OutOfOffHeapMemoryListener ooohml, OffHeapMemoryStats ohms, UnsafeMemoryChunk[] slab) {
+  private SimpleMemoryAllocatorImpl createAllocator(OutOfOffHeapMemoryListener ooohml, OffHeapMemoryStats ohms, SlabImpl[] slab) {
     try {
        return SimpleMemoryAllocatorImpl.createForUnitTest(ooohml, ohms, slab);
     } catch (IllegalStateException e) {
@@ -161,8 +161,8 @@ public class LifecycleListenerJUnitTest {
 
     LifecycleListener.addLifecycleListener(this.listener);
 
-    UnsafeMemoryChunk slab = new UnsafeMemoryChunk(1024); // 1k
-    SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new UnsafeMemoryChunk[] { slab });
+    SlabImpl slab = new SlabImpl(1024); // 1k
+    SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new SlabImpl[] { slab });
 
     Assert.assertEquals(1, this.afterCreateCallbacks.size());
     Assert.assertEquals(0, this.afterReuseCallbacks.size());
@@ -174,8 +174,8 @@ public class LifecycleListenerJUnitTest {
     Assert.assertEquals(0, this.afterReuseCallbacks.size());
     Assert.assertEquals(1, this.beforeCloseCallbacks.size());
 
-    slab = new UnsafeMemoryChunk(1024); // 1k
-    SimpleMemoryAllocatorImpl ma2 = SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new UnsafeMemoryChunk[] { slab });
+    slab = new SlabImpl(1024); // 1k
+    SimpleMemoryAllocatorImpl ma2 = SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new SlabImpl[] { slab });
 
     Assert.assertEquals(2, this.afterCreateCallbacks.size());
     Assert.assertEquals(0, this.afterReuseCallbacks.size());

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/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 3d2dbc7..e1c3f4e 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
@@ -49,13 +49,11 @@ public class MemoryBlockNodeJUnitTest {
   private SimpleMemoryAllocatorImpl ma;
   private OutOfOffHeapMemoryListener ooohml;
   private OffHeapMemoryStats stats;
-  private LogWriter lw;
-  private int numSlabs;
-  private AddressableMemoryChunk[] slabs = {
-      new UnsafeMemoryChunk((int)OffHeapStorage.MIN_SLAB_SIZE), 
-      new UnsafeMemoryChunk((int)OffHeapStorage.MIN_SLAB_SIZE * 2)
+  private Slab[] slabs = {
+      new SlabImpl((int)OffHeapStorage.MIN_SLAB_SIZE), 
+      new SlabImpl((int)OffHeapStorage.MIN_SLAB_SIZE * 2)
   };
-  private MemoryChunkWithRefCount storedObject = null;
+  private StoredObject storedObject = null;
 
   static {
     ClassLoader.getSystemClassLoader().setDefaultAssertionStatus(true);
@@ -84,8 +82,6 @@ public class MemoryBlockNodeJUnitTest {
   public void setUp() {
     ooohml = mock(OutOfOffHeapMemoryListener.class);
     stats = mock(OffHeapMemoryStats.class);
-    lw = mock(LogWriter.class);
-    numSlabs = 3;
     ma = (SimpleMemoryAllocatorImpl) SimpleMemoryAllocatorImpl.createForUnitTest(ooohml, stats, slabs);
   }
 
@@ -98,11 +94,11 @@ public class MemoryBlockNodeJUnitTest {
     return Long.valueOf(Long.MAX_VALUE);
   }
 
-  protected MemoryChunkWithRefCount createValueAsUnserializedStoredObject(Object value) {
-    MemoryChunkWithRefCount createdObject = createValueAsUnserializedStoredObject(value, false);
+  protected StoredObject createValueAsUnserializedStoredObject(Object value) {
+    StoredObject createdObject = createValueAsUnserializedStoredObject(value, false);
     return createdObject;
   }
-  protected MemoryChunkWithRefCount createValueAsUnserializedStoredObject(Object value, boolean isCompressed) {
+  protected StoredObject createValueAsUnserializedStoredObject(Object value, boolean isCompressed) {
     byte[] valueInByteArray;
     if (value instanceof Long) {
       valueInByteArray = convertValueToByteArray(value);
@@ -112,7 +108,7 @@ public class MemoryBlockNodeJUnitTest {
 
     boolean isSerialized = false;
 
-    MemoryChunkWithRefCount createdObject = createChunk(valueInByteArray, isSerialized, isCompressed);
+    StoredObject createdObject = createChunk(valueInByteArray, isSerialized, isCompressed);
     return createdObject;
   }
 
@@ -120,23 +116,23 @@ public class MemoryBlockNodeJUnitTest {
     return ByteBuffer.allocate(Long.SIZE / Byte.SIZE).putLong((Long) value).array();
   }
 
-  private MemoryChunkWithRefCount createChunk(byte[] v, boolean isSerialized, boolean isCompressed) {
-    MemoryChunkWithRefCount chunk = (MemoryChunkWithRefCount) ma.allocateAndInitialize(v, isSerialized, isCompressed);
+  private StoredObject createChunk(byte[] v, boolean isSerialized, boolean isCompressed) {
+    StoredObject chunk = (StoredObject) ma.allocateAndInitialize(v, isSerialized, isCompressed);
     return chunk;
   }
 
 
-  protected MemoryChunkWithRefCount createValueAsSerializedStoredObject(Object value) {
-    MemoryChunkWithRefCount createdObject = createValueAsSerializedStoredObject(value, false);
+  protected StoredObject createValueAsSerializedStoredObject(Object value) {
+    StoredObject createdObject = createValueAsSerializedStoredObject(value, false);
     return createdObject;
   }
   
-  protected MemoryChunkWithRefCount createValueAsSerializedStoredObject(Object value, boolean isCompressed) {
+  protected StoredObject createValueAsSerializedStoredObject(Object value, boolean isCompressed) {
     byte[] valueInSerializedByteArray = EntryEventImpl.serialize(value);
 
     boolean isSerialized = true;
 
-    MemoryChunkWithRefCount createdObject = createChunk(valueInSerializedByteArray, isSerialized, isCompressed);
+    StoredObject createdObject = createChunk(valueInSerializedByteArray, isSerialized, isCompressed);
     return createdObject;
   }
   
@@ -151,7 +147,7 @@ public class MemoryBlockNodeJUnitTest {
     expectedException.expect(NullPointerException.class);
     
     MemoryBlock mb = new MemoryBlockNode(ma, null);
-    Long addr = mb.getMemoryAddress();
+    Long addr = mb.getAddress();
     fail("Operations on MemoryBlockNodes with null block argument expected to throw NullPointerException ");
   }
   
@@ -175,7 +171,7 @@ public class MemoryBlockNodeJUnitTest {
   public void getMemoryAddressReturnsAddressOfBlock() {
     Fragment fragment = new Fragment(slabs[0].getMemoryAddress(), slabs[0].getSize());
     MemoryBlock mb = new MemoryBlockNode(ma, fragment);
-    softly.assertThat(mb.getMemoryAddress()).isEqualTo(fragment.getMemoryAddress());
+    softly.assertThat(mb.getAddress()).isEqualTo(fragment.getAddress());
  }
   
   @Test
@@ -315,7 +311,7 @@ public class MemoryBlockNodeJUnitTest {
     storedObject = createValueAsSerializedStoredObject(obj, true);
     MemoryBlock mb = new MemoryBlockNode(ma, (MemoryBlock) storedObject);
     byte[] storedObjectBytes = new byte[storedObject.getValueSizeInBytes()];
-    storedObject.readBytes(0, storedObjectBytes);
+    storedObject.readDataBytes(0, storedObjectBytes);
     softly.assertThat(mb.getDataValue()).isEqualTo(storedObjectBytes);
   }
   
@@ -325,7 +321,7 @@ public class MemoryBlockNodeJUnitTest {
     storedObject = createValueAsUnserializedStoredObject(obj, true);
     MemoryBlock mb = new MemoryBlockNode(ma, (MemoryBlock) storedObject);
     byte[] storedObjectBytes = new byte[storedObject.getValueSizeInBytes()];
-    storedObject.readBytes(0, storedObjectBytes);
+    storedObject.readDataBytes(0, storedObjectBytes);
     softly.assertThat(mb.getDataValue()).isEqualTo(storedObjectBytes);
   }
   
@@ -335,7 +331,7 @@ public class MemoryBlockNodeJUnitTest {
     storedObject = createValueAsUnserializedStoredObject(obj, false);
     MemoryBlock mb = new MemoryBlockNode(ma, (MemoryBlock) storedObject);
     byte[] storedObjectBytes = new byte[storedObject.getValueSizeInBytes()];
-    storedObject.readBytes(0, storedObjectBytes);
+    storedObject.readDataBytes(0, storedObjectBytes);
     softly.assertThat(mb.getDataValue()).isEqualTo(storedObjectBytes);
   }
   
@@ -343,7 +339,7 @@ public class MemoryBlockNodeJUnitTest {
   public void getDataValueWithIllegalDataTypeCatchesIOException() {
     Object obj = getValue();
     storedObject = createValueAsSerializedStoredObject(obj);
-    storedObject.writeByte(0, DSCODE.ILLEGAL);
+    storedObject.writeDataByte(0, DSCODE.ILLEGAL);
     MemoryBlock mb = new MemoryBlockNode(ma, (MemoryBlock) storedObject);
     ByteArrayOutputStream errContent = new ByteArrayOutputStream();
     System.setErr(new PrintStream(errContent));
@@ -356,7 +352,7 @@ public class MemoryBlockNodeJUnitTest {
     storedObject = createValueAsSerializedStoredObject(obj);
     StoredObject spyStoredObject = spy(storedObject);
     MemoryBlock mb = new MemoryBlockNode(ma, (MemoryBlock) spyStoredObject);
-    when(((ObjectChunk)spyStoredObject).getRawBytes()).thenCallRealMethod().thenThrow(new CacheClosedException("Unit test forced exception"));
+    when(((OffHeapStoredObject)spyStoredObject).getRawBytes()).thenCallRealMethod().thenThrow(new CacheClosedException("Unit test forced exception"));
     ByteArrayOutputStream errContent = new ByteArrayOutputStream();
     System.setErr(new PrintStream(errContent));
     softly.assertThat(mb.getDataValue()).isEqualTo("CacheClosedException:Unit test forced exception");
@@ -369,7 +365,7 @@ public class MemoryBlockNodeJUnitTest {
     storedObject = createValueAsSerializedStoredObject(obj);
     StoredObject spyStoredObject = spy(storedObject);
     MemoryBlock mb = new MemoryBlockNode(ma, (MemoryBlock) spyStoredObject);
-    when(((ObjectChunk)spyStoredObject).getRawBytes()).thenCallRealMethod().thenThrow(ClassNotFoundException.class);
+    when(((OffHeapStoredObject)spyStoredObject).getRawBytes()).thenCallRealMethod().thenThrow(ClassNotFoundException.class);
     ByteArrayOutputStream errContent = new ByteArrayOutputStream();
     System.setErr(new PrintStream(errContent));
     softly.assertThat(mb.getDataValue()).isEqualTo("ClassNotFoundException:null");

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/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/3087c86f/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/MemoryChunkTestSuite.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/MemoryChunkTestSuite.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/MemoryChunkTestSuite.java
deleted file mode 100644
index 2908126..0000000
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/MemoryChunkTestSuite.java
+++ /dev/null
@@ -1,32 +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 org.junit.runner.RunWith;
-import org.junit.runners.Suite;
-
-@Suite.SuiteClasses({
-  ByteArrayMemoryChunkJUnitTest.class,
-  DirectByteBufferMemoryChunkJUnitTest.class,
-  HeapByteBufferMemoryChunkJUnitTest.class,
-  UnsafeMemoryChunkJUnitTest.class,
-})
-@RunWith(Suite.class)
-public class MemoryChunkTestSuite {
-
-}