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/09 00:52:54 UTC

[1/8] incubator-geode git commit: GEODE-982: refactor off-heap

Repository: incubator-geode
Updated Branches:
  refs/heads/develop 8d7a00ee4 -> 3087c86f7


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/TinyMemoryBlockJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/TinyMemoryBlockJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/TinyMemoryBlockJUnitTest.java
index 1626a15..d444865 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/TinyMemoryBlockJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/TinyMemoryBlockJUnitTest.java
@@ -43,14 +43,14 @@ public class TinyMemoryBlockJUnitTest {
   private OutOfOffHeapMemoryListener ooohml;
   private OffHeapMemoryStats stats;
 
-  private AddressableMemoryChunk[] slabs = {
-      new UnsafeMemoryChunk((int)OffHeapStorage.MIN_SLAB_SIZE),
-      new UnsafeMemoryChunk((int)OffHeapStorage.MIN_SLAB_SIZE),
-      new UnsafeMemoryChunk((int)OffHeapStorage.MIN_SLAB_SIZE)
+  private Slab[] slabs = {
+      new SlabImpl((int)OffHeapStorage.MIN_SLAB_SIZE),
+      new SlabImpl((int)OffHeapStorage.MIN_SLAB_SIZE),
+      new SlabImpl((int)OffHeapStorage.MIN_SLAB_SIZE)
   };
 
   private static class TestableFreeListManager extends FreeListManager {
-    TestableFreeListManager(SimpleMemoryAllocatorImpl ma, final AddressableMemoryChunk[] slabs) {
+    TestableFreeListManager(SimpleMemoryAllocatorImpl ma, final Slab[] slabs) {
       super (ma, slabs);
     }
   }
@@ -85,17 +85,17 @@ public class TinyMemoryBlockJUnitTest {
     return Long.valueOf(Long.MAX_VALUE);
   }
 
-  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;
   }
 
-  private MemoryChunkWithRefCount createValueAsSerializedStoredObject(Object value, boolean isCompressed) {
+  private 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;
   }
 
@@ -103,7 +103,7 @@ public class TinyMemoryBlockJUnitTest {
     return ByteBuffer.allocate(Long.SIZE / Byte.SIZE).putLong((Long) value).array();
   }
 
-  private MemoryChunkWithRefCount createValueAsUnserializedStoredObject(Object value, boolean isCompressed) {
+  private StoredObject createValueAsUnserializedStoredObject(Object value, boolean isCompressed) {
     byte[] valueInByteArray;
     if (value instanceof Long) {
       valueInByteArray = convertValueToByteArray(value);
@@ -113,7 +113,7 @@ public class TinyMemoryBlockJUnitTest {
 
     boolean isSerialized = false;
 
-    MemoryChunkWithRefCount createdObject = createChunk(valueInByteArray, isSerialized, isCompressed);
+    StoredObject createdObject = createChunk(valueInByteArray, isSerialized, isCompressed);
     return createdObject;
   }
 
@@ -132,12 +132,12 @@ public class TinyMemoryBlockJUnitTest {
   @Test
   public void getMemoryAddressReturnsAddressBlockWasContructedFrom() {
     MemoryBlock mb = new TestableFreeListManager.TinyMemoryBlock(slabs[0].getMemoryAddress(), 0);
-    softly.assertThat(mb.getMemoryAddress()).isEqualTo(slabs[0].getMemoryAddress());
+    softly.assertThat(mb.getAddress()).isEqualTo(slabs[0].getMemoryAddress());
   }
 
   @Test
   public void getBlockSizeReturnsReturnsSizeOfUnderlyingChunk() {
-    MemoryBlock mb = new TestableFreeListManager.TinyMemoryBlock(new ObjectChunk(slabs[0].getMemoryAddress(), slabs[0].getSize()).getMemoryAddress(), 0);
+    MemoryBlock mb = new TestableFreeListManager.TinyMemoryBlock(new OffHeapStoredObject(slabs[0].getMemoryAddress(), slabs[0].getSize()).getAddress(), 0);
     softly.assertThat(mb.getBlockSize()).isEqualTo(slabs[0].getSize());
   }
 
@@ -145,7 +145,7 @@ public class TinyMemoryBlockJUnitTest {
   public void getNextBlockThrowsUnsupportedOperationException() {
     expectedException.expect(UnsupportedOperationException.class);
 
-    MemoryBlock mb = new TestableFreeListManager.TinyMemoryBlock(new ObjectChunk(slabs[0].getMemoryAddress(), slabs[0].getSize()).getMemoryAddress(), 0);
+    MemoryBlock mb = new TestableFreeListManager.TinyMemoryBlock(new OffHeapStoredObject(slabs[0].getMemoryAddress(), slabs[0].getSize()).getAddress(), 0);
     mb.getNextBlock();
     fail("getNextBlock failed to throw UnsupportedOperationException");
   }
@@ -154,23 +154,23 @@ public class TinyMemoryBlockJUnitTest {
   public void getSlabIdThrowsUnsupportedOperationException() {
     expectedException.expect(UnsupportedOperationException.class);
 
-    MemoryBlock mb = new TestableFreeListManager.TinyMemoryBlock(new ObjectChunk(slabs[0].getMemoryAddress(), slabs[0].getSize()).getMemoryAddress(), 0);
+    MemoryBlock mb = new TestableFreeListManager.TinyMemoryBlock(new OffHeapStoredObject(slabs[0].getMemoryAddress(), slabs[0].getSize()).getAddress(), 0);
     mb.getSlabId();
     fail("getSlabId failed to throw UnsupportedOperationException");
   }
 
   @Test
   public void getFreeListIdReturnsIdBlockWasConstructedWith() {
-    MemoryBlock mb0 = new TestableFreeListManager.TinyMemoryBlock(new ObjectChunk(slabs[0].getMemoryAddress(), slabs[0].getSize()).getMemoryAddress(), 0);
-    MemoryBlock mb1 = new TestableFreeListManager.TinyMemoryBlock(new ObjectChunk(slabs[1].getMemoryAddress(), slabs[1].getSize()).getMemoryAddress(), 1);
+    MemoryBlock mb0 = new TestableFreeListManager.TinyMemoryBlock(new OffHeapStoredObject(slabs[0].getMemoryAddress(), slabs[0].getSize()).getAddress(), 0);
+    MemoryBlock mb1 = new TestableFreeListManager.TinyMemoryBlock(new OffHeapStoredObject(slabs[1].getMemoryAddress(), slabs[1].getSize()).getAddress(), 1);
     softly.assertThat(mb0.getFreeListId()).isEqualTo(0);
     softly.assertThat(mb1.getFreeListId()).isEqualTo(1);
   }
 
   @Test
   public void getRefCountReturnsZero() {
-    MemoryBlock mb0 = new TestableFreeListManager.TinyMemoryBlock(new ObjectChunk(slabs[0].getMemoryAddress(), slabs[0].getSize()).getMemoryAddress(), 0);
-    MemoryBlock mb1 = new TestableFreeListManager.TinyMemoryBlock(new ObjectChunk(slabs[1].getMemoryAddress(), slabs[1].getSize()).getMemoryAddress(), 1);
+    MemoryBlock mb0 = new TestableFreeListManager.TinyMemoryBlock(new OffHeapStoredObject(slabs[0].getMemoryAddress(), slabs[0].getSize()).getAddress(), 0);
+    MemoryBlock mb1 = new TestableFreeListManager.TinyMemoryBlock(new OffHeapStoredObject(slabs[1].getMemoryAddress(), slabs[1].getSize()).getAddress(), 1);
     softly.assertThat(mb0.getRefCount()).isEqualTo(0);
     softly.assertThat(mb1.getRefCount()).isEqualTo(0);
   }
@@ -180,8 +180,8 @@ public class TinyMemoryBlockJUnitTest {
     Object obj = getValue();
     boolean compressed = false;
 
-    MemoryChunkWithRefCount storedObject0 = createValueAsSerializedStoredObject(obj, compressed);
-    MemoryBlock mb = new TestableFreeListManager.TinyMemoryBlock(((MemoryBlock)storedObject0).getMemoryAddress(), 0);
+    StoredObject storedObject0 = createValueAsSerializedStoredObject(obj, compressed);
+    MemoryBlock mb = new TestableFreeListManager.TinyMemoryBlock(((MemoryBlock)storedObject0).getAddress(), 0);
     softly.assertThat(mb.getDataType()).isEqualTo("N/A");
   }
 
@@ -190,8 +190,8 @@ public class TinyMemoryBlockJUnitTest {
     Object obj = getValue();
     boolean compressed = false;
 
-    MemoryChunkWithRefCount storedObject0 = createValueAsSerializedStoredObject(obj, compressed);
-    MemoryBlock mb = new TestableFreeListManager.TinyMemoryBlock(((MemoryBlock)storedObject0).getMemoryAddress(), 0);
+    StoredObject storedObject0 = createValueAsSerializedStoredObject(obj, compressed);
+    MemoryBlock mb = new TestableFreeListManager.TinyMemoryBlock(((MemoryBlock)storedObject0).getAddress(), 0);
     softly.assertThat(mb.getDataValue()).isNull();
   }
 
@@ -200,10 +200,10 @@ public class TinyMemoryBlockJUnitTest {
     Object obj = getValue();
     boolean compressed = false;
 
-    MemoryChunkWithRefCount storedObject0 = createValueAsSerializedStoredObject(obj, compressed);
-    MemoryChunkWithRefCount storedObject1 = createValueAsUnserializedStoredObject(obj, compressed);
-    MemoryBlock mb0 = new TestableFreeListManager.TinyMemoryBlock(((MemoryBlock)storedObject0).getMemoryAddress(), 0);
-    MemoryBlock mb1 = new TestableFreeListManager.TinyMemoryBlock(((MemoryBlock)storedObject1).getMemoryAddress(), 0);
+    StoredObject storedObject0 = createValueAsSerializedStoredObject(obj, compressed);
+    StoredObject storedObject1 = createValueAsUnserializedStoredObject(obj, compressed);
+    MemoryBlock mb0 = new TestableFreeListManager.TinyMemoryBlock(((MemoryBlock)storedObject0).getAddress(), 0);
+    MemoryBlock mb1 = new TestableFreeListManager.TinyMemoryBlock(((MemoryBlock)storedObject1).getAddress(), 0);
     softly.assertThat(mb0.isSerialized()).isFalse();
     softly.assertThat(mb1.isSerialized()).isFalse();
   }
@@ -212,10 +212,10 @@ public class TinyMemoryBlockJUnitTest {
   public void isCompressedReturnsFalse() {
     Object obj = getValue();
     boolean compressed = false;
-    MemoryChunkWithRefCount storedObject0 = createValueAsUnserializedStoredObject(obj, compressed);
-    MemoryChunkWithRefCount storedObject1 = createValueAsUnserializedStoredObject(obj, compressed = true);
-    MemoryBlock mb0 = new TestableFreeListManager.TinyMemoryBlock(((MemoryBlock)storedObject0).getMemoryAddress(), 0);
-    MemoryBlock mb1 = new TestableFreeListManager.TinyMemoryBlock(((MemoryBlock)storedObject1).getMemoryAddress(), 0);
+    StoredObject storedObject0 = createValueAsUnserializedStoredObject(obj, compressed);
+    StoredObject storedObject1 = createValueAsUnserializedStoredObject(obj, compressed = true);
+    MemoryBlock mb0 = new TestableFreeListManager.TinyMemoryBlock(((MemoryBlock)storedObject0).getAddress(), 0);
+    MemoryBlock mb1 = new TestableFreeListManager.TinyMemoryBlock(((MemoryBlock)storedObject1).getAddress(), 0);
     softly.assertThat(mb0.isCompressed()).isFalse();
     softly.assertThat(mb1.isCompressed()).isFalse();
   }
@@ -238,7 +238,7 @@ public class TinyMemoryBlockJUnitTest {
   @Test
   public void hashCodeReturnsHashOfUnderlyingMemory() {
     MemoryBlock mb = new TestableFreeListManager.TinyMemoryBlock(slabs[0].getMemoryAddress(), 0);
-    softly.assertThat(mb.hashCode()).isEqualTo(new ObjectChunk(slabs[0].getMemoryAddress(), slabs[0].getSize()).hashCode());
+    softly.assertThat(mb.hashCode()).isEqualTo(new OffHeapStoredObject(slabs[0].getMemoryAddress(), slabs[0].getSize()).hashCode());
   }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/TinyStoredObjectJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/TinyStoredObjectJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/TinyStoredObjectJUnitTest.java
new file mode 100644
index 0000000..94559d6
--- /dev/null
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/TinyStoredObjectJUnitTest.java
@@ -0,0 +1,353 @@
+/*
+ * 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.TinyStoredObject;
+
+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 TinyStoredObjectJUnitTest 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 TinyStoredObject 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 TinyStoredObject(encodedAddress);
+    }
+
+    @Override
+    public TinyStoredObject createValueAsSerializedStoredObject(Object value) {
+        byte[] valueInSerializedByteArray = EntryEventImpl.serialize(value);
+        //encode a serialized entry value to address
+        long encodedAddress = OffHeapRegionEntryHelper.encodeDataAsAddress(valueInSerializedByteArray, true, false);
+        return new TinyStoredObject(encodedAddress);
+    }
+
+    public TinyStoredObject 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 TinyStoredObject(encodedAddress);
+    }
+
+    public TinyStoredObject 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 TinyStoredObject(encodedAddress);
+    }
+
+    @Test
+    public void shouldReturnCorrectEncodingAddress() {
+
+        TinyStoredObject address1 = new TinyStoredObject(10001L);
+        assertNotNull(address1);
+        assertEquals("Encoding address should be:", 10001, address1.getAddress());
+
+        TinyStoredObject address2 = new TinyStoredObject(10002L);
+        assertNotNull(address2);
+        assertEquals("Returning always 10001 expected 10002", 10002, address2.getAddress());
+    }
+
+    @Test
+    public void twoAddressesShouldBeEqualIfEncodingAddressIsSame() {
+        TinyStoredObject address1 = new TinyStoredObject(10001L);
+        TinyStoredObject address2 = new TinyStoredObject(10001L);
+
+        assertEquals("Two addresses are equal if encoding address is same", true, address1.equals(address2));
+    }
+
+    @Test
+    public void twoAddressesShouldNotBeEqualIfEncodingAddressIsNotSame() {
+        TinyStoredObject address1 = new TinyStoredObject(10001L);
+        TinyStoredObject address2 = new TinyStoredObject(10002L);
+
+        assertEquals("Two addresses are not equal if encoding address is not same", false, address1.equals(address2));
+    }
+
+    @Test
+    public void twoAddressesAreNotEqualIfTheyAreNotTypeDataAsAddress() {
+        TinyStoredObject address1 = new TinyStoredObject(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() {
+        TinyStoredObject address1 = new TinyStoredObject(10001L);
+        assertEquals("", 10001, address1.hashCode());
+    }
+
+    @Test
+    public void getSizeInBytesAlwaysReturnsZero() {
+        TinyStoredObject address1 = new TinyStoredObject(10001L);
+        TinyStoredObject address2 = new TinyStoredObject(10002L);
+
+        assertEquals("getSizeInBytes", 0, address1.getSizeInBytes());
+        assertEquals("getSizeInBytes", 0, address2.getSizeInBytes());
+    }
+
+    @Test
+    public void getValueSizeInBytesAlwaysReturnsZero() {
+        TinyStoredObject address1 = new TinyStoredObject(10001L);
+        TinyStoredObject address2 = new TinyStoredObject(10002L);
+
+        assertEquals("getSizeInBytes", 0, address1.getValueSizeInBytes());
+        assertEquals("getSizeInBytes", 0, address2.getValueSizeInBytes());
+    }
+
+    @Test
+    public void isCompressedShouldReturnTrueIfCompressed() {
+        Object regionEntryValue = getValue();
+
+        TinyStoredObject offheapAddress = createValueAsCompressedStoredObject(regionEntryValue);
+
+        assertEquals("Should return true as it is compressed", true, offheapAddress.isCompressed());
+    }
+
+    @Test
+    public void isCompressedShouldReturnFalseIfNotCompressed() {
+        Object regionEntryValue = getValue();
+
+        TinyStoredObject offheapAddress = createValueAsUncompressedStoredObject(regionEntryValue);
+
+        assertEquals("Should return false as it is compressed", false, offheapAddress.isCompressed());
+    }
+
+    @Test
+    public void isSerializedShouldReturnTrueIfSeriazlied() {
+        Object regionEntryValue = getValue();
+
+        TinyStoredObject offheapAddress = createValueAsSerializedStoredObject(regionEntryValue);
+
+        assertEquals("Should return true as it is serialized", true, offheapAddress.isSerialized());
+    }
+
+    @Test
+    public void isSerializedShouldReturnFalseIfNotSeriazlied() {
+        Object regionEntryValue = getValue();
+
+        TinyStoredObject 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);
+        TinyStoredObject offheapAddress = new TinyStoredObject(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();
+
+        TinyStoredObject 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();
+
+        TinyStoredObject offheapAddress = createValueAsUnserializedStoredObject(regionEntryValueAsBytes);
+        byte[] actual = offheapAddress.getRawBytes();
+
+        assertArrayEquals(regionEntryValueAsBytes, actual);
+    }
+
+    @Test
+    public void getSerializedValueShouldReturnASerializedByteArray() {
+        Object regionEntryValue = getValue();
+
+        TinyStoredObject offheapAddress = createValueAsSerializedStoredObject(regionEntryValue);
+
+        byte[] actualSerializedValue = offheapAddress.getSerializedValue();
+
+        Object actualRegionEntryValue = convertSerializedByteArrayToObject(actualSerializedValue);
+
+        assertEquals(regionEntryValue, actualRegionEntryValue);
+    }
+
+    @Test
+    public void getDeserializedObjectShouldReturnADeserializedObject() {
+        Object regionEntryValue = getValue();
+
+        TinyStoredObject offheapAddress = createValueAsSerializedStoredObject(regionEntryValue);
+
+        Integer actualRegionEntryValue = (Integer) offheapAddress.getDeserializedValue(null, null);
+
+        assertEquals(regionEntryValue, actualRegionEntryValue);
+    }
+
+    @Test
+    public void getDeserializedObjectShouldReturnAByteArrayAsIsIfNotSerialized() {
+        byte[] regionEntryValueAsBytes = getValueAsByteArray();
+
+        TinyStoredObject 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);
+
+        TinyStoredObject offheapAddress = new TinyStoredObject(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);
+
+        TinyStoredObject offheapAddress = new TinyStoredObject(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);
+
+        TinyStoredObject offheapAddress = new TinyStoredObject(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/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/3087c86f/geode-core/src/test/java/com/gemstone/gemfire/management/OffHeapManagementDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/OffHeapManagementDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/OffHeapManagementDUnitTest.java
index 3d06e11..9ded958 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/OffHeapManagementDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/OffHeapManagementDUnitTest.java
@@ -36,7 +36,7 @@ import com.gemstone.gemfire.cache30.CacheTestCase;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
 import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
-import com.gemstone.gemfire.internal.offheap.ObjectChunk;
+import com.gemstone.gemfire.internal.offheap.OffHeapStoredObject;
 import com.gemstone.gemfire.internal.offheap.OffHeapMemoryStats;
 import com.gemstone.gemfire.internal.offheap.OffHeapStorage;
 import com.gemstone.gemfire.management.internal.MBeanJMXAdapter;
@@ -240,7 +240,7 @@ public class OffHeapManagementDUnitTest extends CacheTestCase {
       
       // After allocating large chunk (equal to total memory) 
       // we should still have no fragmentation
-      int largeChunk = (int) TOTAL_MEMORY - ObjectChunk.OFF_HEAP_HEADER_SIZE;
+      int largeChunk = (int) TOTAL_MEMORY - OffHeapStoredObject.HEADER_SIZE;
       doPutOnVm(vm, KEY, new byte[largeChunk], OFF_HEAP_REGION_NAME, false);
       // No compaction has run, so fragmentation should be zero
       assertFragmentationStatOnVm(vm,0,ASSERT_OP.EQUAL);
@@ -255,7 +255,7 @@ public class OffHeapManagementDUnitTest extends CacheTestCase {
       assertFragmentationStatOnVm(vm,0,ASSERT_OP.EQUAL);
       
       // Allocate HALF_TOTAL_MEMORY twice and release one to create one fragment
-      int halfChunk = HALF_TOTAL_MEMORY - ObjectChunk.OFF_HEAP_HEADER_SIZE;
+      int halfChunk = HALF_TOTAL_MEMORY - OffHeapStoredObject.HEADER_SIZE;
       doPutOnVm(vm, KEY + "0", new byte[halfChunk], OFF_HEAP_REGION_NAME, false);
       doPutOnVm(vm, KEY + "1", new byte[halfChunk], OFF_HEAP_REGION_NAME, false);
       doDestroyOnVm(vm, KEY + "0", OFF_HEAP_REGION_NAME);
@@ -267,7 +267,7 @@ public class OffHeapManagementDUnitTest extends CacheTestCase {
       
       // Consume the available fragment as below
       // [16][262120][16][262120][16] = [524288] (HALF_TOTAL_MEMORY)
-      int smallChunk = ObjectChunk.MIN_CHUNK_SIZE - ObjectChunk.OFF_HEAP_HEADER_SIZE;
+      int smallChunk = OffHeapStoredObject.MIN_CHUNK_SIZE - OffHeapStoredObject.HEADER_SIZE;
       int mediumChunk = 262112; //(262120 - ObjectChunk.OFF_HEAP_HEADER_SIZE)
       doPutOnVm(vm, KEY + "S1", new byte[smallChunk], OFF_HEAP_REGION_NAME, false);
       doPutOnVm(vm, KEY + "M1", new byte[mediumChunk], OFF_HEAP_REGION_NAME, false);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/geode-core/src/test/java/com/gemstone/gemfire/pdx/OffHeapByteBufferByteSourceJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/pdx/OffHeapByteBufferByteSourceJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/pdx/OffHeapByteBufferByteSourceJUnitTest.java
index 0f918cb..8380f57 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/pdx/OffHeapByteBufferByteSourceJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/pdx/OffHeapByteBufferByteSourceJUnitTest.java
@@ -22,7 +22,7 @@ import java.nio.ByteBuffer;
 
 import org.junit.experimental.categories.Category;
 
-import com.gemstone.gemfire.internal.offheap.ObjectChunk;
+import com.gemstone.gemfire.internal.offheap.OffHeapStoredObject;
 import com.gemstone.gemfire.internal.offheap.SimpleMemoryAllocatorImpl;
 import com.gemstone.gemfire.internal.offheap.StoredObject;
 import com.gemstone.gemfire.internal.tcp.ByteBufferInputStream.ByteSource;
@@ -35,8 +35,8 @@ public class OffHeapByteBufferByteSourceJUnitTest extends OffHeapByteSourceJUnit
   @Override
   protected ByteSource createByteSource(byte[] bytes) {
     StoredObject so = SimpleMemoryAllocatorImpl.getAllocator().allocateAndInitialize(bytes, false, false);
-    if (so instanceof ObjectChunk) {
-      ObjectChunk c = (ObjectChunk) so;
+    if (so instanceof OffHeapStoredObject) {
+      OffHeapStoredObject c = (OffHeapStoredObject) so;
       ByteBuffer bb = c.createDirectByteBuffer();
       if (bb == null) {
         fail("could not create a direct ByteBuffer for an off-heap Chunk");

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/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..2111f79 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
@@ -20,12 +20,12 @@ import org.junit.After;
 import org.junit.Before;
 import org.junit.experimental.categories.Category;
 
-import com.gemstone.gemfire.internal.offheap.ObjectChunk;
+import com.gemstone.gemfire.internal.offheap.OffHeapStoredObject;
 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
@@ -52,9 +52,9 @@ public class OffHeapByteSourceJUnitTest extends ByteSourceJUnitTest {
   @Override
   protected ByteSource createByteSource(byte[] bytes) {
     StoredObject so = SimpleMemoryAllocatorImpl.getAllocator().allocateAndInitialize(bytes, false, false);
-    if (so instanceof ObjectChunk) {
+    if (so instanceof OffHeapStoredObject) {
       // bypass the factory to make sure that OffHeapByteSource is tested
-      return new OffHeapByteSource((ObjectChunk)so);
+      return new OffHeapByteSource(so);
     } else {
       // bytes are so small they can be encoded in a long (see DataAsAddress).
       // So for this test just wrap the original bytes.

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/geode-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/sanctionedDataSerializables.txt
----------------------------------------------------------------------
diff --git a/geode-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/sanctionedDataSerializables.txt b/geode-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/sanctionedDataSerializables.txt
index dd48df4..bb9f350 100644
--- a/geode-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/sanctionedDataSerializables.txt
+++ b/geode-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/sanctionedDataSerializables.txt
@@ -1030,7 +1030,7 @@ toData,17,2ab400022bb800042ab400032bb80004b1
 
 com/gemstone/gemfire/internal/cache/EntryEventImpl,2
 fromData,214,2a2bb8001dc0001eb5001f2bb8001d4d2bb8001d4e2abb0020592c2d01b70021b500222a2bb900230100b80024b500252a2bb900260100b500082ab400222bb8001db600272a2bb8001dc00028b500092bb9002901009900112a2bb8001dc0002ab50007a700322bb9002901009900212a2bb8002bb5002c2a2ab4002cb500052a2ab4002cb8002db50004a7000b2a2bb8001db500042bb9002901009900192a2bb8002bb5002e2a2ab4002eb8002db50006a7000b2a2bb8001db500062a2bb8002fb500302a2bb80031b5000a2a2bb80032b50016b1
-toData,326,2ab4001f2bb801612ab600902bb801612ab40022b6018a2bb801612b2ab40025b4018bb9018c02002b2ab4000811c03f7eb9018d02002ab6004d2bb801612ab400092bb801612ab40007c6000704a70004033d2b1cb9018e02001c99000e2ab400072bb80161a7006f2ab600434e2dc10086360415049900152dc1008499000e2dc00084b900bb010036042b1504b9018e0200150499003b2ab4002cc6000e2ab4002c2bb8018fa7002e2ab40005c6000e2ab400052bb8018fa7001c2dc000863a051905b900c701002bb80190a700082d2bb801612ab700454d2cc100863e1d9900142cc1008499000d2cc00084b900bb01003e2b1db9018e02001d9900292ab4002ec6000e2ab4002e2bb8018fa7001c2cc000863a041904b900c701002bb80190a700082c2bb801612ab40030c001912bb801922ab6005a2bb801612ab400162bb80193b1
+toData,312,2ab400202bb801632ab6008f2bb801632ab40023b6018c2bb801632b2ab40026b4018db9018e02002b2ab4000911c03f7eb9018f02002ab6004e2bb801632ab4000a2bb801632ab40008c6000704a70004033d2b1cb9019002001c99000e2ab400082bb80163a700682ab600444e2dc100853604150499000e2dc00085b900bb010036042b1504b901900200150499003b2ab4002dc6000e2ab4002d2bb80191a7002e2ab40006c6000e2ab400062bb80191a7001c2dc000853a051905b900c901002bb80192a700082d2bb801632ab700464d2cc100853e1d99000d2cc00085b900bb01003e2b1db9019002001d9900292ab4002fc6000e2ab4002f2bb80191a7001c2cc000853a041904b900c901002bb80192a700082c2bb801632ab40031c001932bb801942ab6005b2bb801632ab400172bb80195b1
 
 com/gemstone/gemfire/internal/cache/EntrySnapshot,2
 fromData,50,2a03b500052bb9004101003d1c9900112abb000759b70042b50004a7000e2abb000359b70043b500042ab400042bb60044b1



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

Posted by ds...@apache.org.
GEODE-982: refactor off-heap

- removed unused MemoryChunk implementations
- collapsed OffHeapCacheDeserializable into StoredObject
- Added Slab, SlabFactory, SlabImpl, and AddressableMemoryManager.
- collapsed MemoryChunkWithRefCount into StoredObject
- methods that access and modify the data now have Data in their name
- collapsed AddressableStoredObject into StoredObject
- changed product code to use the StoredObject interface
  instead of internal class implementations of it
- renamed DataAsAddress to TinyStoredObject
- renamed ObjectChunk to OffHeapStoredObject
- renamed ObjectChunkWithHeapForm to OffHeapStoredObjectWithHeapForm
- renamed allocateChunk to allocateOffHeapStoredObject
- renamed FakeChunk to SearchMarker
- renamed ObjectChunkSlice to OffHeapStoredObjectSlice
- renamed SyncChunkStack to OffHeapStoredObjectAddressStack
- renamed ChunkValueWrapper to OffHeapValueWrapper


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

Branch: refs/heads/develop
Commit: 3087c86f729785ad9fa021f4437d25ca5ef9231d
Parents: 8d7a00e
Author: Darrel Schneider <ds...@pivotal.io>
Authored: Tue Mar 8 15:44:03 2016 -0800
Committer: Darrel Schneider <ds...@pivotal.io>
Committed: Tue Mar 8 15:44:03 2016 -0800

----------------------------------------------------------------------
 .../gemfire/cache/client/internal/PutAllOp.java |   8 +-
 .../gemfire/cache/client/internal/PutOp.java    |  15 +-
 .../internal/GetOperationContextImpl.java       |  24 +-
 .../query/internal/index/AbstractIndex.java     |   8 +-
 .../query/internal/index/DummyQRegion.java      |  14 +-
 .../cache/query/internal/index/HashIndex.java   |   6 +-
 .../internal/cache/AbstractRegionEntry.java     |  85 +-
 .../internal/cache/AbstractRegionMap.java       |  81 +-
 .../gemfire/internal/cache/BucketRegion.java    |   5 +-
 .../cache/BytesAndBitsForCompactor.java         |  18 +-
 .../internal/cache/CachedDeserializable.java    |   8 +
 .../gemfire/internal/cache/DiskEntry.java       | 113 +--
 .../gemfire/internal/cache/DiskStoreImpl.java   |   4 -
 .../internal/cache/DistributedRegion.java       |  10 -
 .../gemfire/internal/cache/EntryEventImpl.java  | 161 ++--
 .../gemfire/internal/cache/LocalRegion.java     |  11 +-
 .../gemstone/gemfire/internal/cache/Oplog.java  |  12 +-
 .../internal/cache/PartitionedRegion.java       |   1 -
 .../cache/PreferBytesCachedDeserializable.java  |  11 +-
 .../gemfire/internal/cache/RegionEntry.java     |   8 +-
 .../internal/cache/RemoteDestroyMessage.java    |   4 +-
 .../cache/SearchLoadAndWriteProcessor.java      |   8 +-
 .../cache/StoreAllCachedDeserializable.java     |  11 +-
 .../internal/cache/VMCachedDeserializable.java  |   9 +-
 .../SnappyCompressedCachedDeserializable.java   |  10 +
 .../internal/cache/partitioned/PutMessage.java  |   8 +-
 .../internal/cache/tier/sockets/Part.java       |  51 +-
 .../cache/tier/sockets/command/Get70.java       |  19 +-
 .../cache/tier/sockets/command/Request.java     |   8 +-
 .../cache/wan/GatewaySenderEventImpl.java       |  12 +-
 .../internal/offheap/AbstractStoredObject.java  |  24 +
 .../offheap/AddressableMemoryChunk.java         |  29 -
 .../offheap/AddressableMemoryChunkFactory.java  |  27 -
 .../offheap/AddressableMemoryManager.java       | 261 ++++++
 .../internal/offheap/ByteArrayMemoryChunk.java  |  77 --
 .../internal/offheap/ByteBufferMemoryChunk.java |  90 --
 .../gemfire/internal/offheap/DataAsAddress.java | 131 ---
 .../gemfire/internal/offheap/Fragment.java      |  10 +-
 .../internal/offheap/FreeListManager.java       | 180 ++--
 .../internal/offheap/MemoryAllocator.java       |  18 +-
 .../gemfire/internal/offheap/MemoryBlock.java   |   2 +-
 .../internal/offheap/MemoryBlockNode.java       |  22 +-
 .../gemfire/internal/offheap/MemoryChunk.java   |  47 -
 .../offheap/MemoryChunkWithRefCount.java        |  34 -
 .../gemfire/internal/offheap/ObjectChunk.java   | 737 ---------------
 .../internal/offheap/ObjectChunkSlice.java      |  44 -
 .../offheap/ObjectChunkWithHeapForm.java        |  40 -
 .../offheap/OffHeapCachedDeserializable.java    | 142 ---
 .../gemfire/internal/offheap/OffHeapHelper.java |  24 +-
 .../offheap/OffHeapRegionEntryHelper.java       |  28 +-
 .../internal/offheap/OffHeapStoredObject.java   | 718 +++++++++++++++
 .../OffHeapStoredObjectAddressStack.java        | 141 +++
 .../offheap/OffHeapStoredObjectSlice.java       |  44 +
 .../OffHeapStoredObjectWithHeapForm.java        |  41 +
 .../offheap/SimpleMemoryAllocatorImpl.java      |  77 +-
 .../gemstone/gemfire/internal/offheap/Slab.java |  39 +
 .../gemfire/internal/offheap/SlabFactory.java   |  27 +
 .../gemfire/internal/offheap/SlabImpl.java      |  61 ++
 .../gemfire/internal/offheap/StoredObject.java  | 117 ++-
 .../internal/offheap/SyncChunkStack.java        | 141 ---
 .../internal/offheap/TinyStoredObject.java      | 229 +++++
 .../internal/offheap/UnsafeMemoryChunk.java     | 217 -----
 .../internal/tcp/ByteBufferInputStream.java     |  74 +-
 .../tcp/ImmutableByteBufferInputStream.java     |   4 +-
 .../gemfire/internal/util/BlobHelper.java       |   4 +-
 .../gemfire/pdx/internal/PdxInputStream.java    |   4 +-
 .../gemfire/cache30/MultiVMRegionTestCase.java  |  14 +-
 .../cache/ChunkValueWrapperJUnitTest.java       | 188 ----
 .../gemfire/internal/cache/OffHeapTestUtil.java |   2 +-
 .../cache/OffHeapValueWrapperJUnitTest.java     | 188 ++++
 .../cache/OldValueImporterTestBase.java         |  22 +-
 .../cache/tier/sockets/MessageJUnitTest.java    |   1 -
 .../offheap/ByteArrayMemoryChunkJUnitTest.java  |  30 -
 .../offheap/DataAsAddressJUnitTest.java         | 368 --------
 .../DirectByteBufferMemoryChunkJUnitTest.java   |  33 -
 .../internal/offheap/FragmentJUnitTest.java     |  22 +-
 .../internal/offheap/FreeListManagerTest.java   | 322 +++----
 .../offheap/FreeListOffHeapRegionJUnitTest.java |   2 +-
 .../HeapByteBufferMemoryChunkJUnitTest.java     |  33 -
 .../offheap/LifecycleListenerJUnitTest.java     |  24 +-
 .../offheap/MemoryBlockNodeJUnitTest.java       |  48 +-
 .../offheap/MemoryChunkJUnitTestBase.java       | 290 ------
 .../internal/offheap/MemoryChunkTestSuite.java  |  32 -
 .../internal/offheap/ObjectChunkJUnitTest.java  | 902 -------------------
 .../offheap/ObjectChunkSliceJUnitTest.java      |  72 --
 .../ObjectChunkWithHeapFormJUnitTest.java       |  64 --
 .../offheap/OffHeapHelperJUnitTest.java         |  14 +-
 .../internal/offheap/OffHeapRegionBase.java     |  16 +-
 .../OffHeapRegionEntryHelperJUnitTest.java      |  88 +-
 ...ffHeapStoredObjectAddressStackJUnitTest.java | 289 ++++++
 .../offheap/OffHeapStoredObjectJUnitTest.java   | 869 ++++++++++++++++++
 .../OffHeapStoredObjectSliceJUnitTest.java      |  72 ++
 ...ffHeapStoredObjectWithHeapFormJUnitTest.java |  64 ++
 .../offheap/OffHeapValidationJUnitTest.java     |   8 +-
 .../OffHeapWriteObjectAsByteArrayJUnitTest.java |  12 +-
 .../OldFreeListOffHeapRegionJUnitTest.java      |   2 +-
 ...moryAllocatorFillPatternIntegrationTest.java |  16 +-
 ...mpleMemoryAllocatorFillPatternJUnitTest.java |  24 +-
 .../offheap/SimpleMemoryAllocatorJUnitTest.java | 114 +--
 .../internal/offheap/StoredObjectTestSuite.java |   8 +-
 .../offheap/SyncChunkStackJUnitTest.java        | 289 ------
 .../offheap/TinyMemoryBlockJUnitTest.java       |  64 +-
 .../offheap/TinyStoredObjectJUnitTest.java      | 353 ++++++++
 .../offheap/UnsafeMemoryChunkJUnitTest.java     |  87 --
 .../management/OffHeapManagementDUnitTest.java  |   8 +-
 .../OffHeapByteBufferByteSourceJUnitTest.java   |   6 +-
 .../gemfire/pdx/OffHeapByteSourceJUnitTest.java |  10 +-
 .../sanctionedDataSerializables.txt             |   2 +-
 108 files changed, 4503 insertions(+), 5217 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/geode-core/src/main/java/com/gemstone/gemfire/cache/client/internal/PutAllOp.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/cache/client/internal/PutAllOp.java b/geode-core/src/main/java/com/gemstone/gemfire/cache/client/internal/PutAllOp.java
index e81cd0c..2f06589 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/cache/client/internal/PutAllOp.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/cache/client/internal/PutAllOp.java
@@ -45,7 +45,6 @@ import com.gemstone.gemfire.internal.cache.tier.sockets.Message;
 import com.gemstone.gemfire.internal.cache.tier.sockets.Part;
 import com.gemstone.gemfire.internal.cache.tier.sockets.VersionedObjectList;
 import com.gemstone.gemfire.internal.logging.LogService;
-import com.gemstone.gemfire.internal.offheap.StoredObject;
 
 /**
  * Does a region putAll on a server
@@ -307,11 +306,12 @@ public class PutAllOp {
         getMessage().addStringOrObjPart(key);
         Object value = mapEntry.getValue();
         if (value instanceof CachedDeserializable) {
-          if (value instanceof StoredObject && !((StoredObject) value).isSerialized()) {
+          CachedDeserializable cd = (CachedDeserializable) value;
+          if (!cd.isSerialized()) {
             // it is a byte[]
-            getMessage().addObjPart(((StoredObject) value).getDeserializedForReading());
+            getMessage().addObjPart(cd.getDeserializedForReading());
           } else {
-            Object cdValue = ((CachedDeserializable)value).getValue();
+            Object cdValue = cd.getValue();
             if (cdValue instanceof byte[]) {
               getMessage().addRawPart((byte[])cdValue, true);
             } else {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/geode-core/src/main/java/com/gemstone/gemfire/cache/client/internal/PutOp.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/cache/client/internal/PutOp.java b/geode-core/src/main/java/com/gemstone/gemfire/cache/client/internal/PutOp.java
index ef4c9d3..c0273c5 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/cache/client/internal/PutOp.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/cache/client/internal/PutOp.java
@@ -34,7 +34,6 @@ import com.gemstone.gemfire.internal.cache.tier.sockets.Message;
 import com.gemstone.gemfire.internal.cache.tier.sockets.Part;
 import com.gemstone.gemfire.internal.cache.versions.VersionTag;
 import com.gemstone.gemfire.internal.logging.LogService;
-import com.gemstone.gemfire.internal.offheap.StoredObject;
 
 import java.io.ByteArrayInputStream;
 import java.io.DataInputStream;
@@ -221,13 +220,14 @@ public class PutOp {
           }
         }
         else if (value instanceof CachedDeserializable) {
-          if (value instanceof StoredObject && !((StoredObject) value).isSerialized()) {
+          CachedDeserializable cd = (CachedDeserializable) value;
+          if (!cd.isSerialized()) {
             // it is a byte[]
             getMessage().addObjPart(Boolean.FALSE);
-            getMessage().addObjPart(((StoredObject) value).getDeserializedForReading());
+            getMessage().addObjPart(cd.getDeserializedForReading());
           } else {
             getMessage().addObjPart(Boolean.FALSE);
-            Object cdValue = ((CachedDeserializable)value).getValue();
+            Object cdValue = cd.getValue();
             if (cdValue instanceof byte[]) {
               getMessage().addRawPart((byte[])cdValue, true);
             }
@@ -283,13 +283,14 @@ public class PutOp {
         }
       }
       else if (value instanceof CachedDeserializable) {
-        if (value instanceof StoredObject && !((StoredObject) value).isSerialized()) {
+        CachedDeserializable cd = (CachedDeserializable) value;
+        if (!cd.isSerialized()) {
           // it is a byte[]
           getMessage().addObjPart(Boolean.FALSE);
-          getMessage().addObjPart(((StoredObject) value).getDeserializedForReading());
+          getMessage().addObjPart(cd.getDeserializedForReading());
         } else {
           getMessage().addObjPart(Boolean.FALSE);
-          Object cdValue = ((CachedDeserializable)value).getValue();
+          Object cdValue = cd.getValue();
           if (cdValue instanceof byte[]) {
             getMessage().addRawPart((byte[])cdValue, true);
           }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/geode-core/src/main/java/com/gemstone/gemfire/cache/operations/internal/GetOperationContextImpl.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/cache/operations/internal/GetOperationContextImpl.java b/geode-core/src/main/java/com/gemstone/gemfire/cache/operations/internal/GetOperationContextImpl.java
index 11d9248..3257051 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/cache/operations/internal/GetOperationContextImpl.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/cache/operations/internal/GetOperationContextImpl.java
@@ -18,7 +18,6 @@ package com.gemstone.gemfire.cache.operations.internal;
 
 import com.gemstone.gemfire.SerializationException;
 import com.gemstone.gemfire.cache.operations.GetOperationContext;
-import com.gemstone.gemfire.internal.offheap.ObjectChunk;
 import com.gemstone.gemfire.internal.offheap.Releasable;
 import com.gemstone.gemfire.internal.offheap.StoredObject;
 import com.gemstone.gemfire.internal.offheap.annotations.Unretained;
@@ -68,9 +67,9 @@ public class GetOperationContextImpl extends GetOperationContext implements Rele
     super.setValue(value, isObject);
   }
 
-  private void checkForReleasedOffHeapValue(Object v) {
-    // Note that we only care about Chunk (instead of all StoredObject) because it is the only one using a refcount
-    if (this.released && v instanceof ObjectChunk) {
+  private void checkForReleasedOffHeapValue(StoredObject so) {
+    // Note that we only care about stored objects with a ref count
+    if (this.released && so.hasRefCount()) {
       throw new IllegalStateException("Attempt to access off-heap value after the OperationContext callback returned.");
     }
   }
@@ -81,8 +80,9 @@ public class GetOperationContextImpl extends GetOperationContext implements Rele
     if (result == null) {
       Object v = super.getValue();
       if (v instanceof StoredObject) {
-        checkForReleasedOffHeapValue(v);
-        result = ((StoredObject) v).getValueAsHeapByteArray();
+        StoredObject so = (StoredObject) v;
+        checkForReleasedOffHeapValue(so);
+        result = so.getValueAsHeapByteArray();
       }
     }
     return result;
@@ -92,8 +92,9 @@ public class GetOperationContextImpl extends GetOperationContext implements Rele
   public Object getDeserializedValue() throws SerializationException {
     Object result = super.getDeserializedValue();
     if (result instanceof StoredObject) {
-      checkForReleasedOffHeapValue(result);
-      result = ((StoredObject) result).getValueAsDeserializedHeapObject();
+      StoredObject so = (StoredObject) result;
+      checkForReleasedOffHeapValue(so);
+      result = so.getValueAsDeserializedHeapObject();
     }
     return result;
   }
@@ -102,9 +103,10 @@ public class GetOperationContextImpl extends GetOperationContext implements Rele
   public Object getValue() {
     Object result = super.getValue();
     if (result instanceof StoredObject) {
-      checkForReleasedOffHeapValue(result);
+      StoredObject so = (StoredObject) result;
+      checkForReleasedOffHeapValue(so);
       // since they called getValue they don't care if it is serialized or deserialized so return it as serialized
-      result = ((StoredObject) result).getValueAsHeapByteArray();
+      result = so.getValueAsHeapByteArray();
     }
     return result;
   }
@@ -116,7 +118,7 @@ public class GetOperationContextImpl extends GetOperationContext implements Rele
     // our value (since this context did not retain it)
     // but we do make sure that any future attempt to access
     // the off-heap value fails.
-    if (super.getValue() instanceof ObjectChunk) {
+    if (super.getValue() instanceof StoredObject) {
       this.released = true;
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/geode-core/src/main/java/com/gemstone/gemfire/cache/query/internal/index/AbstractIndex.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/cache/query/internal/index/AbstractIndex.java b/geode-core/src/main/java/com/gemstone/gemfire/cache/query/internal/index/AbstractIndex.java
index aab99cb..0923327 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/cache/query/internal/index/AbstractIndex.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/cache/query/internal/index/AbstractIndex.java
@@ -75,7 +75,7 @@ import com.gemstone.gemfire.internal.cache.RegionEntry;
 import com.gemstone.gemfire.internal.cache.persistence.query.CloseableIterator;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 import com.gemstone.gemfire.internal.logging.LogService;
-import com.gemstone.gemfire.internal.offheap.ObjectChunk;
+import com.gemstone.gemfire.internal.offheap.StoredObject;
 import com.gemstone.gemfire.internal.offheap.annotations.Released;
 import com.gemstone.gemfire.internal.offheap.annotations.Retained;
 import com.gemstone.gemfire.pdx.PdxInstance;
@@ -1477,9 +1477,9 @@ public abstract class AbstractIndex implements IndexProtocol
         valueInIndex = verifyAndGetPdxDomainObject(value);
       } else{
         @Released Object val = re.getValueInVM(context.getPartitionedRegion());
-        ObjectChunk valToFree = null;
-        if (val instanceof ObjectChunk) {
-          valToFree = (ObjectChunk)val;
+        StoredObject valToFree = null;
+        if (val instanceof StoredObject) {
+          valToFree = (StoredObject)val;
         }
         try {
         if (val instanceof CachedDeserializable) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/geode-core/src/main/java/com/gemstone/gemfire/cache/query/internal/index/DummyQRegion.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/cache/query/internal/index/DummyQRegion.java b/geode-core/src/main/java/com/gemstone/gemfire/cache/query/internal/index/DummyQRegion.java
index bde948d..b888f99 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/cache/query/internal/index/DummyQRegion.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/cache/query/internal/index/DummyQRegion.java
@@ -41,7 +41,7 @@ import com.gemstone.gemfire.internal.cache.LocalRegion;
 import com.gemstone.gemfire.internal.cache.RegionEntry;
 import com.gemstone.gemfire.internal.cache.RegionEntryContext;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
-import com.gemstone.gemfire.internal.offheap.ObjectChunk;
+import com.gemstone.gemfire.internal.offheap.StoredObject;
 import com.gemstone.gemfire.internal.offheap.annotations.Released;
 import com.gemstone.gemfire.internal.offheap.annotations.Retained;
 
@@ -134,8 +134,8 @@ public class DummyQRegion extends QRegion {
     }
     valueInList.clear();
     Object val = this.entry.getValueOffHeapOrDiskWithoutFaultIn((LocalRegion) getRegion());
-    if (val instanceof ObjectChunk) {
-      @Retained @Released ObjectChunk ohval = (ObjectChunk) val;
+    if (val instanceof StoredObject) {
+      @Retained @Released StoredObject ohval = (StoredObject) val;
       try {
         // TODO OFFHEAP: val may be off-heap PdxInstance
         val = ohval.getDeserializedValue(getRegion(), this.entry);
@@ -155,8 +155,8 @@ public class DummyQRegion extends QRegion {
       valueInArray = new  Object[1];      
     }   
     Object val = this.entry.getValueOffHeapOrDiskWithoutFaultIn((LocalRegion) getRegion());
-    if (val instanceof ObjectChunk) {      
-      @Retained @Released ObjectChunk ohval = (ObjectChunk) val;
+    if (val instanceof StoredObject) {      
+      @Retained @Released StoredObject ohval = (StoredObject) val;
       try {
         // TODO OFFHEAP: val may be off-heap PdxInstance
         val = ohval.getDeserializedValue(getRegion(), this.entry);
@@ -178,8 +178,8 @@ public class DummyQRegion extends QRegion {
     }
     values.clear();
     Object val = this.entry.getValueOffHeapOrDiskWithoutFaultIn((LocalRegion) getRegion());
-    if (val instanceof ObjectChunk) {
-      @Retained @Released ObjectChunk ohval = (ObjectChunk) val;
+    if (val instanceof StoredObject) {
+      @Retained @Released StoredObject ohval = (StoredObject) val;
       try {
         // TODO OFFHEAP: val may be off-heap PdxInstance
         val = ohval.getDeserializedValue(getRegion(), this.entry);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/geode-core/src/main/java/com/gemstone/gemfire/cache/query/internal/index/HashIndex.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/cache/query/internal/index/HashIndex.java b/geode-core/src/main/java/com/gemstone/gemfire/cache/query/internal/index/HashIndex.java
index a30a264..f8a1b8d 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/cache/query/internal/index/HashIndex.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/cache/query/internal/index/HashIndex.java
@@ -79,7 +79,7 @@ import com.gemstone.gemfire.internal.cache.Token;
 import com.gemstone.gemfire.internal.cache.persistence.query.CloseableIterator;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 import com.gemstone.gemfire.internal.logging.LogService;
-import com.gemstone.gemfire.internal.offheap.ObjectChunk;
+import com.gemstone.gemfire.internal.offheap.StoredObject;
 import com.gemstone.gemfire.internal.offheap.annotations.Released;
 import com.gemstone.gemfire.internal.offheap.annotations.Retained;
 import com.gemstone.gemfire.pdx.internal.PdxString;
@@ -876,8 +876,8 @@ public class HashIndex extends AbstractIndex {
     if (this.indexOnValues) {
       Object o = entry.getValueOffHeapOrDiskWithoutFaultIn((LocalRegion) getRegion());
       try {
-        if (o instanceof ObjectChunk) {
-          ObjectChunk ohval = (ObjectChunk) o;
+        if (o instanceof StoredObject) {
+          StoredObject ohval = (StoredObject) o;
           try {
             o = ohval.getDeserializedForReading();
           } finally {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/AbstractRegionEntry.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/AbstractRegionEntry.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/AbstractRegionEntry.java
index 558ea37..a103e96 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/AbstractRegionEntry.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/AbstractRegionEntry.java
@@ -61,11 +61,7 @@ import com.gemstone.gemfire.internal.lang.StringUtils;
 import com.gemstone.gemfire.internal.logging.LogService;
 import com.gemstone.gemfire.internal.logging.log4j.LocalizedMessage;
 import com.gemstone.gemfire.internal.logging.log4j.LogMarker;
-import com.gemstone.gemfire.internal.offheap.ObjectChunk;
-import com.gemstone.gemfire.internal.offheap.ObjectChunkWithHeapForm;
-import com.gemstone.gemfire.internal.offheap.ObjectChunk;
 import com.gemstone.gemfire.internal.offheap.MemoryAllocator;
-import com.gemstone.gemfire.internal.offheap.OffHeapCachedDeserializable;
 import com.gemstone.gemfire.internal.offheap.OffHeapHelper;
 import com.gemstone.gemfire.internal.offheap.ReferenceCountHelper;
 import com.gemstone.gemfire.internal.offheap.SimpleMemoryAllocatorImpl;
@@ -334,14 +330,14 @@ public abstract class AbstractRegionEntry implements RegionEntry,
 //        // For SQLFire we prefer eager deserialized
 //        dst.setEagerDeserialize();         
 //      }
-      
-      if (v instanceof StoredObject && !((StoredObject) v).isSerialized()) {
-        dst.value = ((StoredObject) v).getDeserializedForReading();
+      CachedDeserializable cd = (CachedDeserializable) v;
+      if (!cd.isSerialized()) {
+        dst.value = cd.getDeserializedForReading();
       } else {
         /*if (v instanceof ByteSource && CachedDeserializableFactory.preferObject()) {
           dst.value = v;
         } else */ {
-          Object tmp = ((CachedDeserializable) v).getValue();
+          Object tmp = cd.getValue();
           if (tmp instanceof byte[]) {
             byte[] bb = (byte[]) tmp;
             dst.value = bb;
@@ -989,10 +985,10 @@ public abstract class AbstractRegionEntry implements RegionEntry,
       return checkPdxEquals((PdxInstance)v1, v2);
     } else if (v2 instanceof PdxInstance) {
       return checkPdxEquals((PdxInstance)v2, v1);
-    } else if (v1 instanceof OffHeapCachedDeserializable) {
-      return checkOffHeapEquals((OffHeapCachedDeserializable)v1, v2);
-    } else if (v2 instanceof OffHeapCachedDeserializable) {
-      return checkOffHeapEquals((OffHeapCachedDeserializable)v2, v1);
+    } else if (v1 instanceof StoredObject) {
+      return checkOffHeapEquals((StoredObject)v1, v2);
+    } else if (v2 instanceof StoredObject) {
+      return checkOffHeapEquals((StoredObject)v2, v1);
     } else if (v1 instanceof CachedDeserializable) {
       return checkCDEquals((CachedDeserializable)v1, v2, isCompressedOffHeap);
     } else if (v2 instanceof CachedDeserializable) {
@@ -1001,35 +997,29 @@ public abstract class AbstractRegionEntry implements RegionEntry,
       return basicEquals(v1, v2);
     }
   }
-  private static boolean checkOffHeapEquals(@Unretained OffHeapCachedDeserializable cd, @Unretained Object obj) {
-    if (cd.isSerializedPdxInstance()) {
-      PdxInstance pi = InternalDataSerializer.readPdxInstance(cd.getSerializedValue(), GemFireCacheImpl.getForPdx("Could not check value equality"));
+  private static boolean checkOffHeapEquals(@Unretained StoredObject ohVal, @Unretained Object obj) {
+    if (ohVal.isSerializedPdxInstance()) {
+      PdxInstance pi = InternalDataSerializer.readPdxInstance(ohVal.getSerializedValue(), GemFireCacheImpl.getForPdx("Could not check value equality"));
       return checkPdxEquals(pi, obj);
     }
-    if (obj instanceof OffHeapCachedDeserializable) {
-      return cd.checkDataEquals((OffHeapCachedDeserializable)obj);
+    if (obj instanceof StoredObject) {
+      return ohVal.checkDataEquals((StoredObject)obj);
     } else {
       byte[] serializedObj;
       if (obj instanceof CachedDeserializable) {
-        if (!cd.isSerialized()) {
-          if (obj instanceof StoredObject && !((StoredObject) obj).isSerialized()) {
-            // both are byte[]
-            // obj must be DataAsAddress since it was not OffHeapCachedDeserializable
-            // so its byte[] will be small.
-            byte[] objBytes = (byte[]) ((StoredObject) obj).getDeserializedForReading();
-            return cd.checkDataEquals(objBytes);
-          } else {
-            return false;
-          }
+        CachedDeserializable cdObj = (CachedDeserializable) obj;
+        if (!ohVal.isSerialized()) {
+          assert cdObj.isSerialized();
+          return false;
         }
-        serializedObj = ((CachedDeserializable) obj).getSerializedValue();
+        serializedObj = cdObj.getSerializedValue();
       } else if (obj instanceof byte[]) {
-        if (cd.isSerialized()) {
+        if (ohVal.isSerialized()) {
           return false;
         }
         serializedObj = (byte[]) obj;
       } else {
-        if (!cd.isSerialized()) {
+        if (!ohVal.isSerialized()) {
           return false;
         }
         if (obj == null || obj == Token.NOT_AVAILABLE
@@ -1038,19 +1028,20 @@ public abstract class AbstractRegionEntry implements RegionEntry,
         }
         serializedObj = EntryEventImpl.serialize(obj);
       }
-      return cd.checkDataEquals(serializedObj);
+      return ohVal.checkDataEquals(serializedObj);
     }
   }
   
   private static boolean checkCDEquals(CachedDeserializable cd, Object obj, boolean isCompressedOffHeap) {
-    if (cd instanceof StoredObject && !((StoredObject) cd).isSerialized()) {
+    if (!cd.isSerialized()) {
       // cd is an actual byte[].
       byte[] ba2;
-      if (obj instanceof StoredObject) {
-        if (!((StoredObject) obj).isSerialized()) {
+      if (obj instanceof CachedDeserializable) {
+        CachedDeserializable cdObj = (CachedDeserializable) obj;
+        if (!cdObj.isSerialized()) {
           return false;
         }
-        ba2 = (byte[]) ((StoredObject) obj).getDeserializedForReading();
+        ba2 = (byte[]) cdObj.getDeserializedForReading();
       } else if (obj instanceof byte[]) {
         ba2 = (byte[]) obj;
       } else {
@@ -1128,11 +1119,12 @@ public abstract class AbstractRegionEntry implements RegionEntry,
     if (!(obj instanceof PdxInstance)) {
       // obj may be a CachedDeserializable in which case we want to convert it to a PdxInstance even if we are not readSerialized.
       if (obj instanceof CachedDeserializable) {
-        if (obj instanceof StoredObject && !((StoredObject) obj).isSerialized()) {
+        CachedDeserializable cdObj = (CachedDeserializable) obj;
+        if (!cdObj.isSerialized()) {
           // obj is actually a byte[] which will never be equal to a PdxInstance
           return false;
         }
-        Object cdVal = ((CachedDeserializable) obj).getValue();
+        Object cdVal = cdObj.getValue();
         if (cdVal instanceof byte[]) {
           byte[] cdValBytes = (byte[]) cdVal;
           PdxInstance pi = InternalDataSerializer.readPdxInstance(cdValBytes, GemFireCacheImpl.getForPdx("Could not check value equality"));
@@ -1311,9 +1303,9 @@ public abstract class AbstractRegionEntry implements RegionEntry,
           }
           return prepareValueForCache(r, heapValue, event, isEntryUpdate);
         }
-        if (val instanceof ObjectChunk) {
+        if (soVal.hasRefCount()) {
           // if the reused guy has a refcount then need to inc it
-          if (!((ObjectChunk)val).retain()) {
+          if (!soVal.retain()) {
             throw new IllegalStateException("Could not use an off heap value because it was freed");
           }
         }
@@ -1346,16 +1338,8 @@ public abstract class AbstractRegionEntry implements RegionEntry,
         boolean isCompressed = compressedData != data;
         ReferenceCountHelper.setReferenceCountOwner(this);
         MemoryAllocator ma = SimpleMemoryAllocatorImpl.getAllocator(); // fix for bug 47875
-        val = ma.allocateAndInitialize(compressedData, isSerialized, isCompressed); // TODO:KIRK:48068 race happens right after this line
+        val = ma.allocateAndInitialize(compressedData, isSerialized, isCompressed, data);
         ReferenceCountHelper.setReferenceCountOwner(null);
-        if (val instanceof ObjectChunk) {
-          val = new ObjectChunkWithHeapForm((ObjectChunk)val, data);
-        }
-//        if (val instanceof Chunk && r instanceof LocalRegion) {
-//          Chunk c = (Chunk) val;
-//          LocalRegion lr = (LocalRegion) r;
-//          SimpleMemoryAllocatorImpl.debugLog("allocated @" + Long.toHexString(c.getMemoryAddress()) + " reg=" + lr.getFullPath(), false);
-//        }
       }
       return val;
     }
@@ -1365,8 +1349,7 @@ public abstract class AbstractRegionEntry implements RegionEntry,
       byte[] data = ((StoredObject) nv).getSerializedValue();
       nv = CachedDeserializableFactory.create(data);
     }
-    // don't bother checking for SQLFire
-    if (!GemFireCacheImpl.sqlfSystem() && nv instanceof PdxInstanceImpl) {
+    if (nv instanceof PdxInstanceImpl) {
       // We do not want to put PDXs in the cache as values.
       // So get the serialized bytes and use a CachedDeserializable.
       try {
@@ -1574,7 +1557,7 @@ public abstract class AbstractRegionEntry implements RegionEntry,
 
   protected StringBuilder appendFieldsToString(final StringBuilder sb) {
     sb.append("key=").append(getKey()).append("; rawValue=")
-        .append(_getValue()); // OFFHEAP _getValue ok: the current toString on OffHeapCachedDeserializable is safe to use without incing refcount.
+        .append(_getValue()); // OFFHEAP _getValue ok: the current toString on ObjectChunk is safe to use without incing refcount.
     VersionStamp stamp = getVersionStamp();
     if (stamp != null) {
       sb.append("; version=").append(stamp.asVersionTag()+";member="+stamp.getMemberID());

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/AbstractRegionMap.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/AbstractRegionMap.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/AbstractRegionMap.java
index 699de2f..9058984 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/AbstractRegionMap.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/AbstractRegionMap.java
@@ -75,7 +75,6 @@ import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 import com.gemstone.gemfire.internal.logging.LogService;
 import com.gemstone.gemfire.internal.logging.log4j.LocalizedMessage;
 import com.gemstone.gemfire.internal.logging.log4j.LogMarker;
-import com.gemstone.gemfire.internal.offheap.ObjectChunk;
 import com.gemstone.gemfire.internal.offheap.OffHeapHelper;
 import com.gemstone.gemfire.internal.offheap.OffHeapRegionEntryHelper;
 import com.gemstone.gemfire.internal.offheap.ReferenceCountHelper;
@@ -832,6 +831,7 @@ public abstract class AbstractRegionMap implements RegionMap {
                                        boolean deferLRUCallback,
                                        VersionTag entryVersion, InternalDistributedMember sender, boolean isSynchronizing)
   {
+    assert indexUpdater == null : "indexUpdater should only exist if sqlfire";
     boolean result = false;
     boolean done = false;
     boolean cleared = false;
@@ -922,73 +922,7 @@ public abstract class AbstractRegionMap implements RegionMap {
                 }
                 final boolean oldIsTombstone = oldRe.isTombstone();
                 final int oldSize = owner.calculateRegionEntryValueSize(oldRe);
-                // Neeraj: The below if block is to handle the special
-                // scenario witnessed in SqlFabric for now. (Though its
-                // a general scenario). The scenario is that during GII
-                // it is possible that updates start coming before the
-                // base value reaches through GII. In that scenario the deltas
-                // for that particular key is kept on being added to a list
-                // of deltas. When the base value arrives through this path
-                // of GII the oldValue will be that list of deltas. When the
-                // base values arrives the deltas are applied one by one on that list.
-                // The same scenario is applicable for GemFire also but the below 
-                // code will be executed only in case of sqlfabric now. Probably
-                // the code can be made more generic for both SQL Fabric and GemFire.
-                if (indexUpdater != null) {
-                  oldValue = oldRe.getValueInVM(owner); // OFFHEAP: ListOfDeltas
-                  if (oldValue instanceof ListOfDeltas) {
-                  // apply the deltas on this new value. update index
-                  // Make a new event object
-                  // make it an insert operation
-                  LocalRegion rgn = owner;
-                  if (owner instanceof BucketRegion) {
-                    rgn = ((BucketRegion)owner).getPartitionedRegion();
-                  }
-                  event = EntryEventImpl.create(rgn, Operation.CREATE, key, null,
-                      Boolean.TRUE /* indicate that GII is in progress */,
-                      false, null);
-                  try {
-                  event.setOldValue(newValue);
-                  if (logger.isDebugEnabled()) {
-                    logger.debug("initialImagePut: received base value for list of deltas; event: {}", event);
-                  }
-                  ((ListOfDeltas)oldValue).apply(event);
-                  Object preparedNewValue =oldRe.prepareValueForCache(owner,
-                      event.getNewValueAsOffHeapDeserializedOrRaw(), true);
-                  if(preparedNewValue instanceof ObjectChunk) {
-                    event.setNewValue(preparedNewValue);
-                  }
-                  oldRe.setValue(owner, preparedNewValue, event);
-                  //event.setNewValue(event.getOldValue());
-                  event.setOldValue(null);
-                  try {
-                    indexUpdater.onEvent(owner, event, oldRe);
-                    lruEntryUpdate(oldRe);
-                    owner.updateSizeOnPut(key, oldSize, owner.calculateRegionEntryValueSize(oldRe));
-                    EntryLogger.logInitialImagePut(_getOwnerObject(), key, newValue);
-                    result = true;
-                    done = true;
-                    break;
-                  } finally {
-                    // this must be done within the oldRe sync block
-                    indexUpdater.postEvent(owner, event, oldRe, done);
-                  }
-                  } finally {
-                    if (event != null) {
-                      event.release();
-                      event = null;
-                    }
-                  }
-                  }
-                }
                 try {
-                  if (indexUpdater != null) {
-                    event = EntryEventImpl.create(owner, Operation.CREATE, key,
-                        newValue,
-                        Boolean.TRUE /* indicate that GII is in progress */,
-                        false, null);
-                    indexUpdater.onEvent(owner, event, oldRe);
-                  }
                   result = oldRe.initialImagePut(owner, lastModified, newValue, wasRecovered, acceptedVersionTag);
                   if (result) {
                     if (oldIsTombstone) {
@@ -1025,9 +959,6 @@ public abstract class AbstractRegionMap implements RegionMap {
                   }
                   done = true;
                 } finally {
-                  if (indexUpdater != null) {
-                    indexUpdater.postEvent(owner, event, oldRe, result);
-                  }
                   if (event != null) {
                     event.release();
                     event = null;
@@ -1052,13 +983,6 @@ public abstract class AbstractRegionMap implements RegionMap {
                 true, wasRecovered, versionTagAccepted);
             try {
               if (result) {
-                if (indexUpdater != null) {
-                  event = EntryEventImpl.create(owner, Operation.CREATE, key,
-                      newValue,
-                      Boolean.TRUE /* indicate that GII is in progress */,
-                      false, null);
-                  indexUpdater.onEvent(owner, event, newRe);
-                }
                 if (newValue == Token.TOMBSTONE) {
                   owner.scheduleTombstone(newRe, entryVersion);
                 } else {
@@ -1075,9 +999,6 @@ public abstract class AbstractRegionMap implements RegionMap {
               }
               done = true;
             } finally {
-              if (result && indexUpdater != null) {
-                indexUpdater.postEvent(owner, event, newRe, done);
-              }
               if (event != null) {
                 event.release();
                 event = null;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/BucketRegion.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/BucketRegion.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/BucketRegion.java
index 5b17040..413fc87 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/BucketRegion.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/BucketRegion.java
@@ -1829,11 +1829,12 @@ implements Bucket
       if (!(rawNewValue instanceof CachedDeserializable)) {
         return;
       }
-      if (rawNewValue instanceof StoredObject && !((StoredObject) rawNewValue).isSerialized()) {
+      CachedDeserializable cd = (CachedDeserializable) rawNewValue;
+      if (!cd.isSerialized()) {
         // it is a byte[]; not a Delta
         return;
       }
-      Object instance = ((CachedDeserializable)rawNewValue).getValue();
+      Object instance = cd.getValue();
       if (instance instanceof com.gemstone.gemfire.Delta
           && ((com.gemstone.gemfire.Delta)instance).hasDelta()) {
         try {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/BytesAndBitsForCompactor.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/BytesAndBitsForCompactor.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/BytesAndBitsForCompactor.java
index cc358f5..2746e29 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/BytesAndBitsForCompactor.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/BytesAndBitsForCompactor.java
@@ -16,7 +16,7 @@
  */
 package com.gemstone.gemfire.internal.cache;
 
-import com.gemstone.gemfire.internal.offheap.ObjectChunk;
+import com.gemstone.gemfire.internal.offheap.StoredObject;
 import com.gemstone.gemfire.internal.offheap.annotations.Unretained;
 
 /**
@@ -32,11 +32,11 @@ import com.gemstone.gemfire.internal.offheap.annotations.Unretained;
  */
 public class BytesAndBitsForCompactor {
   /**
-   * If dataChunk is set then ignore the "data" and "validLength" fields.
-   * The dataChunk field is unretained so it can only be used while the RegionEntry is still synced.
-   * When done with the dataChunk, null it out if you want to reuse the byte[] later.
+   * If offHeapData is set then ignore the "data" and "validLength" fields.
+   * The offHeapData field is unretained so it can only be used while the RegionEntry is still synced.
+   * When done with the offHeapData, null it out if you want to reuse the byte[] later.
    */
-  private @Unretained ObjectChunk dataChunk;
+  private @Unretained StoredObject offHeapData;
   private  byte[] data;
   private  byte userBits=0;
   // length of the data present in the byte array 
@@ -56,8 +56,8 @@ public class BytesAndBitsForCompactor {
   }
 
   
-  public final ObjectChunk getDataChunk() {
-    return this.dataChunk;
+  public final StoredObject getOffHeapData() {
+    return this.offHeapData;
   }
   public final byte[] getBytes() {
     return this.data;
@@ -87,8 +87,8 @@ public class BytesAndBitsForCompactor {
     this.validLength = validLength;    
     this.isReusable = isReusable;
   }
-  public void setChunkData(ObjectChunk c, byte userBits) {
-    this.dataChunk = c;
+  public void setOffHeapData(StoredObject so, byte userBits) {
+    this.offHeapData = so;
     this.userBits = userBits;
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/CachedDeserializable.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/CachedDeserializable.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/CachedDeserializable.java
index 04fa515..9effbf6 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/CachedDeserializable.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/CachedDeserializable.java
@@ -111,4 +111,12 @@ public interface CachedDeserializable extends Sizeable
    * added by this wrapper class.
    */
   public int getValueSizeInBytes();
+  /**
+   * Returns true if the value stored in this memory chunk is a serialized object. Returns false if it is a byte array.
+   */
+  public boolean isSerialized();
+  /**
+   * Return true if the value uses the java heap; false if not.
+   */
+  public boolean usesHeapForStorage();
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/DiskEntry.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/DiskEntry.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/DiskEntry.java
index 327279b..26e49c9 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/DiskEntry.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/DiskEntry.java
@@ -28,7 +28,6 @@ import com.gemstone.gemfire.internal.Assert;
 import com.gemstone.gemfire.internal.ByteArrayDataInput;
 import com.gemstone.gemfire.internal.HeapDataOutputStream;
 import com.gemstone.gemfire.internal.Version;
-import com.gemstone.gemfire.internal.cache.DiskEntry.Helper.ValueWrapper;
 import com.gemstone.gemfire.internal.cache.DiskStoreImpl.AsyncDiskEntry;
 import com.gemstone.gemfire.internal.cache.lru.EnableLRU;
 import com.gemstone.gemfire.internal.cache.lru.LRUClockNode;
@@ -40,11 +39,9 @@ import com.gemstone.gemfire.internal.cache.versions.VersionStamp;
 import com.gemstone.gemfire.internal.cache.versions.VersionTag;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 import com.gemstone.gemfire.internal.logging.LogService;
-import com.gemstone.gemfire.internal.offheap.ObjectChunk;
+import com.gemstone.gemfire.internal.offheap.AddressableMemoryManager;
 import com.gemstone.gemfire.internal.offheap.OffHeapHelper;
 import com.gemstone.gemfire.internal.offheap.ReferenceCountHelper;
-import com.gemstone.gemfire.internal.offheap.Releasable;
-import com.gemstone.gemfire.internal.offheap.UnsafeMemoryChunk;
 import com.gemstone.gemfire.internal.offheap.StoredObject;
 import com.gemstone.gemfire.internal.offheap.annotations.Released;
 import com.gemstone.gemfire.internal.offheap.annotations.Retained;
@@ -200,7 +197,7 @@ public interface DiskEntry extends RegionEntry {
     /**
      * Get the value of an entry that is on disk without
      * faulting it in . It checks for the presence in the buffer also.
-     * This method is used for concurrent map operations, SQLFabric and CQ processing
+     * This method is used for concurrent map operations and CQ processing
      * 
      * @throws DiskAccessException
      * @since 5.1
@@ -208,18 +205,11 @@ public interface DiskEntry extends RegionEntry {
     static Object getValueOnDiskOrBuffer(DiskEntry entry, DiskRegion dr, RegionEntryContext context) {
       @Released Object v = getOffHeapValueOnDiskOrBuffer(entry, dr, context);
       if (v instanceof CachedDeserializable) {
-        if (v instanceof ObjectChunk) {
-          @Released ObjectChunk ohv = (ObjectChunk) v;
-          try {
-            v = ohv.getDeserializedValue(null, null);
-            if (v == ohv) {
-              throw new IllegalStateException("sqlf tried to use getValueOnDiskOrBuffer");
-            }
-          } finally {
-            ohv.release(); // OFFHEAP the offheap ref is decremented here
-          }
-        } else {
-          v = ((CachedDeserializable)v).getDeserializedValue(null, null);
+        CachedDeserializable cd = (CachedDeserializable) v;
+        try {
+          v = cd.getDeserializedValue(null, null);
+        } finally {
+          OffHeapHelper.release(cd); // If v was off-heap it is released here
         }
       }
       return v;
@@ -391,10 +381,11 @@ public interface DiskEntry extends RegionEntry {
         // fix for bug 31757
         return false;
       } else if (v instanceof CachedDeserializable) {
+        CachedDeserializable cd = (CachedDeserializable) v;
         try {
-          if (v instanceof StoredObject && !((StoredObject) v).isSerialized()) {
+          if (!cd.isSerialized()) {
             entry.setSerialized(false);
-            entry.value = ((StoredObject) v).getDeserializedForReading();
+            entry.value = cd.getDeserializedForReading();
             
             //For SQLFire we prefer eager deserialized
 //            if(v instanceof ByteSource) {
@@ -403,7 +394,7 @@ public interface DiskEntry extends RegionEntry {
           } else {
             // don't serialize here if it is not already serialized
             
-            Object tmp = ((CachedDeserializable)v).getValue();
+            Object tmp = cd.getValue();
           //For SQLFire we prefer eager deserialized
 //            if(v instanceof ByteSource) {
 //              entry.setEagerDeserialize();
@@ -679,27 +670,28 @@ public interface DiskEntry extends RegionEntry {
     }
     
     /**
-     * Note that the Chunk this ValueWrapper is created with
+     * Note that the StoredObject this ValueWrapper is created with
      * is unretained so it must be used before the owner of
-     * the chunk releases it.
+     * the StoredObject releases it.
      * Since the RegionEntry that has the value we are writing to
      * disk has it retained we are ok as long as this ValueWrapper's
      * life ends before the RegionEntry sync is released.
-     * Note that this class is only used with uncompressed chunks.
+     * Note that this class is only used with uncompressed StoredObjects.
      */
-    public static class ChunkValueWrapper implements ValueWrapper {
-      private final @Unretained ObjectChunk chunk;
-      public ChunkValueWrapper(ObjectChunk c) {
-        assert !c.isCompressed();
-        this.chunk = c;
+    public static class OffHeapValueWrapper implements ValueWrapper {
+      private final @Unretained StoredObject offHeapData;
+      public OffHeapValueWrapper(StoredObject so) {
+        assert so.hasRefCount();
+        assert !so.isCompressed();
+        this.offHeapData = so;
       }
       @Override
       public boolean isSerialized() {
-        return this.chunk.isSerialized();
+        return this.offHeapData.isSerialized();
       }
       @Override
       public int getLength() {
-        return this.chunk.getDataSize();
+        return this.offHeapData.getDataSize();
       }
       @Override
       public byte getUserBits() {
@@ -716,21 +708,21 @@ public interface DiskEntry extends RegionEntry {
           return;
         }
         if (maxOffset > bb.capacity()) {
-          ByteBuffer chunkbb = this.chunk.createDirectByteBuffer();
+          ByteBuffer chunkbb = this.offHeapData.createDirectByteBuffer();
           if (chunkbb != null) {
             flushable.flush(bb, chunkbb);
             return;
           }
         }
-        final long bbAddress = ObjectChunk.getDirectByteBufferAddress(bb);
+        final long bbAddress = AddressableMemoryManager.getDirectByteBufferAddress(bb);
         if (bbAddress != 0L) {
           int bytesRemaining = maxOffset;
           int availableSpace = bb.remaining();
           long addrToWrite = bbAddress + bb.position();
-          long addrToRead = this.chunk.getAddressForReading(0, maxOffset);
+          long addrToRead = this.offHeapData.getAddressForReadingData(0, maxOffset);
           if (bytesRemaining > availableSpace) {
             do {
-              UnsafeMemoryChunk.copyMemory(addrToRead, addrToWrite, availableSpace);
+              AddressableMemoryManager.copyMemory(addrToRead, addrToWrite, availableSpace);
               bb.position(bb.position()+availableSpace);
               addrToRead += availableSpace;
               bytesRemaining -= availableSpace;
@@ -739,13 +731,13 @@ public interface DiskEntry extends RegionEntry {
               availableSpace = bb.remaining();
             } while (bytesRemaining > availableSpace);
           }
-          UnsafeMemoryChunk.copyMemory(addrToRead, addrToWrite, bytesRemaining);
+          AddressableMemoryManager.copyMemory(addrToRead, addrToWrite, bytesRemaining);
           bb.position(bb.position()+bytesRemaining);
         } else {
-          long addr = this.chunk.getAddressForReading(0, maxOffset);
+          long addr = this.offHeapData.getAddressForReadingData(0, maxOffset);
           final long endAddr = addr + maxOffset;
           while (addr != endAddr) {
-            bb.put(UnsafeMemoryChunk.readAbsoluteByte(addr));
+            bb.put(AddressableMemoryManager.readByte(addr));
             addr++;
             if (!bb.hasRemaining()) {
               flushable.flush();
@@ -755,10 +747,28 @@ public interface DiskEntry extends RegionEntry {
       }
       @Override
       public String getBytesAsString() {
-        return this.chunk.getStringForm();
+        return this.offHeapData.getStringForm();
       }
     }
 
+    /**
+     * Returns true if the given object is off-heap
+     * and it is worth wrapping a reference to it
+     * instead of copying its data to the heap.
+     * Currently all StoredObject's with a refCount are
+     * wrapped.
+     */
+    public static boolean wrapOffHeapReference(Object o) {
+      if (o instanceof StoredObject) {
+        StoredObject so = (StoredObject) o;
+        if (so.hasRefCount()) {
+          // 
+          return true;
+        }
+      }
+      return false;
+    }
+    
     public static ValueWrapper createValueWrapper(Object value, EntryEventImpl event) {
       if (value == Token.INVALID) {
         // even though it is not serialized we say it is because
@@ -782,19 +792,14 @@ public interface DiskEntry extends RegionEntry {
         byte[] bytes;
         if (value instanceof CachedDeserializable) {
           CachedDeserializable proxy = (CachedDeserializable)value;
-          if (proxy instanceof ObjectChunk) {
-            return new ChunkValueWrapper((ObjectChunk) proxy);
+          if (wrapOffHeapReference(proxy)) {
+            return new OffHeapValueWrapper((StoredObject) proxy);
           }
-          if (proxy instanceof StoredObject) {
-            StoredObject ohproxy = (StoredObject) proxy;
-            isSerializedObject = ohproxy.isSerialized();
-            if (isSerializedObject) {
-              bytes = ohproxy.getSerializedValue();
-            } else {
-              bytes = (byte[]) ohproxy.getDeserializedForReading();
-            }
-          } else {
+          isSerializedObject = proxy.isSerialized();
+          if (isSerializedObject) {
             bytes = proxy.getSerializedValue();
+          } else {
+            bytes = (byte[]) proxy.getDeserializedForReading();
           }
           if (event != null && isSerializedObject) {
             event.setCachedSerializedNewValue(bytes);
@@ -826,15 +831,15 @@ public interface DiskEntry extends RegionEntry {
         // For off-heap it should be faster to pass a reference to the
         // StoredObject instead of using the cached byte[] (unless it is also compressed).
         // Since NIO is used if the chunk of memory is large we can write it
-        // to the file with using the off-heap memory with no extra copying.
+        // to the file using the off-heap memory with no extra copying.
         // So we give preference to getRawNewValue over getCachedSerializedNewValue
         Object rawValue = null;
         if (!event.hasDelta()) {
           // We don't do this for the delta case because getRawNewValue returns delta
           // and we want to write the entire new value to disk.
           rawValue = event.getRawNewValue();
-          if (rawValue instanceof ObjectChunk) {
-            return new ChunkValueWrapper((ObjectChunk) rawValue);
+          if (wrapOffHeapReference(rawValue)) {
+            return new OffHeapValueWrapper((StoredObject) rawValue);
           }
         }
         if (event.getCachedSerializedNewValue() != null) {
@@ -1161,10 +1166,6 @@ public interface DiskEntry extends RegionEntry {
       if (result instanceof CachedDeserializable) {
         result = ((CachedDeserializable)result).getDeserializedValue(null, null);
       }
-      if (result instanceof StoredObject) {
-        ((StoredObject) result).release();
-        throw new IllegalStateException("sqlf tried to use getValueInVMOrDiskWithoutFaultIn");
-      }
       return result;
     }
     

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/DiskStoreImpl.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/DiskStoreImpl.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/DiskStoreImpl.java
index e4ef21d..e813058 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/DiskStoreImpl.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/DiskStoreImpl.java
@@ -110,10 +110,6 @@ import com.gemstone.gemfire.internal.logging.LogService;
 import com.gemstone.gemfire.internal.logging.LoggingThreadGroup;
 import com.gemstone.gemfire.internal.logging.log4j.LocalizedMessage;
 import com.gemstone.gemfire.internal.logging.log4j.LogMarker;
-import com.gemstone.gemfire.internal.offheap.MemoryChunkWithRefCount;
-import com.gemstone.gemfire.internal.offheap.OffHeapHelper;
-import com.gemstone.gemfire.internal.offheap.annotations.Released;
-import com.gemstone.gemfire.internal.offheap.annotations.Retained;
 import com.gemstone.gemfire.internal.util.BlobHelper;
 import com.gemstone.gemfire.pdx.internal.EnumInfo;
 import com.gemstone.gemfire.pdx.internal.PdxField;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/DistributedRegion.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/DistributedRegion.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/DistributedRegion.java
index 19496da..111a6d5 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/DistributedRegion.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/DistributedRegion.java
@@ -121,9 +121,6 @@ import com.gemstone.gemfire.internal.cache.wan.parallel.ConcurrentParallelGatewa
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 import com.gemstone.gemfire.internal.logging.LogService;
 import com.gemstone.gemfire.internal.logging.log4j.LocalizedMessage;
-import com.gemstone.gemfire.internal.offheap.ObjectChunk;
-import com.gemstone.gemfire.internal.offheap.OffHeapHelper;
-import com.gemstone.gemfire.internal.offheap.annotations.Released;
 import com.gemstone.gemfire.internal.offheap.annotations.Retained;
 import com.gemstone.gemfire.internal.sequencelog.RegionLogger;
 import com.gemstone.gemfire.internal.util.concurrent.StoppableCountDownLatch;
@@ -2449,7 +2446,6 @@ public class DistributedRegion extends LocalRegion implements
     boolean fromServer = false;
     EntryEventImpl event = null;
     @Retained Object result = null;
-    boolean incrementUseCountForSqlf = false;
     try {
     {
       if (this.srp != null) {
@@ -2515,7 +2511,6 @@ public class DistributedRegion extends LocalRegion implements
           	((BucketRegion)this).handleWANEvent(event);
           }
           re = basicPutEntry(event, lastModified);
-          incrementUseCountForSqlf = GemFireCacheImpl.sqlfSystem() ;
         } catch (ConcurrentCacheModificationException e) {
           // the cache was modified while we were searching for this entry and
           // the netsearch result was elided.  Return the current value from the cache
@@ -2547,11 +2542,6 @@ public class DistributedRegion extends LocalRegion implements
     } else {
       result = event.getNewValue();     
     }
-    //For SQLFire , we need to increment the use count so that returned
-    //object has use count 2
-    if( incrementUseCountForSqlf && result instanceof ObjectChunk) {
-      ((ObjectChunk)result).retain();
-    }
     return result;
     } finally {
       if (event != null) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/EntryEventImpl.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/EntryEventImpl.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/EntryEventImpl.java
index 6c58790..dfd20ef 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/EntryEventImpl.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/EntryEventImpl.java
@@ -73,7 +73,6 @@ import com.gemstone.gemfire.internal.lang.StringUtils;
 import com.gemstone.gemfire.internal.logging.LogService;
 import com.gemstone.gemfire.internal.logging.log4j.LocalizedMessage;
 import com.gemstone.gemfire.internal.logging.log4j.LogMarker;
-import com.gemstone.gemfire.internal.offheap.ObjectChunk;
 import com.gemstone.gemfire.internal.offheap.OffHeapHelper;
 import com.gemstone.gemfire.internal.offheap.OffHeapRegionEntryHelper;
 import com.gemstone.gemfire.internal.offheap.ReferenceCountHelper;
@@ -834,10 +833,7 @@ public class EntryEventImpl
       }
       boolean doCopyOnRead = getRegion().isCopyOnRead();
       if (ov != null) {
-        if (ov instanceof StoredObject) {
-          // TODO OFFHEAP: returns off-heap PdxInstance
-          return ((StoredObject) ov).getValueAsDeserializedHeapObject();
-        } else
+        // TODO OFFHEAP: returns off-heap PdxInstance
         if (ov instanceof CachedDeserializable) {
           CachedDeserializable cd = (CachedDeserializable)ov;
           if (doCopyOnRead) {
@@ -930,9 +926,9 @@ public class EntryEventImpl
     if (this.offHeapOk) {
       OffHeapHelper.releaseAndTrackOwner(this.newValue, this);
     }
-    if (v instanceof ObjectChunk) {
+    if (isOffHeapReference(v)) {
       ReferenceCountHelper.setReferenceCountOwner(this);
-      if (!((ObjectChunk) v).retain()) {
+      if (!((StoredObject) v).retain()) {
         ReferenceCountHelper.setReferenceCountOwner(null);
         this.newValue = null;
         return;
@@ -946,19 +942,23 @@ public class EntryEventImpl
    * Returns true if this event has a reference to an off-heap new or old value.
    */
   public boolean hasOffHeapValue() {
-    return (this.newValue instanceof ObjectChunk) || (this.oldValue instanceof ObjectChunk);
+    return isOffHeapReference(this.newValue) || isOffHeapReference(this.oldValue);
   }
   
   @Unretained
   protected final Object basicGetNewValue() {
     Object result = this.newValue;
-    if (!this.offHeapOk && result instanceof ObjectChunk) {
+    if (!this.offHeapOk && isOffHeapReference(result)) {
       //this.region.getCache().getLogger().info("DEBUG new value already freed " + System.identityHashCode(result));
       throw new IllegalStateException("Attempt to access off heap value after the EntryEvent was released.");
     }
     return result;
   }
   
+  private static boolean isOffHeapReference(Object ref) {
+    return (ref instanceof StoredObject) && ((StoredObject)ref).hasRefCount();
+  }
+  
   private class OldValueOwner {
     private EntryEventImpl getEvent() {
       return EntryEventImpl.this;
@@ -992,12 +992,10 @@ public class EntryEventImpl
     @Released final Object curOldValue = this.oldValue;
     if (v == curOldValue) return;
     if (this.offHeapOk) {
-      if (curOldValue instanceof ObjectChunk) {
-        if (ReferenceCountHelper.trackReferenceCounts()) {
-          OffHeapHelper.releaseAndTrackOwner(curOldValue, new OldValueOwner());
-        } else {
-          OffHeapHelper.release(curOldValue);
-        }
+      if (ReferenceCountHelper.trackReferenceCounts()) {
+        OffHeapHelper.releaseAndTrackOwner(curOldValue, new OldValueOwner());
+      } else {
+        OffHeapHelper.release(curOldValue);
       }
     }
     
@@ -1008,17 +1006,18 @@ public class EntryEventImpl
   private void retainAndSetOldValue(@Retained(ENTRY_EVENT_OLD_VALUE) Object v) {
     if (v == this.oldValue) return;
     
-    if (v instanceof ObjectChunk) {
+    if (isOffHeapReference(v)) {
+      StoredObject so = (StoredObject) v;
       if (ReferenceCountHelper.trackReferenceCounts()) {
         ReferenceCountHelper.setReferenceCountOwner(new OldValueOwner());
-        boolean couldNotRetain = (!((ObjectChunk) v).retain());
+        boolean couldNotRetain = (!so.retain());
         ReferenceCountHelper.setReferenceCountOwner(null);
         if (couldNotRetain) {
           this.oldValue = null;
           return;
         }
       } else {
-        if (!((ObjectChunk) v).retain()) {
+        if (!so.retain()) {
           this.oldValue = null;
           return;
         }
@@ -1031,7 +1030,7 @@ public class EntryEventImpl
   private Object basicGetOldValue() {
     @Unretained(ENTRY_EVENT_OLD_VALUE)
     Object result = this.oldValue;
-    if (!this.offHeapOk && result instanceof ObjectChunk) {
+    if (!this.offHeapOk && isOffHeapReference(result)) {
       //this.region.getCache().getLogger().info("DEBUG old value already freed " + System.identityHashCode(result));
       throw new IllegalStateException("Attempt to access off heap value after the EntryEvent was released.");
     }
@@ -1056,7 +1055,6 @@ public class EntryEventImpl
   }
   /**
    * Just like getRawOldValue except if the raw old value is off-heap deserialize it.
-   * Note that in some cases sqlf ignores the request to deserialize.
    */
   @Unretained(ENTRY_EVENT_OLD_VALUE)
   public final Object getOldValueAsOffHeapDeserializedOrRaw() {
@@ -1107,11 +1105,8 @@ public class EntryEventImpl
         // I'm not sure this can even happen
         return AbstractRegion.handleNotAvailable(nv);
       }
-      if (nv instanceof StoredObject) {
-        // TODO OFFHEAP currently we copy offheap new value to the heap here. Check callers of this method to see if they can be optimized to use offheap values.
-        // TODO OFFHEAP: returns off-heap PdxInstance
-        return ((StoredObject) nv).getValueAsDeserializedHeapObject();
-      } else
+      // TODO OFFHEAP currently we copy offheap new value to the heap here. Check callers of this method to see if they can be optimized to use offheap values.
+      // TODO OFFHEAP: returns off-heap PdxInstance
       if (nv instanceof CachedDeserializable) {
         CachedDeserializable cd = (CachedDeserializable)nv;
         Object v = null;
@@ -1284,18 +1279,16 @@ public class EntryEventImpl
     @Unretained(ENTRY_EVENT_NEW_VALUE)
     final Object tmp = basicGetNewValue();
     if (tmp instanceof CachedDeserializable) {
-      if (tmp instanceof StoredObject) {
-        if (!((StoredObject) tmp).isSerialized()) {
-          // TODO OFFHEAP can we handle offheap byte[] better?
-          return null;
-        }
+      CachedDeserializable cd = (CachedDeserializable) tmp;
+      if (!cd.isSerialized()) {
+        // TODO OFFHEAP can we handle offheap byte[] better?
+        return null;
       }
       byte[] bytes = this.newValueBytes;
       if (bytes == null) {
         bytes = this.cachedSerializedNewValue;
       }
-      return new SerializedCacheValueImpl(this, getRegion(), this.re,
-          (CachedDeserializable)tmp, bytes);
+      return new SerializedCacheValueImpl(this, getRegion(), this.re, cd, bytes);
     } else {
       // Note we return null even if cachedSerializedNewValue is not null.
       // This is because some callers of this method use it to indicate
@@ -1360,7 +1353,7 @@ public class EntryEventImpl
       @Unretained(ENTRY_EVENT_NEW_VALUE)
       final StoredObject so = (StoredObject) nv;
       final boolean isSerialized = so.isSerialized();
-      if (nv instanceof ObjectChunk) {
+      if (so.hasRefCount()) {
         if (importer.isUnretainedNewReferenceOk()) {
           importer.importNewObject(nv, isSerialized);
         } else {
@@ -1447,7 +1440,7 @@ public class EntryEventImpl
     if (ov instanceof StoredObject) {
       final StoredObject so = (StoredObject) ov;
       final boolean isSerialized = so.isSerialized();
-      if (ov instanceof ObjectChunk) {
+      if (so.hasRefCount()) {
         if (importer.isUnretainedOldReferenceOk()) {
           importer.importOldObject(ov, isSerialized);
         } else {
@@ -1501,7 +1494,6 @@ public class EntryEventImpl
   }
   /**
    * Just like getRawNewValue(true) except if the raw new value is off-heap deserialize it.
-   * Note that in some cases sqlf ignores the request to deserialize.
    */
   @Unretained(ENTRY_EVENT_NEW_VALUE)
   public final Object getNewValueAsOffHeapDeserializedOrRaw() {
@@ -1518,16 +1510,7 @@ public class EntryEventImpl
    */
   @Retained(ENTRY_EVENT_NEW_VALUE)
   public StoredObject getOffHeapNewValue() {
-    final Object tmp = basicGetNewValue();
-    if (tmp instanceof StoredObject) {
-      StoredObject result = (StoredObject) tmp;
-      if (!result.retain()) {
-        return null;
-      }
-      return result;
-    } else {
-      return null;
-    }
+    return convertToStoredObject(basicGetNewValue());
   }
   
   /**
@@ -1536,28 +1519,23 @@ public class EntryEventImpl
    */
   @Retained(ENTRY_EVENT_OLD_VALUE)
   public StoredObject getOffHeapOldValue() {
-    final Object tmp = basicGetOldValue();
-    if (tmp instanceof StoredObject) {
-      StoredObject result = (StoredObject) tmp;
-      if (!result.retain()) {
-        return null;
-      }
-      return result;
-    } else {
+    return convertToStoredObject(basicGetOldValue());
+  }
+
+  private static StoredObject convertToStoredObject(final Object tmp) {
+    if (!(tmp instanceof StoredObject)) {
+      return null;
+    }
+    StoredObject result = (StoredObject) tmp;
+    if (!result.retain()) {
       return null;
     }
+    return result;
   }
-
-  /**
-   * Result may be unretained because sqlf getDeserializedForReading returns unretained.
-   */
+  
   public final Object getDeserializedValue() {
     if (this.delta == null) {
       final Object val = basicGetNewValue();
-      if (val instanceof StoredObject) {
-        // TODO OFFHEAP: returns off-heap PdxInstance
-        return ((StoredObject) val).getValueAsDeserializedHeapObject();
-      } else 
       if (val instanceof CachedDeserializable) {
         return ((CachedDeserializable)val).getDeserializedForReading();
       }
@@ -1869,8 +1847,8 @@ public class EntryEventImpl
     Object preparedV = reentry.prepareValueForCache(this.region, v, this, this.hasDelta());
     if (preparedV != v) {
       v = preparedV;
-      if (v instanceof ObjectChunk) {
-        if (!((ObjectChunk) v).isCompressed()) { // fix bug 52109
+      if (v instanceof StoredObject) {
+        if (!((StoredObject) v).isCompressed()) { // fix bug 52109
           // If we put it off heap and it is not compressed then remember that value.
           // Otherwise we want to remember the decompressed value in the event.
           basicSetNewValue(v);
@@ -1931,8 +1909,8 @@ public class EntryEventImpl
       success = true;
     }
     } finally {
-      if (!success && reentry instanceof OffHeapRegionEntry && v instanceof ObjectChunk) {
-        OffHeapRegionEntryHelper.releaseEntry((OffHeapRegionEntry)reentry, (ObjectChunk)v);
+      if (!success && reentry instanceof OffHeapRegionEntry && v instanceof StoredObject) {
+        OffHeapRegionEntryHelper.releaseEntry((OffHeapRegionEntry)reentry, (StoredObject)v);
       }      
     }
     if (logger.isTraceEnabled()) {
@@ -2230,9 +2208,9 @@ public class EntryEventImpl
 
   /**
    * If a PdxInstance is returned then it will have an unretained reference
-   * to Chunk's off-heap address.
+   * to the StoredObject's off-heap address.
    */
-  public static @Unretained Object deserializeChunk(ObjectChunk bytes) {
+  public static @Unretained Object deserializeOffHeap(StoredObject bytes) {
     if (bytes == null)
       return null;
     try {
@@ -2418,9 +2396,7 @@ public class EntryEventImpl
         Object nv = basicGetNewValue();
         boolean newValueSerialized = nv instanceof CachedDeserializable;
         if (newValueSerialized) {
-          if (nv instanceof StoredObject) {
-            newValueSerialized = ((StoredObject) nv).isSerialized();
-          }
+          newValueSerialized = ((CachedDeserializable) nv).isSerialized();
         }
         out.writeBoolean(newValueSerialized);
         if (newValueSerialized) {
@@ -2443,9 +2419,7 @@ public class EntryEventImpl
       Object ov = basicGetOldValue();
       boolean oldValueSerialized = ov instanceof CachedDeserializable;
       if (oldValueSerialized) {
-        if (ov instanceof StoredObject) {
-          oldValueSerialized = ((StoredObject) ov).isSerialized();
-        }
+        oldValueSerialized = ((CachedDeserializable) ov).isSerialized();
       }
       out.writeBoolean(oldValueSerialized);
       if (oldValueSerialized) {
@@ -2517,14 +2491,12 @@ public class EntryEventImpl
     @Unretained(ENTRY_EVENT_OLD_VALUE)
     final Object tmp = basicGetOldValue();
     if (tmp instanceof CachedDeserializable) {
-      if (tmp instanceof StoredObject) {
-        if (!((StoredObject) tmp).isSerialized()) {
-          // TODO OFFHEAP can we handle offheap byte[] better?
-          return null;
-        }
+      CachedDeserializable cd = (CachedDeserializable) tmp;
+      if (!cd.isSerialized()) {
+        // TODO OFFHEAP can we handle offheap byte[] better?
+        return null;
       }
-      return new SerializedCacheValueImpl(this, this.region, this.re,
-          (CachedDeserializable)tmp, this.oldValueBytes);
+      return new SerializedCacheValueImpl(this, this.region, this.re, cd, this.oldValueBytes);
     }
     else {
       return null;
@@ -2881,7 +2853,7 @@ public class EntryEventImpl
     private final byte[] serializedValue;
     
     SerializedCacheValueImpl(EntryEventImpl event, Region r, RegionEntry re, @Unretained CachedDeserializable cd, byte[] serializedBytes) {
-      if (cd instanceof ObjectChunk) {
+      if (isOffHeapReference(cd)) {
         this.event = event;
       } else {
         this.event = null;
@@ -2958,6 +2930,16 @@ public class EntryEventImpl
     public void sendTo(DataOutput out) throws IOException {
       DataSerializer.writeObject(getCd(), out);
     }
+
+    @Override
+    public boolean isSerialized() {
+      return getCd().isSerialized();
+    }
+
+    @Override
+    public boolean usesHeapForStorage() {
+      return getCd().usesHeapForStorage();
+    }
   }
 //////////////////////////////////////////////////////////////////////////////////////////
   
@@ -3060,14 +3042,14 @@ public class EntryEventImpl
     Object nv = basicGetNewValue();
     this.offHeapOk = false;
     
-    if (ov instanceof ObjectChunk) {
+    if (ov instanceof StoredObject) {
       //this.region.getCache().getLogger().info("DEBUG freeing ref to old value on " + System.identityHashCode(ov));
       if (ReferenceCountHelper.trackReferenceCounts()) {
         ReferenceCountHelper.setReferenceCountOwner(new OldValueOwner());
-        ((ObjectChunk) ov).release();
+        ((StoredObject) ov).release();
         ReferenceCountHelper.setReferenceCountOwner(null);
       } else {
-        ((ObjectChunk) ov).release();
+        ((StoredObject) ov).release();
       }
     }
     OffHeapHelper.releaseAndTrackOwner(nv, this);
@@ -3078,7 +3060,7 @@ public class EntryEventImpl
    * Once this is called on an event it does not need to have release called.
    */
   public void disallowOffHeapValues() {
-    if (this.newValue instanceof ObjectChunk || this.oldValue instanceof ObjectChunk) {
+    if (isOffHeapReference(this.newValue) || isOffHeapReference(this.oldValue)) {
       throw new IllegalStateException("This event does not support off-heap values");
     }
     this.offHeapOk = false;
@@ -3087,12 +3069,11 @@ public class EntryEventImpl
   /**
    * This copies the off-heap new and/or old value to the heap.
    * As a result the current off-heap new/old will be released.
-   * @throws IllegalStateException if called with an event for sqlf data.
    */
   @Released({ENTRY_EVENT_NEW_VALUE, ENTRY_EVENT_OLD_VALUE})
   public void copyOffHeapToHeap() {
     Object ov = basicGetOldValue();
-    if (ov instanceof ObjectChunk) {
+    if (isOffHeapReference(ov)) {
       if (ReferenceCountHelper.trackReferenceCounts()) {
         ReferenceCountHelper.setReferenceCountOwner(new OldValueOwner());
         this.oldValue = OffHeapHelper.copyAndReleaseIfNeeded(ov);
@@ -3102,19 +3083,19 @@ public class EntryEventImpl
       }
     }
     Object nv = basicGetNewValue();
-    if (nv instanceof ObjectChunk) {
+    if (isOffHeapReference(nv)) {
       ReferenceCountHelper.setReferenceCountOwner(this);
       this.newValue = OffHeapHelper.copyAndReleaseIfNeeded(nv);
       ReferenceCountHelper.setReferenceCountOwner(null);
     }
-    if (this.newValue instanceof ObjectChunk || this.oldValue instanceof ObjectChunk) {
+    if (isOffHeapReference(this.newValue) || isOffHeapReference(this.oldValue)) {
       throw new IllegalStateException("event's old/new value still off-heap after calling copyOffHeapToHeap");
     }
     this.offHeapOk = false;
   }
 
   public boolean isOldValueOffHeap() {
-    return this.oldValue instanceof ObjectChunk;
+    return isOffHeapReference(this.oldValue);
   }
   public final boolean isFetchFromHDFS() {
     return fetchFromHDFS;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/LocalRegion.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/LocalRegion.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/LocalRegion.java
index b6d8c49..966130a 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/LocalRegion.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/LocalRegion.java
@@ -203,7 +203,6 @@ import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 import com.gemstone.gemfire.internal.logging.LogService;
 import com.gemstone.gemfire.internal.logging.log4j.LocalizedMessage;
 import com.gemstone.gemfire.internal.logging.log4j.LogMarker;
-import com.gemstone.gemfire.internal.offheap.ObjectChunk;
 import com.gemstone.gemfire.internal.offheap.OffHeapHelper;
 import com.gemstone.gemfire.internal.offheap.ReferenceCountHelper;
 import com.gemstone.gemfire.internal.offheap.StoredObject;
@@ -1423,6 +1422,7 @@ public class LocalRegion extends AbstractRegion
   public Object getRetained(Object key, Object aCallbackArgument,
       boolean generateCallbacks, boolean disableCopyOnRead,
       ClientProxyMembershipID requestingClient, EntryEventImpl clientEvent, boolean returnTombstones, boolean opScopeIsLocal) throws TimeoutException, CacheLoaderException {
+    // TODO OFFHEAP: the last parameter "retainResult" should be true for getRetained. Need to look into what it is being set to false.
     return get(key, aCallbackArgument, generateCallbacks, disableCopyOnRead, true, requestingClient, clientEvent, returnTombstones, opScopeIsLocal, true, false);
   }
   /**
@@ -1585,14 +1585,7 @@ public class LocalRegion extends AbstractRegion
           } else if (!disableCopyOnRead) {
             result = conditionalCopy(result);
           }
-          //For sqlf since the deserialized value is nothing but chunk
-          // before returning the found value increase its use count
-          if(GemFireCacheImpl.sqlfSystem() && result instanceof ObjectChunk) {
-            if(!((ObjectChunk)result).retain()) {
-              return null;
-            }
-          }
-          // what was a miss is now a hit
+         // what was a miss is now a hit
           RegionEntry re = null;
           if (isCreate) {
             re = basicGetEntry(keyInfo.getKey());

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/Oplog.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/Oplog.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/Oplog.java
index 3a2d1ed..94e61fb 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/Oplog.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/Oplog.java
@@ -4592,8 +4592,8 @@ public final class Oplog implements CompactableOplog, Flushable {
         // TODO: compaction needs to get version?
         byte userBits = wrapper.getBits();
         ValueWrapper vw;
-        if (wrapper.getDataChunk() != null) {
-          vw = new DiskEntry.Helper.ChunkValueWrapper(wrapper.getDataChunk());
+        if (wrapper.getOffHeapData() != null) {
+          vw = new DiskEntry.Helper.OffHeapValueWrapper(wrapper.getOffHeapData());
         } else {
           vw = new DiskEntry.Helper.CompactorValueWrapper(wrapper.getBytes(), wrapper.getValidLength());
         }
@@ -4613,8 +4613,8 @@ public final class Oplog implements CompactableOplog, Flushable {
             LocalizedStrings.Oplog_FAILED_WRITING_KEY_TO_0_DUE_TO_FAILURE_IN_ACQUIRING_READ_LOCK_FOR_ASYNCH_WRITING
                 .toLocalizedString(this.diskFile.getPath()), ie, getParent());
       } finally {
-        if (wrapper.getDataChunk() != null) {
-          wrapper.setChunkData(null, (byte) 0);
+        if (wrapper.getOffHeapData() != null) {
+          wrapper.setOffHeapData(null, (byte) 0);
         }
         if (exceptionOccured) {
           did.setValueLength(len);
@@ -6127,8 +6127,8 @@ public final class Oplog implements CompactableOplog, Flushable {
                     // skip this guy his oplogId changed
                     if (!wrapper.isReusable()) {
                       wrapper = new BytesAndBitsForCompactor();
-                    } else if (wrapper.getDataChunk() != null) {
-                      wrapper.setChunkData(null, (byte) 0);
+                    } else if (wrapper.getOffHeapData() != null) {
+                      wrapper.setOffHeapData(null, (byte) 0);
                     }
                     continue;
                   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/PartitionedRegion.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/PartitionedRegion.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/PartitionedRegion.java
index 541c453..5a5de2c 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/PartitionedRegion.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/PartitionedRegion.java
@@ -252,7 +252,6 @@ import com.gemstone.gemfire.internal.logging.LogService;
 import com.gemstone.gemfire.internal.logging.LoggingThreadGroup;
 import com.gemstone.gemfire.internal.logging.log4j.LocalizedMessage;
 import com.gemstone.gemfire.internal.logging.log4j.LogMarker;
-import com.gemstone.gemfire.internal.offheap.ObjectChunk;
 import com.gemstone.gemfire.internal.offheap.annotations.Unretained;
 import com.gemstone.gemfire.internal.sequencelog.RegionLogger;
 import com.gemstone.gemfire.internal.util.TransformUtils;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/PreferBytesCachedDeserializable.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/PreferBytesCachedDeserializable.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/PreferBytesCachedDeserializable.java
index 7ed88b5..933362d 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/PreferBytesCachedDeserializable.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/PreferBytesCachedDeserializable.java
@@ -150,8 +150,17 @@ public final class PreferBytesCachedDeserializable implements CachedDeserializab
 
   @Override
   public Version[] getSerializationVersions() {
-    // TODO Auto-generated method stub
     return null;
   }
 
+  @Override
+  public boolean isSerialized() {
+    return true;
+  }
+
+  @Override
+  public boolean usesHeapForStorage() {
+    return true;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/RegionEntry.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/RegionEntry.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/RegionEntry.java
index e2cfb90..08101c2 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/RegionEntry.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/RegionEntry.java
@@ -31,7 +31,6 @@ import com.gemstone.gemfire.internal.cache.lru.NewLRUClockHand;
 import com.gemstone.gemfire.internal.cache.versions.VersionSource;
 import com.gemstone.gemfire.internal.cache.versions.VersionStamp;
 import com.gemstone.gemfire.internal.cache.versions.VersionTag;
-import com.gemstone.gemfire.internal.offheap.MemoryChunkWithRefCount;
 import com.gemstone.gemfire.internal.offheap.annotations.Released;
 import com.gemstone.gemfire.internal.offheap.annotations.Retained;
 import com.gemstone.gemfire.internal.offheap.annotations.Unretained;
@@ -237,10 +236,9 @@ public interface RegionEntry {
    */
   public void setValue(RegionEntryContext context, Object value, EntryEventImpl event) throws RegionClearedException;
   /**
-   * Obtain and return the value of this entry using {@link #_getValue()}.
-   * If the value is a MemoryChunkWithRefCount then increment its refcount.
-   * WARNING: if a MemoryChunkWithRefCount is returned then the caller MUST
-   * call {@link MemoryChunkWithRefCount#release()}.
+   * Obtain, retain and return the value of this entry.
+   * WARNING: if a StoredObject is returned and it has a refCount then the caller MUST
+   * make sure that {@link StoredObject#release()} before the returned object is garbage collected.
    * 
    * This is only retained in off-heap subclasses.  However, it's marked as
    * Retained here so that callers are aware that the value may be retained.

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/RemoteDestroyMessage.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/RemoteDestroyMessage.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/RemoteDestroyMessage.java
index a3c9402..4a8c101 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/RemoteDestroyMessage.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/RemoteDestroyMessage.java
@@ -210,10 +210,10 @@ public class RemoteDestroyMessage extends RemoteOperationMessageWithDirectReply
       this.hasOldValue = true;
       CachedDeserializable cd = (CachedDeserializable) event.getSerializedOldValue();
       if (cd != null) {
-        if (cd instanceof StoredObject && !((StoredObject) cd).isSerialized()) {
+        if (!cd.isSerialized()) {
           // it is a byte[]
           this.oldValueIsSerialized = false;
-          setOldValBytes((byte[]) ((StoredObject) cd).getDeserializedForReading());
+          setOldValBytes((byte[]) cd.getDeserializedForReading());
         } else {
           this.oldValueIsSerialized = true;
           Object o = cd.getValue();



[5/8] incubator-geode git commit: GEODE-982: refactor off-heap

Posted by ds...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/SimpleMemoryAllocatorImpl.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/SimpleMemoryAllocatorImpl.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/SimpleMemoryAllocatorImpl.java
index 14bde59..209a4a4 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/SimpleMemoryAllocatorImpl.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/SimpleMemoryAllocatorImpl.java
@@ -90,17 +90,17 @@ public class SimpleMemoryAllocatorImpl implements MemoryAllocator {
   public static MemoryAllocator create(OutOfOffHeapMemoryListener ooohml, OffHeapMemoryStats stats, LogWriter lw, 
       int slabCount, long offHeapMemorySize, long maxSlabSize) {
     return create(ooohml, stats, lw, slabCount, offHeapMemorySize, maxSlabSize,
-        null, new AddressableMemoryChunkFactory() {
+        null, new SlabFactory() {
       @Override
-      public AddressableMemoryChunk create(int size) {
-        return new UnsafeMemoryChunk(size);
+      public Slab create(int size) {
+        return new SlabImpl(size);
       }
     });
   }
 
   private static SimpleMemoryAllocatorImpl create(OutOfOffHeapMemoryListener ooohml, OffHeapMemoryStats stats, LogWriter lw, 
       int slabCount, long offHeapMemorySize, long maxSlabSize, 
-      AddressableMemoryChunk[] slabs, AddressableMemoryChunkFactory memChunkFactory) {
+      Slab[] slabs, SlabFactory slabFactory) {
     SimpleMemoryAllocatorImpl result = singleton;
     boolean created = false;
     try {
@@ -118,16 +118,16 @@ public class SimpleMemoryAllocatorImpl implements MemoryAllocator {
         if (lw != null) {
           lw.config("Allocating " + offHeapMemorySize + " bytes of off-heap memory. The maximum size of a single off-heap object is " + maxSlabSize + " bytes.");
         }
-        slabs = new UnsafeMemoryChunk[slabCount];
+        slabs = new SlabImpl[slabCount];
         long uncreatedMemory = offHeapMemorySize;
         for (int i=0; i < slabCount; i++) {
           try {
             if (uncreatedMemory >= maxSlabSize) {
-              slabs[i] = memChunkFactory.create((int) maxSlabSize);
+              slabs[i] = slabFactory.create((int) maxSlabSize);
               uncreatedMemory -= maxSlabSize;
             } else {
               // the last slab can be smaller then maxSlabSize
-              slabs[i] = memChunkFactory.create((int) uncreatedMemory);
+              slabs[i] = slabFactory.create((int) uncreatedMemory);
             }
           } catch (OutOfMemoryError err) {
             if (i > 0) {
@@ -137,7 +137,7 @@ public class SimpleMemoryAllocatorImpl implements MemoryAllocator {
             }
             for (int j=0; j < i; j++) {
               if (slabs[j] != null) {
-                slabs[j].release();
+                slabs[j].free();
               }
             }
             throw err;
@@ -163,11 +163,11 @@ public class SimpleMemoryAllocatorImpl implements MemoryAllocator {
     return result;
   }
   static SimpleMemoryAllocatorImpl createForUnitTest(OutOfOffHeapMemoryListener ooohml, OffHeapMemoryStats stats, LogWriter lw, 
-      int slabCount, long offHeapMemorySize, long maxSlabSize, AddressableMemoryChunkFactory memChunkFactory) {
+      int slabCount, long offHeapMemorySize, long maxSlabSize, SlabFactory memChunkFactory) {
     return create(ooohml, stats, lw, slabCount, offHeapMemorySize, maxSlabSize, 
         null, memChunkFactory);
   }
-  public static SimpleMemoryAllocatorImpl createForUnitTest(OutOfOffHeapMemoryListener oooml, OffHeapMemoryStats stats, AddressableMemoryChunk[] slabs) {
+  public static SimpleMemoryAllocatorImpl createForUnitTest(OutOfOffHeapMemoryListener oooml, OffHeapMemoryStats stats, Slab[] slabs) {
     int slabCount = 0;
     long offHeapMemorySize = 0;
     long maxSlabSize = 0;
@@ -185,7 +185,7 @@ public class SimpleMemoryAllocatorImpl implements MemoryAllocator {
   }
   
   
-  private void reuse(OutOfOffHeapMemoryListener oooml, LogWriter lw, OffHeapMemoryStats newStats, long offHeapMemorySize, AddressableMemoryChunk[] slabs) {
+  private void reuse(OutOfOffHeapMemoryListener oooml, LogWriter lw, OffHeapMemoryStats newStats, long offHeapMemorySize, Slab[] slabs) {
     if (isClosed()) {
       throw new IllegalStateException("Can not reuse a closed off-heap memory manager.");
     }
@@ -205,7 +205,7 @@ public class SimpleMemoryAllocatorImpl implements MemoryAllocator {
     this.stats = newStats;
   }
 
-  private SimpleMemoryAllocatorImpl(final OutOfOffHeapMemoryListener oooml, final OffHeapMemoryStats stats, final AddressableMemoryChunk[] slabs) {
+  private SimpleMemoryAllocatorImpl(final OutOfOffHeapMemoryListener oooml, final OffHeapMemoryStats stats, final Slab[] slabs) {
     if (oooml == null) {
       throw new IllegalArgumentException("OutOfOffHeapMemoryListener is null");
     }
@@ -224,20 +224,20 @@ public class SimpleMemoryAllocatorImpl implements MemoryAllocator {
     this.stats.incFreeMemory(this.freeList.getTotalMemory());
   }
   
-  public List<ObjectChunk> getLostChunks() {
-    List<ObjectChunk> liveChunks = this.freeList.getLiveChunks();
-    List<ObjectChunk> regionChunks = getRegionLiveChunks();
-    Set<ObjectChunk> liveChunksSet = new HashSet<>(liveChunks);
-    Set<ObjectChunk> regionChunksSet = new HashSet<>(regionChunks);
+  public List<OffHeapStoredObject> getLostChunks() {
+    List<OffHeapStoredObject> liveChunks = this.freeList.getLiveChunks();
+    List<OffHeapStoredObject> regionChunks = getRegionLiveChunks();
+    Set<OffHeapStoredObject> liveChunksSet = new HashSet<>(liveChunks);
+    Set<OffHeapStoredObject> regionChunksSet = new HashSet<>(regionChunks);
     liveChunksSet.removeAll(regionChunksSet);
-    return new ArrayList<ObjectChunk>(liveChunksSet);
+    return new ArrayList<OffHeapStoredObject>(liveChunksSet);
   }
   
   /**
    * Returns a possibly empty list that contains all the Chunks used by regions.
    */
-  private List<ObjectChunk> getRegionLiveChunks() {
-    ArrayList<ObjectChunk> result = new ArrayList<ObjectChunk>();
+  private List<OffHeapStoredObject> getRegionLiveChunks() {
+    ArrayList<OffHeapStoredObject> result = new ArrayList<OffHeapStoredObject>();
     RegionService gfc = GemFireCacheImpl.getInstance();
     if (gfc != null) {
       Iterator<Region<?,?>> rootIt = gfc.rootRegions().iterator();
@@ -253,7 +253,7 @@ public class SimpleMemoryAllocatorImpl implements MemoryAllocator {
     return result;
   }
 
-  private void getRegionLiveChunks(Region<?,?> r, List<ObjectChunk> result) {
+  private void getRegionLiveChunks(Region<?,?> r, List<OffHeapStoredObject> result) {
     if (r.getAttributes().getOffHeap()) {
 
       if (r instanceof PartitionedRegion) {
@@ -277,7 +277,7 @@ public class SimpleMemoryAllocatorImpl implements MemoryAllocator {
 
   }
   
-  private void basicGetRegionLiveChunks(LocalRegion r, List<ObjectChunk> result) {
+  private void basicGetRegionLiveChunks(LocalRegion r, List<OffHeapStoredObject> result) {
     for (Object key : r.keySet()) {
       RegionEntry re = ((LocalRegion) r).getRegionEntry(key);
       if (re != null) {
@@ -286,30 +286,30 @@ public class SimpleMemoryAllocatorImpl implements MemoryAllocator {
          */
         @Unretained(OffHeapIdentifier.GATEWAY_SENDER_EVENT_IMPL_VALUE)
         Object value = re._getValue();
-        if (value instanceof ObjectChunk) {
-          result.add((ObjectChunk) value);
+        if (value instanceof OffHeapStoredObject) {
+          result.add((OffHeapStoredObject) value);
         }
       }
     }
   }
 
-  private ObjectChunk allocateChunk(int size) {
-    ObjectChunk result = this.freeList.allocate(size);
+  private OffHeapStoredObject allocateOffHeapStoredObject(int size) {
+    OffHeapStoredObject result = this.freeList.allocate(size);
     int resultSize = result.getSize();
     stats.incObjects(1);
     stats.incUsedMemory(resultSize);
     stats.incFreeMemory(-resultSize);
     notifyListeners();
     if (ReferenceCountHelper.trackReferenceCounts()) {
-      ReferenceCountHelper.refCountChanged(result.getMemoryAddress(), false, 1);
+      ReferenceCountHelper.refCountChanged(result.getAddress(), false, 1);
     }
     return result;
   }
   
   @Override
-  public MemoryChunk allocate(int size) {
+  public StoredObject allocate(int size) {
     //System.out.println("allocating " + size);
-    ObjectChunk result = allocateChunk(size);
+    OffHeapStoredObject result = allocateOffHeapStoredObject(size);
     //("allocated off heap object of size " + size + " @" + Long.toHexString(result.getMemoryAddress()), true);
     return result;
   }
@@ -324,16 +324,23 @@ public class SimpleMemoryAllocatorImpl implements MemoryAllocator {
   
   @Override
   public StoredObject allocateAndInitialize(byte[] v, boolean isSerialized, boolean isCompressed) {
+    return allocateAndInitialize(v, isSerialized, isCompressed, null);
+  }
+  @Override
+  public StoredObject allocateAndInitialize(byte[] v, boolean isSerialized, boolean isCompressed, byte[] originalHeapData) {
     long addr = OffHeapRegionEntryHelper.encodeDataAsAddress(v, isSerialized, isCompressed);
     if (addr != 0L) {
-      return new DataAsAddress(addr);
+      return new TinyStoredObject(addr);
     }
-    ObjectChunk result = allocateChunk(v.length);
+    OffHeapStoredObject result = allocateOffHeapStoredObject(v.length);
     //debugLog("allocated off heap object of size " + v.length + " @" + Long.toHexString(result.getMemoryAddress()), true);
     //debugLog("allocated off heap object of size " + v.length + " @" + Long.toHexString(result.getMemoryAddress()) +  "chunkSize=" + result.getSize() + " isSerialized=" + isSerialized + " v=" + Arrays.toString(v), true);
     result.setSerializedValue(v);
     result.setSerialized(isSerialized);
     result.setCompressed(isCompressed);
+    if (originalHeapData != null) {
+      result = new OffHeapStoredObjectWithHeapForm(result, originalHeapData);
+    }
     return result;
   }
   
@@ -485,18 +492,18 @@ public class SimpleMemoryAllocatorImpl implements MemoryAllocator {
   }
 
   public synchronized List<MemoryBlock> getOrphans() {
-    List<ObjectChunk> liveChunks = this.freeList.getLiveChunks();
-    List<ObjectChunk> regionChunks = getRegionLiveChunks();
+    List<OffHeapStoredObject> liveChunks = this.freeList.getLiveChunks();
+    List<OffHeapStoredObject> regionChunks = getRegionLiveChunks();
     liveChunks.removeAll(regionChunks);
     List<MemoryBlock> orphans = new ArrayList<MemoryBlock>();
-    for (ObjectChunk chunk: liveChunks) {
+    for (OffHeapStoredObject chunk: liveChunks) {
       orphans.add(new MemoryBlockNode(this, chunk));
     }
     Collections.sort(orphans,
         new Comparator<MemoryBlock>() {
           @Override
           public int compare(MemoryBlock o1, MemoryBlock o2) {
-            return Long.valueOf(o1.getMemoryAddress()).compareTo(o2.getMemoryAddress());
+            return Long.valueOf(o1.getAddress()).compareTo(o2.getAddress());
           }
         });
     //this.memoryBlocks = new WeakReference<List<MemoryBlock>>(orphans);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/Slab.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/Slab.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/Slab.java
new file mode 100644
index 0000000..000b8cb
--- /dev/null
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/Slab.java
@@ -0,0 +1,39 @@
+/*
+ * 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;
+
+/**
+ * A "slab" of memory.
+ * Slabs can be created by calling {@link AddressableMemoryManager#allocateSlab(int)}.
+ * Slabs have an address, a size, and can be freed.
+ */
+public interface Slab {
+  /**
+   * Return the address of the memory of this slab.
+   */
+  public long getMemoryAddress();
+  /**
+   * Returns the size of this memory chunk in bytes.
+   */
+  public int getSize();
+  /**
+   * Returns any memory allocated for this slab.
+   * Note that after free is called the address of
+   * this slab should no longer be used.
+   */
+  public void free();
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/SlabFactory.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/SlabFactory.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/SlabFactory.java
new file mode 100644
index 0000000..a3f457d
--- /dev/null
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/SlabFactory.java
@@ -0,0 +1,27 @@
+/*
+ * 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;
+
+/**
+ * Used to create Slab instances.
+ */
+public interface SlabFactory {
+  /** Create and return a Slab
+   * @throws OutOfMemoryError if the create fails
+   */
+  public Slab create(int size);
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/SlabImpl.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/SlabImpl.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/SlabImpl.java
new file mode 100644
index 0000000..1c88bde
--- /dev/null
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/SlabImpl.java
@@ -0,0 +1,61 @@
+/*
+ * 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;
+
+/**
+ * Implements the Slab interface using AddressableMemoryManager.
+ * 
+ * @since 9.0
+ */
+public class SlabImpl implements Slab {
+  private final long address;
+  private final int size;
+  
+  public SlabImpl(int size) {
+    this(AddressableMemoryManager.allocate(size), size);
+  }
+
+  public SlabImpl(long addr, int size) {
+    this.address = addr;
+    this.size = size;
+  }
+  
+  @Override
+  public int getSize() {
+    return this.size;
+  }
+  
+  @Override
+  public long getMemoryAddress() {
+    return this.address;
+  }
+  
+  @Override
+  public void free() {
+    AddressableMemoryManager.free(this.address);
+  }
+
+  @Override
+  public String toString() {
+    final StringBuilder sb = new StringBuilder(getClass().getSimpleName());
+    sb.append("{");
+    sb.append("MemoryAddress=").append(getMemoryAddress());
+    sb.append(", Size=").append(getSize());
+    sb.append("}");
+    return sb.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/StoredObject.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/StoredObject.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/StoredObject.java
index 4d93a07..26cb81f 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/StoredObject.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/StoredObject.java
@@ -18,38 +18,27 @@ package com.gemstone.gemfire.internal.offheap;
 
 import java.io.DataOutput;
 import java.io.IOException;
+import java.nio.ByteBuffer;
 
 import com.gemstone.gemfire.internal.Sendable;
 import com.gemstone.gemfire.internal.cache.CachedDeserializable;
 import com.gemstone.gemfire.internal.offheap.annotations.Retained;
+import com.gemstone.gemfire.internal.offheap.annotations.Unretained;
 
 /**
  * Represents an object stored in the cache.
  * Currently this interface is only used for values stored in off-heap regions.
+ * This interface provides methods that let you read and write the bytes
+ * of addressable memory used to store the bytes of the object.
+ * A reference count is used to determine if the object is still allocated.
+ * To increment the count call {@link #retain()}.
+ * To decrement the count call {@link #release()}.
  * At some point in the future it may also be used for values stored in heap regions. 
  * 
  * @author darrel
  * @since 9.0
  */
-public interface StoredObject extends Releasable, Sendable, CachedDeserializable {
-  /**
-   * Call to indicate that this object's memory is in use by the caller.
-   * The memory will stay allocated until {@link #release()} is called.
-   * It is ok for a thread other than the one that called this method to call release.
-   * This method is called implicitly at the time the chunk is allocated.
-   * Note: @Retained tells you that "this" is retained by this method.
-   * 
-   * @throws IllegalStateException if the max ref count is exceeded.
-   * @return true if we are able to retain this chunk; false if we need to retry
-   */
-  @Retained
-  public boolean retain();
-
-  /**
-   * Returns true if the value stored in this memory chunk is a serialized object. Returns false if it is a byte array.
-   */
-  public boolean isSerialized();
-
+public interface StoredObject extends Sendable, CachedDeserializable, Releasable {
   /**
    * Returns true if the value stored in this memory chunk is compressed. Returns false if it is uncompressed.
    */
@@ -92,4 +81,94 @@ public interface StoredObject extends Releasable, Sendable, CachedDeserializable
    * @throws IOException
    */
   void sendAsCachedDeserializable(DataOutput out) throws IOException;
+  
+  /**
+   * Call to indicate that this object's memory is in use by the caller.
+   * The memory will stay allocated until {@link #release()} is called.
+   * It is ok for a thread other than the one that called this method to call release.
+   * This method is called implicitly at the time the chunk is allocated.
+   * Note: @Retained tells you that "this" is retained by this method.
+   * 
+   * @throws IllegalStateException if the max ref count is exceeded.
+   * @return true if we are able to retain this chunk; false if we need to retry
+   */
+  @Retained
+  public boolean retain();
+
+  /**
+   * Returns true if this type of StoredObject uses a references count; false otherwise.
+   */
+  public boolean hasRefCount();
+   /**
+   * Returns the number of users of this memory. If this type of StoredObject does not
+   * have a reference count then -1 is returned.
+   */
+  public int getRefCount();
+  
+  /**
+   * Returns the address of the memory used to store this object.
+   * This address may not be to the first byte of stored data since
+   * the implementation may store some internal data in the first bytes of the memory.
+   * This address can be used with AddressableMemoryManager.
+   */
+  public long getAddress();
+
+  /**
+   * Returns the number of bytes of memory used by this object to store an object.
+   * This size includes any bytes used for padding and meta-information.
+   */
+  public int getSize();
+  
+  /**
+   * Returns the number of bytes of memory used to store the object.
+   * This size does not include any bytes used for padding.
+   */
+  public int getDataSize();
+  public byte readDataByte(int offset);
+  public void writeDataByte(int offset, byte value);
+  public void readDataBytes(int offset, byte[] bytes);
+  public void writeDataBytes(int offset, byte[] bytes);
+  public void readDataBytes(int offset, byte[] bytes, int bytesOffset, int size);
+  public void writeDataBytes(int offset, byte[] bytes, int bytesOffset, int size);
+  /**
+   * Returns an address that can read data from this StoredObject at the given offset.
+   */
+  public long getAddressForReadingData(int offset, int size);
+  
+  /**
+   * Returns a StoredObject that acts as if its data is our data starting
+   * at the given offset and limited to the given number of bytes.
+   */
+  public StoredObject slice(int offset, int limit);
+  
+  /**
+   * Returns true if our data is equal to other's data; false otherwise.
+   */
+  public boolean checkDataEquals(StoredObject other);
+  /**
+   * Returns true if the given bytes are equal to our data bytes; false otherwise
+   */
+  public boolean checkDataEquals(byte[] serializedObj);
+
+  /**
+   * Creates and returns a direct ByteBuffer that contains the data of this stored object.
+   * Note that the returned ByteBuffer has a reference to the
+   * address of this stored object so it can only be used while this stored object is retained.
+   * @return the created direct byte buffer or null if it could not be created.
+   */
+  @Unretained
+  public ByteBuffer createDirectByteBuffer();
+  /**
+   * Returns true if the data is serialized with PDX
+   */
+  public boolean isSerializedPdxInstance();
+  
+  /**
+   * Returns a StoredObject that does not cache the heap form.
+   * If a StoredObject is going to be kept around for a while then
+   * it is good to call this so that it will not also keep the heap
+   * form in memory.
+   */
+  public StoredObject getStoredObjectWithoutHeapForm();
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/SyncChunkStack.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/SyncChunkStack.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/SyncChunkStack.java
deleted file mode 100644
index 99fd96f..0000000
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/SyncChunkStack.java
+++ /dev/null
@@ -1,141 +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.LogWriter;
-
-/**
- * A "stack" of "chunk" instances. The chunks are not kept
- * in java object form but instead each "chunk" is just an
- * off-heap address.
- * This class is used for each "tiny" free-list of the off-heap memory allocator.
- */
-public class SyncChunkStack {
-  // Ok to read without sync but must be synced on write
-  private volatile long topAddr;
-  
-  public SyncChunkStack(long addr) {
-    if (addr != 0L) SimpleMemoryAllocatorImpl.validateAddress(addr);
-    this.topAddr = addr;
-  }
-  public SyncChunkStack() {
-    this.topAddr = 0L;
-  }
-  public boolean isEmpty() {
-    return this.topAddr == 0L;
-  }
-  public void offer(long e) {
-    assert e != 0;
-    SimpleMemoryAllocatorImpl.validateAddress(e);
-    synchronized (this) {
-      ObjectChunk.setNext(e, this.topAddr);
-      this.topAddr = e;
-    }
-  }
-  public long poll() {
-    long result;
-    synchronized (this) {
-      result = this.topAddr;
-      if (result != 0L) {
-        this.topAddr = ObjectChunk.getNext(result);
-      }
-    }
-    return result;
-  }
-  /**
-   * Returns the address of the "top" item in this stack.
-   */
-  public long getTopAddress() {
-    return this.topAddr;
-  }
-  /**
-   * Removes all the Chunks from this stack
-   * and returns the address of the first chunk.
-   * The caller owns all the Chunks after this call.
-   */
-  public long clear() {
-    long result;
-    synchronized (this) {
-      result = this.topAddr;
-      if (result != 0L) {
-        this.topAddr = 0L;
-      }
-    }
-    return result;
-  }
-  public void logSizes(LogWriter lw, String msg) {
-    long headAddr = this.topAddr;
-    long addr;
-    boolean concurrentModDetected;
-    do {
-      concurrentModDetected = false;
-      addr = headAddr;
-      while (addr != 0L) {
-        int curSize = ObjectChunk.getSize(addr);
-        addr = ObjectChunk.getNext(addr);
-        testHookDoConcurrentModification();
-        long curHead = this.topAddr;
-        if (curHead != headAddr) {
-          headAddr = curHead;
-          concurrentModDetected = true;
-          // Someone added or removed from the stack.
-          // So we break out of the inner loop and start
-          // again at the new head.
-          break;
-        }
-        // TODO construct a single log msg
-        // that gets reset when concurrentModDetected.
-        lw.info(msg + curSize);
-      }
-    } while (concurrentModDetected);
-  }
-  public long computeTotalSize() {
-    long result;
-    long headAddr = this.topAddr;
-    long addr;
-    boolean concurrentModDetected;
-    do {
-      concurrentModDetected = false;
-      result = 0;
-      addr = headAddr;
-      while (addr != 0L) {
-        result += ObjectChunk.getSize(addr);
-        addr = ObjectChunk.getNext(addr);
-        testHookDoConcurrentModification();
-        long curHead = this.topAddr;
-        if (curHead != headAddr) {
-          headAddr = curHead;
-          concurrentModDetected = true;
-          // Someone added or removed from the stack.
-          // So we break out of the inner loop and start
-          // again at the new head.
-          break;
-        }
-      }
-    } while (concurrentModDetected);
-    return result;
-  }
-  
-  /**
-   * This method allows tests to override it
-   * and do a concurrent modification to the stack.
-   * For production code it will be a noop.
-   */
-  protected void testHookDoConcurrentModification() {
-    // nothing needed in production code
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/TinyStoredObject.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/TinyStoredObject.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/TinyStoredObject.java
new file mode 100644
index 0000000..e8878fa
--- /dev/null
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/TinyStoredObject.java
@@ -0,0 +1,229 @@
+/*
+ * 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 java.util.Arrays;
+
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.internal.cache.BytesAndBitsForCompactor;
+import com.gemstone.gemfire.internal.cache.EntryBits;
+import com.gemstone.gemfire.internal.cache.EntryEventImpl;
+import com.gemstone.gemfire.internal.cache.RegionEntry;
+import com.gemstone.gemfire.internal.cache.RegionEntryContext;
+
+/**
+ * Used to represent stored objects that can be stored
+ * in the address field.
+ * The RegionEntry for an off-heap region uses a primitive
+ * long to store the off-heap address of the entry's value.
+ * If the value can be encoded as a long (i.e. its serialized
+ * representation will fit in the 8 bytes of a long without looking
+ * like an actual off-heap address) then these tiny values on an
+ * off-heap regions are actually stored on the heap in the primitive
+ * long field. When these values are "objectified" they will be an
+ * instance of this class.
+ * Instances of this class have a very short lifetime.
+ */
+public class TinyStoredObject extends AbstractStoredObject {
+  private final long address;
+  
+  public TinyStoredObject(long addr) {
+    this.address = addr;
+  }
+  
+  @Override
+  public long getAddress() {
+    return this.address;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o instanceof TinyStoredObject) {
+      return getAddress() == ((TinyStoredObject) o).getAddress();
+    }
+    return false;
+  }
+  
+  @Override
+  public int hashCode() {
+    long value = getAddress();
+    return (int)(value ^ (value >>> 32));
+  }
+
+  @Override
+  public int getSizeInBytes() {
+    return 0;
+  }
+
+  public byte[] getDecompressedBytes(RegionEntryContext r) {
+    byte[] bytes = OffHeapRegionEntryHelper.decodeAddressToBytes(getAddress(), true, true);
+    if (isCompressed()) {
+        long time = r.getCachePerfStats().startDecompression();
+        bytes = r.getCompressor().decompress(bytes);
+        r.getCachePerfStats().endDecompression(time);
+    }
+    return bytes;
+  }
+
+  /**
+   * If we contain a byte[] return it.
+   * Otherwise return the serialize bytes in us in a byte array.
+   */
+  public byte[] getRawBytes() {
+    return OffHeapRegionEntryHelper.decodeAddressToBytes(getAddress(), true, false);
+  }
+  
+  @Override
+  public byte[] getSerializedValue() {
+    byte[] value = OffHeapRegionEntryHelper.decodeAddressToBytes(this.address, true, false);
+    if (!isSerialized()) {
+      value = EntryEventImpl.serialize(value);
+    }
+    return value;
+  }
+
+  @Override
+  public Object getDeserializedValue(Region r, RegionEntry re) {
+    return OffHeapRegionEntryHelper.decodeAddressToObject(this.address);
+  }
+
+  @Override
+  public void fillSerializedValue(BytesAndBitsForCompactor wrapper,
+      byte userBits) {
+    byte[] value;
+    if (isSerialized()) {
+      value = getSerializedValue();
+      userBits = EntryBits.setSerialized(userBits, true);
+    } else {
+      value = (byte[]) getDeserializedForReading();
+    }
+    wrapper.setData(value, userBits, value.length, true);
+  }
+
+  @Override
+  public int getValueSizeInBytes() {
+    return 0;
+  }
+  
+  @Override
+  public boolean isSerialized() {
+    return OffHeapRegionEntryHelper.isSerialized(this.address);
+  }
+
+  @Override
+  public boolean isCompressed() {
+    return OffHeapRegionEntryHelper.isCompressed(this.address);
+  }
+
+  @Override
+  public void release() {
+    // nothing needed
+  }
+
+  @Override
+  public boolean retain() {
+    return true;
+  }
+
+  @Override
+  public int getRefCount() {
+    return -1;
+  }
+
+  @Override
+  public int getSize() {
+    return Long.BYTES;
+  }
+
+  @Override
+  public int getDataSize() {
+    return OffHeapRegionEntryHelper.decodeAddressToDataSize(this.address);
+  }
+
+  @Override
+  public byte readDataByte(int offset) {
+    // TODO OFFHEAP: what if the data is compressed?
+    return getRawBytes()[offset];
+  }
+
+  @Override
+  public void writeDataByte(int offset, byte value) {
+    throw new UnsupportedOperationException("ObjectStoredAsAddress does not support modifying the data bytes");
+  }
+
+  @Override
+  public void readDataBytes(int offset, byte[] bytes) {
+    readDataBytes(offset, bytes, 0, bytes.length);
+  }
+
+  @Override
+  public void writeDataBytes(int offset, byte[] bytes) {
+    writeDataBytes(offset, bytes, 0, bytes.length);
+  }
+
+  @Override
+  public void readDataBytes(int offset, byte[] bytes, int bytesOffset, int size) {
+    // TODO OFFHEAP: what if the data is compressed?
+    byte[] src = getRawBytes();
+    int dstIdx = bytesOffset;
+    for (int i = offset; i < offset+size; i++) {
+      bytes[dstIdx++] = src[i];
+    }
+  }
+
+  @Override
+  public void writeDataBytes(int offset, byte[] bytes, int bytesOffset, int size) {
+    throw new UnsupportedOperationException("ObjectStoredAsAddress does not support modifying the data bytes");
+  }
+
+  @Override
+  public ByteBuffer createDirectByteBuffer() {
+    return null;
+  }
+
+  @Override
+  public boolean hasRefCount() {
+    return false;
+  }
+
+  @Override
+  public boolean checkDataEquals(StoredObject so) {
+    // TODO OFFHEAP: what if the data is compressed?
+    return equals(so);
+  }
+
+  @Override
+  public boolean checkDataEquals(byte[] serializedObj) {
+    // TODO OFFHEAP: what if the data is compressed?
+    byte[] myBytes = getSerializedValue();
+    return Arrays.equals(myBytes, serializedObj);
+  }
+
+  @Override
+  public long getAddressForReadingData(int offset, int size) {
+    throw new UnsupportedOperationException("ObjectStoredAsAddress does not support reading at an address");
+  }
+
+  @Override
+  public StoredObject slice(int offset, int limit) {
+    throw new UnsupportedOperationException("ObjectStoredAsAddress does not support slice");
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/UnsafeMemoryChunk.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/UnsafeMemoryChunk.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/UnsafeMemoryChunk.java
deleted file mode 100644
index aebc459..0000000
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/UnsafeMemoryChunk.java
+++ /dev/null
@@ -1,217 +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.internal.SharedLibrary;
-import com.gemstone.gemfire.pdx.internal.unsafe.UnsafeWrapper;
-
-/**
- * Represents a single addressable chunk of off-heap memory. The size specifies
- * the number of bytes stored at the address.
- * 
- * @since 9.0
- */
-public class UnsafeMemoryChunk implements AddressableMemoryChunk {
-  private static final UnsafeWrapper unsafe;
-  private static final int ARRAY_BYTE_BASE_OFFSET;
-  private static String reason;
-  static {
-	UnsafeWrapper tmp = null;
-	try {
-	  tmp = new UnsafeWrapper();
-	  reason = null;
-	} catch (RuntimeException ignore) {
-      reason = ignore.toString();
-	} catch (Error ignore) {
-      reason = ignore.toString();
-	}
-	unsafe = tmp;
-    ARRAY_BYTE_BASE_OFFSET = unsafe != null ? unsafe.arrayBaseOffset(byte[].class) : 0;
-  }
-  
-  private final long data;
-  private final int size;
-  
-  public UnsafeMemoryChunk(int size) {
-	if (unsafe == null) {
-      throw new OutOfMemoryError("Off-heap memory is not available because: " + reason);
-	}
-    try {
-    this.data = unsafe.allocateMemory(size);
-    this.size = size;
-    } catch (OutOfMemoryError err) {
-      String msg = "Failed creating " + size + " bytes of off-heap memory during cache creation.";
-      if (err.getMessage() != null && !err.getMessage().isEmpty()) {
-        msg += " Cause: " + err.getMessage();
-      }
-      if (!SharedLibrary.is64Bit() && size >= (1024*1024*1024)) {
-        msg += " The JVM looks like a 32-bit one. For large amounts of off-heap memory a 64-bit JVM is needed.";
-      }
-      throw new OutOfMemoryError(msg);
-    }
-  }
-
-  @Override
-  public int getSize() {
-    return (int)this.size;
-  }
-  
-  /* (non-Javadoc)
-   * @see com.gemstone.gemfire.internal.offheap.AddressableMemoryChunk#getMemoryAddress()
-   */
-  @Override
-  public long getMemoryAddress() {
-    return this.data;
-  }
-  
-  public static byte readAbsoluteByte(long addr) {
-    return unsafe.getByte(addr);
-  }
-  public static char readAbsoluteChar(long addr) {
-    return unsafe.getChar(null, addr);
-  }
-  public static short readAbsoluteShort(long addr) {
-    return unsafe.getShort(null, addr);
-  }
-  public static int readAbsoluteInt(long addr) {
-    return unsafe.getInt(null, addr);
-  }
-  public static int readAbsoluteIntVolatile(long addr) {
-    return unsafe.getIntVolatile(null, addr);
-  }
-  public static long readAbsoluteLong(long addr) {
-    return unsafe.getLong(null, addr);
-  }
-  public static long readAbsoluteLongVolatile(long addr) {
-    return unsafe.getLongVolatile(null, addr);
-  }
-
-  @Override
-  public byte readByte(int offset) {
-    return readAbsoluteByte(this.data+offset);
-  }
-
-  public static void writeAbsoluteByte(long addr, byte value) {
-    unsafe.putByte(addr, value);
-  }
-       
-  public static void writeAbsoluteInt(long addr, int value) {
-    unsafe.putInt(null, addr, value);
-  }
-  public static void writeAbsoluteIntVolatile(long addr, int value) {
-    unsafe.putIntVolatile(null, addr, value);
-  }
-  public static boolean writeAbsoluteIntVolatile(long addr, int expected, int value) {
-    return unsafe.compareAndSwapInt(null, addr, expected, value);
-  }
-  public static void writeAbsoluteLong(long addr, long value) {
-    unsafe.putLong(null, addr, value);
-  }
-  public static void writeAbsoluteLongVolatile(long addr, long value) {
-    unsafe.putLongVolatile(null, addr, value);
-  }
-  public static boolean writeAbsoluteLongVolatile(long addr, long expected, long value) {
-    return unsafe.compareAndSwapLong(null, addr, expected, value);
-  }
-
-  @Override
-  public void writeByte(int offset, byte value) {
-    writeAbsoluteByte(this.data+offset, value);
-  }
-
-  @Override
-  public void readBytes(int offset, byte[] bytes) {
-    readBytes(offset, bytes, 0, bytes.length);
-  }
-
-  @Override
-  public void writeBytes(int offset, byte[] bytes) {
-    writeBytes(offset, bytes, 0, bytes.length);
-  }
-
-  public static void readAbsoluteBytes(long addr, byte[] bytes, int bytesOffset, int size) {
-    // Throwing an Error instead of using the "assert" keyword because passing < 0 to
-    // copyMemory(...) can lead to a core dump with some JVMs and we don't want to
-    // require the -ea JVM flag.
-    if (size < 0) {
-      throw new AssertionError("Size=" + size + ", but size must be >= 0");
-    }
-    
-    assert bytesOffset >= 0 : "byteOffset=" + bytesOffset;
-    assert bytesOffset + size <= bytes.length : "byteOffset=" + bytesOffset + ",size=" + size + ",bytes.length=" + bytes.length;
-    
-    if (size == 0) {
-      return; // No point in wasting time copying 0 bytes
-    }
-    unsafe.copyMemory(null, addr, bytes, ARRAY_BYTE_BASE_OFFSET+bytesOffset, size);
-  }
-
-  @Override
-  public void readBytes(int offset, byte[] bytes, int bytesOffset, int size) {
-    readAbsoluteBytes(this.data+offset, bytes, bytesOffset, size);
-  }
-
-  public static void copyMemory(long srcAddr, long dstAddr, long size) {
-    unsafe.copyMemory(srcAddr, dstAddr, size);
-  }
-  
-  public static void writeAbsoluteBytes(long addr, byte[] bytes, int bytesOffset, int size) {
-    // Throwing an Error instead of using the "assert" keyword because passing < 0 to
-    // copyMemory(...) can lead to a core dump with some JVMs and we don't want to
-    // require the -ea JVM flag.
-    if (size < 0) {
-      throw new AssertionError("Size=" + size + ", but size must be >= 0");
-    }
-
-    assert bytesOffset >= 0 : "byteOffset=" + bytesOffset;
-    assert bytesOffset + size <= bytes.length : "byteOffset=" + bytesOffset + ",size=" + size + ",bytes.length=" + bytes.length;
-    
-    if (size == 0) {
-      return; // No point in wasting time copying 0 bytes
-    }
-    unsafe.copyMemory(bytes, ARRAY_BYTE_BASE_OFFSET+bytesOffset, null, addr, size);
-  }
-
-  public static void fill(long addr, int size, byte fill) {
-    unsafe.setMemory(addr, size, fill);
-  }
-  
-  @Override
-  public void writeBytes(int offset, byte[] bytes, int bytesOffset, int size) {
-    writeAbsoluteBytes(this.data+offset, bytes, bytesOffset, size);
-  }
-
-  @Override
-  public void release() {
-    unsafe.freeMemory(this.data);
-  }
-
-  @Override
-  public void copyBytes(int src, int dst, int size) {
-    unsafe.copyMemory(this.data+src, this.data+dst, size);
-  }
-  
-  @Override
-  public String toString() {
-    final StringBuilder sb = new StringBuilder(getClass().getSimpleName());
-    sb.append("{");
-    sb.append("MemoryAddress=").append(getMemoryAddress());
-    sb.append(", Size=").append(getSize());
-    sb.append("}");
-    return sb.toString();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/geode-core/src/main/java/com/gemstone/gemfire/internal/tcp/ByteBufferInputStream.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/tcp/ByteBufferInputStream.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/tcp/ByteBufferInputStream.java
index cfc05f2..d7ec947 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/tcp/ByteBufferInputStream.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/tcp/ByteBufferInputStream.java
@@ -31,8 +31,8 @@ import java.nio.ByteOrder;
 
 import com.gemstone.gemfire.internal.ByteBufferWriter;
 import com.gemstone.gemfire.internal.HeapDataOutputStream;
-import com.gemstone.gemfire.internal.offheap.ObjectChunk;
-import com.gemstone.gemfire.internal.offheap.UnsafeMemoryChunk;
+import com.gemstone.gemfire.internal.offheap.AddressableMemoryManager;
+import com.gemstone.gemfire.internal.offheap.StoredObject;
 
 /**
  * <p>
@@ -109,15 +109,15 @@ public class ByteBufferInputStream extends InputStream implements DataInput, jav
     public static ByteSource create(ByteBuffer bb) {
       return new ByteBufferByteSource(bb);
     }
-    public static ByteSource create(ObjectChunk chunk) {
-      // Since I found a way to create a DirectByteBuffer (using reflection) from a Chunk
+    public static ByteSource create(StoredObject so) {
+      // Since I found a way to create a DirectByteBuffer (using reflection) from a StoredObject
       // we might not even need the ByteSource abstraction any more.
       // But it is possible that createByteBuffer will not work on a different jdk so keep it for now.
-      ByteBuffer bb = chunk.createDirectByteBuffer();
+      ByteBuffer bb = so.createDirectByteBuffer();
       if (bb != null) {
         return create(bb);
       } else {
-        return new OffHeapByteSource(chunk);
+        return new OffHeapByteSource(so);
       }
     }
   }
@@ -323,10 +323,10 @@ public class ByteBufferInputStream extends InputStream implements DataInput, jav
   public static class OffHeapByteSource implements ByteSource {
     private int position;
     private int limit;
-    private final ObjectChunk chunk;
+    private final StoredObject chunk;
 
-    public OffHeapByteSource(ObjectChunk c) {
-      this.chunk = c;
+    public OffHeapByteSource(StoredObject so) {
+      this.chunk = so;
       this.position = 0;
       this.limit = capacity();
     }
@@ -474,17 +474,17 @@ public class ByteBufferInputStream extends InputStream implements DataInput, jav
       }
       int p = this.position;
       this.position += length;
-      this.chunk.readBytes(p, dst, offset, length);
+      this.chunk.readDataBytes(p, dst, offset, length);
     }
 
     @Override
     public byte get() {
-      return this.chunk.readByte(nextGetIndex());
+      return this.chunk.readDataByte(nextGetIndex());
     }
     @Override
     public byte get(int pos) {
       checkIndex(pos);
-      return this.chunk.readByte(pos);
+      return this.chunk.readDataByte(pos);
     }
 
     /**
@@ -513,16 +513,16 @@ public class ByteBufferInputStream extends InputStream implements DataInput, jav
       return basicGetShort(pos);
     }
     private short basicGetShort(int pos) {
-      long addr = this.chunk.getAddressForReading(pos, 2);
+      long addr = this.chunk.getAddressForReadingData(pos, 2);
       if (unaligned) {
-        short result = UnsafeMemoryChunk.readAbsoluteShort(addr);
+        short result = AddressableMemoryManager.readShort(addr);
         if (ByteOrder.nativeOrder() == ByteOrder.LITTLE_ENDIAN) {
           result = Short.reverseBytes(result);
         }
         return result;
       } else {
-        int ch1 = UnsafeMemoryChunk.readAbsoluteByte(addr++);
-        int ch2 = UnsafeMemoryChunk.readAbsoluteByte(addr);
+        int ch1 = AddressableMemoryManager.readByte(addr++);
+        int ch2 = AddressableMemoryManager.readByte(addr);
         return (short)((ch1 << 8) + (ch2 << 0));
       }
     }
@@ -537,16 +537,16 @@ public class ByteBufferInputStream extends InputStream implements DataInput, jav
       return basicGetChar(pos);
     }
     private char basicGetChar(int pos) {
-      long addr = this.chunk.getAddressForReading(pos, 2);
+      long addr = this.chunk.getAddressForReadingData(pos, 2);
       if (unaligned) {
-        char result = UnsafeMemoryChunk.readAbsoluteChar(addr);
+        char result = AddressableMemoryManager.readChar(addr);
         if (ByteOrder.nativeOrder() == ByteOrder.LITTLE_ENDIAN) {
           result = Character.reverseBytes(result);
         }
         return result;
       } else {
-        int ch1 = UnsafeMemoryChunk.readAbsoluteByte(addr++);
-        int ch2 = UnsafeMemoryChunk.readAbsoluteByte(addr);
+        int ch1 = AddressableMemoryManager.readByte(addr++);
+        int ch2 = AddressableMemoryManager.readByte(addr);
         return (char)((ch1 << 8) + (ch2 << 0));
       }
     }
@@ -562,18 +562,18 @@ public class ByteBufferInputStream extends InputStream implements DataInput, jav
     }
     
     private int basicGetInt(final int pos) {
-      long addr = this.chunk.getAddressForReading(pos, 4);
+      long addr = this.chunk.getAddressForReadingData(pos, 4);
       if (unaligned) {
-        int result = UnsafeMemoryChunk.readAbsoluteInt(addr);
+        int result = AddressableMemoryManager.readInt(addr);
         if (ByteOrder.nativeOrder() == ByteOrder.LITTLE_ENDIAN) {
           result = Integer.reverseBytes(result);
         }
         return result;
       } else {
-        byte b0 = UnsafeMemoryChunk.readAbsoluteByte(addr++);
-        byte b1 = UnsafeMemoryChunk.readAbsoluteByte(addr++);
-        byte b2 = UnsafeMemoryChunk.readAbsoluteByte(addr++);
-        byte b3 = UnsafeMemoryChunk.readAbsoluteByte(addr);
+        byte b0 = AddressableMemoryManager.readByte(addr++);
+        byte b1 = AddressableMemoryManager.readByte(addr++);
+        byte b2 = AddressableMemoryManager.readByte(addr++);
+        byte b3 = AddressableMemoryManager.readByte(addr);
         return (b0 << 24) + ((b1 & 255) << 16) + ((b2 & 255) << 8) + ((b3 & 255) << 0);
       }
     }
@@ -588,22 +588,22 @@ public class ByteBufferInputStream extends InputStream implements DataInput, jav
       return basicGetLong(pos);
     }
     private long basicGetLong(final int pos) {
-      long addr = this.chunk.getAddressForReading(pos, 8);
+      long addr = this.chunk.getAddressForReadingData(pos, 8);
       if (unaligned) {
-        long result = UnsafeMemoryChunk.readAbsoluteLong(addr);
+        long result = AddressableMemoryManager.readLong(addr);
         if (ByteOrder.nativeOrder() == ByteOrder.LITTLE_ENDIAN) {
           result = Long.reverseBytes(result);
         }
         return result;
       } else {
-        byte b0 = UnsafeMemoryChunk.readAbsoluteByte(addr++);
-        byte b1 = UnsafeMemoryChunk.readAbsoluteByte(addr++);
-        byte b2 = UnsafeMemoryChunk.readAbsoluteByte(addr++);
-        byte b3 = UnsafeMemoryChunk.readAbsoluteByte(addr++);
-        byte b4 = UnsafeMemoryChunk.readAbsoluteByte(addr++);
-        byte b5 = UnsafeMemoryChunk.readAbsoluteByte(addr++);
-        byte b6 = UnsafeMemoryChunk.readAbsoluteByte(addr++);
-        byte b7 = UnsafeMemoryChunk.readAbsoluteByte(addr);
+        byte b0 = AddressableMemoryManager.readByte(addr++);
+        byte b1 = AddressableMemoryManager.readByte(addr++);
+        byte b2 = AddressableMemoryManager.readByte(addr++);
+        byte b3 = AddressableMemoryManager.readByte(addr++);
+        byte b4 = AddressableMemoryManager.readByte(addr++);
+        byte b5 = AddressableMemoryManager.readByte(addr++);
+        byte b6 = AddressableMemoryManager.readByte(addr++);
+        byte b7 = AddressableMemoryManager.readByte(addr);
         return (((long)b0 << 56) +
             ((long)(b1 & 255) << 48) +
             ((long)(b2 & 255) << 40) +
@@ -724,7 +724,7 @@ public class ByteBufferInputStream extends InputStream implements DataInput, jav
     this.buffer = copy.buffer.duplicate();
   }
 
-  public ByteBufferInputStream(ObjectChunk blob) {
+  public ByteBufferInputStream(StoredObject blob) {
     this.buffer = ByteSourceFactory.create(blob);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/geode-core/src/main/java/com/gemstone/gemfire/internal/tcp/ImmutableByteBufferInputStream.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/tcp/ImmutableByteBufferInputStream.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/tcp/ImmutableByteBufferInputStream.java
index d632158..ff0871a 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/tcp/ImmutableByteBufferInputStream.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/tcp/ImmutableByteBufferInputStream.java
@@ -18,7 +18,7 @@ package com.gemstone.gemfire.internal.tcp;
 
 import java.nio.ByteBuffer;
 
-import com.gemstone.gemfire.internal.offheap.ObjectChunk;
+import com.gemstone.gemfire.internal.offheap.StoredObject;
 
 /**
  * You should only create an instance of this class if the bytes this buffer reads
@@ -67,7 +67,7 @@ public class ImmutableByteBufferInputStream extends ByteBufferInputStream {
     // for serialization
   }
   
-  public ImmutableByteBufferInputStream(ObjectChunk blob) {
+  public ImmutableByteBufferInputStream(StoredObject blob) {
     super(blob);
   }
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/geode-core/src/main/java/com/gemstone/gemfire/internal/util/BlobHelper.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/util/BlobHelper.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/util/BlobHelper.java
index 40015a4..28252c3 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/util/BlobHelper.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/util/BlobHelper.java
@@ -27,7 +27,7 @@ import com.gemstone.gemfire.internal.DSCODE;
 import com.gemstone.gemfire.internal.HeapDataOutputStream;
 import com.gemstone.gemfire.internal.Version;
 import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
-import com.gemstone.gemfire.internal.offheap.ObjectChunk;
+import com.gemstone.gemfire.internal.offheap.StoredObject;
 import com.gemstone.gemfire.internal.offheap.annotations.Unretained;
 import com.gemstone.gemfire.pdx.internal.PdxInputStream;
 
@@ -140,7 +140,7 @@ public class BlobHelper {
    * If a PdxInstance is returned then it will refer to Chunk's off-heap memory
    * with an unretained reference.
    */
-  public static @Unretained Object deserializeOffHeapBlob(ObjectChunk blob) throws IOException, ClassNotFoundException {
+  public static @Unretained Object deserializeOffHeapBlob(StoredObject blob) throws IOException, ClassNotFoundException {
     Object result;
     final long start = startDeserialization();
     // For both top level and nested pdxs we just want a reference to this off-heap blob.

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/geode-core/src/main/java/com/gemstone/gemfire/pdx/internal/PdxInputStream.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/pdx/internal/PdxInputStream.java b/geode-core/src/main/java/com/gemstone/gemfire/pdx/internal/PdxInputStream.java
index 4a5a9df..85c6cd5 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/pdx/internal/PdxInputStream.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/pdx/internal/PdxInputStream.java
@@ -26,7 +26,7 @@ import java.util.Date;
 import com.gemstone.gemfire.DataSerializer;
 import com.gemstone.gemfire.InternalGemFireException;
 import com.gemstone.gemfire.pdx.PdxSerializationException;
-import com.gemstone.gemfire.internal.offheap.ObjectChunk;
+import com.gemstone.gemfire.internal.offheap.StoredObject;
 import com.gemstone.gemfire.internal.tcp.ByteBufferInputStream;
 import com.gemstone.gemfire.internal.tcp.ImmutableByteBufferInputStream;
 
@@ -76,7 +76,7 @@ public class PdxInputStream extends ImmutableByteBufferInputStream {
     // for serialization
   }
 
-  public PdxInputStream(ObjectChunk blob) {
+  public PdxInputStream(StoredObject blob) {
     super(blob);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/geode-core/src/test/java/com/gemstone/gemfire/cache30/MultiVMRegionTestCase.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache30/MultiVMRegionTestCase.java b/geode-core/src/test/java/com/gemstone/gemfire/cache30/MultiVMRegionTestCase.java
index 7a4d09e..dce68cf 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache30/MultiVMRegionTestCase.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache30/MultiVMRegionTestCase.java
@@ -110,8 +110,8 @@ import com.gemstone.gemfire.internal.cache.versions.RegionVersionVector;
 import com.gemstone.gemfire.internal.cache.versions.VMRegionVersionVector;
 import com.gemstone.gemfire.internal.cache.versions.VersionTag;
 import com.gemstone.gemfire.internal.logging.LogService;
-import com.gemstone.gemfire.internal.offheap.MemoryChunkWithRefCount;
 import com.gemstone.gemfire.internal.offheap.SimpleMemoryAllocatorImpl;
+import com.gemstone.gemfire.internal.offheap.StoredObject;
 import com.gemstone.gemfire.test.dunit.AsyncInvocation;
 import com.gemstone.gemfire.test.dunit.DistributedTestCase;
 import com.gemstone.gemfire.test.dunit.DistributedTestUtils;
@@ -2004,8 +2004,8 @@ public abstract class MultiVMRegionTestCase extends RegionTestCase {
               LocalRegion reRegion;
               reRegion = (LocalRegion) region;
               RegionEntry re = reRegion.getRegionEntry(key2);
-              MemoryChunkWithRefCount mc = (MemoryChunkWithRefCount) re._getValue();
-              assertEquals(1, mc.getRefCount());
+              StoredObject so = (StoredObject) re._getValue();
+              assertEquals(1, so.getRefCount());
               assertEquals(1, ma.getStats().getObjects());
             }
           }
@@ -2091,8 +2091,8 @@ public abstract class MultiVMRegionTestCase extends RegionTestCase {
             assertEquals(2, ma.getStats().getObjects());
             LocalRegion reRegion;
             reRegion = (LocalRegion) region;
-            MemoryChunkWithRefCount mc = (MemoryChunkWithRefCount) reRegion.getRegionEntry(key)._getValue();
-            assertEquals(1, mc.getRefCount());
+            StoredObject so = (StoredObject) reRegion.getRegionEntry(key)._getValue();
+            assertEquals(1, so.getRefCount());
           }
         }
       });
@@ -2157,8 +2157,8 @@ public abstract class MultiVMRegionTestCase extends RegionTestCase {
               assertEquals(2, ma.getStats().getObjects());
               LocalRegion reRegion;
               reRegion = (LocalRegion) region;
-              MemoryChunkWithRefCount mc = (MemoryChunkWithRefCount) reRegion.getRegionEntry(key)._getValue();
-              assertEquals(1, mc.getRefCount());
+              StoredObject so = (StoredObject) reRegion.getRegionEntry(key)._getValue();
+              assertEquals(1, so.getRefCount());
             }
           }
         }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/ChunkValueWrapperJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/ChunkValueWrapperJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/ChunkValueWrapperJUnitTest.java
deleted file mode 100644
index b69f82e..0000000
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/ChunkValueWrapperJUnitTest.java
+++ /dev/null
@@ -1,188 +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.cache;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import com.gemstone.gemfire.internal.cache.DiskEntry.Helper.ChunkValueWrapper;
-import com.gemstone.gemfire.internal.cache.DiskEntry.Helper.Flushable;
-import com.gemstone.gemfire.internal.offheap.ObjectChunk;
-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.UnsafeMemoryChunk;
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
-
-@Category(UnitTest.class)
-public class ChunkValueWrapperJUnitTest {
-
-  private static ChunkValueWrapper createChunkValueWrapper(byte[] bytes, boolean isSerialized) {
-    ObjectChunk c = (ObjectChunk)SimpleMemoryAllocatorImpl.getAllocator().allocateAndInitialize(bytes, isSerialized, false);
-    return new ChunkValueWrapper(c);
-  }
-
-  @Before
-  public void setUp() throws Exception {
-    SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new UnsafeMemoryChunk[]{new UnsafeMemoryChunk(1024*1024)});
-  }
-
-  @After
-  public void tearDown() throws Exception {
-    SimpleMemoryAllocatorImpl.freeOffHeapMemory();
-  }
-
-  @Test
-  public void testIsSerialized() {
-    assertEquals(true, createChunkValueWrapper(new byte[16], true).isSerialized());
-    assertEquals(false, createChunkValueWrapper(new byte[16], false).isSerialized());
-  }
-  
-  @Test
-  public void testGetUserBits() {
-    assertEquals((byte)1, createChunkValueWrapper(new byte[16], true).getUserBits());
-    assertEquals((byte)0, createChunkValueWrapper(new byte[16], false).getUserBits());
-  }
-  
-  @Test
-  public void testGetLength() {
-    assertEquals(32, createChunkValueWrapper(new byte[32], true).getLength());
-    assertEquals(17, createChunkValueWrapper(new byte[17], false).getLength());
-  }
-  
-  @Test
-  public void testGetBytesAsString() {
-    assertEquals("byte[0, 0, 0, 0, 0, 0, 0, 0]", createChunkValueWrapper(new byte[8], false).getBytesAsString());
-  }
-  
-  @Test
-  public void testSendTo() throws IOException {
-    final ByteBuffer bb = ByteBuffer.allocateDirect(18);
-    bb.limit(8);
-    ChunkValueWrapper vw = createChunkValueWrapper(new byte[]{1,2,3,4,5,6,7,8}, false);
-    vw.sendTo(bb, new Flushable() {
-      @Override
-      public void flush() throws IOException {
-        fail("should not have been called");
-      }
-
-      @Override
-      public void flush(ByteBuffer bb, ByteBuffer chunkbb) throws IOException {
-        fail("should not have been called");
-      }
-    });
-    assertEquals(8, bb.position());
-    bb.flip();
-    assertEquals(1, bb.get());
-    assertEquals(2, bb.get());
-    assertEquals(3, bb.get());
-    assertEquals(4, bb.get());
-    assertEquals(5, bb.get());
-    assertEquals(6, bb.get());
-    assertEquals(7, bb.get());
-    assertEquals(8, bb.get());
-    
-    bb.clear();
-    bb.limit(8);
-    vw = createChunkValueWrapper(new byte[]{1,2,3,4,5,6,7,8,9}, false);
-    final int[] flushCalls = new int[1];
-    vw.sendTo(bb, new Flushable() {
-      @Override
-      public void flush() throws IOException {
-        if (flushCalls[0] != 0) {
-          fail("expected flush to only be called once");
-        }
-        flushCalls[0]++;
-        assertEquals(8, bb.position());
-        for (int i=0; i < 8; i++) {
-          assertEquals(i+1, bb.get(i));
-        }
-        bb.clear();
-        bb.limit(8);
-      }
-
-      @Override
-      public void flush(ByteBuffer bb, ByteBuffer chunkbb) throws IOException {
-        fail("should not have been called");
-      }
-    });
-    assertEquals(1, bb.position());
-    bb.flip();
-    assertEquals(9, bb.get());
-    
-    bb.clear();
-    bb.limit(8);
-    flushCalls[0] = 0;
-    vw = createChunkValueWrapper(new byte[]{1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17}, false);
-    vw.sendTo(bb, new Flushable() {
-      @Override
-      public void flush() throws IOException {
-        if (flushCalls[0] > 1) {
-          fail("expected flush to only be called twice");
-        }
-        assertEquals(8, bb.position());
-        for (int i=0; i < 8; i++) {
-          assertEquals((flushCalls[0]*8)+i+1, bb.get(i));
-        }
-        flushCalls[0]++;
-        bb.clear();
-        bb.limit(8);
-      }
-
-      @Override
-      public void flush(ByteBuffer bb, ByteBuffer chunkbb) throws IOException {
-        fail("should not have been called");
-      }
-    });
-    assertEquals(1, bb.position());
-    bb.flip();
-    assertEquals(17, bb.get());
-    
-    // now test with a chunk that will not fit in bb.
-    bb.clear();
-    flushCalls[0] = 0;
-    bb.put((byte) 0);
-    vw = createChunkValueWrapper(new byte[]{1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19}, false);
-    vw.sendTo(bb, new Flushable() {
-      @Override
-      public void flush() throws IOException {
-        fail("should not have been called");
-      }
-
-      @Override
-      public void flush(ByteBuffer bb, ByteBuffer chunkbb) throws IOException {
-        flushCalls[0]++;
-        assertEquals(1, bb.position());
-        bb.flip();
-        assertEquals(0, bb.get());
-        assertEquals(19, chunkbb.remaining());
-        for (int i=1; i <= 19; i++) {
-          assertEquals(i, chunkbb.get());
-        }
-      }
-    });
-    assertEquals(1, flushCalls[0]);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/OffHeapTestUtil.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/OffHeapTestUtil.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/OffHeapTestUtil.java
index cbf3bf6..8fd6895 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/OffHeapTestUtil.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/OffHeapTestUtil.java
@@ -52,7 +52,7 @@ public class OffHeapTestUtil {
     }
     
     if(orphans != null && ! orphans.isEmpty()) {
-      List<RefCountChangeInfo> info = ReferenceCountHelper.getRefCountInfo(orphans.get(0).getMemoryAddress());
+      List<RefCountChangeInfo> info = ReferenceCountHelper.getRefCountInfo(orphans.get(0).getAddress());
       System.out.println("FOUND ORPHAN!!");
       System.out.println("Sample orphan: " + orphans.get(0));
       System.out.println("Orphan info: " + info);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/OffHeapValueWrapperJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/OffHeapValueWrapperJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/OffHeapValueWrapperJUnitTest.java
new file mode 100644
index 0000000..0829009
--- /dev/null
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/OffHeapValueWrapperJUnitTest.java
@@ -0,0 +1,188 @@
+/*
+ * 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.cache;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.gemstone.gemfire.internal.cache.DiskEntry.Helper.OffHeapValueWrapper;
+import com.gemstone.gemfire.internal.cache.DiskEntry.Helper.Flushable;
+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.SlabImpl;
+import com.gemstone.gemfire.internal.offheap.StoredObject;
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+
+@Category(UnitTest.class)
+public class OffHeapValueWrapperJUnitTest {
+
+  private static OffHeapValueWrapper createChunkValueWrapper(byte[] bytes, boolean isSerialized) {
+    StoredObject c = SimpleMemoryAllocatorImpl.getAllocator().allocateAndInitialize(bytes, isSerialized, false);
+    return new OffHeapValueWrapper(c);
+  }
+
+  @Before
+  public void setUp() throws Exception {
+    SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new SlabImpl[]{new SlabImpl(1024*1024)});
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    SimpleMemoryAllocatorImpl.freeOffHeapMemory();
+  }
+
+  @Test
+  public void testIsSerialized() {
+    assertEquals(true, createChunkValueWrapper(new byte[16], true).isSerialized());
+    assertEquals(false, createChunkValueWrapper(new byte[16], false).isSerialized());
+  }
+  
+  @Test
+  public void testGetUserBits() {
+    assertEquals((byte)1, createChunkValueWrapper(new byte[16], true).getUserBits());
+    assertEquals((byte)0, createChunkValueWrapper(new byte[16], false).getUserBits());
+  }
+  
+  @Test
+  public void testGetLength() {
+    assertEquals(32, createChunkValueWrapper(new byte[32], true).getLength());
+    assertEquals(17, createChunkValueWrapper(new byte[17], false).getLength());
+  }
+  
+  @Test
+  public void testGetBytesAsString() {
+    assertEquals("byte[0, 0, 0, 0, 0, 0, 0, 0]", createChunkValueWrapper(new byte[8], false).getBytesAsString());
+  }
+  
+  @Test
+  public void testSendTo() throws IOException {
+    final ByteBuffer bb = ByteBuffer.allocateDirect(18);
+    bb.limit(8);
+    OffHeapValueWrapper vw = createChunkValueWrapper(new byte[]{1,2,3,4,5,6,7,8}, false);
+    vw.sendTo(bb, new Flushable() {
+      @Override
+      public void flush() throws IOException {
+        fail("should not have been called");
+      }
+
+      @Override
+      public void flush(ByteBuffer bb, ByteBuffer chunkbb) throws IOException {
+        fail("should not have been called");
+      }
+    });
+    assertEquals(8, bb.position());
+    bb.flip();
+    assertEquals(1, bb.get());
+    assertEquals(2, bb.get());
+    assertEquals(3, bb.get());
+    assertEquals(4, bb.get());
+    assertEquals(5, bb.get());
+    assertEquals(6, bb.get());
+    assertEquals(7, bb.get());
+    assertEquals(8, bb.get());
+    
+    bb.clear();
+    bb.limit(8);
+    vw = createChunkValueWrapper(new byte[]{1,2,3,4,5,6,7,8,9}, false);
+    final int[] flushCalls = new int[1];
+    vw.sendTo(bb, new Flushable() {
+      @Override
+      public void flush() throws IOException {
+        if (flushCalls[0] != 0) {
+          fail("expected flush to only be called once");
+        }
+        flushCalls[0]++;
+        assertEquals(8, bb.position());
+        for (int i=0; i < 8; i++) {
+          assertEquals(i+1, bb.get(i));
+        }
+        bb.clear();
+        bb.limit(8);
+      }
+
+      @Override
+      public void flush(ByteBuffer bb, ByteBuffer chunkbb) throws IOException {
+        fail("should not have been called");
+      }
+    });
+    assertEquals(1, bb.position());
+    bb.flip();
+    assertEquals(9, bb.get());
+    
+    bb.clear();
+    bb.limit(8);
+    flushCalls[0] = 0;
+    vw = createChunkValueWrapper(new byte[]{1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17}, false);
+    vw.sendTo(bb, new Flushable() {
+      @Override
+      public void flush() throws IOException {
+        if (flushCalls[0] > 1) {
+          fail("expected flush to only be called twice");
+        }
+        assertEquals(8, bb.position());
+        for (int i=0; i < 8; i++) {
+          assertEquals((flushCalls[0]*8)+i+1, bb.get(i));
+        }
+        flushCalls[0]++;
+        bb.clear();
+        bb.limit(8);
+      }
+
+      @Override
+      public void flush(ByteBuffer bb, ByteBuffer chunkbb) throws IOException {
+        fail("should not have been called");
+      }
+    });
+    assertEquals(1, bb.position());
+    bb.flip();
+    assertEquals(17, bb.get());
+    
+    // now test with a chunk that will not fit in bb.
+    bb.clear();
+    flushCalls[0] = 0;
+    bb.put((byte) 0);
+    vw = createChunkValueWrapper(new byte[]{1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19}, false);
+    vw.sendTo(bb, new Flushable() {
+      @Override
+      public void flush() throws IOException {
+        fail("should not have been called");
+      }
+
+      @Override
+      public void flush(ByteBuffer bb, ByteBuffer chunkbb) throws IOException {
+        flushCalls[0]++;
+        assertEquals(1, bb.position());
+        bb.flip();
+        assertEquals(0, bb.get());
+        assertEquals(19, chunkbb.remaining());
+        for (int i=1; i <= 19; i++) {
+          assertEquals(i, chunkbb.get());
+        }
+      }
+    });
+    assertEquals(1, flushCalls[0]);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/OldValueImporterTestBase.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/OldValueImporterTestBase.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/OldValueImporterTestBase.java
index 690b55a..84d7fc7 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/OldValueImporterTestBase.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/OldValueImporterTestBase.java
@@ -26,12 +26,12 @@ import org.junit.Test;
 
 import com.gemstone.gemfire.internal.HeapDataOutputStream;
 import com.gemstone.gemfire.internal.cache.EntryEventImpl.OldValueImporter;
-import com.gemstone.gemfire.internal.offheap.ObjectChunk;
-import com.gemstone.gemfire.internal.offheap.DataAsAddress;
+import com.gemstone.gemfire.internal.offheap.OffHeapStoredObject;
+import com.gemstone.gemfire.internal.offheap.TinyStoredObject;
 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.UnsafeMemoryChunk;
+import com.gemstone.gemfire.internal.offheap.SlabImpl;
 import com.gemstone.gemfire.internal.util.BlobHelper;
 
 public abstract class OldValueImporterTestBase {
@@ -110,10 +110,10 @@ public abstract class OldValueImporterTestBase {
     // off-heap DataAsAddress byte array
     {
       SimpleMemoryAllocatorImpl sma =
-          SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new UnsafeMemoryChunk[]{new UnsafeMemoryChunk(1024*1024)});
+          SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new SlabImpl[]{new SlabImpl(1024*1024)});
       try {
         byte[] baValue = new byte[] {1,2};
-        DataAsAddress baValueSO = (DataAsAddress) sma.allocateAndInitialize(baValue, false, false);
+        TinyStoredObject baValueSO = (TinyStoredObject) sma.allocateAndInitialize(baValue, false, false);
         OldValueImporter omsg = createImporter();
         omsg.importOldObject(baValueSO, false);
         hdos = new HeapDataOutputStream(bytes);
@@ -127,10 +127,10 @@ public abstract class OldValueImporterTestBase {
     // off-heap Chunk byte array
     {
       SimpleMemoryAllocatorImpl sma =
-          SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new UnsafeMemoryChunk[]{new UnsafeMemoryChunk(1024*1024)});
+          SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new SlabImpl[]{new SlabImpl(1024*1024)});
       try {
         byte[] baValue = new byte[] {1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17};
-        ObjectChunk baValueSO = (ObjectChunk) sma.allocateAndInitialize(baValue, false, false);
+        OffHeapStoredObject baValueSO = (OffHeapStoredObject) sma.allocateAndInitialize(baValue, false, false);
         OldValueImporter omsg = createImporter();
         omsg.importOldObject(baValueSO, false);
         hdos = new HeapDataOutputStream(bytes);
@@ -144,11 +144,11 @@ public abstract class OldValueImporterTestBase {
     // off-heap DataAsAddress String
     {
       SimpleMemoryAllocatorImpl sma =
-          SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new UnsafeMemoryChunk[]{new UnsafeMemoryChunk(1024*1024)});
+          SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new SlabImpl[]{new SlabImpl(1024*1024)});
       try {
         String baValue = "12";
         byte[] baValueBlob = BlobHelper.serializeToBlob(baValue);
-        DataAsAddress baValueSO = (DataAsAddress) sma.allocateAndInitialize(baValueBlob, true, false);
+        TinyStoredObject baValueSO = (TinyStoredObject) sma.allocateAndInitialize(baValueBlob, true, false);
         OldValueImporter omsg = createImporter();
         omsg.importOldObject(baValueSO, true);
         hdos = new HeapDataOutputStream(bytes);
@@ -162,11 +162,11 @@ public abstract class OldValueImporterTestBase {
     // off-heap Chunk String
     {
       SimpleMemoryAllocatorImpl sma =
-          SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new UnsafeMemoryChunk[]{new UnsafeMemoryChunk(1024*1024)});
+          SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new SlabImpl[]{new SlabImpl(1024*1024)});
       try {
         String baValue = "12345678";
         byte[] baValueBlob = BlobHelper.serializeToBlob(baValue);
-        ObjectChunk baValueSO = (ObjectChunk) sma.allocateAndInitialize(baValueBlob, true, false);
+        OffHeapStoredObject baValueSO = (OffHeapStoredObject) sma.allocateAndInitialize(baValueBlob, true, false);
         OldValueImporter omsg = createImporter();
         omsg.importOldObject(baValueSO, true);
         hdos = new HeapDataOutputStream(bytes);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/MessageJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/MessageJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/MessageJUnitTest.java
index b7bd47a..8caf3f6 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/MessageJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/MessageJUnitTest.java
@@ -28,7 +28,6 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import com.gemstone.gemfire.internal.Version;
-import com.gemstone.gemfire.internal.offheap.HeapByteBufferMemoryChunkJUnitTest;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
 @Category(UnitTest.class)

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


[7/8] incubator-geode git commit: GEODE-982: refactor off-heap

Posted by ds...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/SearchLoadAndWriteProcessor.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/SearchLoadAndWriteProcessor.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/SearchLoadAndWriteProcessor.java
index e96d99d..4dba6c5 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/SearchLoadAndWriteProcessor.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/SearchLoadAndWriteProcessor.java
@@ -74,7 +74,6 @@ import com.gemstone.gemfire.internal.cache.versions.VersionTag;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 import com.gemstone.gemfire.internal.logging.LogService;
 import com.gemstone.gemfire.internal.logging.log4j.LocalizedMessage;
-import com.gemstone.gemfire.internal.offheap.StoredObject;
 
 
 /**
@@ -1938,13 +1937,14 @@ public class SearchLoadAndWriteProcessor implements MembershipListener {
                       long lastModified = entry.getLastModified();
                       lastModifiedCacheTime = lastModified;
                       if (eov instanceof CachedDeserializable) {
-                        if (eov instanceof StoredObject && !((StoredObject) eov).isSerialized()) {
+                        CachedDeserializable cd = (CachedDeserializable) eov;
+                        if (!cd.isSerialized()) {
                           isSer = false;
-                          ebv = (byte[]) ((StoredObject)eov).getDeserializedForReading();
+                          ebv = (byte[]) cd.getDeserializedForReading();
                           ebvLen = ebv.length;
                         } else {
                           // don't serialize here if it is not already serialized
-                          Object tmp = ((CachedDeserializable)eov).getValue();
+                          Object tmp = cd.getValue();
                           if (tmp instanceof byte[]) {
                             byte[] bb = (byte[])tmp;
                             ebv = bb;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/StoreAllCachedDeserializable.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/StoreAllCachedDeserializable.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/StoreAllCachedDeserializable.java
index 7d94281..3519e72 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/StoreAllCachedDeserializable.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/StoreAllCachedDeserializable.java
@@ -153,7 +153,16 @@ public class StoreAllCachedDeserializable implements CachedDeserializable, DataS
 
   @Override
   public Version[] getSerializationVersions() {
-    // TODO Auto-generated method stub
     return null;
   }
+
+  @Override
+  public boolean isSerialized() {
+    return true;
+  }
+
+  @Override
+  public boolean usesHeapForStorage() {
+    return true;
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/VMCachedDeserializable.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/VMCachedDeserializable.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/VMCachedDeserializable.java
index c5d3c6e..b96f6d1 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/VMCachedDeserializable.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/VMCachedDeserializable.java
@@ -256,8 +256,15 @@ public final class VMCachedDeserializable implements CachedDeserializable, DataS
   }
   @Override
   public Version[] getSerializationVersions() {
-    // TODO Auto-generated method stub
     return null;
   }
+  @Override
+  public boolean isSerialized() {
+    return true;
+  }
+  @Override
+  public boolean usesHeapForStorage() {
+    return true;
+  }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/compression/SnappyCompressedCachedDeserializable.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/compression/SnappyCompressedCachedDeserializable.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/compression/SnappyCompressedCachedDeserializable.java
index 05b65dd..6c6ec6f 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/compression/SnappyCompressedCachedDeserializable.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/compression/SnappyCompressedCachedDeserializable.java
@@ -80,4 +80,14 @@ public class SnappyCompressedCachedDeserializable extends
   public Version[] getSerializationVersions() {
     return null;
   }
+
+  @Override
+  public boolean isSerialized() {
+    return true;
+  }
+
+  @Override
+  public boolean usesHeapForStorage() {
+    return true;
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/PutMessage.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/PutMessage.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/PutMessage.java
index c2ff411..d6f5293 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/PutMessage.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/PutMessage.java
@@ -67,7 +67,6 @@ import com.gemstone.gemfire.internal.cache.versions.VersionTag;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 import com.gemstone.gemfire.internal.logging.LogService;
 import com.gemstone.gemfire.internal.logging.log4j.LogMarker;
-import com.gemstone.gemfire.internal.offheap.StoredObject;
 import com.gemstone.gemfire.internal.offheap.annotations.Unretained;
 import com.gemstone.gemfire.internal.util.BlobHelper;
 
@@ -231,10 +230,11 @@ public final class PutMessage extends PartitionMessageWithDirectReply implements
     }
     else{
       if(original.valObj instanceof CachedDeserializable) {
-        if (original.valObj instanceof StoredObject && !((StoredObject)original.valObj).isSerialized()) {
-          this.valObj = ((StoredObject)original.valObj).getDeserializedForReading();
+        CachedDeserializable cd = (CachedDeserializable) original.valObj;
+        if (!cd.isSerialized()) {
+          this.valObj = cd.getDeserializedForReading();
         } else {
-          Object val = ((CachedDeserializable) original.valObj).getValue();
+          Object val = cd.getValue();
           if(val instanceof byte[]) {
             this.valBytes = (byte[]) val; 
           } else {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/Part.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/Part.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/Part.java
index 80b5c0a..2c36b32 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/Part.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/Part.java
@@ -17,11 +17,8 @@
 package com.gemstone.gemfire.internal.cache.tier.sockets;
 
 import com.gemstone.gemfire.internal.*;
-import com.gemstone.gemfire.internal.cache.CachedDeserializable;
-import com.gemstone.gemfire.internal.offheap.ObjectChunk;
-import com.gemstone.gemfire.internal.offheap.DataAsAddress;
+import com.gemstone.gemfire.internal.offheap.AddressableMemoryManager;
 import com.gemstone.gemfire.internal.offheap.StoredObject;
-import com.gemstone.gemfire.internal.offheap.UnsafeMemoryChunk;
 
 import java.io.*;
 import java.nio.*;
@@ -121,17 +118,17 @@ public class Part {
   public void setPartState(StoredObject so, boolean isObject) {
     if (isObject) {
       this.typeCode = OBJECT_CODE;
-    } else if (so.getValueSizeInBytes() == 0) {
+    } else if (so.getDataSize() == 0) {
       this.typeCode = EMPTY_BYTEARRAY_CODE;
       this.part = EMPTY_BYTE_ARRAY;
       return;
     } else {
       this.typeCode = BYTE_CODE;
     }
-    if (so instanceof DataAsAddress) {
-      this.part = ((DataAsAddress)so).getRawBytes();
+    if (so.hasRefCount()) {
+      this.part = so;
     } else {
-      this.part = (ObjectChunk)so;
+      this.part = so.getValueAsHeapByteArray();
     }
   }
   public byte getTypeCode() {
@@ -146,8 +143,8 @@ public class Part {
       return 0;
     } else if (this.part instanceof byte[]) {
       return ((byte[])this.part).length;
-    } else if (this.part instanceof ObjectChunk) {
-      return ((ObjectChunk) this.part).getValueSizeInBytes();
+    } else if (this.part instanceof StoredObject) {
+      return ((StoredObject) this.part).getDataSize();
     } else {
       return ((HeapDataOutputStream)this.part).size();
     }
@@ -289,19 +286,19 @@ public class Part {
       if (this.part instanceof byte[]) {
         byte[] bytes = (byte[])this.part;
         out.write(bytes, 0, bytes.length);
-      } else if (this.part instanceof ObjectChunk) {
-        ObjectChunk c = (ObjectChunk) this.part;
-        ByteBuffer cbb = c.createDirectByteBuffer();
-        if (cbb != null) {
-          HeapDataOutputStream.writeByteBufferToStream(out,  buf, cbb);
+      } else if (this.part instanceof StoredObject) {
+        StoredObject so = (StoredObject) this.part;
+        ByteBuffer sobb = so.createDirectByteBuffer();
+        if (sobb != null) {
+          HeapDataOutputStream.writeByteBufferToStream(out,  buf, sobb);
         } else {
-          int bytesToSend = c.getDataSize();
-          long addr = c.getAddressForReading(0, bytesToSend);
+          int bytesToSend = so.getDataSize();
+          long addr = so.getAddressForReadingData(0, bytesToSend);
           while (bytesToSend > 0) {
             if (buf.remaining() == 0) {
               HeapDataOutputStream.flushStream(out,  buf);
             }
-            buf.put(UnsafeMemoryChunk.readAbsoluteByte(addr));
+            buf.put(AddressableMemoryManager.readByte(addr));
             addr++;
             bytesToSend--;
           }
@@ -322,16 +319,16 @@ public class Part {
     if (getLength() > 0) {
       if (this.part instanceof byte[]) {
         buf.put((byte[])this.part);
-      } else if (this.part instanceof ObjectChunk) {
-        ObjectChunk c = (ObjectChunk) this.part;
+      } else if (this.part instanceof StoredObject) {
+        StoredObject c = (StoredObject) this.part;
         ByteBuffer bb = c.createDirectByteBuffer();
         if (bb != null) {
           buf.put(bb);
         } else {
           int bytesToSend = c.getDataSize();
-          long addr = c.getAddressForReading(0, bytesToSend);
+          long addr = c.getAddressForReadingData(0, bytesToSend);
           while (bytesToSend > 0) {
-            buf.put(UnsafeMemoryChunk.readAbsoluteByte(addr));
+            buf.put(AddressableMemoryManager.readByte(addr));
             addr++;
             bytesToSend--;
           }
@@ -372,10 +369,10 @@ public class Part {
           }
           buf.clear();
         }
-      } else if (this.part instanceof ObjectChunk) {
-        // instead of copying the Chunk to buf try to create a direct ByteBuffer and
+      } else if (this.part instanceof StoredObject) {
+        // instead of copying the StoredObject to buf try to create a direct ByteBuffer and
         // just write it directly to the socket channel.
-        ObjectChunk c = (ObjectChunk) this.part;
+        StoredObject c = (StoredObject) this.part;
         ByteBuffer bb = c.createDirectByteBuffer();
         if (bb != null) {
           while (bb.remaining() > 0) {
@@ -383,7 +380,7 @@ public class Part {
           }
         } else {
           int len = c.getDataSize();
-          long addr = c.getAddressForReading(0, len);
+          long addr = c.getAddressForReadingData(0, len);
           buf.clear();
           while (len > 0) {
             int bytesThisTime = len;
@@ -392,7 +389,7 @@ public class Part {
             }
             len -= bytesThisTime;
             while (bytesThisTime > 0) {
-              buf.put(UnsafeMemoryChunk.readAbsoluteByte(addr));
+              buf.put(AddressableMemoryManager.readByte(addr));
               addr++;
               bytesThisTime--;
             }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/Get70.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/Get70.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/Get70.java
index b87903e..e382c57 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/Get70.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/Get70.java
@@ -42,7 +42,6 @@ import com.gemstone.gemfire.internal.cache.tier.sockets.ServerConnection;
 import com.gemstone.gemfire.internal.cache.versions.VersionTag;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 import com.gemstone.gemfire.internal.offheap.OffHeapHelper;
-import com.gemstone.gemfire.internal.offheap.StoredObject;
 import com.gemstone.gemfire.internal.offheap.annotations.Retained;
 import com.gemstone.gemfire.internal.offheap.annotations.Unretained;
 import com.gemstone.gemfire.internal.security.AuthorizeRequest;
@@ -321,12 +320,13 @@ public class Get70 extends BaseCommand {
     // disk. If it is already a byte[], set isObject to false.
     boolean wasInvalid = false;
     if (data instanceof CachedDeserializable) {
-      if (data instanceof StoredObject && !((StoredObject) data).isSerialized()) {
+      CachedDeserializable cd = (CachedDeserializable) data;
+      if (!cd.isSerialized()) {
         // it is a byte[]
         isObject = false;
-        data = ((StoredObject) data).getDeserializedForReading();
+        data = cd.getDeserializedForReading();
       } else {
-        data = ((CachedDeserializable)data).getValue();
+        data = cd.getValue();
       }
     }
     else if (data == Token.REMOVED_PHASE1 || data == Token.REMOVED_PHASE2 || data == Token.DESTROYED) {
@@ -391,13 +391,10 @@ public class Get70 extends BaseCommand {
     else if (data instanceof byte[]) {
       isObject = false;
     } else if (data instanceof CachedDeserializable) {
-      if (data instanceof StoredObject) {
-        // it is off-heap so do not unwrap it.
-        if (!((StoredObject) data).isSerialized()) {
-          isObject = false;
-        }
-      } else {
-        data = ((CachedDeserializable)data).getValue();
+      CachedDeserializable cd = (CachedDeserializable) data;
+      isObject = cd.isSerialized();
+      if (cd.usesHeapForStorage()) {
+        data = cd.getValue();
       }
     }
     Entry result = new Entry();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/Request.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/Request.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/Request.java
index 6c806c8..b40b6e5 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/Request.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/Request.java
@@ -38,7 +38,6 @@ import com.gemstone.gemfire.internal.cache.tier.sockets.Message;
 import com.gemstone.gemfire.internal.cache.tier.sockets.Part;
 import com.gemstone.gemfire.internal.cache.tier.sockets.ServerConnection;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
-import com.gemstone.gemfire.internal.offheap.StoredObject;
 import com.gemstone.gemfire.internal.security.AuthorizeRequest;
 import com.gemstone.gemfire.internal.security.AuthorizeRequestPP;
 import com.gemstone.gemfire.security.NotAuthorizedException;
@@ -253,12 +252,13 @@ public class Request extends BaseCommand {
     // disk. If it is already a byte[], set isObject to false.
     // TODO OFFHEAP: optimize
     if (data instanceof CachedDeserializable) {
-      if (data instanceof StoredObject && !((StoredObject) data).isSerialized()) {
+      CachedDeserializable cd = (CachedDeserializable) data;
+      if (!cd.isSerialized()) {
         // it is a byte[]
         isObject = false;
-        data = ((StoredObject) data).getDeserializedForReading();
+        data = cd.getDeserializedForReading();
       } else {
-        data = ((CachedDeserializable)data).getValue();
+        data = cd.getValue();
       }
     }
     else if (data == Token.REMOVED_PHASE1 || data == Token.REMOVED_PHASE2 || data == Token.TOMBSTONE || data == Token.DESTROYED) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/wan/GatewaySenderEventImpl.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/wan/GatewaySenderEventImpl.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/wan/GatewaySenderEventImpl.java
index 0e506f7..b64a654 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/wan/GatewaySenderEventImpl.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/wan/GatewaySenderEventImpl.java
@@ -50,8 +50,6 @@ import com.gemstone.gemfire.internal.cache.WrappedCallbackArgument;
 import com.gemstone.gemfire.internal.cache.lru.Sizeable;
 import com.gemstone.gemfire.internal.cache.tier.sockets.CacheServerHelper;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
-import com.gemstone.gemfire.internal.offheap.ObjectChunk;
-import com.gemstone.gemfire.internal.offheap.ObjectChunkWithHeapForm;
 import com.gemstone.gemfire.internal.offheap.OffHeapHelper;
 import com.gemstone.gemfire.internal.offheap.ReferenceCountHelper;
 import com.gemstone.gemfire.internal.offheap.Releasable;
@@ -554,11 +552,11 @@ public class GatewaySenderEventImpl implements
       result = this.substituteValue;
       if (result == null) {
       result = this.valueObj;
-      if (result instanceof ObjectChunk) {
+      if (result instanceof StoredObject && ((StoredObject) result).hasRefCount()) {
         if (this.valueObjReleased) {
           result = null;
         } else {
-          ObjectChunk ohref = (ObjectChunk) result;
+          StoredObject ohref = (StoredObject) result;
           if (!ohref.retain()) {
             result = null;
           } else if (this.valueObjReleased) {
@@ -966,9 +964,7 @@ public class GatewaySenderEventImpl implements
 //    if (so != null  && !event.hasDelta()) {
       // Since GatewaySenderEventImpl instances can live for a long time in the gateway region queue
       // we do not want the StoredObject to be one that keeps the heap form cached.
-      if (so instanceof ObjectChunkWithHeapForm) {
-        so = ((ObjectChunkWithHeapForm) so).getChunkWithoutHeapForm(); // fixes 51999
-      }
+      so = so.getStoredObjectWithoutHeapForm(); // fixes 51999
       this.valueObj = so;
       if (!so.isSerialized()) {
         this.valueIsObject = 0x00;
@@ -1280,7 +1276,7 @@ public class GatewaySenderEventImpl implements
           return this;
         }
       }
-      if (v instanceof ObjectChunk) {
+      if (v instanceof StoredObject && ((StoredObject) v).hasRefCount()) {
         try {
           return makeCopy();
         } catch (IllegalStateException ex) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/AbstractStoredObject.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/AbstractStoredObject.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/AbstractStoredObject.java
index 6dad277..22e2d55 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/AbstractStoredObject.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/AbstractStoredObject.java
@@ -21,6 +21,7 @@ import java.io.IOException;
 
 import com.gemstone.gemfire.DataSerializer;
 import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.internal.DSCODE;
 import com.gemstone.gemfire.internal.DataSerializableFixedID;
 import com.gemstone.gemfire.internal.InternalDataSerializer;
 import com.gemstone.gemfire.internal.cache.RegionEntry;
@@ -104,4 +105,27 @@ public abstract class AbstractStoredObject implements StoredObject {
     InternalDataSerializer.writeDSFIDHeader(DataSerializableFixedID.VM_CACHED_DESERIALIZABLE, out);
     sendAsByteArray(out);
   }
+  
+  @Override
+  public boolean usesHeapForStorage() {
+    return false;
+  }
+  
+  @Override
+  public boolean isSerializedPdxInstance() {
+    if (!isSerialized()) {
+      return false;
+    }
+    // TODO OFFHEAP: what if the data is compressed?
+    byte dsCode = this.readDataByte(0);
+    return dsCode == DSCODE.PDX || dsCode == DSCODE.PDX_ENUM || dsCode == DSCODE.PDX_INLINE_ENUM;
+  }
+
+  @Override
+  public StoredObject getStoredObjectWithoutHeapForm() {
+    // the only implementation that needs to override this
+    // is OffHeapStoredObjectWithHeapForm.
+    return this;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/AddressableMemoryChunk.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/AddressableMemoryChunk.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/AddressableMemoryChunk.java
deleted file mode 100644
index 7916e1f..0000000
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/AddressableMemoryChunk.java
+++ /dev/null
@@ -1,29 +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;
-
-/**
- * A memory chunk that also has an address of its memory.
- */
-public interface AddressableMemoryChunk extends MemoryChunk {
-
-  /**
-   * Return the address of the memory of this chunk.
-   */
-  public long getMemoryAddress();
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/AddressableMemoryChunkFactory.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/AddressableMemoryChunkFactory.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/AddressableMemoryChunkFactory.java
deleted file mode 100644
index fa2dd78..0000000
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/AddressableMemoryChunkFactory.java
+++ /dev/null
@@ -1,27 +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;
-
-/**
- * Used to create AddressableMemoryChunk instances.
- */
-public interface AddressableMemoryChunkFactory {
-  /** Create and return an AddressableMemoryChunk.
-   * @throws OutOfMemoryError if the create fails
-   */
-  public AddressableMemoryChunk create(int size);
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/AddressableMemoryManager.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/AddressableMemoryManager.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/AddressableMemoryManager.java
new file mode 100644
index 0000000..3b6f58d
--- /dev/null
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/AddressableMemoryManager.java
@@ -0,0 +1,261 @@
+/*
+ * 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.lang.reflect.Constructor;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.nio.ByteBuffer;
+
+import com.gemstone.gemfire.internal.SharedLibrary;
+import com.gemstone.gemfire.pdx.internal.unsafe.UnsafeWrapper;
+
+/**
+ * This class supports allocating and freeing large amounts of addressable memory
+ * (i.e. slabs). It also supports using an "address" to operate on the memory.
+ * Note that this class's implementation is currently a singleton so all the methods
+ * on it are static.
+ */
+public class AddressableMemoryManager {
+  private static final UnsafeWrapper unsafe;
+  private static final int ARRAY_BYTE_BASE_OFFSET;
+  private static final String reason;
+  static {
+    UnsafeWrapper tmp = null;
+    String tmpReason = null;
+    try {
+      tmp = new UnsafeWrapper();
+    } catch (RuntimeException ignore) {
+      tmpReason = ignore.toString();
+    } catch (Error ignore) {
+      tmpReason = ignore.toString();
+    }
+    reason = tmpReason;
+    unsafe = tmp;
+    ARRAY_BYTE_BASE_OFFSET = unsafe != null ? unsafe.arrayBaseOffset(byte[].class) : 0;
+  }
+  
+  public static long allocate(int size) {
+    if (unsafe == null) {
+      throw new OutOfMemoryError("Off-heap memory is not available because: " + reason);
+    }
+    try {
+      return unsafe.allocateMemory(size);
+    } catch (OutOfMemoryError err) {
+      String msg = "Failed creating " + size + " bytes of off-heap memory during cache creation.";
+      if (err.getMessage() != null && !err.getMessage().isEmpty()) {
+        msg += " Cause: " + err.getMessage();
+      }
+      if (!SharedLibrary.is64Bit() && size >= (1024*1024*1024)) {
+        msg += " The JVM looks like a 32-bit one. For large amounts of off-heap memory a 64-bit JVM is needed.";
+      }
+      throw new OutOfMemoryError(msg);
+    }
+  }
+
+  public static void free(long addr) {
+    unsafe.freeMemory(addr);
+  }
+  
+  public static Slab allocateSlab(int size) {
+    return new SlabImpl(size);
+  }
+
+  public static byte readByte(long addr) {
+    return unsafe.getByte(addr);
+  }
+  public static char readChar(long addr) {
+    return unsafe.getChar(null, addr);
+  }
+  public static short readShort(long addr) {
+    return unsafe.getShort(null, addr);
+  }
+  public static int readInt(long addr) {
+    return unsafe.getInt(null, addr);
+  }
+  public static int readIntVolatile(long addr) {
+    return unsafe.getIntVolatile(null, addr);
+  }
+  public static long readLong(long addr) {
+    return unsafe.getLong(null, addr);
+  }
+  public static long readLongVolatile(long addr) {
+    return unsafe.getLongVolatile(null, addr);
+  }
+  public static void writeByte(long addr, byte value) {
+    unsafe.putByte(addr, value);
+  }
+  public static void writeInt(long addr, int value) {
+    unsafe.putInt(null, addr, value);
+  }
+  public static void writeIntVolatile(long addr, int value) {
+    unsafe.putIntVolatile(null, addr, value);
+  }
+  public static boolean writeIntVolatile(long addr, int expected, int value) {
+    return unsafe.compareAndSwapInt(null, addr, expected, value);
+  }
+  public static void writeLong(long addr, long value) {
+    unsafe.putLong(null, addr, value);
+  }
+  public static void writeLongVolatile(long addr, long value) {
+    unsafe.putLongVolatile(null, addr, value);
+  }
+  public static boolean writeLongVolatile(long addr, long expected, long value) {
+    return unsafe.compareAndSwapLong(null, addr, expected, value);
+  }
+  public static void readBytes(long addr, byte[] bytes, int bytesOffset, int size) {
+    // Throwing an Error instead of using the "assert" keyword because passing < 0 to
+    // copyMemory(...) can lead to a core dump with some JVMs and we don't want to
+    // require the -ea JVM flag.
+    if (size < 0) {
+      throw new AssertionError("Size=" + size + ", but size must be >= 0");
+    }
+    
+    assert bytesOffset >= 0 : "byteOffset=" + bytesOffset;
+    assert bytesOffset + size <= bytes.length : "byteOffset=" + bytesOffset + ",size=" + size + ",bytes.length=" + bytes.length;
+    
+    if (size == 0) {
+      return; // No point in wasting time copying 0 bytes
+    }
+    unsafe.copyMemory(null, addr, bytes, ARRAY_BYTE_BASE_OFFSET+bytesOffset, size);
+  }
+  public static void copyMemory(long srcAddr, long dstAddr, long size) {
+    unsafe.copyMemory(srcAddr, dstAddr, size);
+  }
+  public static void writeBytes(long addr, byte[] bytes, int bytesOffset, int size) {
+    // Throwing an Error instead of using the "assert" keyword because passing < 0 to
+    // copyMemory(...) can lead to a core dump with some JVMs and we don't want to
+    // require the -ea JVM flag.
+    if (size < 0) {
+      throw new AssertionError("Size=" + size + ", but size must be >= 0");
+    }
+  
+    assert bytesOffset >= 0 : "byteOffset=" + bytesOffset;
+    assert bytesOffset + size <= bytes.length : "byteOffset=" + bytesOffset + ",size=" + size + ",bytes.length=" + bytes.length;
+    
+    if (size == 0) {
+      return; // No point in wasting time copying 0 bytes
+    }
+    unsafe.copyMemory(bytes, ARRAY_BYTE_BASE_OFFSET+bytesOffset, null, addr, size);
+  }
+  public static void fill(long addr, int size, byte fill) {
+    unsafe.setMemory(addr, size, fill);
+  }
+
+  @SuppressWarnings("rawtypes")
+  private static volatile Class dbbClass = null;
+  @SuppressWarnings("rawtypes")
+  private static volatile Constructor dbbCtor = null;
+  private static volatile boolean dbbCreateFailed = false;
+  private static volatile Method dbbAddressMethod = null;
+  private static volatile boolean dbbAddressFailed = false;
+
+  /**
+   * Returns the address of the Unsafe memory for the first byte of a direct ByteBuffer.
+   * If the buffer is not direct or the address can not be obtained return 0.
+   */
+  @SuppressWarnings({ "rawtypes", "unchecked" })
+  public static long getDirectByteBufferAddress(ByteBuffer bb) {
+    if (!bb.isDirect()) {
+      return 0L;
+    }
+    if (dbbAddressFailed) {
+      return 0L;
+    }
+    Method m = dbbAddressMethod;
+    if (m == null) {
+      Class c = dbbClass;
+      if (c == null) {
+        try {
+          c = Class.forName("java.nio.DirectByteBuffer");
+        } catch (ClassNotFoundException e) {
+          //throw new IllegalStateException("Could not find java.nio.DirectByteBuffer", e);
+          dbbCreateFailed = true;
+          dbbAddressFailed = true;
+          return 0L;
+        }
+        dbbClass = c;
+      }
+      try {
+        m = c.getDeclaredMethod("address");
+      } catch (NoSuchMethodException | SecurityException e) {
+        //throw new IllegalStateException("Could not get method DirectByteBuffer.address()", e);
+        dbbClass = null;
+        dbbAddressFailed = true;
+        return 0L;
+      }
+      m.setAccessible(true);
+      dbbAddressMethod = m;
+    }
+    try {
+      return (Long)m.invoke(bb);
+    } catch (IllegalAccessException | IllegalArgumentException | InvocationTargetException e) {
+      //throw new IllegalStateException("Could not create an invoke DirectByteBuffer.address()", e);
+      dbbClass = null;
+      dbbAddressMethod = null;
+      dbbAddressFailed = true;
+      return 0L;
+    }
+  }
+
+  /**
+   * Create a direct byte buffer given its address and size.
+   * The returned ByteBuffer will be direct and use the memory at the given address.
+   * @return the created direct byte buffer or null if it could not be created.
+   */
+  @SuppressWarnings({ "rawtypes", "unchecked" })
+  static ByteBuffer createDirectByteBuffer(long address, int size) {
+    if (dbbCreateFailed) {
+      return null;
+    }
+    Constructor ctor = dbbCtor;
+    if (ctor == null) {
+      Class c = dbbClass;
+      if (c == null) {
+        try {
+          c = Class.forName("java.nio.DirectByteBuffer");
+        } catch (ClassNotFoundException e) {
+          //throw new IllegalStateException("Could not find java.nio.DirectByteBuffer", e);
+          dbbCreateFailed = true;
+          dbbAddressFailed = true;
+          return null;
+        }
+        dbbClass = c;
+      }
+      try {
+        ctor = c.getDeclaredConstructor(long.class, int.class);
+      } catch (NoSuchMethodException | SecurityException e) {
+        //throw new IllegalStateException("Could not get constructor DirectByteBuffer(long, int)", e);
+        dbbClass = null;
+        dbbCreateFailed = true;
+        return null;
+      }
+      ctor.setAccessible(true);
+      dbbCtor = ctor;
+    }
+    try {
+      return (ByteBuffer)ctor.newInstance(address, size);
+    } catch (InstantiationException | IllegalAccessException | IllegalArgumentException | InvocationTargetException e) {
+      //throw new IllegalStateException("Could not create an instance using DirectByteBuffer(long, int)", e);
+      dbbClass = null;
+      dbbCtor = null;
+      dbbCreateFailed = true;
+      return null;
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/ByteArrayMemoryChunk.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/ByteArrayMemoryChunk.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/ByteArrayMemoryChunk.java
deleted file mode 100644
index cd02259..0000000
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/ByteArrayMemoryChunk.java
+++ /dev/null
@@ -1,77 +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;
-
-/**
- * The read and write methods on this implementation will throw ArrayIndexOutOfBoundsException
- * if the offset extends past the end of the underlying array of if an attempt is made to read or write past the end of the array.
- * 
- * @author darrel
- * @since 9.0
- */
-public class ByteArrayMemoryChunk implements MemoryChunk {
-
-  private final byte[] data;
-  
-  public ByteArrayMemoryChunk(int size) {
-    this.data = new byte[size];
-  }
-  
-  @Override
-  public int getSize() {
-    return this.data.length;
-  }
-
-  @Override
-  public byte readByte(int offset) {
-    return this.data[offset];
-  }
-
-  @Override
-  public void writeByte(int offset, byte value) {
-    this.data[offset] = value;
-  }
-
-  @Override
-  public void readBytes(int offset, byte[] bytes) {
-    readBytes(offset, bytes, 0, bytes.length);
-  }
-
-  @Override
-  public void writeBytes(int offset, byte[] bytes) {
-    writeBytes(offset, bytes, 0, bytes.length);
-  }
-
-  @Override
-  public void readBytes(int offset, byte[] bytes, int bytesOffset, int size) {
-    System.arraycopy(this.data, offset, bytes, bytesOffset, size);
-  }
-
-  @Override
-  public void writeBytes(int offset, byte[] bytes, int bytesOffset, int size) {
-    System.arraycopy(bytes, bytesOffset, this.data, offset, size);
-  }
-
-  @Override
-  public void release() {
-  }
-
-  @Override
-  public void copyBytes(int src, int dst, int size) {
-    System.arraycopy(this.data, src, this.data, dst, size);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/ByteBufferMemoryChunk.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/ByteBufferMemoryChunk.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/ByteBufferMemoryChunk.java
deleted file mode 100644
index 1c030f7..0000000
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/ByteBufferMemoryChunk.java
+++ /dev/null
@@ -1,90 +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;
-
-/**
- * This implementation may throw IndexOutOfBoundsException or IllegalArgumentException if the wrong offset is given to the read and write methods.
- * BufferUnderflowException will be thrown if an attempt to read more data than exists is made.
- * BufferOverflowException will be thrown if an attempt to write past the end of the chunk is made.
- * 
- * @author darrel
- * @since 9.0
- */
-public class ByteBufferMemoryChunk implements MemoryChunk {
-
-  private final ByteBuffer data;
-  
-  public ByteBufferMemoryChunk(ByteBuffer bb) {
-    this.data = bb;
-  }
-  
-  @Override
-  public int getSize() {
-    return this.data.capacity();
-  }
-
-  @Override
-  public byte readByte(int offset) {
-    return this.data.get(offset);
-  }
-
-  @Override
-  public void writeByte(int offset, byte value) {
-    this.data.put(offset, value);
-  }
-
-  @Override
-  public void readBytes(int offset, byte[] bytes) {
-    readBytes(offset, bytes, 0, bytes.length);
-  }
-
-  @Override
-  public void writeBytes(int offset, byte[] bytes) {
-    writeBytes(offset, bytes, 0, bytes.length);
-  }
-
-  @Override
-  public void readBytes(int offset, byte[] bytes, int bytesOffset, int size) {
-    // NOT THREAD SAFE
-    this.data.position(offset);
-    this.data.get(bytes, bytesOffset, size);
-  }
-
-  @Override
-  public void writeBytes(int offset, byte[] bytes, int bytesOffset, int size) {
-    // NOT THREAD SAFE
-    this.data.position(offset);
-    this.data.put(bytes, bytesOffset, size);
-  }
-
-  @Override
-  public void release() {
-  }
-
-  @Override
-  public void copyBytes(int src, int dst, int size) {
-    // NOT THREAD SAFE
-    this.data.position(src);
-    ByteBuffer srcBuff = this.data.slice();
-    srcBuff.limit(size);
-
-    this.data.position(dst);
-    this.data.put(srcBuff);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/DataAsAddress.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/DataAsAddress.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/DataAsAddress.java
deleted file mode 100644
index 96957ac..0000000
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/DataAsAddress.java
+++ /dev/null
@@ -1,131 +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.cache.Region;
-import com.gemstone.gemfire.internal.cache.BytesAndBitsForCompactor;
-import com.gemstone.gemfire.internal.cache.EntryBits;
-import com.gemstone.gemfire.internal.cache.EntryEventImpl;
-import com.gemstone.gemfire.internal.cache.RegionEntry;
-import com.gemstone.gemfire.internal.cache.RegionEntryContext;
-
-/**
- * Used to represent offheap addresses whose
- * value encodes actual data instead a memory
- * location.
- * Instances of this class have a very short lifetime.
- */
-public class DataAsAddress extends AbstractStoredObject {
-  private final long address;
-  
-  public DataAsAddress(long addr) {
-    this.address = addr;
-  }
-  
-  public long getEncodedAddress() {
-    return this.address;
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (o instanceof DataAsAddress) {
-      return getEncodedAddress() == ((DataAsAddress) o).getEncodedAddress();
-    }
-    return false;
-  }
-  
-  @Override
-  public int hashCode() {
-    long value = getEncodedAddress();
-    return (int)(value ^ (value >>> 32));
-  }
-
-  @Override
-  public int getSizeInBytes() {
-    return 0;
-  }
-
-  public byte[] getDecompressedBytes(RegionEntryContext r) {
-    byte[] bytes = OffHeapRegionEntryHelper.decodeAddressToBytes(getEncodedAddress(), true, true);
-    if (isCompressed()) {
-        long time = r.getCachePerfStats().startDecompression();
-        bytes = r.getCompressor().decompress(bytes);
-        r.getCachePerfStats().endDecompression(time);
-    }
-    return bytes;
-  }
-
-  /**
-   * If we contain a byte[] return it.
-   * Otherwise return the serialize bytes in us in a byte array.
-   */
-  public byte[] getRawBytes() {
-    return OffHeapRegionEntryHelper.decodeAddressToBytes(getEncodedAddress(), true, false);
-  }
-  
-  @Override
-  public byte[] getSerializedValue() {
-    byte[] value = OffHeapRegionEntryHelper.decodeAddressToBytes(this.address, true, false);
-    if (!isSerialized()) {
-      value = EntryEventImpl.serialize(value);
-    }
-    return value;
-  }
-
-  @Override
-  public Object getDeserializedValue(Region r, RegionEntry re) {
-    return OffHeapRegionEntryHelper.decodeAddressToObject(this.address);
-  }
-
-  @Override
-  public void fillSerializedValue(BytesAndBitsForCompactor wrapper,
-      byte userBits) {
-    byte[] value;
-    if (isSerialized()) {
-      value = getSerializedValue();
-      userBits = EntryBits.setSerialized(userBits, true);
-    } else {
-      value = (byte[]) getDeserializedForReading();
-    }
-    wrapper.setData(value, userBits, value.length, true);
-  }
-
-  @Override
-  public int getValueSizeInBytes() {
-    return 0;
-  }
-  
-  @Override
-  public boolean isSerialized() {
-    return OffHeapRegionEntryHelper.isSerialized(this.address);
-  }
-
-  @Override
-  public boolean isCompressed() {
-    return OffHeapRegionEntryHelper.isCompressed(this.address);
-  }
-  
-  @Override
-  public boolean retain() {
-    // nothing needed
-    return true;
-  }
-  @Override
-  public void release() {
-    // nothing needed
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/Fragment.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/Fragment.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/Fragment.java
index d337cfc..d36a71c 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/Fragment.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/Fragment.java
@@ -30,7 +30,7 @@ import java.util.concurrent.atomic.AtomicIntegerFieldUpdater;
  *
  */
 public class Fragment implements MemoryBlock {
-  private static final byte FILL_BYTE = ObjectChunk.FILL_BYTE;
+  private static final byte FILL_BYTE = OffHeapStoredObject.FILL_BYTE;
   private final long baseAddr;
   private final int size;
   @SuppressWarnings("unused")
@@ -60,7 +60,7 @@ public class Fragment implements MemoryBlock {
     return this.size;
   }
 
-  public long getMemoryAddress() {
+  public long getAddress() {
     return this.baseAddr;
   }
 
@@ -115,20 +115,20 @@ public class Fragment implements MemoryBlock {
   }
   
   public void fill() {
-    UnsafeMemoryChunk.fill(this.baseAddr, this.size, FILL_BYTE);
+    AddressableMemoryManager.fill(this.baseAddr, this.size, FILL_BYTE);
   }
 
   @Override
   public boolean equals(Object o) {
     if (o instanceof Fragment) {
-      return getMemoryAddress() == ((Fragment) o).getMemoryAddress();
+      return getAddress() == ((Fragment) o).getAddress();
     }
     return false;
   }
   
   @Override
   public int hashCode() {
-    long value = this.getMemoryAddress();
+    long value = this.getAddress();
     return (int)(value ^ (value >>> 32));
   }
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/FreeListManager.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/FreeListManager.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/FreeListManager.java
index 3859d58..ed7035a 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/FreeListManager.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/FreeListManager.java
@@ -40,35 +40,35 @@ public class FreeListManager {
   /** The MemoryChunks that this allocator is managing by allocating smaller chunks of them.
    * The contents of this array never change.
    */
-  private final AddressableMemoryChunk[] slabs;
+  private final Slab[] slabs;
   private final long totalSlabSize;
   
-  final private AtomicReferenceArray<SyncChunkStack> tinyFreeLists = new AtomicReferenceArray<SyncChunkStack>(TINY_FREE_LIST_COUNT);
+  final private AtomicReferenceArray<OffHeapStoredObjectAddressStack> tinyFreeLists = new AtomicReferenceArray<OffHeapStoredObjectAddressStack>(TINY_FREE_LIST_COUNT);
   // hugeChunkSet is sorted by chunk size in ascending order. It will only contain chunks larger than MAX_TINY.
-  private final ConcurrentSkipListSet<ObjectChunk> hugeChunkSet = new ConcurrentSkipListSet<ObjectChunk>();
+  private final ConcurrentSkipListSet<OffHeapStoredObject> hugeChunkSet = new ConcurrentSkipListSet<OffHeapStoredObject>();
   private final AtomicLong allocatedSize = new AtomicLong(0L);
 
   private int getNearestTinyMultiple(int size) {
     return (size-1)/TINY_MULTIPLE;
   }
-  List<ObjectChunk> getLiveChunks() {
-    ArrayList<ObjectChunk> result = new ArrayList<ObjectChunk>();
+  List<OffHeapStoredObject> getLiveChunks() {
+    ArrayList<OffHeapStoredObject> result = new ArrayList<OffHeapStoredObject>();
     for (int i=0; i < slabs.length; i++) {
       getLiveChunks(slabs[i], result);
     }
     return result;
   }
-  private void getLiveChunks(AddressableMemoryChunk slab, List<ObjectChunk> result) {
+  private void getLiveChunks(Slab slab, List<OffHeapStoredObject> result) {
     long addr = slab.getMemoryAddress();
-    while (addr <= (slab.getMemoryAddress() + slab.getSize() - ObjectChunk.MIN_CHUNK_SIZE)) {
+    while (addr <= (slab.getMemoryAddress() + slab.getSize() - OffHeapStoredObject.MIN_CHUNK_SIZE)) {
       Fragment f = isAddrInFragmentFreeSpace(addr);
       if (f != null) {
-        addr = f.getMemoryAddress() + f.getSize();
+        addr = f.getAddress() + f.getSize();
       } else {
-        int curChunkSize = ObjectChunk.getSize(addr);
-        int refCount = ObjectChunk.getRefCount(addr);
+        int curChunkSize = OffHeapStoredObject.getSize(addr);
+        int refCount = OffHeapStoredObject.getRefCount(addr);
         if (refCount > 0) {
-          result.add(new ObjectChunk(addr));
+          result.add(new OffHeapStoredObject(addr));
         }
         addr += curChunkSize;
       }
@@ -79,7 +79,7 @@ public class FreeListManager {
    */
   private Fragment isAddrInFragmentFreeSpace(long addr) {
     for (Fragment f: this.fragmentList) {
-      if (addr >= (f.getMemoryAddress() + f.getFreeIndex()) && addr < (f.getMemoryAddress() + f.getSize())) {
+      if (addr >= (f.getAddress() + f.getFreeIndex()) && addr < (f.getAddress() + f.getSize())) {
         return f;
       }
     }
@@ -95,7 +95,7 @@ public class FreeListManager {
     long result = 0;
     for (Fragment f: this.fragmentList) {
       int freeSpace = f.freeSpace();
-      if (freeSpace >= ObjectChunk.MIN_CHUNK_SIZE) {
+      if (freeSpace >= OffHeapStoredObject.MIN_CHUNK_SIZE) {
         result += freeSpace;
       }
     }
@@ -104,7 +104,7 @@ public class FreeListManager {
   long getFreeTinyMemory() {
     long tinyFree = 0;
     for (int i=0; i < this.tinyFreeLists.length(); i++) {
-      SyncChunkStack cl = this.tinyFreeLists.get(i);
+      OffHeapStoredObjectAddressStack cl = this.tinyFreeLists.get(i);
       if (cl != null) {
         tinyFree += cl.computeTotalSize();
       }
@@ -113,7 +113,7 @@ public class FreeListManager {
   }
   long getFreeHugeMemory() {
     long hugeFree = 0;
-    for (ObjectChunk c: this.hugeChunkSet) {
+    for (OffHeapStoredObject c: this.hugeChunkSet) {
       hugeFree += c.getSize();
     }
     return hugeFree;
@@ -126,7 +126,7 @@ public class FreeListManager {
   private final CopyOnWriteArrayList<Fragment> fragmentList;
   private final SimpleMemoryAllocatorImpl ma;
 
-  public FreeListManager(SimpleMemoryAllocatorImpl ma, final AddressableMemoryChunk[] slabs) {
+  public FreeListManager(SimpleMemoryAllocatorImpl ma, final Slab[] slabs) {
     this.ma = ma;
     this.slabs = slabs;
     long total = 0;
@@ -178,10 +178,10 @@ public class FreeListManager {
    * @throws IllegalStateException if a chunk can not be allocated.
    */
   @SuppressWarnings("synthetic-access")
-  public ObjectChunk allocate(int size) {
+  public OffHeapStoredObject allocate(int size) {
     assert size > 0;
     
-    ObjectChunk result = basicAllocate(size, true);
+    OffHeapStoredObject result = basicAllocate(size, true);
 
     result.setDataSize(size);
     this.allocatedSize.addAndGet(result.getSize());
@@ -190,13 +190,13 @@ public class FreeListManager {
     return result;
   }
 
-  private ObjectChunk basicAllocate(int size, boolean useSlabs) {
+  private OffHeapStoredObject basicAllocate(int size, boolean useSlabs) {
     if (useSlabs) {
       // Every object stored off heap has a header so we need
       // to adjust the size so that the header gets allocated.
       // If useSlabs is false then the incoming size has already
       // been adjusted.
-      size += ObjectChunk.OFF_HEAP_HEADER_SIZE;
+      size += OffHeapStoredObject.HEADER_SIZE;
     }
     if (size <= MAX_TINY) {
       return allocateTiny(size, useSlabs);
@@ -205,17 +205,17 @@ public class FreeListManager {
     }
   }
 
-  private ObjectChunk allocateFromFragments(int chunkSize) {
+  private OffHeapStoredObject allocateFromFragments(int chunkSize) {
     do {
       final int lastAllocationId = this.lastFragmentAllocation.get();
       for (int i=lastAllocationId; i < this.fragmentList.size(); i++) {
-        ObjectChunk result = allocateFromFragment(i, chunkSize);
+        OffHeapStoredObject result = allocateFromFragment(i, chunkSize);
         if (result != null) {
           return result;
         }
       }
       for (int i=0; i < lastAllocationId; i++) {
-        ObjectChunk result = allocateFromFragment(i, chunkSize);
+        OffHeapStoredObject result = allocateFromFragment(i, chunkSize);
         if (result != null) {
           return result;
         }
@@ -247,13 +247,13 @@ public class FreeListManager {
   }
 
   private void logHugeState(LogWriter lw) {
-    for (ObjectChunk c: this.hugeChunkSet) {
+    for (OffHeapStoredObject c: this.hugeChunkSet) {
       lw.info("Free huge of size " + c.getSize());
     }
   }
   private void logTinyState(LogWriter lw) {
     for (int i=0; i < this.tinyFreeLists.length(); i++) {
-      SyncChunkStack cl = this.tinyFreeLists.get(i);
+      OffHeapStoredObjectAddressStack cl = this.tinyFreeLists.get(i);
       if (cl != null) {
         cl.logSizes(lw, "Free tiny of size ");
       }
@@ -263,7 +263,7 @@ public class FreeListManager {
     for (Fragment f: this.fragmentList) {
       int freeSpace = f.freeSpace();
       if (freeSpace > 0) {
-        lw.info("Fragment at " + f.getMemoryAddress() + " of size " + f.getSize() + " has " + freeSpace + " bytes free.");
+        lw.info("Fragment at " + f.getAddress() + " of size " + f.getSize() + " has " + freeSpace + " bytes free.");
       }
     }
   }
@@ -323,14 +323,14 @@ public class FreeListManager {
           // So just return true causing the caller to retry the allocation.
           return true;
         }
-        ArrayList<SyncChunkStack> freeChunks = new ArrayList<SyncChunkStack>();
+        ArrayList<OffHeapStoredObjectAddressStack> freeChunks = new ArrayList<OffHeapStoredObjectAddressStack>();
         collectFreeChunks(freeChunks);
         final int SORT_ARRAY_BLOCK_SIZE = 128;
         long[] sorted = new long[SORT_ARRAY_BLOCK_SIZE];
         int sortedSize = 0;
         boolean result = false;
         int largestFragment = 0;
-        for (SyncChunkStack l: freeChunks) {
+        for (OffHeapStoredObjectAddressStack l: freeChunks) {
           long addr = l.poll();
           while (addr != 0) {
             int idx = Arrays.binarySearch(sorted, 0, sortedSize, addr);
@@ -345,10 +345,10 @@ public class FreeListManager {
               } else {
                 // see if we can conflate into sorted[idx]
                 long lowAddr = sorted[idx-1];
-                int lowSize = ObjectChunk.getSize(lowAddr);
+                int lowSize = OffHeapStoredObject.getSize(lowAddr);
                 if (lowAddr + lowSize == addr) {
                   // append the addr chunk to lowAddr
-                  ObjectChunk.setSize(lowAddr, lowSize + ObjectChunk.getSize(addr));
+                  OffHeapStoredObject.setSize(lowAddr, lowSize + OffHeapStoredObject.getSize(addr));
                 } else {
                   if (sortedSize >= sorted.length) {
                     long[] newSorted = new long[sorted.length+SORT_ARRAY_BLOCK_SIZE];
@@ -360,11 +360,11 @@ public class FreeListManager {
                 }
               }
             } else {
-              int addrSize = ObjectChunk.getSize(addr);
+              int addrSize = OffHeapStoredObject.getSize(addr);
               long highAddr = sorted[idx];
               if (addr + addrSize == highAddr) {
                 // append highAddr chunk to addr
-                ObjectChunk.setSize(addr, addrSize + ObjectChunk.getSize(highAddr));
+                OffHeapStoredObject.setSize(addr, addrSize + OffHeapStoredObject.getSize(highAddr));
                 sorted[idx] = addr;
               } else {
                 boolean insert = idx==0;
@@ -374,10 +374,10 @@ public class FreeListManager {
                   //                    long[] tmp = Arrays.copyOf(sorted, sortedSize);
                   //                    throw new IllegalStateException("addr was zero at idx=" + (idx-1) + " sorted="+ Arrays.toString(tmp));
                   //                  }
-                  int lowSize = ObjectChunk.getSize(lowAddr);
+                  int lowSize = OffHeapStoredObject.getSize(lowAddr);
                   if (lowAddr + lowSize == addr) {
                     // append the addr chunk to lowAddr
-                    ObjectChunk.setSize(lowAddr, lowSize + addrSize);
+                    OffHeapStoredObject.setSize(lowAddr, lowSize + addrSize);
                   } else {
                     insert = true;
                   }
@@ -403,10 +403,10 @@ public class FreeListManager {
         for (int i=sortedSize-1; i > 0; i--) {
           long addr = sorted[i];
           long lowAddr = sorted[i-1];
-          int lowSize = ObjectChunk.getSize(lowAddr);
+          int lowSize = OffHeapStoredObject.getSize(lowAddr);
           if (lowAddr + lowSize == addr) {
             // append addr chunk to lowAddr
-            ObjectChunk.setSize(lowAddr, lowSize + ObjectChunk.getSize(addr));
+            OffHeapStoredObject.setSize(lowAddr, lowSize + OffHeapStoredObject.getSize(addr));
             sorted[i] = 0L;
           }
         }
@@ -415,7 +415,7 @@ public class FreeListManager {
         for (int i=sortedSize-1; i >= 0; i--) {
           long addr = sorted[i];
           if (addr == 0L) continue;
-          int addrSize = ObjectChunk.getSize(addr);
+          int addrSize = OffHeapStoredObject.getSize(addr);
           Fragment f = createFragment(addr, addrSize);
           if (addrSize >= chunkSize) {
             result = true;
@@ -492,15 +492,15 @@ public class FreeListManager {
       } else {
         //more than 1 fragment is available so freeMemory is > ObjectChunk.MIN_CHUNK_SIZE
         long freeMemory = getFreeMemory();
-        assert freeMemory > ObjectChunk.MIN_CHUNK_SIZE;
-        long maxPossibleFragments = freeMemory / ObjectChunk.MIN_CHUNK_SIZE;
+        assert freeMemory > OffHeapStoredObject.MIN_CHUNK_SIZE;
+        long maxPossibleFragments = freeMemory / OffHeapStoredObject.MIN_CHUNK_SIZE;
         double fragmentation = ((double) availableFragments /(double) maxPossibleFragments) * 100d;
         return (int) Math.rint(fragmentation);
       }
     }
   }
 
-  private void collectFreeChunks(List<SyncChunkStack> l) {
+  private void collectFreeChunks(List<OffHeapStoredObjectAddressStack> l) {
     collectFreeFragmentChunks(l);
     collectFreeHugeChunks(l);
     collectFreeTinyChunks(l);
@@ -508,17 +508,17 @@ public class FreeListManager {
   List<Fragment> getFragmentList() {
     return this.fragmentList;
   }
-  private void collectFreeFragmentChunks(List<SyncChunkStack> l) {
+  private void collectFreeFragmentChunks(List<OffHeapStoredObjectAddressStack> l) {
     if (this.fragmentList.size() == 0) return;
-    SyncChunkStack result = new SyncChunkStack();
+    OffHeapStoredObjectAddressStack result = new OffHeapStoredObjectAddressStack();
     for (Fragment f: this.fragmentList) {
       int offset;
       int diff;
       do {
         offset = f.getFreeIndex();
         diff = f.getSize() - offset;
-      } while (diff >= ObjectChunk.MIN_CHUNK_SIZE && !f.allocate(offset, offset+diff));
-      if (diff < ObjectChunk.MIN_CHUNK_SIZE) {
+      } while (diff >= OffHeapStoredObject.MIN_CHUNK_SIZE && !f.allocate(offset, offset+diff));
+      if (diff < OffHeapStoredObject.MIN_CHUNK_SIZE) {
         // If diff > 0 then that memory will be lost during compaction.
         // This should never happen since we keep the sizes rounded
         // based on MIN_CHUNK_SIZE.
@@ -526,8 +526,8 @@ public class FreeListManager {
         // The current fragment is completely allocated so just skip it.
         continue;
       }
-      long chunkAddr = f.getMemoryAddress()+offset;
-      ObjectChunk.setSize(chunkAddr, diff);
+      long chunkAddr = f.getAddress()+offset;
+      OffHeapStoredObject.setSize(chunkAddr, diff);
       result.offer(chunkAddr);
     }
     // All the fragments have been turned in to chunks so now clear them
@@ -537,31 +537,31 @@ public class FreeListManager {
       l.add(result);
     }
   }
-  private void collectFreeTinyChunks(List<SyncChunkStack> l) {
+  private void collectFreeTinyChunks(List<OffHeapStoredObjectAddressStack> l) {
     for (int i=0; i < this.tinyFreeLists.length(); i++) {
-      SyncChunkStack cl = this.tinyFreeLists.get(i);
+      OffHeapStoredObjectAddressStack cl = this.tinyFreeLists.get(i);
       if (cl != null) {
         long head = cl.clear();
         if (head != 0L) {
-          l.add(new SyncChunkStack(head));
+          l.add(new OffHeapStoredObjectAddressStack(head));
         }
       }
     }
   }
-  private void collectFreeHugeChunks(List<SyncChunkStack> l) {
-    ObjectChunk c = this.hugeChunkSet.pollFirst();
-    SyncChunkStack result = null;
+  private void collectFreeHugeChunks(List<OffHeapStoredObjectAddressStack> l) {
+    OffHeapStoredObject c = this.hugeChunkSet.pollFirst();
+    OffHeapStoredObjectAddressStack result = null;
     while (c != null) {
       if (result == null) {
-        result = new SyncChunkStack();
+        result = new OffHeapStoredObjectAddressStack();
         l.add(result);
       }
-      result.offer(c.getMemoryAddress());
+      result.offer(c.getAddress());
       c = this.hugeChunkSet.pollFirst();
     }
   }
 
-  ObjectChunk allocateFromFragment(final int fragIdx, final int chunkSize) {
+  OffHeapStoredObject allocateFromFragment(final int fragIdx, final int chunkSize) {
     if (fragIdx >= this.fragmentList.size()) return null;
     final Fragment fragment;
     try {
@@ -580,7 +580,7 @@ public class FreeListManager {
         // this fragment has room
         int newOffset = oldOffset + chunkSize;
         int extraSize = fragmentSize - newOffset;
-        if (extraSize < ObjectChunk.MIN_CHUNK_SIZE) {
+        if (extraSize < OffHeapStoredObject.MIN_CHUNK_SIZE) {
           // include these last few bytes of the fragment in the allocation.
           // If we don't then they will be lost forever.
           // The extraSize bytes only apply to the first chunk we allocate (not the batch ones).
@@ -591,11 +591,11 @@ public class FreeListManager {
         if (fragment.allocate(oldOffset, newOffset)) {
           // We did the allocate!
           this.lastFragmentAllocation.set(fragIdx);
-          ObjectChunk result = new ObjectChunk(fragment.getMemoryAddress()+oldOffset, chunkSize+extraSize);
+          OffHeapStoredObject result = new OffHeapStoredObject(fragment.getAddress()+oldOffset, chunkSize+extraSize);
           checkDataIntegrity(result);
           return result;
         } else {
-          ObjectChunk result = basicAllocate(chunkSize, false);
+          OffHeapStoredObject result = basicAllocate(chunkSize, false);
           if (result != null) {
             return result;
           }
@@ -609,15 +609,15 @@ public class FreeListManager {
   private int round(int multiple, int value) {
     return (int) ((((long)value + (multiple-1)) / multiple) * multiple);
   }
-  private ObjectChunk allocateTiny(int size, boolean useFragments) {
+  private OffHeapStoredObject allocateTiny(int size, boolean useFragments) {
     return basicAllocate(getNearestTinyMultiple(size), TINY_MULTIPLE, 0, this.tinyFreeLists, useFragments);
   }
-  private ObjectChunk basicAllocate(int idx, int multiple, int offset, AtomicReferenceArray<SyncChunkStack> freeLists, boolean useFragments) {
-    SyncChunkStack clq = freeLists.get(idx);
+  private OffHeapStoredObject basicAllocate(int idx, int multiple, int offset, AtomicReferenceArray<OffHeapStoredObjectAddressStack> freeLists, boolean useFragments) {
+    OffHeapStoredObjectAddressStack clq = freeLists.get(idx);
     if (clq != null) {
       long memAddr = clq.poll();
       if (memAddr != 0) {
-        ObjectChunk result = new ObjectChunk(memAddr);
+        OffHeapStoredObject result = new OffHeapStoredObject(memAddr);
         checkDataIntegrity(result);
         result.readyForAllocation();
         return result;
@@ -629,13 +629,13 @@ public class FreeListManager {
       return null;
     }
   }
-  private ObjectChunk allocateHuge(int size, boolean useFragments) {
+  private OffHeapStoredObject allocateHuge(int size, boolean useFragments) {
     // sizeHolder is a fake Chunk used to search our sorted hugeChunkSet.
-    ObjectChunk sizeHolder = new FakeChunk(size);
-    NavigableSet<ObjectChunk> ts = this.hugeChunkSet.tailSet(sizeHolder);
-    ObjectChunk result = ts.pollFirst();
+    OffHeapStoredObject sizeHolder = new SearchMarker(size);
+    NavigableSet<OffHeapStoredObject> ts = this.hugeChunkSet.tailSet(sizeHolder);
+    OffHeapStoredObject result = ts.pollFirst();
     if (result != null) {
-      if (result.getSize() - (HUGE_MULTIPLE - ObjectChunk.OFF_HEAP_HEADER_SIZE) < size) {
+      if (result.getSize() - (HUGE_MULTIPLE - OffHeapStoredObject.HEADER_SIZE) < size) {
         // close enough to the requested size; just return it.
         checkDataIntegrity(result);
         result.readyForAllocation();
@@ -653,19 +653,19 @@ public class FreeListManager {
     }
   }
   
-  private void checkDataIntegrity(ObjectChunk data) {
+  private void checkDataIntegrity(OffHeapStoredObject data) {
     if (this.validateMemoryWithFill) {
       data.validateFill();
     }
   }
   /**
    * Used by the FreeListManager to easily search its
-   * ConcurrentSkipListSet. This is not a real chunk
+   * ConcurrentSkipListSet. This is not a real OffHeapStoredObject
    * but only used for searching.
    */
-  private static class FakeChunk extends ObjectChunk {
+  private static class SearchMarker extends OffHeapStoredObject {
     private final int size;
-    public FakeChunk(int size) {
+    public SearchMarker(int size) {
       super();
       this.size = size;
     }
@@ -678,14 +678,14 @@ public class FreeListManager {
   @SuppressWarnings("synthetic-access")
   public void free(long addr) {
     if (this.validateMemoryWithFill) {
-      ObjectChunk.fill(addr);
+      OffHeapStoredObject.fill(addr);
     }
     
     free(addr, true);
   }
 
   private void free(long addr, boolean updateStats) {
-    int cSize = ObjectChunk.getSize(addr);
+    int cSize = OffHeapStoredObject.getSize(addr);
     if (updateStats) {
       OffHeapMemoryStats stats = this.ma.getStats();
       stats.incObjects(-1);
@@ -703,8 +703,8 @@ public class FreeListManager {
   private void freeTiny(long addr, int cSize) {
     basicFree(addr, getNearestTinyMultiple(cSize), this.tinyFreeLists);
   }
-  private void basicFree(long addr, int idx, AtomicReferenceArray<SyncChunkStack> freeLists) {
-    SyncChunkStack clq = freeLists.get(idx);
+  private void basicFree(long addr, int idx, AtomicReferenceArray<OffHeapStoredObjectAddressStack> freeLists) {
+    OffHeapStoredObjectAddressStack clq = freeLists.get(idx);
     if (clq != null) {
       clq.offer(addr);
     } else {
@@ -719,12 +719,12 @@ public class FreeListManager {
   /**
    * Tests override this method to simulate concurrent modification
    */
-  protected SyncChunkStack createFreeListForEmptySlot(AtomicReferenceArray<SyncChunkStack> freeLists, int idx) {
-    return new SyncChunkStack();
+  protected OffHeapStoredObjectAddressStack createFreeListForEmptySlot(AtomicReferenceArray<OffHeapStoredObjectAddressStack> freeLists, int idx) {
+    return new OffHeapStoredObjectAddressStack();
   }
   
   private void freeHuge(long addr, int cSize) {
-    this.hugeChunkSet.add(new ObjectChunk(addr)); // TODO make this a collection of longs
+    this.hugeChunkSet.add(new OffHeapStoredObject(addr)); // TODO make this a collection of longs
   }
 
   List<MemoryBlock> getOrderedBlocks() {
@@ -737,7 +737,7 @@ public class FreeListManager {
         new Comparator<MemoryBlock>() {
           @Override
           public int compare(MemoryBlock o1, MemoryBlock o2) {
-            return Long.valueOf(o1.getMemoryAddress()).compareTo(o2.getMemoryAddress());
+            return Long.valueOf(o1.getAddress()).compareTo(o2.getAddress());
           }
     });
     return value;
@@ -748,8 +748,8 @@ public class FreeListManager {
     }
   }
   
-  private void addBlocksFromChunks(Collection<ObjectChunk> src, List<MemoryBlock> dest) {
-    for (ObjectChunk chunk : src) {
+  private void addBlocksFromChunks(Collection<OffHeapStoredObject> src, List<MemoryBlock> dest) {
+    for (OffHeapStoredObject chunk : src) {
       dest.add(new MemoryBlockNode(this.ma, chunk));
     }
   }
@@ -768,7 +768,7 @@ public class FreeListManager {
       long addr = this.tinyFreeLists.get(i).getTopAddress();
       while (addr != 0L) {
         value.add(new MemoryBlockNode(sma, new TinyMemoryBlock(addr, i)));
-        addr = ObjectChunk.getNext(addr);
+        addr = OffHeapStoredObject.getNext(addr);
       }
     }
     return value;
@@ -780,7 +780,7 @@ public class FreeListManager {
         new Comparator<MemoryBlock>() {
           @Override
           public int compare(MemoryBlock o1, MemoryBlock o2) {
-            return Long.valueOf(o1.getMemoryAddress()).compareTo(o2.getMemoryAddress());
+            return Long.valueOf(o1.getAddress()).compareTo(o2.getAddress());
           }
     });
     return value;
@@ -803,13 +803,13 @@ public class FreeListManager {
     }
 
     @Override
-    public long getMemoryAddress() {
+    public long getAddress() {
       return address;
     }
 
     @Override
     public int getBlockSize() {
-      return ObjectChunk.getSize(address);
+      return OffHeapStoredObject.getSize(address);
     }
 
     @Override
@@ -855,14 +855,14 @@ public class FreeListManager {
     @Override
     public boolean equals(Object o) {
       if (o instanceof TinyMemoryBlock) {
-        return getMemoryAddress() == ((TinyMemoryBlock) o).getMemoryAddress();
+        return getAddress() == ((TinyMemoryBlock) o).getAddress();
       }
       return false;
     }
     
     @Override
     public int hashCode() {
-      long value = this.getMemoryAddress();
+      long value = this.getAddress();
       return (int)(value ^ (value >>> 32));
     }
   }
@@ -873,7 +873,7 @@ public class FreeListManager {
   
   void freeSlabs() {
     for (int i=0; i < slabs.length; i++) {
-      slabs[i].release();
+      slabs[i].free();
     }
   }
   /**
@@ -885,7 +885,7 @@ public class FreeListManager {
    * be used. Note that this code does not bother
    * comparing the contents of the arrays.
    */
-  boolean okToReuse(AddressableMemoryChunk[] newSlabs) {
+  boolean okToReuse(Slab[] newSlabs) {
     return newSlabs == null || newSlabs == this.slabs;
   }
   
@@ -894,7 +894,7 @@ public class FreeListManager {
   }
   int findSlab(long addr) {
     for (int i=0; i < this.slabs.length; i++) {
-      AddressableMemoryChunk slab = this.slabs[i];
+      Slab slab = this.slabs[i];
       long slabAddr = slab.getMemoryAddress();
       if (addr >= slabAddr) {
         if (addr < slabAddr + slab.getSize()) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/MemoryAllocator.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/MemoryAllocator.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/MemoryAllocator.java
index 0c063ac..0f776e7 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/MemoryAllocator.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/MemoryAllocator.java
@@ -29,18 +29,30 @@ public interface MemoryAllocator {
    * @return the allocated chunk of memory.
    * @throws IllegalStateException if the heap does not have enough memory to grant the request
    */
-  public MemoryChunk allocate(int size);
+  public StoredObject allocate(int size);
   
   /**
-   * Allocates off heap memory for the given data and returns a MemoryChunk
+   * Allocates off heap memory for the given data and returns a StoredObject
    * that is backed by this allocated memory and that contains the data.
-   * @param data the bytes of the data to put in the allocated CachedDeserializable
+   * @param data the bytes of the data to put in the allocated StoredObject
    * @param isSerialized true if data contains a serialized object; false if it is an actual byte array.
    * @param isCompressed true if data is compressed; false if it is uncompressed.
    * @throws IllegalStateException if the heap does not have enough memory to grant the request
    */
   public StoredObject allocateAndInitialize(byte[] data, boolean isSerialized, boolean isCompressed);
   
+  /**
+   * Allocates off heap memory for the given data and returns a StoredObject
+   * that is backed by this allocated memory and that contains the data
+   * and keeps a reference to the original heap data.
+   * @param data the bytes of the data to put in the allocated StoredObject
+   * @param isSerialized true if data contains a serialized object; false if it is an actual byte array.
+   * @param originalHeapData the original uncompressed heap data
+   * @param isCompressed true if data is compressed; false if it is uncompressed.
+   * @throws IllegalStateException if the heap does not have enough memory to grant the request
+   */
+  public StoredObject allocateAndInitialize(byte[] data, boolean isSerialized, boolean isCompressed, byte[] originalHeapData);
+  
   public long getFreeMemory();
   
   public long getUsedMemory();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/MemoryBlock.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/MemoryBlock.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/MemoryBlock.java
index d8cb80a..878d06f 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/MemoryBlock.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/MemoryBlock.java
@@ -40,7 +40,7 @@ public interface MemoryBlock {
   /**
    * Returns the unsafe memory address of the first byte of this block.
    */
-  public long getMemoryAddress();
+  public long getAddress();
   
   /**
    * Returns the size of this memory block in bytes.

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/MemoryBlockNode.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/MemoryBlockNode.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/MemoryBlockNode.java
index b41d429..6e2414f 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/MemoryBlockNode.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/MemoryBlockNode.java
@@ -37,8 +37,8 @@ public class MemoryBlockNode implements MemoryBlock {
     return this.block.getState();
   }
   @Override
-  public long getMemoryAddress() {
-    return this.block.getMemoryAddress();
+  public long getAddress() {
+    return this.block.getAddress();
   }
   @Override
   public int getBlockSize() {
@@ -49,7 +49,7 @@ public class MemoryBlockNode implements MemoryBlock {
     return this.ma.getMemoryInspector().getBlockAfter(this);
   }
   public int getSlabId() {
-    return this.ma.findSlab(getMemoryAddress());
+    return this.ma.findSlab(getAddress());
   }
   @Override
   public int getFreeListId() {
@@ -65,15 +65,15 @@ public class MemoryBlockNode implements MemoryBlock {
     if (!isSerialized()) {
       // byte array
       if (isCompressed()) {
-        return "compressed byte[" + ((ObjectChunk)this.block).getDataSize() + "]";
+        return "compressed byte[" + ((OffHeapStoredObject)this.block).getDataSize() + "]";
       } else {
-        return "byte[" + ((ObjectChunk)this.block).getDataSize() + "]";
+        return "byte[" + ((OffHeapStoredObject)this.block).getDataSize() + "]";
       }
     } else if (isCompressed()) {
-      return "compressed object of size " + ((ObjectChunk)this.block).getDataSize();
+      return "compressed object of size " + ((OffHeapStoredObject)this.block).getDataSize();
     }
     //Object obj = EntryEventImpl.deserialize(((Chunk)this.block).getRawBytes());
-    byte[] bytes = ((ObjectChunk)this.block).getRawBytes();
+    byte[] bytes = ((OffHeapStoredObject)this.block).getRawBytes();
     return DataType.getDataType(bytes);
   }
   public boolean isSerialized() {
@@ -88,14 +88,14 @@ public class MemoryBlockNode implements MemoryBlock {
     if (dataType == null || dataType.equals("N/A")) {
       return null;
     } else if (isCompressed()) {
-      return ((ObjectChunk)this.block).getCompressedBytes();
+      return ((OffHeapStoredObject)this.block).getCompressedBytes();
     } else if (!isSerialized()) {
       // byte array
       //return "byte[" + ((Chunk)this.block).getDataSize() + "]";
-      return ((ObjectChunk)this.block).getRawBytes();
+      return ((OffHeapStoredObject)this.block).getRawBytes();
     } else {
       try {
-        byte[] bytes = ((ObjectChunk)this.block).getRawBytes();
+        byte[] bytes = ((OffHeapStoredObject)this.block).getRawBytes();
         return DataSerializer.readObject(DataType.getDataInput(bytes));
       } catch (IOException e) {
         e.printStackTrace();
@@ -113,7 +113,7 @@ public class MemoryBlockNode implements MemoryBlock {
   public String toString() {
     final StringBuilder sb = new StringBuilder(MemoryBlock.class.getSimpleName());
     sb.append("{");
-    sb.append("MemoryAddress=").append(getMemoryAddress());
+    sb.append("MemoryAddress=").append(getAddress());
     sb.append(", State=").append(getState());
     sb.append(", BlockSize=").append(getBlockSize());
     sb.append(", SlabId=").append(getSlabId());

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/MemoryChunk.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/MemoryChunk.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/MemoryChunk.java
deleted file mode 100644
index 012fbe5..0000000
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/MemoryChunk.java
+++ /dev/null
@@ -1,47 +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;
-
-/**
- * Represents a chunk of allocated memory that is not on the heap.
- * This interface provides methods that let you read and write to the chunk.
- * 
- * @author darrel
- * @since 9.0
- */
-public interface MemoryChunk extends Releasable {
-  
-  /**
-   * Returns the size of this memory chunk in bytes.
-   */
-  public int getSize();
-  
-  public byte readByte(int offset);
-  public void writeByte(int offset, byte value);
-  
-  public void readBytes(int offset, byte[] bytes);
-  public void writeBytes(int offset, byte[] bytes);
-  public void readBytes(int offset, byte[] bytes, int bytesOffset, int size);
-  public void writeBytes(int offset, byte[] bytes, int bytesOffset, int size);
-  
-  /**
-   * Read the bytes in this range [src..src+size]
-   * and write them to the range that starts at dst.
-   * The number of bytes copied is size.
-   */
-  public void copyBytes(int src, int dst, int size);
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/MemoryChunkWithRefCount.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/MemoryChunkWithRefCount.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/MemoryChunkWithRefCount.java
deleted file mode 100644
index e3ba6ab..0000000
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/MemoryChunkWithRefCount.java
+++ /dev/null
@@ -1,34 +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;
-
-/**
- * Adds a reference count to the basic MemoryChunk.
- * Also an Object can be stored in one of these.
- * To increment the count call {@link #retain()}.
- * To decrement the count call {@link #release()}.
- * 
- * @author darrel
- * @since 9.0
- */
-public interface MemoryChunkWithRefCount extends MemoryChunk, StoredObject {
-
-  /**
-   * Returns the number of users of this memory.
-   */
-  public int getRefCount();
-}


[4/8] incubator-geode git commit: GEODE-982: refactor off-heap

Posted by ds...@apache.org.
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 {
-
-}


[3/8] incubator-geode git commit: GEODE-982: refactor off-heap

Posted by ds...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/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
deleted file mode 100644
index 9271b53..0000000
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/ObjectChunkJUnitTest.java
+++ /dev/null
@@ -1,902 +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.assertj.core.api.Assertions.assertThat;
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.mockito.Mockito.atLeastOnce;
-import static org.mockito.Mockito.doNothing;
-import static org.mockito.Mockito.doReturn;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.spy;
-import static org.mockito.Mockito.times;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import com.gemstone.gemfire.LogWriter;
-import com.gemstone.gemfire.compression.Compressor;
-import com.gemstone.gemfire.internal.DSCODE;
-import com.gemstone.gemfire.internal.HeapDataOutputStream;
-import com.gemstone.gemfire.internal.Version;
-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.MemoryBlock.State;
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
-
-@Category(UnitTest.class)
-public class ObjectChunkJUnitTest extends AbstractStoredObjectTestBase {
-
-  private MemoryAllocator ma;
-
-  static {
-    ClassLoader.getSystemClassLoader().setDefaultAssertionStatus(true);
-  }
-
-  @Before
-  public void setUp() {
-    OutOfOffHeapMemoryListener ooohml = mock(OutOfOffHeapMemoryListener.class);
-    OffHeapMemoryStats stats = mock(OffHeapMemoryStats.class);
-    LogWriter lw = mock(LogWriter.class);
-
-    ma = SimpleMemoryAllocatorImpl.create(ooohml, stats, lw, 3, OffHeapStorage.MIN_SLAB_SIZE * 3, OffHeapStorage.MIN_SLAB_SIZE);
-  }
-
-  @After
-  public void tearDown() {
-    SimpleMemoryAllocatorImpl.freeOffHeapMemory();
-  }
-
-  @Override
-  public Object getValue() {
-    return Long.valueOf(Long.MAX_VALUE);
-  }
-
-  @Override
-  public byte[] getValueAsByteArray() {
-    return convertValueToByteArray(getValue());
-  }
-
-  private byte[] convertValueToByteArray(Object value) {
-    return ByteBuffer.allocate(Long.SIZE / Byte.SIZE).putLong((Long) value).array();
-  }
-
-  @Override
-  public Object convertByteArrayToObject(byte[] valueInByteArray) {
-    return ByteBuffer.wrap(valueInByteArray).getLong();
-  }
-
-  @Override
-  public Object convertSerializedByteArrayToObject(byte[] valueInSerializedByteArray) {
-    return EntryEventImpl.deserialize(valueInSerializedByteArray);
-  }
-
-  @Override
-  public ObjectChunk createValueAsUnserializedStoredObject(Object value) {
-    byte[] valueInByteArray;
-    if (value instanceof Long) {
-      valueInByteArray = convertValueToByteArray(value);
-    } else {
-      valueInByteArray = (byte[]) value;
-    }
-
-    boolean isSerialized = false;
-    boolean isCompressed = false;
-
-    return createChunk(valueInByteArray, isSerialized, isCompressed);
-  }
-
-  @Override
-  public ObjectChunk createValueAsSerializedStoredObject(Object value) {
-    byte[] valueInSerializedByteArray = EntryEventImpl.serialize(value);
-
-    boolean isSerialized = true;
-    boolean isCompressed = false;
-
-    return createChunk(valueInSerializedByteArray, isSerialized, isCompressed);
-  }
-
-  private ObjectChunk createChunk(byte[] v, boolean isSerialized, boolean isCompressed) {
-    ObjectChunk chunk = (ObjectChunk) ma.allocateAndInitialize(v, isSerialized, isCompressed);
-    return chunk;
-  }
-
-  @Test
-  public void chunkCanBeCreatedFromAnotherChunk() {
-    ObjectChunk chunk = createValueAsUnserializedStoredObject(getValue());
-
-    ObjectChunk newChunk = new ObjectChunk(chunk);
-
-    assertNotNull(newChunk);
-    assertThat(newChunk.getMemoryAddress()).isEqualTo(chunk.getMemoryAddress());
-
-    chunk.release();
-  }
-
-  @Test
-  public void chunkCanBeCreatedWithOnlyMemoryAddress() {
-    ObjectChunk chunk = createValueAsUnserializedStoredObject(getValue());
-
-    ObjectChunk newChunk = new ObjectChunk(chunk.getMemoryAddress());
-
-    assertNotNull(newChunk);
-    assertThat(newChunk.getMemoryAddress()).isEqualTo(chunk.getMemoryAddress());
-
-    chunk.release();
-  }
-
-  @Test
-  public void chunkSliceCanBeCreatedFromAnotherChunk() {
-    ObjectChunk chunk = createValueAsUnserializedStoredObject(getValue());
-
-    int position = 1;
-    int end = 2;
-
-    ObjectChunk newChunk = (ObjectChunk) chunk.slice(position, end);
-
-    assertNotNull(newChunk);
-    assertThat(newChunk.getClass()).isEqualTo(ObjectChunkSlice.class);
-    assertThat(newChunk.getMemoryAddress()).isEqualTo(chunk.getMemoryAddress());
-
-    chunk.release();
-  }
-
-  @Test
-  public void fillSerializedValueShouldFillWrapperWithSerializedValueIfValueIsSerialized() {
-    ObjectChunk chunk = createValueAsSerializedStoredObject(getValue());
-
-    // mock the things
-    BytesAndBitsForCompactor wrapper = mock(BytesAndBitsForCompactor.class);
-
-    byte userBits = 0;
-    byte serializedUserBits = 1;
-    chunk.fillSerializedValue(wrapper, userBits);
-
-    verify(wrapper, times(1)).setChunkData(chunk, serializedUserBits);
-
-    chunk.release();
-  }
-
-  @Test
-  public void fillSerializedValueShouldFillWrapperWithDeserializedValueIfValueIsNotSerialized() {
-    ObjectChunk chunk = createValueAsUnserializedStoredObject(getValue());
-
-    // mock the things
-    BytesAndBitsForCompactor wrapper = mock(BytesAndBitsForCompactor.class);
-
-    byte userBits = 1;
-    chunk.fillSerializedValue(wrapper, userBits);
-
-    verify(wrapper, times(1)).setChunkData(chunk, userBits);
-
-    chunk.release();
-  }
-
-  @Test
-  public void getShortClassNameShouldReturnShortClassName() {
-    ObjectChunk chunk = createValueAsUnserializedStoredObject(getValue());
-    assertThat(chunk.getShortClassName()).isEqualTo("ObjectChunk");
-
-    chunk.release();
-  }
-
-  @Test
-  public void chunksAreEqualsOnlyByAddress() {
-    ObjectChunk chunk = createValueAsSerializedStoredObject(getValue());
-
-    ObjectChunk newChunk = new ObjectChunk(chunk.getMemoryAddress());
-    assertThat(chunk.equals(newChunk)).isTrue();
-
-    ObjectChunk chunkWithSameValue = createValueAsUnserializedStoredObject(getValue());
-    assertThat(chunk.equals(chunkWithSameValue)).isFalse();
-
-    Object someObject = getValue();
-    assertThat(chunk.equals(someObject)).isFalse();
-
-    chunk.release();
-    chunkWithSameValue.release();
-  }
-
-  @Test
-  public void chunksShouldBeComparedBySize() {
-    ObjectChunk chunk1 = createValueAsSerializedStoredObject(getValue());
-
-    ObjectChunk chunk2 = chunk1;
-    assertThat(chunk1.compareTo(chunk2)).isEqualTo(0);
-
-    ObjectChunk chunkWithSameValue = createValueAsSerializedStoredObject(getValue());
-    assertThat(chunk1.compareTo(chunkWithSameValue)).isEqualTo(Long.signum(chunk1.getMemoryAddress() - chunkWithSameValue.getMemoryAddress()));
-
-    ObjectChunk chunk3 = createValueAsSerializedStoredObject(Long.MAX_VALUE);
-    ObjectChunk chunk4 = createValueAsSerializedStoredObject(Long.MAX_VALUE);
-
-    int newSizeForChunk3 = 2;
-    int newSizeForChunk4 = 3;
-
-    assertThat(chunk3.compareTo(chunk4)).isEqualTo(Integer.signum(newSizeForChunk3 - newSizeForChunk4));
-
-    chunk1.release();
-    chunk4.release();
-  }
-
-  @Test
-  public void setSerializedShouldSetTheSerializedBit() {
-    Object regionEntryValue = getValue();
-    byte[] regionEntryValueAsBytes = convertValueToByteArray(regionEntryValue);
-
-    boolean isSerialized = false;
-    boolean isCompressed = false;
-
-    ObjectChunk chunk = (ObjectChunk) ma.allocateAndInitialize(regionEntryValueAsBytes, isSerialized, isCompressed);
-
-    int headerBeforeSerializedBitSet = UnsafeMemoryChunk.readAbsoluteIntVolatile(chunk.getMemoryAddress() + ObjectChunk.REF_COUNT_OFFSET);
-
-    assertThat(chunk.isSerialized()).isFalse();
-
-    chunk.setSerialized(true); // set to true
-
-    assertThat(chunk.isSerialized()).isTrue();
-
-    int headerAfterSerializedBitSet = UnsafeMemoryChunk.readAbsoluteIntVolatile(chunk.getMemoryAddress() + ObjectChunk.REF_COUNT_OFFSET);
-
-    assertThat(headerAfterSerializedBitSet).isEqualTo(headerBeforeSerializedBitSet | ObjectChunk.IS_SERIALIZED_BIT);
-
-    chunk.release();
-  }
-
-  @Test(expected = IllegalStateException.class)
-  public void setSerialziedShouldThrowExceptionIfChunkIsAlreadyReleased() {
-    ObjectChunk chunk = createValueAsUnserializedStoredObject(getValue());
-    chunk.release();
-    chunk.setSerialized(true);
-
-    chunk.release();
-  }
-
-  @Test
-  public void setCompressedShouldSetTheCompressedBit() {
-    Object regionEntryValue = getValue();
-    byte[] regionEntryValueAsBytes = convertValueToByteArray(regionEntryValue);
-
-    boolean isSerialized = false;
-    boolean isCompressed = false;
-
-    ObjectChunk chunk = (ObjectChunk) ma.allocateAndInitialize(regionEntryValueAsBytes, isSerialized, isCompressed);
-
-    int headerBeforeCompressedBitSet = UnsafeMemoryChunk.readAbsoluteIntVolatile(chunk.getMemoryAddress() + ObjectChunk.REF_COUNT_OFFSET);
-
-    assertThat(chunk.isCompressed()).isFalse();
-
-    chunk.setCompressed(true); // set to true
-
-    assertThat(chunk.isCompressed()).isTrue();
-
-    int headerAfterCompressedBitSet = UnsafeMemoryChunk.readAbsoluteIntVolatile(chunk.getMemoryAddress() + ObjectChunk.REF_COUNT_OFFSET);
-
-    assertThat(headerAfterCompressedBitSet).isEqualTo(headerBeforeCompressedBitSet | ObjectChunk.IS_COMPRESSED_BIT);
-
-    chunk.release();
-  }
-
-  @Test(expected = IllegalStateException.class)
-  public void setCompressedShouldThrowExceptionIfChunkIsAlreadyReleased() {
-    ObjectChunk chunk = createValueAsUnserializedStoredObject(getValue());
-    chunk.release();
-    chunk.setCompressed(true);
-
-    chunk.release();
-  }
-
-  @Test
-  public void setDataSizeShouldSetTheDataSizeBits() {
-    ObjectChunk chunk = createValueAsUnserializedStoredObject(getValue());
-
-    int beforeSize = chunk.getDataSize();
-
-    chunk.setDataSize(2);
-
-    int afterSize = chunk.getDataSize();
-
-    assertThat(afterSize).isEqualTo(2);
-    assertThat(afterSize).isNotEqualTo(beforeSize);
-
-    chunk.release();
-  }
-
-  @Test(expected = IllegalStateException.class)
-  public void setDataSizeShouldThrowExceptionIfChunkIsAlreadyReleased() {
-    ObjectChunk chunk = createValueAsUnserializedStoredObject(getValue());
-    chunk.release();
-    chunk.setDataSize(1);
-
-    chunk.release();
-  }
-
-  @Test(expected = IllegalStateException.class)
-  public void initializeUseCountShouldThrowIllegalStateExceptionIfChunkIsAlreadyRetained() {
-    ObjectChunk chunk = createValueAsUnserializedStoredObject(getValue());
-    chunk.retain();
-    chunk.initializeUseCount();
-
-    chunk.release();
-  }
-
-  @Test(expected = IllegalStateException.class)
-  public void initializeUseCountShouldThrowIllegalStateExceptionIfChunkIsAlreadyReleased() {
-    ObjectChunk chunk = createValueAsUnserializedStoredObject(getValue());
-    chunk.release();
-    chunk.initializeUseCount();
-
-    chunk.release();
-  }
-
-  @Test
-  public void isSerializedPdxInstanceShouldReturnTrueIfItsPDXInstance() {
-    ObjectChunk chunk = createValueAsSerializedStoredObject(getValue());
-
-    byte[] serailizedValue = chunk.getSerializedValue();
-    serailizedValue[0] = DSCODE.PDX;
-    chunk.setSerializedValue(serailizedValue);
-
-    assertThat(chunk.isSerializedPdxInstance()).isTrue();
-
-    serailizedValue = chunk.getSerializedValue();
-    serailizedValue[0] = DSCODE.PDX_ENUM;
-    chunk.setSerializedValue(serailizedValue);
-
-    assertThat(chunk.isSerializedPdxInstance()).isTrue();
-
-    serailizedValue = chunk.getSerializedValue();
-    serailizedValue[0] = DSCODE.PDX_INLINE_ENUM;
-    chunk.setSerializedValue(serailizedValue);
-
-    assertThat(chunk.isSerializedPdxInstance()).isTrue();
-
-    chunk.release();
-  }
-
-  @Test
-  public void isSerializedPdxInstanceShouldReturnFalseIfItsNotPDXInstance() {
-    ObjectChunk chunk = createValueAsSerializedStoredObject(getValue());
-    assertThat(chunk.isSerializedPdxInstance()).isFalse();
-
-    chunk.release();
-  }
-
-  @Test
-  public void checkDataEqualsByChunk() {
-    ObjectChunk chunk1 = createValueAsSerializedStoredObject(getValue());
-    ObjectChunk sameAsChunk1 = chunk1;
-
-    assertThat(chunk1.checkDataEquals(sameAsChunk1)).isTrue();
-
-    ObjectChunk unserializedChunk = createValueAsUnserializedStoredObject(getValue());
-    assertThat(chunk1.checkDataEquals(unserializedChunk)).isFalse();
-
-    ObjectChunk chunkDifferBySize = createValueAsSerializedStoredObject(getValue());
-    chunkDifferBySize.setSize(0);
-    assertThat(chunk1.checkDataEquals(chunkDifferBySize)).isFalse();
-
-    ObjectChunk chunkDifferByValue = createValueAsSerializedStoredObject(Long.MAX_VALUE - 1);
-    assertThat(chunk1.checkDataEquals(chunkDifferByValue)).isFalse();
-
-    ObjectChunk newChunk1 = createValueAsSerializedStoredObject(getValue());
-    assertThat(chunk1.checkDataEquals(newChunk1)).isTrue();
-
-    chunk1.release();
-    unserializedChunk.release();
-    chunkDifferBySize.release();
-    chunkDifferByValue.release();
-    newChunk1.release();
-  }
-
-  @Test
-  public void checkDataEqualsBySerializedValue() {
-    ObjectChunk chunk = createValueAsSerializedStoredObject(getValue());
-    assertThat(chunk.checkDataEquals(new byte[1])).isFalse();
-
-    ObjectChunk chunkDifferByValue = createValueAsSerializedStoredObject(Long.MAX_VALUE - 1);
-    assertThat(chunk.checkDataEquals(chunkDifferByValue.getSerializedValue())).isFalse();
-
-    ObjectChunk newChunk = createValueAsSerializedStoredObject(getValue());
-    assertThat(chunk.checkDataEquals(newChunk.getSerializedValue())).isTrue();
-
-    chunk.release();
-    chunkDifferByValue.release();
-    newChunk.release();
-  }
-
-  @Test
-  public void getDecompressedBytesShouldReturnDecompressedBytesIfCompressed() {
-    Object regionEntryValue = getValue();
-    byte[] regionEntryValueAsBytes = convertValueToByteArray(regionEntryValue);
-
-    boolean isSerialized = true;
-    boolean isCompressed = true;
-
-    ObjectChunk chunk = (ObjectChunk) ma.allocateAndInitialize(regionEntryValueAsBytes, isSerialized, isCompressed);
-
-    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 = chunk.getDecompressedBytes(regionContext);
-
-    // verify the thing happened
-    verify(cacheStats, atLeastOnce()).startDecompression();
-    verify(compressor, times(1)).decompress(regionEntryValueAsBytes);
-    verify(cacheStats, atLeastOnce()).endDecompression(startTime);
-
-    assertArrayEquals(regionEntryValueAsBytes, bytes);
-
-    chunk.release();
-  }
-
-  @Test
-  public void incSizeShouldIncrementSize() {
-    ObjectChunk chunk = createValueAsSerializedStoredObject(getValue());
-
-    int beforeSize = chunk.getSize();
-
-    chunk.incSize(1);
-    assertThat(chunk.getSize()).isEqualTo(beforeSize + 1);
-
-    chunk.incSize(2);
-    assertThat(chunk.getSize()).isEqualTo(beforeSize + 1 + 2);
-
-    chunk.release();
-  }
-
-  @Test
-  public void readyForFreeShouldResetTheRefCount() {
-    ObjectChunk chunk = createValueAsSerializedStoredObject(getValue());
-
-    int refCountBeforeFreeing = chunk.getRefCount();
-    assertThat(refCountBeforeFreeing).isEqualTo(1);
-
-    chunk.readyForFree();
-
-    int refCountAfterFreeing = chunk.getRefCount();
-    assertThat(refCountAfterFreeing).isEqualTo(0);
-  }
-
-  @Test(expected = IllegalStateException.class)
-  public void readyForAllocationShouldThrowExceptionIfAlreadyAllocated() {
-    ObjectChunk chunk = createValueAsSerializedStoredObject(getValue());
-
-    // chunk is already allocated when we created it, so calling readyForAllocation should throw exception.
-    chunk.readyForAllocation();
-
-    chunk.release();
-  }
-
-  @Test
-  public void checkIsAllocatedShouldReturnIfAllocated() {
-    ObjectChunk chunk = createValueAsSerializedStoredObject(getValue());
-    chunk.checkIsAllocated();
-
-    chunk.release();
-  }
-
-  @Test(expected = IllegalStateException.class)
-  public void checkIsAllocatedShouldThrowExceptionIfNotAllocated() {
-    ObjectChunk chunk = createValueAsSerializedStoredObject(getValue());
-    chunk.release();
-    chunk.checkIsAllocated();
-
-    chunk.release();
-  }
-
-  @Test
-  public void sendToShouldWriteSerializedValueToDataOutputIfValueIsSerialized() throws IOException {
-    ObjectChunk chunk = createValueAsSerializedStoredObject(getValue());
-    ObjectChunk spyChunk = spy(chunk);
-
-    HeapDataOutputStream dataOutput = mock(HeapDataOutputStream.class);
-    ByteBuffer directByteBuffer = ByteBuffer.allocate(1024);
-
-    doReturn(directByteBuffer).when(spyChunk).createDirectByteBuffer();
-    doNothing().when(dataOutput).write(directByteBuffer);
-
-    spyChunk.sendTo(dataOutput);
-
-    verify(dataOutput, times(1)).write(directByteBuffer);
-
-    chunk.release();
-  }
-
-  @Test
-  public void sendToShouldWriteUnserializedValueToDataOutputIfValueIsUnserialized() throws IOException {
-    byte[] regionEntryValue = getValueAsByteArray();
-    ObjectChunk chunk = createValueAsUnserializedStoredObject(regionEntryValue);
-
-    // writeByte is a final method and cannot be mocked, so creating a real one
-    HeapDataOutputStream dataOutput = new HeapDataOutputStream(Version.CURRENT);
-
-    chunk.sendTo(dataOutput);
-
-    byte[] actual = dataOutput.toByteArray();
-
-    byte[] expected = new byte[regionEntryValue.length + 2];
-    expected[0] = DSCODE.BYTE_ARRAY;
-    expected[1] = (byte) regionEntryValue.length;
-    System.arraycopy(regionEntryValue, 0, expected, 2, regionEntryValue.length);
-
-    assertNotNull(dataOutput);
-    assertThat(actual).isEqualTo(expected);
-
-    chunk.release();
-  }
-
-  @Test
-  public void sendAsByteArrayShouldWriteValueToDataOutput() throws IOException {
-    byte[] regionEntryValue = getValueAsByteArray();
-    ObjectChunk chunk = createValueAsUnserializedStoredObject(regionEntryValue);
-
-    // writeByte is a final method and cannot be mocked, so creating a real one
-    HeapDataOutputStream dataOutput = new HeapDataOutputStream(Version.CURRENT);
-
-    chunk.sendAsByteArray(dataOutput);
-
-    byte[] actual = dataOutput.toByteArray();
-
-    byte[] expected = new byte[regionEntryValue.length + 1];
-    expected[0] = (byte) regionEntryValue.length;
-    System.arraycopy(regionEntryValue, 0, expected, 1, regionEntryValue.length);
-
-    assertNotNull(dataOutput);
-    assertThat(actual).isEqualTo(expected);
-
-    chunk.release();
-  }
-
-  @Test
-  public void createDirectByteBufferShouldCreateAByteBuffer() {
-    byte[] regionEntryValue = getValueAsByteArray();
-
-    ObjectChunk chunk = createValueAsUnserializedStoredObject(regionEntryValue);
-
-    ByteBuffer buffer = chunk.createDirectByteBuffer();
-
-    byte[] actual = new byte[regionEntryValue.length];
-    buffer.get(actual);
-
-    assertArrayEquals(regionEntryValue, actual);
-
-    chunk.release();
-  }
-
-  @Test
-  public void getDirectByteBufferShouldCreateAByteBuffer() {
-    byte[] regionEntryValue = getValueAsByteArray();
-    ObjectChunk chunk = createValueAsUnserializedStoredObject(regionEntryValue);
-
-    ByteBuffer buffer = chunk.createDirectByteBuffer();
-    long bufferAddress = ObjectChunk.getDirectByteBufferAddress(buffer);
-
-    // returned address should be starting of the value (after skipping HEADER_SIZE bytes)
-    assertEquals(chunk.getMemoryAddress() + ObjectChunk.OFF_HEAP_HEADER_SIZE, bufferAddress);
-
-    chunk.release();
-  }
-
-  @Test(expected = AssertionError.class)
-  public void getAddressForReadingShouldFailIfItsOutsideOfChunk() {
-    ObjectChunk chunk = createValueAsSerializedStoredObject(getValue());
-    chunk.getAddressForReading(0, chunk.getDataSize() + 1);
-
-    chunk.release();
-  }
-
-  @Test
-  public void getAddressForReadingShouldReturnDataAddressFromGivenOffset() {
-    ObjectChunk chunk = createValueAsSerializedStoredObject(getValue());
-
-    int offset = 1;
-    long requestedAddress = chunk.getAddressForReading(offset, 1);
-
-    assertThat(requestedAddress).isEqualTo(chunk.getBaseDataAddress() + offset);
-
-    chunk.release();
-  }
-
-  @Test
-  public void getSizeInBytesShouldReturnSize() {
-    ObjectChunk chunk = createValueAsSerializedStoredObject(getValue());
-    assertThat(chunk.getSizeInBytes()).isEqualTo(chunk.getSize());
-
-    chunk.release();
-  }
-
-  @Test(expected = AssertionError.class)
-  public void getUnsafeAddressShouldFailIfOffsetIsNegative() {
-    ObjectChunk chunk = createValueAsSerializedStoredObject(getValue());
-    chunk.getUnsafeAddress(-1, 1);
-
-    chunk.release();
-  }
-
-  @Test(expected = AssertionError.class)
-  public void getUnsafeAddressShouldFailIfSizeIsNegative() {
-    ObjectChunk chunk = createValueAsSerializedStoredObject(getValue());
-    chunk.getUnsafeAddress(1, -1);
-
-    chunk.release();
-  }
-
-  @Test(expected = AssertionError.class)
-  public void getUnsafeAddressShouldFailIfItsOutsideOfChunk() {
-    ObjectChunk chunk = createValueAsSerializedStoredObject(getValue());
-    chunk.getUnsafeAddress(0, chunk.getDataSize() + 1);
-
-    chunk.release();
-  }
-
-  @Test
-  public void getUnsafeAddressShouldReturnUnsafeAddress() {
-    ObjectChunk chunk = createValueAsSerializedStoredObject(getValue());
-
-    int offset = 1;
-    long unsafeAddress = chunk.getUnsafeAddress(offset, 1);
-
-    assertThat(unsafeAddress).isEqualTo(chunk.getBaseDataAddress() + offset);
-
-    chunk.release();
-  }
-
-  @Test(expected = AssertionError.class)
-  public void readByteAndWriteByteShouldFailIfOffsetIsOutside() {
-    ObjectChunk chunk = createValueAsSerializedStoredObject(getValue());
-
-    chunk.readByte(chunk.getDataSize() + 1);
-
-    chunk.writeByte(chunk.getDataSize() + 1, Byte.MAX_VALUE);
-
-    chunk.release();
-  }
-
-  @Test
-  public void writeByteShouldWriteAtCorrectLocation() {
-    ObjectChunk chunk = createValueAsSerializedStoredObject(getValue());
-
-    byte valueBeforeWrite = chunk.readByte(2);
-
-    Byte expected = Byte.MAX_VALUE;
-    chunk.writeByte(2, expected);
-
-    Byte actual = chunk.readByte(2);
-
-    assertThat(actual).isNotEqualTo(valueBeforeWrite);
-    assertThat(actual).isEqualTo(expected);
-
-    chunk.release();
-  }
-
-  @Test
-  public void retainShouldIncrementRefCount() {
-    ObjectChunk chunk = createValueAsUnserializedStoredObject(getValue());
-    assertThat(chunk.getRefCount()).isEqualTo(1);
-
-    chunk.retain();
-    assertThat(chunk.getRefCount()).isEqualTo(2);
-
-    chunk.retain();
-    assertThat(chunk.getRefCount()).isEqualTo(3);
-
-    chunk.release();
-    chunk.release();
-    chunk.release();
-    boolean retainAfterRelease = chunk.retain();
-
-    assertThat(retainAfterRelease).isFalse();
-  }
-
-  @Test(expected = IllegalStateException.class)
-  public void retainShouldThrowExceptionAfterMaxNumberOfTimesRetained() {
-    ObjectChunk chunk = createValueAsUnserializedStoredObject(getValue());
-
-    // loop though and invoke retain for MAX_REF_COUNT-1 times, as create chunk above counted as one reference
-    for (int i = 0; i < ObjectChunk.MAX_REF_COUNT - 1; i++)
-      chunk.retain();
-
-    // invoke for the one more time should throw exception
-    chunk.retain();
-  }
-
-  @Test
-  public void releaseShouldDecrementRefCount() {
-    ObjectChunk chunk = createValueAsUnserializedStoredObject(getValue());
-    assertThat(chunk.getRefCount()).isEqualTo(1);
-
-    chunk.retain();
-    chunk.retain();
-    assertThat(chunk.getRefCount()).isEqualTo(3);
-
-    chunk.release();
-    assertThat(chunk.getRefCount()).isEqualTo(2);
-
-    chunk.release();
-    assertThat(chunk.getRefCount()).isEqualTo(1);
-
-    chunk.retain();
-    chunk.release();
-    assertThat(chunk.getRefCount()).isEqualTo(1);
-
-    chunk.release();
-    assertThat(chunk.getRefCount()).isEqualTo(0);
-  }
-
-  @Test(expected = IllegalStateException.class)
-  public void releaseShouldThrowExceptionIfChunkIsAlreadyReleased() {
-    ObjectChunk chunk = createValueAsUnserializedStoredObject(getValue());
-    chunk.release();
-    chunk.release();
-  }
-
-  @Test
-  public void testToStringForOffHeapByteSource() {
-    ObjectChunk chunk = createValueAsUnserializedStoredObject(getValue());
-
-    String expected = ":<dataSize=" + chunk.getDataSize() + " refCount=" + chunk.getRefCount() + " addr=" + Long.toHexString(chunk.getMemoryAddress()) + ">";
-    assertThat(chunk.toStringForOffHeapByteSource()).endsWith(expected);
-
-    // test toString
-    ObjectChunk spy = spy(chunk);
-    spy.toString();
-    verify(spy, times(1)).toStringForOffHeapByteSource();
-
-    chunk.release();
-  }
-
-  @Test
-  public void getStateShouldReturnAllocatedIfRefCountIsGreaterThanZero() {
-    ObjectChunk chunk = createValueAsUnserializedStoredObject(getValue());
-    assertEquals(State.ALLOCATED, chunk.getState());
-
-    chunk.release();
-  }
-
-  @Test
-  public void getStateShouldReturnDeallocatedIfRefCountIsZero() {
-    ObjectChunk chunk = createValueAsUnserializedStoredObject(getValue());
-    chunk.release();
-    assertEquals(State.DEALLOCATED, chunk.getState());
-  }
-
-  @Test(expected = UnsupportedOperationException.class)
-  public void getNextBlockShouldThrowUnSupportedOperationException() {
-    ObjectChunk chunk = createValueAsUnserializedStoredObject(getValue());
-    chunk.getNextBlock();
-
-    chunk.release();
-  }
-
-  @Test
-  public void getBlockSizeShouldBeSameSameGetSize() {
-    ObjectChunk chunk = createValueAsUnserializedStoredObject(getValue());
-    assertEquals(chunk.getSize(), chunk.getBlockSize());
-
-    chunk.release();
-  }
-
-  @Test(expected = UnsupportedOperationException.class)
-  public void copyBytesShouldThrowUnSupportedOperationException() {
-    ObjectChunk chunk = createValueAsUnserializedStoredObject(getValue());
-    chunk.copyBytes(1, 2, 1);
-
-    chunk.release();
-  }
-
-  @Test(expected = UnsupportedOperationException.class)
-  public void getSlabIdShouldThrowUnSupportedOperationException() {
-    ObjectChunk chunk = createValueAsUnserializedStoredObject(getValue());
-    chunk.getSlabId();
-
-    chunk.release();
-  }
-
-  @Test
-  public void getFreeListIdShouldReturnMinusOne() {
-    ObjectChunk chunk = createValueAsUnserializedStoredObject(getValue());
-    assertThat(chunk.getFreeListId()).isEqualTo(-1);
-
-    chunk.release();
-  }
-
-  @Test
-  public void getDataTypeShouldReturnNull() {
-    ObjectChunk chunk = createValueAsUnserializedStoredObject(getValue());
-    assertThat(chunk.getDataType()).isNull();
-
-    chunk.release();
-  }
-
-  @Test
-  public void getDataDataShouldReturnNull() {
-    ObjectChunk chunk = createValueAsUnserializedStoredObject(getValue());
-    assertThat(chunk.getDataValue()).isNull();
-  }
-
-  @Test(expected = UnsupportedOperationException.class)
-  public void getRawBytesShouldThrowExceptionIfValueIsCompressed() {
-    Object regionEntryValue = getValue();
-    byte[] regionEntryValueAsBytes = convertValueToByteArray(regionEntryValue);
-
-    boolean isSerialized = true;
-    boolean isCompressed = true;
-
-    ObjectChunk chunk = (ObjectChunk) ma.allocateAndInitialize(regionEntryValueAsBytes, isSerialized, isCompressed);
-
-    chunk.getRawBytes();
-
-    chunk.release();
-  }
-
-  @Test
-  public void getSerializedValueShouldSerializeTheValue() {
-    Object regionEntryValue = getValue();
-    byte[] regionEntryValueAsBytes = convertValueToByteArray(regionEntryValue);
-
-    boolean isSerialized = false;
-    boolean isCompressed = false;
-
-    ObjectChunk chunk = (ObjectChunk) ma.allocateAndInitialize(regionEntryValueAsBytes, isSerialized, isCompressed);
-
-    byte[] serializedValue = chunk.getSerializedValue();
-
-    assertThat(serializedValue).isEqualTo(EntryEventImpl.serialize(regionEntryValueAsBytes));
-
-    chunk.release();
-  }
-
-  @Test
-  public void fillShouldFillTheChunk() {
-    boolean isSerialized = false;
-    boolean isCompressed = false;
-
-    ObjectChunk chunk = (ObjectChunk) ma.allocateAndInitialize(new byte[100], isSerialized, isCompressed);
-
-    // first fill the unused part with FILL_PATTERN
-    ObjectChunk.fill(chunk.getMemoryAddress());
-
-    // Validate that it is filled
-    chunk.validateFill();
-
-    chunk.release();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/ObjectChunkSliceJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/ObjectChunkSliceJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/ObjectChunkSliceJUnitTest.java
deleted file mode 100644
index fe55910..0000000
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/ObjectChunkSliceJUnitTest.java
+++ /dev/null
@@ -1,72 +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.assertEquals;
-import static org.junit.Assert.assertNotNull;
-
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
-
-@Category(UnitTest.class)
-public class ObjectChunkSliceJUnitTest extends ObjectChunkJUnitTest {
-
-  @Test
-  public void sliceShouldHaveAValidDataSize() {
-    int position = 1;
-    int end = 2;
-
-    ObjectChunk chunk = createValueAsUnserializedStoredObject(getValue());
-    ObjectChunkSlice slice = (ObjectChunkSlice) chunk.slice(position, end);
-
-    assertNotNull(slice);
-    assertEquals(ObjectChunkSlice.class, slice.getClass());
-
-    assertEquals(end - position, slice.getDataSize());
-  }
-
-  @Test
-  public void sliceShouldHaveAValidBaseDataAddress() {
-    int position = 1;
-    int end = 2;
-
-    ObjectChunk chunk = createValueAsUnserializedStoredObject(getValue());
-    ObjectChunkSlice slice = (ObjectChunkSlice) chunk.slice(position, end);
-
-    assertNotNull(slice);
-    assertEquals(ObjectChunkSlice.class, slice.getClass());
-
-    assertEquals(chunk.getBaseDataAddress() + position, slice.getBaseDataAddress());
-  }
-
-  @Test
-  public void sliceShouldHaveAValidBaseOffset() {
-    int position = 1;
-    int end = 2;
-
-    ObjectChunk chunk = createValueAsUnserializedStoredObject(getValue());
-    ObjectChunkSlice slice = (ObjectChunkSlice) chunk.slice(position, end);
-
-    assertNotNull(slice);
-    assertEquals(ObjectChunkSlice.class, slice.getClass());
-
-    assertEquals(chunk.getBaseDataOffset() + position, slice.getBaseDataOffset());
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/ObjectChunkWithHeapFormJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/ObjectChunkWithHeapFormJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/ObjectChunkWithHeapFormJUnitTest.java
deleted file mode 100644
index 4486845..0000000
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/ObjectChunkWithHeapFormJUnitTest.java
+++ /dev/null
@@ -1,64 +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.assertArrayEquals;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNotSame;
-import static org.junit.Assert.assertSame;
-
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
-
-@Category(UnitTest.class)
-public class ObjectChunkWithHeapFormJUnitTest extends ObjectChunkJUnitTest {
-
-  @Test
-  public void getRawBytesShouldReturnCachedHeapForm() {
-    ObjectChunk chunk = createValueAsUnserializedStoredObject(getValue());
-
-    byte[] valueInBytes = getValueAsByteArray();
-    ObjectChunkWithHeapForm heapForm = new ObjectChunkWithHeapForm(chunk, valueInBytes);
-
-    assertNotNull(heapForm);
-
-    assertSame(valueInBytes, heapForm.getRawBytes());
-  }
-
-  @Test
-  public void getChunkWithoutHeapFormShouldReturnGemFireChunk() {
-    ObjectChunk chunk = createValueAsSerializedStoredObject(getValue());
-
-    byte[] valueInBytes = getValueAsByteArray();
-    ObjectChunkWithHeapForm heapForm = new ObjectChunkWithHeapForm(chunk, valueInBytes);
-
-    ObjectChunk chunkWithOutHeapForm = heapForm.getChunkWithoutHeapForm();
-
-    assertNotNull(chunkWithOutHeapForm);
-    assertEquals(ObjectChunk.class, chunkWithOutHeapForm.getClass());
-
-    assertEquals(chunk, heapForm.getChunkWithoutHeapForm());
-
-    assertEquals(chunk.getMemoryAddress(), chunkWithOutHeapForm.getMemoryAddress());
-    assertArrayEquals(chunk.getRawBytes(), chunkWithOutHeapForm.getRawBytes());
-    assertNotSame(valueInBytes, chunkWithOutHeapForm.getRawBytes());
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/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..989abfc 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;
@@ -102,7 +102,7 @@ public class OffHeapHelperJUnitTest extends AbstractStoredObjectTestBase {
   }
 
   @Override
-  protected MemoryChunkWithRefCount createValueAsUnserializedStoredObject(Object value) {
+  protected StoredObject createValueAsUnserializedStoredObject(Object value) {
     byte[] valueInByteArray;
     if (value instanceof Long) {
       valueInByteArray = convertValueToByteArray(value);
@@ -113,23 +113,23 @@ public class OffHeapHelperJUnitTest extends AbstractStoredObjectTestBase {
     boolean isSerialized = false;
     boolean isCompressed = false;
 
-    MemoryChunkWithRefCount createdObject = createChunk(valueInByteArray, isSerialized, isCompressed);
+    StoredObject createdObject = createChunk(valueInByteArray, isSerialized, isCompressed);
     return createdObject;
   }
 
   @Override
-  protected MemoryChunkWithRefCount createValueAsSerializedStoredObject(Object value) {
+  protected StoredObject createValueAsSerializedStoredObject(Object value) {
     byte[] valueInSerializedByteArray = EntryEventImpl.serialize(value);
 
     boolean isSerialized = true;
     boolean isCompressed = false;
 
-    MemoryChunkWithRefCount createdObject = createChunk(valueInSerializedByteArray, isSerialized, isCompressed);
+    StoredObject createdObject = createChunk(valueInSerializedByteArray, isSerialized, isCompressed);
     return createdObject;
   }
 
-  private ObjectChunk createChunk(byte[] v, boolean isSerialized, boolean isCompressed) {
-    ObjectChunk chunk = (ObjectChunk) ma.allocateAndInitialize(v, isSerialized, isCompressed);
+  private OffHeapStoredObject createChunk(byte[] v, boolean isSerialized, boolean isCompressed) {
+    OffHeapStoredObject chunk = (OffHeapStoredObject) ma.allocateAndInitialize(v, isSerialized, isCompressed);
     return chunk;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/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..ae1b35d 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,11 +163,11 @@ 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);
+      StoredObject mc1 = (StoredObject)ma.allocateAndInitialize(data, false, false);
       assertEquals(data.length+perObjectOverhead(), ma.getUsedMemory());
       assertEquals(offHeapSize-(data.length+perObjectOverhead()), ma.getFreeMemory());
       byte[] data2 = new byte[data.length];
-      mc1.readBytes(0, data2);
+      mc1.readDataBytes(0, data2);
       assertTrue(Arrays.equals(data, data2));
       mc1.release();
       assertEquals(offHeapSize, ma.getFreeMemory());
@@ -540,11 +540,11 @@ public abstract class OffHeapRegionBase {
     @Released(OffHeapIdentifier.TEST_OFF_HEAP_REGION_BASE_LISTENER)
     @Override
     public void close() {
-      if (this.ohOldValue instanceof ObjectChunk) {
-        ((ObjectChunk)this.ohOldValue).release();
+      if (this.ohOldValue instanceof OffHeapStoredObject) {
+        ((OffHeapStoredObject)this.ohOldValue).release();
       }
-      if (this.ohNewValue instanceof ObjectChunk) {
-        ((ObjectChunk)this.ohNewValue).release();
+      if (this.ohNewValue instanceof OffHeapStoredObject) {
+        ((OffHeapStoredObject)this.ohNewValue).release();
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapRegionEntryHelperJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapRegionEntryHelperJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapRegionEntryHelperJUnitTest.java
index 5d53109..540bba5 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapRegionEntryHelperJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapRegionEntryHelperJUnitTest.java
@@ -54,7 +54,7 @@ import com.gemstone.gemfire.test.junit.categories.UnitTest;
 @Category(UnitTest.class)
 @RunWith(PowerMockRunner.class)
 @PowerMockIgnore("*.UnitTest")
-@PrepareForTest({ ObjectChunk.class, OffHeapRegionEntryHelper.class })
+@PrepareForTest({ OffHeapStoredObject.class, OffHeapRegionEntryHelper.class })
 public class OffHeapRegionEntryHelperJUnitTest {
 
   private static final Long VALUE_IS_NOT_ENCODABLE = 0L;
@@ -75,13 +75,13 @@ public class OffHeapRegionEntryHelperJUnitTest {
     SimpleMemoryAllocatorImpl.freeOffHeapMemory();
   }
 
-  private ObjectChunk createChunk(Object value) {
+  private OffHeapStoredObject createChunk(Object value) {
     byte[] v = EntryEventImpl.serialize(value);
 
     boolean isSerialized = true;
     boolean isCompressed = false;
 
-    ObjectChunk chunk = (ObjectChunk) ma.allocateAndInitialize(v, isSerialized, isCompressed);
+    OffHeapStoredObject chunk = (OffHeapStoredObject) ma.allocateAndInitialize(v, isSerialized, isCompressed);
 
     return chunk;
   }
@@ -294,8 +294,8 @@ public class OffHeapRegionEntryHelperJUnitTest {
 
   @Test
   public void isOffHeapShouldReturnTrueIfAddressIsOnOffHeap() {
-    ObjectChunk value = createChunk(Long.MAX_VALUE);
-    assertThat(OffHeapRegionEntryHelper.isOffHeap(value.getMemoryAddress())).isTrue();
+    OffHeapStoredObject value = createChunk(Long.MAX_VALUE);
+    assertThat(OffHeapRegionEntryHelper.isOffHeap(value.getAddress())).isTrue();
   }
 
   @Test
@@ -327,28 +327,28 @@ public class OffHeapRegionEntryHelperJUnitTest {
     long oldAddress = 1L;
 
     // testing when the newValue is a chunk
-    ObjectChunk newValue = createChunk(Long.MAX_VALUE);
+    OffHeapStoredObject newValue = createChunk(Long.MAX_VALUE);
     // mock region entry methods required for test
     when(re.getAddress()).thenReturn(oldAddress);
-    when(re.setAddress(oldAddress, newValue.getMemoryAddress())).thenReturn(Boolean.TRUE);
+    when(re.setAddress(oldAddress, newValue.getAddress())).thenReturn(Boolean.TRUE);
 
     // invoke the method under test
     OffHeapRegionEntryHelper.setValue(re, newValue);
 
     // verify oldAddress is replaced with newAddress
-    verify(re, times(1)).setAddress(oldAddress, newValue.getMemoryAddress());
+    verify(re, times(1)).setAddress(oldAddress, newValue.getAddress());
     // resetting the spy in-order to re-use
     reset(re);
 
     // testing when the newValue is DataAsAddress
-    DataAsAddress newAddress1 = new DataAsAddress(2L);
+    TinyStoredObject newAddress1 = new TinyStoredObject(2L);
     // mock region entry methods required for test
     when(re.getAddress()).thenReturn(oldAddress);
-    when(re.setAddress(oldAddress, newAddress1.getEncodedAddress())).thenReturn(Boolean.TRUE);
+    when(re.setAddress(oldAddress, newAddress1.getAddress())).thenReturn(Boolean.TRUE);
     OffHeapRegionEntryHelper.setValue(re, newAddress1);
 
     // verify oldAddress is replaced with newAddress
-    verify(re, times(1)).setAddress(oldAddress, newAddress1.getEncodedAddress());
+    verify(re, times(1)).setAddress(oldAddress, newAddress1.getAddress());
     reset(re);
 
     // Testing when newValue is Token Objects
@@ -440,27 +440,27 @@ public class OffHeapRegionEntryHelperJUnitTest {
     // mock region entry
     OffHeapRegionEntry re = mock(OffHeapRegionEntry.class);
 
-    ObjectChunk oldValue = createChunk(Long.MAX_VALUE);
-    ObjectChunk newValue = createChunk(Long.MAX_VALUE - 1);
+    OffHeapStoredObject oldValue = createChunk(Long.MAX_VALUE);
+    OffHeapStoredObject newValue = createChunk(Long.MAX_VALUE - 1);
 
     // mock Chunk static methods - in-order to verify that release is called
-    PowerMockito.spy(ObjectChunk.class);
-    PowerMockito.doNothing().when(ObjectChunk.class);
-    ObjectChunk.release(oldValue.getMemoryAddress());
+    PowerMockito.spy(OffHeapStoredObject.class);
+    PowerMockito.doNothing().when(OffHeapStoredObject.class);
+    OffHeapStoredObject.release(oldValue.getAddress());
 
     // mock region entry methods required for test
-    when(re.getAddress()).thenReturn(oldValue.getMemoryAddress());
-    when(re.setAddress(oldValue.getMemoryAddress(), newValue.getMemoryAddress())).thenReturn(Boolean.TRUE);
+    when(re.getAddress()).thenReturn(oldValue.getAddress());
+    when(re.setAddress(oldValue.getAddress(), newValue.getAddress())).thenReturn(Boolean.TRUE);
 
     // invoke the method under test
     OffHeapRegionEntryHelper.setValue(re, newValue);
 
     // verify oldAddress is changed to newAddress
-    verify(re, times(1)).setAddress(oldValue.getMemoryAddress(), newValue.getMemoryAddress());
+    verify(re, times(1)).setAddress(oldValue.getAddress(), newValue.getAddress());
 
     // verify oldAddress is released
     PowerMockito.verifyStatic();
-    ObjectChunk.release(oldValue.getMemoryAddress());
+    OffHeapStoredObject.release(oldValue.getAddress());
   }
 
   @Test
@@ -472,26 +472,26 @@ public class OffHeapRegionEntryHelperJUnitTest {
     long oldAddress = OffHeapRegionEntryHelper.encodeDataAsAddress(oldData, false, false);
 
     byte[] newData = ByteBuffer.allocate(Integer.SIZE / Byte.SIZE).putInt((Integer) Integer.MAX_VALUE - 1).array();
-    DataAsAddress newAddress = new DataAsAddress(OffHeapRegionEntryHelper.encodeDataAsAddress(newData, false, false));
+    TinyStoredObject newAddress = new TinyStoredObject(OffHeapRegionEntryHelper.encodeDataAsAddress(newData, false, false));
 
     // mock Chunk static methods - in-order to verify that release is never called
-    PowerMockito.spy(ObjectChunk.class);
-    PowerMockito.doNothing().when(ObjectChunk.class);
-    ObjectChunk.release(oldAddress);
+    PowerMockito.spy(OffHeapStoredObject.class);
+    PowerMockito.doNothing().when(OffHeapStoredObject.class);
+    OffHeapStoredObject.release(oldAddress);
 
     // mock region entry methods required for test
     when(re.getAddress()).thenReturn(oldAddress);
-    when(re.setAddress(oldAddress, newAddress.getEncodedAddress())).thenReturn(Boolean.TRUE);
+    when(re.setAddress(oldAddress, newAddress.getAddress())).thenReturn(Boolean.TRUE);
 
     // invoke the method under test
     OffHeapRegionEntryHelper.setValue(re, newAddress);
 
     // verify oldAddress is changed to newAddress
-    verify(re, times(1)).setAddress(oldAddress, newAddress.getEncodedAddress());
+    verify(re, times(1)).setAddress(oldAddress, newAddress.getAddress());
 
     // verify that release is never called as the old address is not on offheap
     PowerMockito.verifyStatic(never());
-    ObjectChunk.release(oldAddress);
+    OffHeapStoredObject.release(oldAddress);
   }
 
   @Test
@@ -505,9 +505,9 @@ public class OffHeapRegionEntryHelperJUnitTest {
     long newAddress = OffHeapRegionEntryHelper.REMOVED_PHASE2_ADDRESS;
 
     // mock Chunk static methods - in-order to verify that release is never called
-    PowerMockito.spy(ObjectChunk.class);
-    PowerMockito.doNothing().when(ObjectChunk.class);
-    ObjectChunk.release(oldAddress);
+    PowerMockito.spy(OffHeapStoredObject.class);
+    PowerMockito.doNothing().when(OffHeapStoredObject.class);
+    OffHeapStoredObject.release(oldAddress);
 
     // mock region entry methods required for test
     when(re.getAddress()).thenReturn(oldAddress);
@@ -521,7 +521,7 @@ public class OffHeapRegionEntryHelperJUnitTest {
 
     // verify that release is never called as the old address is not on offheap
     PowerMockito.verifyStatic(never());
-    ObjectChunk.release(oldAddress);
+    OffHeapStoredObject.release(oldAddress);
   }
 
   @Test(expected = IllegalStateException.class)
@@ -541,10 +541,10 @@ public class OffHeapRegionEntryHelperJUnitTest {
     // mock region entry
     OffHeapRegionEntry re = mock(OffHeapRegionEntry.class);
 
-    ObjectChunk chunk = createChunk(Long.MAX_VALUE);
+    OffHeapStoredObject chunk = createChunk(Long.MAX_VALUE);
 
     // mock region entry methods required for test
-    when(re.getAddress()).thenReturn(chunk.getMemoryAddress());
+    when(re.getAddress()).thenReturn(chunk.getAddress());
     Token token = OffHeapRegionEntryHelper.getValueAsToken(re);
 
     assertThat(token).isEqualTo(Token.NOT_A_TOKEN);
@@ -627,10 +627,10 @@ public class OffHeapRegionEntryHelperJUnitTest {
 
   @Test
   public void addressToObjectShouldReturnValueFromChunk() {
-    ObjectChunk expected = createChunk(Long.MAX_VALUE);
-    Object actual = OffHeapRegionEntryHelper.addressToObject(expected.getMemoryAddress(), false, null);
+    OffHeapStoredObject expected = createChunk(Long.MAX_VALUE);
+    Object actual = OffHeapRegionEntryHelper.addressToObject(expected.getAddress(), false, null);
 
-    assertThat(actual).isInstanceOf(ObjectChunk.class);
+    assertThat(actual).isInstanceOf(OffHeapStoredObject.class);
     assertThat(actual).isEqualTo(expected);
   }
 
@@ -640,7 +640,7 @@ public class OffHeapRegionEntryHelperJUnitTest {
     boolean isSerialized = true;
     boolean isCompressed = true;
 
-    ObjectChunk chunk = (ObjectChunk) ma.allocateAndInitialize(data, isSerialized, isCompressed);
+    OffHeapStoredObject chunk = (OffHeapStoredObject) ma.allocateAndInitialize(data, isSerialized, isCompressed);
 
     // create the mock context
     RegionEntryContext regionContext = mock(RegionEntryContext.class);
@@ -655,7 +655,7 @@ public class OffHeapRegionEntryHelperJUnitTest {
     when(regionContext.getCachePerfStats()).thenReturn(cacheStats);
     when(cacheStats.startDecompression()).thenReturn(startTime);
 
-    Object actual = OffHeapRegionEntryHelper.addressToObject(chunk.getMemoryAddress(), true, regionContext);
+    Object actual = OffHeapRegionEntryHelper.addressToObject(chunk.getAddress(), true, regionContext);
 
     assertThat(actual).isInstanceOf(VMCachedDeserializable.class);
 
@@ -669,7 +669,7 @@ public class OffHeapRegionEntryHelperJUnitTest {
     boolean isSerialized = false;
     boolean isCompressed = true;
 
-    ObjectChunk chunk = (ObjectChunk) ma.allocateAndInitialize(data, isSerialized, isCompressed);
+    OffHeapStoredObject chunk = (OffHeapStoredObject) ma.allocateAndInitialize(data, isSerialized, isCompressed);
 
     // create the mock context
     RegionEntryContext regionContext = mock(RegionEntryContext.class);
@@ -684,7 +684,7 @@ public class OffHeapRegionEntryHelperJUnitTest {
     when(regionContext.getCachePerfStats()).thenReturn(cacheStats);
     when(cacheStats.startDecompression()).thenReturn(startTime);
 
-    Object actual = OffHeapRegionEntryHelper.addressToObject(chunk.getMemoryAddress(), true, regionContext);
+    Object actual = OffHeapRegionEntryHelper.addressToObject(chunk.getAddress(), true, regionContext);
 
     assertThat(actual).isInstanceOf(byte[].class);
     assertThat(actual).isEqualTo(data);
@@ -695,10 +695,10 @@ public class OffHeapRegionEntryHelperJUnitTest {
     byte[] data = ByteBuffer.allocate(Integer.SIZE / Byte.SIZE).putInt(Integer.MAX_VALUE).array();
     long address = OffHeapRegionEntryHelper.encodeDataAsAddress(data, false, false);
 
-    DataAsAddress expected = new DataAsAddress(address);
+    TinyStoredObject expected = new TinyStoredObject(address);
     Object actual = OffHeapRegionEntryHelper.addressToObject(address, false, null);
 
-    assertThat(actual).isInstanceOf(DataAsAddress.class);
+    assertThat(actual).isInstanceOf(TinyStoredObject.class);
     assertThat(actual).isEqualTo(expected);
   }
 
@@ -795,7 +795,7 @@ public class OffHeapRegionEntryHelperJUnitTest {
     boolean isCompressed = true;
 
     long address = OffHeapRegionEntryHelper.encodeDataAsAddress(data, isSerialized, isCompressed);
-    DataAsAddress daa = new DataAsAddress(address);
+    TinyStoredObject daa = new TinyStoredObject(address);
 
     int actualLength = OffHeapRegionEntryHelper.getSerializedLengthFromDataAsAddress(daa);
 
@@ -804,7 +804,7 @@ public class OffHeapRegionEntryHelperJUnitTest {
 
   @Test
   public void getSerializedLengthFromDataAsAddressShouldReturnZeroForNonEncodedAddress() {
-    DataAsAddress nonEncodedAddress = new DataAsAddress(100000L);
+    TinyStoredObject nonEncodedAddress = new TinyStoredObject(100000L);
     int actualLength = OffHeapRegionEntryHelper.getSerializedLengthFromDataAsAddress(nonEncodedAddress);
     assertThat(actualLength).isZero();
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapStoredObjectAddressStackJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapStoredObjectAddressStackJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapStoredObjectAddressStackJUnitTest.java
new file mode 100644
index 0000000..8040bf7
--- /dev/null
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapStoredObjectAddressStackJUnitTest.java
@@ -0,0 +1,289 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.gemstone.gemfire.internal.offheap;
+
+import static org.junit.Assert.*;
+import static org.mockito.Mockito.*;
+
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.mockito.listeners.InvocationListener;
+import org.mockito.listeners.MethodInvocationReport;
+
+import com.gemstone.gemfire.LogWriter;
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+
+@Category(UnitTest.class)
+public class OffHeapStoredObjectAddressStackJUnitTest {
+  static {
+    ClassLoader.getSystemClassLoader().setDefaultAssertionStatus(true);
+  }
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+  }
+
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+  }
+
+  @Before
+  public void setUp() throws Exception {
+  }
+
+  @After
+  public void tearDown() throws Exception {
+  }
+
+  @Test
+  public void addressZeroCausesStackToBeEmpty() {
+    OffHeapStoredObjectAddressStack stack = new OffHeapStoredObjectAddressStack(0L);
+    assertEquals(true, stack.isEmpty());
+  }
+
+  @Test
+  public void defaultStackIsEmpty() {
+    OffHeapStoredObjectAddressStack stack = new OffHeapStoredObjectAddressStack();
+    assertEquals(true, stack.isEmpty());
+  }
+
+  @Test
+  public void defaultStackReturnsZeroFromTop() {
+    OffHeapStoredObjectAddressStack stack = new OffHeapStoredObjectAddressStack();
+    assertEquals(0L, stack.getTopAddress());
+  }
+  
+  @Test
+  public void defaultStackReturnsZeroFromPoll() {
+    OffHeapStoredObjectAddressStack stack = new OffHeapStoredObjectAddressStack();
+    assertEquals(0L, stack.poll());
+  }
+  
+  @Test
+  public void defaultStackReturnsZeroFromClear() {
+    OffHeapStoredObjectAddressStack stack = new OffHeapStoredObjectAddressStack();
+    assertEquals(0L, stack.clear());
+    assertEquals(true, stack.isEmpty());
+  }
+  
+  @Test
+  public void defaultStackLogsNothing() {
+    OffHeapStoredObjectAddressStack stack = new OffHeapStoredObjectAddressStack();
+    LogWriter lw = mock(LogWriter.class, withSettings().invocationListeners(new InvocationListener() {
+      @Override
+      public void reportInvocation(MethodInvocationReport methodInvocationReport) {
+        fail("Unexpected invocation");
+      }
+    }));
+    stack.logSizes(lw, "should not be used");
+  }
+  
+  @Test
+  public void defaultStackComputeSizeIsZero() {
+    OffHeapStoredObjectAddressStack stack = new OffHeapStoredObjectAddressStack();
+    assertEquals(0L, stack.computeTotalSize());
+  }
+  
+  @Test
+  public void stackCreatedWithAddressIsNotEmpty() {
+    SlabImpl slab = new SlabImpl(1024);
+    try {
+      SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new SlabImpl[]{slab});
+      OffHeapStoredObject chunk = (OffHeapStoredObject) ma.allocate(100);
+
+      OffHeapStoredObjectAddressStack stack = new OffHeapStoredObjectAddressStack(chunk.getAddress());
+      assertEquals(false, stack.isEmpty());
+    } finally {
+      SimpleMemoryAllocatorImpl.freeOffHeapMemory();
+    }
+  }
+
+  @Test
+  public void stackWithChunkIsNotEmpty() {
+    SlabImpl slab = new SlabImpl(1024);
+    try {
+      SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new SlabImpl[]{slab});
+      OffHeapStoredObject chunk = (OffHeapStoredObject) ma.allocate(100);
+
+      OffHeapStoredObjectAddressStack stack = new OffHeapStoredObjectAddressStack();
+      stack.offer(chunk.getAddress());
+      assertEquals(false, stack.isEmpty());
+    } finally {
+      SimpleMemoryAllocatorImpl.freeOffHeapMemory();
+    }
+  }
+
+  @Test
+  public void stackWithChunkTopEqualsAddress() {
+    SlabImpl slab = new SlabImpl(1024);
+    try {
+      SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new SlabImpl[]{slab});
+      OffHeapStoredObject chunk = (OffHeapStoredObject) ma.allocate(100);
+
+      long addr = chunk.getAddress();
+      OffHeapStoredObjectAddressStack stack = new OffHeapStoredObjectAddressStack();
+      stack.offer(addr);
+      assertEquals(addr, stack.getTopAddress());
+    } finally {
+      SimpleMemoryAllocatorImpl.freeOffHeapMemory();
+    }
+  }
+
+  @Test
+  public void addressZeroOfferCausesFailedAssertion() {
+    OffHeapStoredObjectAddressStack stack = new OffHeapStoredObjectAddressStack(0L);
+    try {
+      stack.offer(0);
+      fail("expected AssertionError");
+    } catch (AssertionError expected) {
+    }
+  }
+
+
+  @Test
+  public void stackWithChunkClearReturnsAddressAndEmptiesStack() {
+    SlabImpl slab = new SlabImpl(1024);
+    try {
+      SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new SlabImpl[]{slab});
+      OffHeapStoredObject chunk = (OffHeapStoredObject) ma.allocate(100);
+
+      long addr = chunk.getAddress();
+      OffHeapStoredObjectAddressStack stack = new OffHeapStoredObjectAddressStack();
+      stack.offer(addr);
+      long clearAddr = stack.clear();
+      assertEquals(addr, clearAddr);
+      assertEquals(true, stack.isEmpty());
+    } finally {
+      SimpleMemoryAllocatorImpl.freeOffHeapMemory();
+    }
+  }
+
+  @Test
+  public void stackWithChunkPollReturnsAddressAndEmptiesStack() {
+    SlabImpl slab = new SlabImpl(1024);
+    try {
+      SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new SlabImpl[]{slab});
+      OffHeapStoredObject chunk = (OffHeapStoredObject) ma.allocate(100);
+
+      long addr = chunk.getAddress();
+      OffHeapStoredObjectAddressStack stack = new OffHeapStoredObjectAddressStack();
+      stack.offer(addr);
+      long pollAddr = stack.poll();
+      assertEquals(addr, pollAddr);
+      assertEquals(true, stack.isEmpty());
+    } finally {
+      SimpleMemoryAllocatorImpl.freeOffHeapMemory();
+    }
+  }
+
+  @Test
+  public void stackWithChunkTotalSizeIsChunkSize() {
+    SlabImpl slab = new SlabImpl(1024);
+    try {
+      SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new SlabImpl[]{slab});
+      OffHeapStoredObject chunk = (OffHeapStoredObject) ma.allocate(100);
+      int chunkSize = chunk.getSize();
+
+      long addr = chunk.getAddress();
+      OffHeapStoredObjectAddressStack stack = new OffHeapStoredObjectAddressStack();
+      stack.offer(addr);
+      assertEquals(chunkSize, stack.computeTotalSize());
+    } finally {
+      SimpleMemoryAllocatorImpl.freeOffHeapMemory();
+    }
+  }
+
+
+  @Test
+  public void stackWithChunkLogShowsMsgAndSize() {
+    SlabImpl slab = new SlabImpl(1024);
+    try {
+      SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new SlabImpl[]{slab});
+      OffHeapStoredObject chunk = (OffHeapStoredObject) ma.allocate(100);
+      int chunkSize = chunk.getSize();
+
+      long addr = chunk.getAddress();
+      OffHeapStoredObjectAddressStack stack = new OffHeapStoredObjectAddressStack();
+      stack.offer(addr);
+      LogWriter lw = mock(LogWriter.class);
+      stack.logSizes(lw, "foo");
+      verify(lw).info("foo"+chunkSize);
+    } finally {
+      SimpleMemoryAllocatorImpl.freeOffHeapMemory();
+    }
+  }
+  
+  private class TestableSyncChunkStack extends OffHeapStoredObjectAddressStack {
+    public boolean doConcurrentMod = true;
+    public int chunk2Size;
+    private SimpleMemoryAllocatorImpl ma;
+    TestableSyncChunkStack(SimpleMemoryAllocatorImpl ma) {
+      this.ma = ma;
+    }
+    @Override
+    protected void testHookDoConcurrentModification() {
+      if (doConcurrentMod) {
+        doConcurrentMod = false;
+        OffHeapStoredObject chunk2 = (OffHeapStoredObject) ma.allocate(50);
+        this.chunk2Size = chunk2.getSize();
+        this.offer(chunk2.getAddress());
+      }
+    }
+  }
+  @Test
+  public void stackWithChunkTotalSizeIsChunkSizeWithConcurrentMod() {
+    SlabImpl slab = new SlabImpl(1024);
+    try {
+      SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new SlabImpl[]{slab});
+      OffHeapStoredObject chunk = (OffHeapStoredObject) ma.allocate(100);
+      int chunkSize = chunk.getSize();
+
+      long addr = chunk.getAddress();
+      TestableSyncChunkStack stack = new TestableSyncChunkStack(ma);
+      stack.offer(addr);
+      long totalSize = stack.computeTotalSize();
+      assertEquals("chunkSize=" + chunkSize + " chunk2Size=" + stack.chunk2Size, chunkSize + stack.chunk2Size, totalSize);
+    } finally {
+      SimpleMemoryAllocatorImpl.freeOffHeapMemory();
+    }
+  }
+
+
+  @Test
+  public void stackWithChunkLogShowsMsgAndSizeWithConcurrentMod() {
+    SlabImpl slab = new SlabImpl(1024);
+    try {
+      SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new SlabImpl[]{slab});
+      OffHeapStoredObject chunk = (OffHeapStoredObject) ma.allocate(100);
+      int chunkSize = chunk.getSize();
+
+      long addr = chunk.getAddress();
+      TestableSyncChunkStack stack = new TestableSyncChunkStack(ma);
+      stack.offer(addr);
+      LogWriter lw = mock(LogWriter.class);
+      stack.logSizes(lw, "foo");
+      verify(lw).info("foo"+chunkSize);
+      verify(lw).info("foo"+stack.chunk2Size);
+    } finally {
+      SimpleMemoryAllocatorImpl.freeOffHeapMemory();
+    }
+  }
+}


[2/8] incubator-geode git commit: GEODE-982: refactor off-heap

Posted by ds...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapStoredObjectJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapStoredObjectJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapStoredObjectJUnitTest.java
new file mode 100644
index 0000000..2f6b32c
--- /dev/null
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapStoredObjectJUnitTest.java
@@ -0,0 +1,869 @@
+/*
+ * 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.assertj.core.api.Assertions.assertThat;
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.mockito.Mockito.atLeastOnce;
+import static org.mockito.Mockito.doNothing;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.gemstone.gemfire.LogWriter;
+import com.gemstone.gemfire.compression.Compressor;
+import com.gemstone.gemfire.internal.DSCODE;
+import com.gemstone.gemfire.internal.HeapDataOutputStream;
+import com.gemstone.gemfire.internal.Version;
+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.MemoryBlock.State;
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+
+@Category(UnitTest.class)
+public class OffHeapStoredObjectJUnitTest extends AbstractStoredObjectTestBase {
+
+  private MemoryAllocator ma;
+
+  static {
+    ClassLoader.getSystemClassLoader().setDefaultAssertionStatus(true);
+  }
+
+  @Before
+  public void setUp() {
+    OutOfOffHeapMemoryListener ooohml = mock(OutOfOffHeapMemoryListener.class);
+    OffHeapMemoryStats stats = mock(OffHeapMemoryStats.class);
+    LogWriter lw = mock(LogWriter.class);
+
+    ma = SimpleMemoryAllocatorImpl.create(ooohml, stats, lw, 3, OffHeapStorage.MIN_SLAB_SIZE * 3, OffHeapStorage.MIN_SLAB_SIZE);
+  }
+
+  @After
+  public void tearDown() {
+    SimpleMemoryAllocatorImpl.freeOffHeapMemory();
+  }
+
+  @Override
+  public Object getValue() {
+    return Long.valueOf(Long.MAX_VALUE);
+  }
+
+  @Override
+  public byte[] getValueAsByteArray() {
+    return convertValueToByteArray(getValue());
+  }
+
+  private byte[] convertValueToByteArray(Object value) {
+    return ByteBuffer.allocate(Long.SIZE / Byte.SIZE).putLong((Long) value).array();
+  }
+
+  @Override
+  public Object convertByteArrayToObject(byte[] valueInByteArray) {
+    return ByteBuffer.wrap(valueInByteArray).getLong();
+  }
+
+  @Override
+  public Object convertSerializedByteArrayToObject(byte[] valueInSerializedByteArray) {
+    return EntryEventImpl.deserialize(valueInSerializedByteArray);
+  }
+
+  @Override
+  public OffHeapStoredObject createValueAsUnserializedStoredObject(Object value) {
+    byte[] valueInByteArray;
+    if (value instanceof Long) {
+      valueInByteArray = convertValueToByteArray(value);
+    } else {
+      valueInByteArray = (byte[]) value;
+    }
+
+    boolean isSerialized = false;
+    boolean isCompressed = false;
+
+    return createChunk(valueInByteArray, isSerialized, isCompressed);
+  }
+
+  @Override
+  public OffHeapStoredObject createValueAsSerializedStoredObject(Object value) {
+    byte[] valueInSerializedByteArray = EntryEventImpl.serialize(value);
+
+    boolean isSerialized = true;
+    boolean isCompressed = false;
+
+    return createChunk(valueInSerializedByteArray, isSerialized, isCompressed);
+  }
+
+  private OffHeapStoredObject createChunk(byte[] v, boolean isSerialized, boolean isCompressed) {
+    OffHeapStoredObject chunk = (OffHeapStoredObject) ma.allocateAndInitialize(v, isSerialized, isCompressed);
+    return chunk;
+  }
+
+  @Test
+  public void chunkCanBeCreatedFromAnotherChunk() {
+    OffHeapStoredObject chunk = createValueAsUnserializedStoredObject(getValue());
+
+    OffHeapStoredObject newChunk = new OffHeapStoredObject(chunk);
+
+    assertNotNull(newChunk);
+    assertThat(newChunk.getAddress()).isEqualTo(chunk.getAddress());
+
+    chunk.release();
+  }
+
+  @Test
+  public void chunkCanBeCreatedWithOnlyMemoryAddress() {
+    OffHeapStoredObject chunk = createValueAsUnserializedStoredObject(getValue());
+
+    OffHeapStoredObject newChunk = new OffHeapStoredObject(chunk.getAddress());
+
+    assertNotNull(newChunk);
+    assertThat(newChunk.getAddress()).isEqualTo(chunk.getAddress());
+
+    chunk.release();
+  }
+
+  @Test
+  public void chunkSliceCanBeCreatedFromAnotherChunk() {
+    OffHeapStoredObject chunk = createValueAsUnserializedStoredObject(getValue());
+
+    int position = 1;
+    int end = 2;
+
+    OffHeapStoredObject newChunk = (OffHeapStoredObject) chunk.slice(position, end);
+
+    assertNotNull(newChunk);
+    assertThat(newChunk.getClass()).isEqualTo(OffHeapStoredObjectSlice.class);
+    assertThat(newChunk.getAddress()).isEqualTo(chunk.getAddress());
+
+    chunk.release();
+  }
+
+  @Test
+  public void fillSerializedValueShouldFillWrapperWithSerializedValueIfValueIsSerialized() {
+    OffHeapStoredObject chunk = createValueAsSerializedStoredObject(getValue());
+
+    // mock the things
+    BytesAndBitsForCompactor wrapper = mock(BytesAndBitsForCompactor.class);
+
+    byte userBits = 0;
+    byte serializedUserBits = 1;
+    chunk.fillSerializedValue(wrapper, userBits);
+
+    verify(wrapper, times(1)).setOffHeapData(chunk, serializedUserBits);
+
+    chunk.release();
+  }
+
+  @Test
+  public void fillSerializedValueShouldFillWrapperWithDeserializedValueIfValueIsNotSerialized() {
+    OffHeapStoredObject chunk = createValueAsUnserializedStoredObject(getValue());
+
+    // mock the things
+    BytesAndBitsForCompactor wrapper = mock(BytesAndBitsForCompactor.class);
+
+    byte userBits = 1;
+    chunk.fillSerializedValue(wrapper, userBits);
+
+    verify(wrapper, times(1)).setOffHeapData(chunk, userBits);
+
+    chunk.release();
+  }
+
+  @Test
+  public void getShortClassNameShouldReturnShortClassName() {
+    OffHeapStoredObject chunk = createValueAsUnserializedStoredObject(getValue());
+    assertThat(chunk.getShortClassName()).isEqualTo("OffHeapStoredObject");
+
+    chunk.release();
+  }
+
+  @Test
+  public void chunksAreEqualsOnlyByAddress() {
+    OffHeapStoredObject chunk = createValueAsSerializedStoredObject(getValue());
+
+    OffHeapStoredObject newChunk = new OffHeapStoredObject(chunk.getAddress());
+    assertThat(chunk.equals(newChunk)).isTrue();
+
+    OffHeapStoredObject chunkWithSameValue = createValueAsUnserializedStoredObject(getValue());
+    assertThat(chunk.equals(chunkWithSameValue)).isFalse();
+
+    Object someObject = getValue();
+    assertThat(chunk.equals(someObject)).isFalse();
+
+    chunk.release();
+    chunkWithSameValue.release();
+  }
+
+  @Test
+  public void chunksShouldBeComparedBySize() {
+    OffHeapStoredObject chunk1 = createValueAsSerializedStoredObject(getValue());
+
+    OffHeapStoredObject chunk2 = chunk1;
+    assertThat(chunk1.compareTo(chunk2)).isEqualTo(0);
+
+    OffHeapStoredObject chunkWithSameValue = createValueAsSerializedStoredObject(getValue());
+    assertThat(chunk1.compareTo(chunkWithSameValue)).isEqualTo(Long.signum(chunk1.getAddress() - chunkWithSameValue.getAddress()));
+
+    OffHeapStoredObject chunk3 = createValueAsSerializedStoredObject(Long.MAX_VALUE);
+    OffHeapStoredObject chunk4 = createValueAsSerializedStoredObject(Long.MAX_VALUE);
+
+    int newSizeForChunk3 = 2;
+    int newSizeForChunk4 = 3;
+
+    assertThat(chunk3.compareTo(chunk4)).isEqualTo(Integer.signum(newSizeForChunk3 - newSizeForChunk4));
+
+    chunk1.release();
+    chunk4.release();
+  }
+
+  @Test
+  public void setSerializedShouldSetTheSerializedBit() {
+    Object regionEntryValue = getValue();
+    byte[] regionEntryValueAsBytes = convertValueToByteArray(regionEntryValue);
+
+    boolean isSerialized = false;
+    boolean isCompressed = false;
+
+    OffHeapStoredObject chunk = (OffHeapStoredObject) ma.allocateAndInitialize(regionEntryValueAsBytes, isSerialized, isCompressed);
+
+    int headerBeforeSerializedBitSet = AddressableMemoryManager.readIntVolatile(chunk.getAddress() + OffHeapStoredObject.REF_COUNT_OFFSET);
+
+    assertThat(chunk.isSerialized()).isFalse();
+
+    chunk.setSerialized(true); // set to true
+
+    assertThat(chunk.isSerialized()).isTrue();
+
+    int headerAfterSerializedBitSet = AddressableMemoryManager.readIntVolatile(chunk.getAddress() + OffHeapStoredObject.REF_COUNT_OFFSET);
+
+    assertThat(headerAfterSerializedBitSet).isEqualTo(headerBeforeSerializedBitSet | OffHeapStoredObject.IS_SERIALIZED_BIT);
+
+    chunk.release();
+  }
+
+  @Test(expected = IllegalStateException.class)
+  public void setSerialziedShouldThrowExceptionIfChunkIsAlreadyReleased() {
+    OffHeapStoredObject chunk = createValueAsUnserializedStoredObject(getValue());
+    chunk.release();
+    chunk.setSerialized(true);
+
+    chunk.release();
+  }
+
+  @Test
+  public void setCompressedShouldSetTheCompressedBit() {
+    Object regionEntryValue = getValue();
+    byte[] regionEntryValueAsBytes = convertValueToByteArray(regionEntryValue);
+
+    boolean isSerialized = false;
+    boolean isCompressed = false;
+
+    OffHeapStoredObject chunk = (OffHeapStoredObject) ma.allocateAndInitialize(regionEntryValueAsBytes, isSerialized, isCompressed);
+
+    int headerBeforeCompressedBitSet = AddressableMemoryManager.readIntVolatile(chunk.getAddress() + OffHeapStoredObject.REF_COUNT_OFFSET);
+
+    assertThat(chunk.isCompressed()).isFalse();
+
+    chunk.setCompressed(true); // set to true
+
+    assertThat(chunk.isCompressed()).isTrue();
+
+    int headerAfterCompressedBitSet = AddressableMemoryManager.readIntVolatile(chunk.getAddress() + OffHeapStoredObject.REF_COUNT_OFFSET);
+
+    assertThat(headerAfterCompressedBitSet).isEqualTo(headerBeforeCompressedBitSet | OffHeapStoredObject.IS_COMPRESSED_BIT);
+
+    chunk.release();
+  }
+
+  @Test(expected = IllegalStateException.class)
+  public void setCompressedShouldThrowExceptionIfChunkIsAlreadyReleased() {
+    OffHeapStoredObject chunk = createValueAsUnserializedStoredObject(getValue());
+    chunk.release();
+    chunk.setCompressed(true);
+
+    chunk.release();
+  }
+
+  @Test
+  public void setDataSizeShouldSetTheDataSizeBits() {
+    OffHeapStoredObject chunk = createValueAsUnserializedStoredObject(getValue());
+
+    int beforeSize = chunk.getDataSize();
+
+    chunk.setDataSize(2);
+
+    int afterSize = chunk.getDataSize();
+
+    assertThat(afterSize).isEqualTo(2);
+    assertThat(afterSize).isNotEqualTo(beforeSize);
+
+    chunk.release();
+  }
+
+  @Test(expected = IllegalStateException.class)
+  public void setDataSizeShouldThrowExceptionIfChunkIsAlreadyReleased() {
+    OffHeapStoredObject chunk = createValueAsUnserializedStoredObject(getValue());
+    chunk.release();
+    chunk.setDataSize(1);
+
+    chunk.release();
+  }
+
+  @Test(expected = IllegalStateException.class)
+  public void initializeUseCountShouldThrowIllegalStateExceptionIfChunkIsAlreadyRetained() {
+    OffHeapStoredObject chunk = createValueAsUnserializedStoredObject(getValue());
+    chunk.retain();
+    chunk.initializeUseCount();
+
+    chunk.release();
+  }
+
+  @Test(expected = IllegalStateException.class)
+  public void initializeUseCountShouldThrowIllegalStateExceptionIfChunkIsAlreadyReleased() {
+    OffHeapStoredObject chunk = createValueAsUnserializedStoredObject(getValue());
+    chunk.release();
+    chunk.initializeUseCount();
+
+    chunk.release();
+  }
+
+  @Test
+  public void isSerializedPdxInstanceShouldReturnTrueIfItsPDXInstance() {
+    OffHeapStoredObject chunk = createValueAsSerializedStoredObject(getValue());
+
+    byte[] serailizedValue = chunk.getSerializedValue();
+    serailizedValue[0] = DSCODE.PDX;
+    chunk.setSerializedValue(serailizedValue);
+
+    assertThat(chunk.isSerializedPdxInstance()).isTrue();
+
+    serailizedValue = chunk.getSerializedValue();
+    serailizedValue[0] = DSCODE.PDX_ENUM;
+    chunk.setSerializedValue(serailizedValue);
+
+    assertThat(chunk.isSerializedPdxInstance()).isTrue();
+
+    serailizedValue = chunk.getSerializedValue();
+    serailizedValue[0] = DSCODE.PDX_INLINE_ENUM;
+    chunk.setSerializedValue(serailizedValue);
+
+    assertThat(chunk.isSerializedPdxInstance()).isTrue();
+
+    chunk.release();
+  }
+
+  @Test
+  public void isSerializedPdxInstanceShouldReturnFalseIfItsNotPDXInstance() {
+    OffHeapStoredObject chunk = createValueAsSerializedStoredObject(getValue());
+    assertThat(chunk.isSerializedPdxInstance()).isFalse();
+
+    chunk.release();
+  }
+
+  @Test
+  public void checkDataEqualsByChunk() {
+    OffHeapStoredObject chunk1 = createValueAsSerializedStoredObject(getValue());
+    OffHeapStoredObject sameAsChunk1 = chunk1;
+
+    assertThat(chunk1.checkDataEquals(sameAsChunk1)).isTrue();
+
+    OffHeapStoredObject unserializedChunk = createValueAsUnserializedStoredObject(getValue());
+    assertThat(chunk1.checkDataEquals(unserializedChunk)).isFalse();
+
+    OffHeapStoredObject chunkDifferBySize = createValueAsSerializedStoredObject(getValue());
+    chunkDifferBySize.setSize(0);
+    assertThat(chunk1.checkDataEquals(chunkDifferBySize)).isFalse();
+
+    OffHeapStoredObject chunkDifferByValue = createValueAsSerializedStoredObject(Long.MAX_VALUE - 1);
+    assertThat(chunk1.checkDataEquals(chunkDifferByValue)).isFalse();
+
+    OffHeapStoredObject newChunk1 = createValueAsSerializedStoredObject(getValue());
+    assertThat(chunk1.checkDataEquals(newChunk1)).isTrue();
+
+    chunk1.release();
+    unserializedChunk.release();
+    chunkDifferBySize.release();
+    chunkDifferByValue.release();
+    newChunk1.release();
+  }
+
+  @Test
+  public void checkDataEqualsBySerializedValue() {
+    OffHeapStoredObject chunk = createValueAsSerializedStoredObject(getValue());
+    assertThat(chunk.checkDataEquals(new byte[1])).isFalse();
+
+    OffHeapStoredObject chunkDifferByValue = createValueAsSerializedStoredObject(Long.MAX_VALUE - 1);
+    assertThat(chunk.checkDataEquals(chunkDifferByValue.getSerializedValue())).isFalse();
+
+    OffHeapStoredObject newChunk = createValueAsSerializedStoredObject(getValue());
+    assertThat(chunk.checkDataEquals(newChunk.getSerializedValue())).isTrue();
+
+    chunk.release();
+    chunkDifferByValue.release();
+    newChunk.release();
+  }
+
+  @Test
+  public void getDecompressedBytesShouldReturnDecompressedBytesIfCompressed() {
+    Object regionEntryValue = getValue();
+    byte[] regionEntryValueAsBytes = convertValueToByteArray(regionEntryValue);
+
+    boolean isSerialized = true;
+    boolean isCompressed = true;
+
+    OffHeapStoredObject chunk = (OffHeapStoredObject) ma.allocateAndInitialize(regionEntryValueAsBytes, isSerialized, isCompressed);
+
+    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 = chunk.getDecompressedBytes(regionContext);
+
+    // verify the thing happened
+    verify(cacheStats, atLeastOnce()).startDecompression();
+    verify(compressor, times(1)).decompress(regionEntryValueAsBytes);
+    verify(cacheStats, atLeastOnce()).endDecompression(startTime);
+
+    assertArrayEquals(regionEntryValueAsBytes, bytes);
+
+    chunk.release();
+  }
+
+  @Test
+  public void incSizeShouldIncrementSize() {
+    OffHeapStoredObject chunk = createValueAsSerializedStoredObject(getValue());
+
+    int beforeSize = chunk.getSize();
+
+    chunk.incSize(1);
+    assertThat(chunk.getSize()).isEqualTo(beforeSize + 1);
+
+    chunk.incSize(2);
+    assertThat(chunk.getSize()).isEqualTo(beforeSize + 1 + 2);
+
+    chunk.release();
+  }
+
+  @Test
+  public void readyForFreeShouldResetTheRefCount() {
+    OffHeapStoredObject chunk = createValueAsSerializedStoredObject(getValue());
+
+    int refCountBeforeFreeing = chunk.getRefCount();
+    assertThat(refCountBeforeFreeing).isEqualTo(1);
+
+    chunk.readyForFree();
+
+    int refCountAfterFreeing = chunk.getRefCount();
+    assertThat(refCountAfterFreeing).isEqualTo(0);
+  }
+
+  @Test(expected = IllegalStateException.class)
+  public void readyForAllocationShouldThrowExceptionIfAlreadyAllocated() {
+    OffHeapStoredObject chunk = createValueAsSerializedStoredObject(getValue());
+
+    // chunk is already allocated when we created it, so calling readyForAllocation should throw exception.
+    chunk.readyForAllocation();
+
+    chunk.release();
+  }
+
+  @Test
+  public void checkIsAllocatedShouldReturnIfAllocated() {
+    OffHeapStoredObject chunk = createValueAsSerializedStoredObject(getValue());
+    chunk.checkIsAllocated();
+
+    chunk.release();
+  }
+
+  @Test(expected = IllegalStateException.class)
+  public void checkIsAllocatedShouldThrowExceptionIfNotAllocated() {
+    OffHeapStoredObject chunk = createValueAsSerializedStoredObject(getValue());
+    chunk.release();
+    chunk.checkIsAllocated();
+
+    chunk.release();
+  }
+
+  @Test
+  public void sendToShouldWriteSerializedValueToDataOutputIfValueIsSerialized() throws IOException {
+    OffHeapStoredObject chunk = createValueAsSerializedStoredObject(getValue());
+    OffHeapStoredObject spyChunk = spy(chunk);
+
+    HeapDataOutputStream dataOutput = mock(HeapDataOutputStream.class);
+    ByteBuffer directByteBuffer = ByteBuffer.allocate(1024);
+
+    doReturn(directByteBuffer).when(spyChunk).createDirectByteBuffer();
+    doNothing().when(dataOutput).write(directByteBuffer);
+
+    spyChunk.sendTo(dataOutput);
+
+    verify(dataOutput, times(1)).write(directByteBuffer);
+
+    chunk.release();
+  }
+
+  @Test
+  public void sendToShouldWriteUnserializedValueToDataOutputIfValueIsUnserialized() throws IOException {
+    byte[] regionEntryValue = getValueAsByteArray();
+    OffHeapStoredObject chunk = createValueAsUnserializedStoredObject(regionEntryValue);
+
+    // writeByte is a final method and cannot be mocked, so creating a real one
+    HeapDataOutputStream dataOutput = new HeapDataOutputStream(Version.CURRENT);
+
+    chunk.sendTo(dataOutput);
+
+    byte[] actual = dataOutput.toByteArray();
+
+    byte[] expected = new byte[regionEntryValue.length + 2];
+    expected[0] = DSCODE.BYTE_ARRAY;
+    expected[1] = (byte) regionEntryValue.length;
+    System.arraycopy(regionEntryValue, 0, expected, 2, regionEntryValue.length);
+
+    assertNotNull(dataOutput);
+    assertThat(actual).isEqualTo(expected);
+
+    chunk.release();
+  }
+
+  @Test
+  public void sendAsByteArrayShouldWriteValueToDataOutput() throws IOException {
+    byte[] regionEntryValue = getValueAsByteArray();
+    OffHeapStoredObject chunk = createValueAsUnserializedStoredObject(regionEntryValue);
+
+    // writeByte is a final method and cannot be mocked, so creating a real one
+    HeapDataOutputStream dataOutput = new HeapDataOutputStream(Version.CURRENT);
+
+    chunk.sendAsByteArray(dataOutput);
+
+    byte[] actual = dataOutput.toByteArray();
+
+    byte[] expected = new byte[regionEntryValue.length + 1];
+    expected[0] = (byte) regionEntryValue.length;
+    System.arraycopy(regionEntryValue, 0, expected, 1, regionEntryValue.length);
+
+    assertNotNull(dataOutput);
+    assertThat(actual).isEqualTo(expected);
+
+    chunk.release();
+  }
+
+  @Test
+  public void createDirectByteBufferShouldCreateAByteBuffer() {
+    byte[] regionEntryValue = getValueAsByteArray();
+
+    OffHeapStoredObject chunk = createValueAsUnserializedStoredObject(regionEntryValue);
+
+    ByteBuffer buffer = chunk.createDirectByteBuffer();
+
+    byte[] actual = new byte[regionEntryValue.length];
+    buffer.get(actual);
+
+    assertArrayEquals(regionEntryValue, actual);
+
+    chunk.release();
+  }
+
+  @Test
+  public void getDirectByteBufferShouldCreateAByteBuffer() {
+    byte[] regionEntryValue = getValueAsByteArray();
+    OffHeapStoredObject chunk = createValueAsUnserializedStoredObject(regionEntryValue);
+
+    ByteBuffer buffer = chunk.createDirectByteBuffer();
+    long bufferAddress = AddressableMemoryManager.getDirectByteBufferAddress(buffer);
+
+    // returned address should be starting of the value (after skipping HEADER_SIZE bytes)
+    assertEquals(chunk.getAddress() + OffHeapStoredObject.HEADER_SIZE, bufferAddress);
+
+    chunk.release();
+  }
+
+  @Test(expected = AssertionError.class)
+  public void getAddressForReadingDataShouldFailIfItsOutsideOfChunk() {
+    OffHeapStoredObject chunk = createValueAsSerializedStoredObject(getValue());
+    chunk.getAddressForReadingData(0, chunk.getDataSize() + 1);
+
+    chunk.release();
+  }
+
+  @Test
+  public void getAddressForReadingDataShouldReturnDataAddressFromGivenOffset() {
+    OffHeapStoredObject chunk = createValueAsSerializedStoredObject(getValue());
+
+    int offset = 1;
+    long requestedAddress = chunk.getAddressForReadingData(offset, 1);
+
+    assertThat(requestedAddress).isEqualTo(chunk.getBaseDataAddress() + offset);
+
+    chunk.release();
+  }
+
+  @Test
+  public void getSizeInBytesShouldReturnSize() {
+    OffHeapStoredObject chunk = createValueAsSerializedStoredObject(getValue());
+    assertThat(chunk.getSizeInBytes()).isEqualTo(chunk.getSize());
+
+    chunk.release();
+  }
+
+  @Test(expected = AssertionError.class)
+  public void getAddressForReadingDataShouldFailIfOffsetIsNegative() {
+    OffHeapStoredObject chunk = createValueAsSerializedStoredObject(getValue());
+    chunk.getAddressForReadingData(-1, 1);
+
+    chunk.release();
+  }
+
+  @Test(expected = AssertionError.class)
+  public void getAddressForReadingDataShouldFailIfSizeIsNegative() {
+    OffHeapStoredObject chunk = createValueAsSerializedStoredObject(getValue());
+    chunk.getAddressForReadingData(1, -1);
+
+    chunk.release();
+  }
+
+  @Test(expected = AssertionError.class)
+  public void readByteAndWriteByteShouldFailIfOffsetIsOutside() {
+    OffHeapStoredObject chunk = createValueAsSerializedStoredObject(getValue());
+
+    chunk.readDataByte(chunk.getDataSize() + 1);
+
+    chunk.writeDataByte(chunk.getDataSize() + 1, Byte.MAX_VALUE);
+
+    chunk.release();
+  }
+
+  @Test
+  public void writeByteShouldWriteAtCorrectLocation() {
+    OffHeapStoredObject chunk = createValueAsSerializedStoredObject(getValue());
+
+    byte valueBeforeWrite = chunk.readDataByte(2);
+
+    Byte expected = Byte.MAX_VALUE;
+    chunk.writeDataByte(2, expected);
+
+    Byte actual = chunk.readDataByte(2);
+
+    assertThat(actual).isNotEqualTo(valueBeforeWrite);
+    assertThat(actual).isEqualTo(expected);
+
+    chunk.release();
+  }
+
+  @Test
+  public void retainShouldIncrementRefCount() {
+    OffHeapStoredObject chunk = createValueAsUnserializedStoredObject(getValue());
+    assertThat(chunk.getRefCount()).isEqualTo(1);
+
+    chunk.retain();
+    assertThat(chunk.getRefCount()).isEqualTo(2);
+
+    chunk.retain();
+    assertThat(chunk.getRefCount()).isEqualTo(3);
+
+    chunk.release();
+    chunk.release();
+    chunk.release();
+    boolean retainAfterRelease = chunk.retain();
+
+    assertThat(retainAfterRelease).isFalse();
+  }
+
+  @Test(expected = IllegalStateException.class)
+  public void retainShouldThrowExceptionAfterMaxNumberOfTimesRetained() {
+    OffHeapStoredObject chunk = createValueAsUnserializedStoredObject(getValue());
+
+    // loop though and invoke retain for MAX_REF_COUNT-1 times, as create chunk above counted as one reference
+    for (int i = 0; i < OffHeapStoredObject.MAX_REF_COUNT - 1; i++)
+      chunk.retain();
+
+    // invoke for the one more time should throw exception
+    chunk.retain();
+  }
+
+  @Test
+  public void releaseShouldDecrementRefCount() {
+    OffHeapStoredObject chunk = createValueAsUnserializedStoredObject(getValue());
+    assertThat(chunk.getRefCount()).isEqualTo(1);
+
+    chunk.retain();
+    chunk.retain();
+    assertThat(chunk.getRefCount()).isEqualTo(3);
+
+    chunk.release();
+    assertThat(chunk.getRefCount()).isEqualTo(2);
+
+    chunk.release();
+    assertThat(chunk.getRefCount()).isEqualTo(1);
+
+    chunk.retain();
+    chunk.release();
+    assertThat(chunk.getRefCount()).isEqualTo(1);
+
+    chunk.release();
+    assertThat(chunk.getRefCount()).isEqualTo(0);
+  }
+
+  @Test(expected = IllegalStateException.class)
+  public void releaseShouldThrowExceptionIfChunkIsAlreadyReleased() {
+    OffHeapStoredObject chunk = createValueAsUnserializedStoredObject(getValue());
+    chunk.release();
+    chunk.release();
+  }
+
+  @Test
+  public void testToString() {
+    OffHeapStoredObject chunk = createValueAsUnserializedStoredObject(getValue());
+
+    String expected = ":<dataSize=" + chunk.getDataSize() + " refCount=" + chunk.getRefCount() + " addr=" + Long.toHexString(chunk.getAddress()) + ">";
+    assertThat(chunk.toString()).endsWith(expected);
+
+    chunk.release();
+  }
+
+  @Test
+  public void getStateShouldReturnAllocatedIfRefCountIsGreaterThanZero() {
+    OffHeapStoredObject chunk = createValueAsUnserializedStoredObject(getValue());
+    assertEquals(State.ALLOCATED, chunk.getState());
+
+    chunk.release();
+  }
+
+  @Test
+  public void getStateShouldReturnDeallocatedIfRefCountIsZero() {
+    OffHeapStoredObject chunk = createValueAsUnserializedStoredObject(getValue());
+    chunk.release();
+    assertEquals(State.DEALLOCATED, chunk.getState());
+  }
+
+  @Test(expected = UnsupportedOperationException.class)
+  public void getNextBlockShouldThrowUnSupportedOperationException() {
+    OffHeapStoredObject chunk = createValueAsUnserializedStoredObject(getValue());
+    chunk.getNextBlock();
+
+    chunk.release();
+  }
+
+  @Test
+  public void getBlockSizeShouldBeSameSameGetSize() {
+    OffHeapStoredObject chunk = createValueAsUnserializedStoredObject(getValue());
+    assertEquals(chunk.getSize(), chunk.getBlockSize());
+
+    chunk.release();
+  }
+
+  @Test(expected = UnsupportedOperationException.class)
+  public void getSlabIdShouldThrowUnSupportedOperationException() {
+    OffHeapStoredObject chunk = createValueAsUnserializedStoredObject(getValue());
+    chunk.getSlabId();
+
+    chunk.release();
+  }
+
+  @Test
+  public void getFreeListIdShouldReturnMinusOne() {
+    OffHeapStoredObject chunk = createValueAsUnserializedStoredObject(getValue());
+    assertThat(chunk.getFreeListId()).isEqualTo(-1);
+
+    chunk.release();
+  }
+
+  @Test
+  public void getDataTypeShouldReturnNull() {
+    OffHeapStoredObject chunk = createValueAsUnserializedStoredObject(getValue());
+    assertThat(chunk.getDataType()).isNull();
+
+    chunk.release();
+  }
+
+  @Test
+  public void getDataDataShouldReturnNull() {
+    OffHeapStoredObject chunk = createValueAsUnserializedStoredObject(getValue());
+    assertThat(chunk.getDataValue()).isNull();
+  }
+
+  @Test(expected = UnsupportedOperationException.class)
+  public void getRawBytesShouldThrowExceptionIfValueIsCompressed() {
+    Object regionEntryValue = getValue();
+    byte[] regionEntryValueAsBytes = convertValueToByteArray(regionEntryValue);
+
+    boolean isSerialized = true;
+    boolean isCompressed = true;
+
+    OffHeapStoredObject chunk = (OffHeapStoredObject) ma.allocateAndInitialize(regionEntryValueAsBytes, isSerialized, isCompressed);
+
+    chunk.getRawBytes();
+
+    chunk.release();
+  }
+
+  @Test
+  public void getSerializedValueShouldSerializeTheValue() {
+    Object regionEntryValue = getValue();
+    byte[] regionEntryValueAsBytes = convertValueToByteArray(regionEntryValue);
+
+    boolean isSerialized = false;
+    boolean isCompressed = false;
+
+    OffHeapStoredObject chunk = (OffHeapStoredObject) ma.allocateAndInitialize(regionEntryValueAsBytes, isSerialized, isCompressed);
+
+    byte[] serializedValue = chunk.getSerializedValue();
+
+    assertThat(serializedValue).isEqualTo(EntryEventImpl.serialize(regionEntryValueAsBytes));
+
+    chunk.release();
+  }
+
+  @Test
+  public void fillShouldFillTheChunk() {
+    boolean isSerialized = false;
+    boolean isCompressed = false;
+
+    OffHeapStoredObject chunk = (OffHeapStoredObject) ma.allocateAndInitialize(new byte[100], isSerialized, isCompressed);
+
+    // first fill the unused part with FILL_PATTERN
+    OffHeapStoredObject.fill(chunk.getAddress());
+
+    // Validate that it is filled
+    chunk.validateFill();
+
+    chunk.release();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapStoredObjectSliceJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapStoredObjectSliceJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapStoredObjectSliceJUnitTest.java
new file mode 100644
index 0000000..1b5bf26
--- /dev/null
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapStoredObjectSliceJUnitTest.java
@@ -0,0 +1,72 @@
+/*
+ * 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.assertEquals;
+import static org.junit.Assert.assertNotNull;
+
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+
+@Category(UnitTest.class)
+public class OffHeapStoredObjectSliceJUnitTest extends OffHeapStoredObjectJUnitTest {
+
+  @Test
+  public void sliceShouldHaveAValidDataSize() {
+    int position = 1;
+    int end = 2;
+
+    OffHeapStoredObject chunk = createValueAsUnserializedStoredObject(getValue());
+    OffHeapStoredObjectSlice slice = (OffHeapStoredObjectSlice) chunk.slice(position, end);
+
+    assertNotNull(slice);
+    assertEquals(OffHeapStoredObjectSlice.class, slice.getClass());
+
+    assertEquals(end - position, slice.getDataSize());
+  }
+
+  @Test
+  public void sliceShouldHaveAValidBaseDataAddress() {
+    int position = 1;
+    int end = 2;
+
+    OffHeapStoredObject chunk = createValueAsUnserializedStoredObject(getValue());
+    OffHeapStoredObjectSlice slice = (OffHeapStoredObjectSlice) chunk.slice(position, end);
+
+    assertNotNull(slice);
+    assertEquals(OffHeapStoredObjectSlice.class, slice.getClass());
+
+    assertEquals(chunk.getBaseDataAddress() + position, slice.getBaseDataAddress());
+  }
+
+  @Test
+  public void sliceShouldHaveAValidBaseOffset() {
+    int position = 1;
+    int end = 2;
+
+    OffHeapStoredObject chunk = createValueAsUnserializedStoredObject(getValue());
+    OffHeapStoredObjectSlice slice = (OffHeapStoredObjectSlice) chunk.slice(position, end);
+
+    assertNotNull(slice);
+    assertEquals(OffHeapStoredObjectSlice.class, slice.getClass());
+
+    assertEquals(chunk.getBaseDataOffset() + position, slice.getBaseDataOffset());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapStoredObjectWithHeapFormJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapStoredObjectWithHeapFormJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapStoredObjectWithHeapFormJUnitTest.java
new file mode 100644
index 0000000..a763e76
--- /dev/null
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapStoredObjectWithHeapFormJUnitTest.java
@@ -0,0 +1,64 @@
+/*
+ * 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.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNotSame;
+import static org.junit.Assert.assertSame;
+
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+
+@Category(UnitTest.class)
+public class OffHeapStoredObjectWithHeapFormJUnitTest extends OffHeapStoredObjectJUnitTest {
+
+  @Test
+  public void getRawBytesShouldReturnCachedHeapForm() {
+    OffHeapStoredObject chunk = createValueAsUnserializedStoredObject(getValue());
+
+    byte[] valueInBytes = getValueAsByteArray();
+    OffHeapStoredObjectWithHeapForm heapForm = new OffHeapStoredObjectWithHeapForm(chunk, valueInBytes);
+
+    assertNotNull(heapForm);
+
+    assertSame(valueInBytes, heapForm.getRawBytes());
+  }
+
+  @Test
+  public void getChunkWithoutHeapFormShouldReturnGemFireChunk() {
+    OffHeapStoredObject chunk = createValueAsSerializedStoredObject(getValue());
+
+    byte[] valueInBytes = getValueAsByteArray();
+    OffHeapStoredObjectWithHeapForm heapForm = new OffHeapStoredObjectWithHeapForm(chunk, valueInBytes);
+
+    OffHeapStoredObject chunkWithOutHeapForm = (OffHeapStoredObject)heapForm.getStoredObjectWithoutHeapForm();
+
+    assertNotNull(chunkWithOutHeapForm);
+    assertEquals(OffHeapStoredObject.class, chunkWithOutHeapForm.getClass());
+
+    assertEquals(chunk, heapForm.getStoredObjectWithoutHeapForm());
+
+    assertEquals(chunk.getAddress(), chunkWithOutHeapForm.getAddress());
+    assertArrayEquals(chunk.getRawBytes(), chunkWithOutHeapForm.getRawBytes());
+    assertNotSame(valueInBytes, chunkWithOutHeapForm.getRawBytes());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapValidationJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapValidationJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapValidationJUnitTest.java
index 630ae22..f8a5c8e 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapValidationJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapValidationJUnitTest.java
@@ -122,7 +122,7 @@ public class OffHeapValidationJUnitTest {
       assertEquals(1024*1024*2, firstBlock.getBlockSize());
       assertEquals("N/A", firstBlock.getDataType());
       assertEquals(-1, firstBlock.getFreeListId());
-      assertTrue(firstBlock.getMemoryAddress() > 0);
+      assertTrue(firstBlock.getAddress() > 0);
       assertNull(firstBlock.getNextBlock());
       assertEquals(0, firstBlock.getRefCount());
       assertEquals(0, firstBlock.getSlabId());
@@ -202,7 +202,7 @@ public class OffHeapValidationJUnitTest {
       assertEquals(i + ":" + values.dataType, values.blockSize, block.getBlockSize());
       assertEquals(i + ":" + values.dataType, values.dataType, block.getDataType());
       assertEquals(i + ":" + values.dataType, values.freeListId, block.getFreeListId());
-      assertEquals(i + ":" + values.dataType, values.memoryAddress, block.getMemoryAddress());
+      assertEquals(i + ":" + values.dataType, values.memoryAddress, block.getAddress());
       assertEquals(i + ":" + values.dataType, values.refCount, block.getRefCount());
       assertEquals(i + ":" + values.dataType, values.slabId, block.getSlabId());
       assertEquals(i + ":" + values.dataType, values.isCompressed, block.isCompressed());
@@ -296,8 +296,8 @@ public class OffHeapValidationJUnitTest {
   
   private long getMemoryAddress(Region region, String key) {
     Object entry = ((LocalRegion) region).getRegionEntry(key)._getValue();
-    assertTrue(entry instanceof ObjectChunk);
-    long memoryAddress = ((ObjectChunk)entry).getMemoryAddress();
+    assertTrue(entry instanceof OffHeapStoredObject);
+    long memoryAddress = ((OffHeapStoredObject)entry).getAddress();
     assertTrue(memoryAddress > 0);
     return memoryAddress;
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/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..7157eaa 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
@@ -64,7 +64,7 @@ public class OffHeapWriteObjectAsByteArrayJUnitTest {
   public void testByteArrayChunk() throws IOException, ClassNotFoundException {
     byte[] expected = new byte[] {1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16};
     StoredObject so = createStoredObject(expected, false, false);
-    assertTrue(so instanceof ObjectChunk);
+    assertTrue(so instanceof OffHeapStoredObject);
     HeapDataOutputStream hdos = new HeapDataOutputStream(new byte[1024]);
     DataSerializer.writeObjectAsByteArray(so, hdos);
     DataInputStream in = createInput(hdos);
@@ -76,7 +76,7 @@ public class OffHeapWriteObjectAsByteArrayJUnitTest {
   public void testByteArrayDataAsAddress() throws IOException, ClassNotFoundException {
     byte[] expected = new byte[] {1, 2, 3};
     StoredObject so = createStoredObject(expected, false, false);
-    assertTrue(so instanceof DataAsAddress);
+    assertTrue(so instanceof TinyStoredObject);
     HeapDataOutputStream hdos = new HeapDataOutputStream(new byte[1024]);
     DataSerializer.writeObjectAsByteArray(so, hdos);
     DataInputStream in = createInput(hdos);
@@ -88,7 +88,7 @@ public class OffHeapWriteObjectAsByteArrayJUnitTest {
   public void testStringChunk() throws IOException, ClassNotFoundException {
     byte[] expected = EntryEventImpl.serialize("1234567890");
     StoredObject so = createStoredObject(expected, true, false);
-    assertTrue(so instanceof ObjectChunk);
+    assertTrue(so instanceof OffHeapStoredObject);
     HeapDataOutputStream hdos = new HeapDataOutputStream(new byte[1024]);
     DataSerializer.writeObjectAsByteArray(so, hdos);
     DataInputStream in = createInput(hdos);
@@ -101,7 +101,7 @@ public class OffHeapWriteObjectAsByteArrayJUnitTest {
   public void testStringDataAsAddress() throws IOException, ClassNotFoundException {
     byte[] expected = EntryEventImpl.serialize("1234");
     StoredObject so = createStoredObject(expected, true, false);
-    assertTrue(so instanceof DataAsAddress);
+    assertTrue(so instanceof TinyStoredObject);
     HeapDataOutputStream hdos = new HeapDataOutputStream(new byte[1024]);
     DataSerializer.writeObjectAsByteArray(so, hdos);
     DataInputStream in = createInput(hdos);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OldFreeListOffHeapRegionJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OldFreeListOffHeapRegionJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OldFreeListOffHeapRegionJUnitTest.java
index d8c35b8..e9ca59d 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OldFreeListOffHeapRegionJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OldFreeListOffHeapRegionJUnitTest.java
@@ -41,7 +41,7 @@ public class OldFreeListOffHeapRegionJUnitTest 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/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..51bc0a2 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});
   }
 
   /**
@@ -163,7 +163,7 @@ public class SimpleMemoryAllocatorFillPatternIntegrationTest {
         private int totalAllocation = 0;
         
         // List of Chunks allocated by this thread
-        private List<ObjectChunk> chunks = new LinkedList<ObjectChunk>();
+        private List<OffHeapStoredObject> chunks = new LinkedList<OffHeapStoredObject>();
         
         // Time to end thread execution
         private long endTime = System.currentTimeMillis() + RUN_TIME_IN_MILLIS;
@@ -173,7 +173,7 @@ public class SimpleMemoryAllocatorFillPatternIntegrationTest {
          */
         private void allocate() {          
           int allocation = chunkSizer.allocationSize();
-          ObjectChunk chunk = (ObjectChunk) allocator.allocate(allocation);
+          OffHeapStoredObject chunk = (OffHeapStoredObject) allocator.allocate(allocation);
           
           // This should always work just after allocation
           chunk.validateFill();
@@ -186,7 +186,7 @@ public class SimpleMemoryAllocatorFillPatternIntegrationTest {
          * Frees a random chunk from the Chunk list.
          */
         private void free() {
-          ObjectChunk chunk = chunks.remove(random.nextInt(chunks.size()));
+          OffHeapStoredObject chunk = chunks.remove(random.nextInt(chunks.size()));
           totalAllocation -= chunk.getSize();
           
           /*
@@ -200,8 +200,8 @@ public class SimpleMemoryAllocatorFillPatternIntegrationTest {
          * Writes canned data to a random Chunk from the Chunk list.
          */
         private void write() {
-          ObjectChunk chunk = chunks.get(random.nextInt(chunks.size()));
-          chunk.writeBytes(0, WRITE_BYTES);
+          OffHeapStoredObject chunk = chunks.get(random.nextInt(chunks.size()));
+          chunk.writeDataBytes(0, WRITE_BYTES);
         }
         
         /**

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/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..c61f2f4 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});
   }
 
   /**
@@ -101,7 +101,7 @@ public class SimpleMemoryAllocatorFillPatternJUnitTest {
      * Pull a chunk off the fragment.  This will have no fill because
      * it is a "fresh" chunk.
      */
-    ObjectChunk chunk = (ObjectChunk) this.allocator.allocate(chunkSize);
+    OffHeapStoredObject chunk = (OffHeapStoredObject) this.allocator.allocate(chunkSize);
 
     /*
      * Chunk should have valid fill from initial fragment allocation.
@@ -109,7 +109,7 @@ public class SimpleMemoryAllocatorFillPatternJUnitTest {
     chunk.validateFill();
          
     // "Dirty" the chunk so the release has something to fill over
-    chunk.writeBytes(ObjectChunk.MIN_CHUNK_SIZE + 1, WRITE_BYTES);
+    chunk.writeDataBytes(OffHeapStoredObject.MIN_CHUNK_SIZE + 1, WRITE_BYTES);
 
     // This should free the Chunk (ref count == 1)
     chunk.release();
@@ -118,24 +118,24 @@ public class SimpleMemoryAllocatorFillPatternJUnitTest {
      * This chunk should have a fill because it was reused from the
      * free list (assuming no fragmentation at this point...)
      */
-    chunk = (ObjectChunk) this.allocator.allocate(chunkSize);
+    chunk = (OffHeapStoredObject) this.allocator.allocate(chunkSize);
     
     // Make sure we have a fill this time
     chunk.validateFill();
     
     // Give the fill code something to write over during the release
-    chunk.writeBytes(ObjectChunk.MIN_CHUNK_SIZE + 1, WRITE_BYTES);
+    chunk.writeDataBytes(OffHeapStoredObject.MIN_CHUNK_SIZE + 1, WRITE_BYTES);
     chunk.release();
 
     // Again, make sure the release implemented the fill
     chunk.validateFill();
 
     // "Dirty up" the free chunk
-    chunk.writeBytes(ObjectChunk.MIN_CHUNK_SIZE + 1, WRITE_BYTES);
+    chunk.writeDataBytes(OffHeapStoredObject.MIN_CHUNK_SIZE + 1, WRITE_BYTES);
     
     catchException(chunk).validateFill();
     assertTrue(caughtException() instanceof IllegalStateException);
-    assertEquals("Fill pattern violated for chunk " + chunk.getMemoryAddress() + " with size " + chunk.getSize(), caughtException().getMessage());
+    assertEquals("Fill pattern violated for chunk " + chunk.getAddress() + " with size " + chunk.getSize(), caughtException().getMessage());
     
   }
 
@@ -149,14 +149,14 @@ public class SimpleMemoryAllocatorFillPatternJUnitTest {
     /*
      * Stores our allocated memory.
      */
-    ObjectChunk[] allocatedChunks = new ObjectChunk[COMPACTION_CHUNKS];
+    OffHeapStoredObject[] allocatedChunks = new OffHeapStoredObject[COMPACTION_CHUNKS];
     
     /*
      * Use up most of our memory
      * Our memory looks like [      ][      ][      ]
      */
     for(int i =0;i < allocatedChunks.length;++i) {
-      allocatedChunks[i] = (ObjectChunk) this.allocator.allocate(COMPACTION_CHUNK_SIZE);
+      allocatedChunks[i] = (OffHeapStoredObject) this.allocator.allocate(COMPACTION_CHUNK_SIZE);
       allocatedChunks[i].validateFill();
     }
 
@@ -173,7 +173,7 @@ public class SimpleMemoryAllocatorFillPatternJUnitTest {
      * our initial chunks.  This should force a compaction causing our
      * memory to look like [            ][      ].
      */
-    ObjectChunk slightlyLargerChunk = (ObjectChunk) this.allocator.allocate(FORCE_COMPACTION_CHUNK_SIZE);
+    OffHeapStoredObject slightlyLargerChunk = (OffHeapStoredObject) this.allocator.allocate(FORCE_COMPACTION_CHUNK_SIZE);
     
     /*
      * Make sure the compacted memory has the fill validation.

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/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..cc791fc 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());
@@ -189,23 +189,23 @@ public class SimpleMemoryAllocatorJUnitTest {
   @Test
   public void testBasics() {
     int BATCH_SIZE = 1;
-    int TINY_MULTIPLE = com.gemstone.gemfire.internal.offheap.FreeListManager.TINY_MULTIPLE;
-    int HUGE_MULTIPLE = com.gemstone.gemfire.internal.offheap.FreeListManager.HUGE_MULTIPLE;
-    int perObjectOverhead = com.gemstone.gemfire.internal.offheap.ObjectChunk.OFF_HEAP_HEADER_SIZE;
-    int maxTiny = com.gemstone.gemfire.internal.offheap.FreeListManager.MAX_TINY-perObjectOverhead;
+    int TINY_MULTIPLE = FreeListManager.TINY_MULTIPLE;
+    int HUGE_MULTIPLE = FreeListManager.HUGE_MULTIPLE;
+    int perObjectOverhead = OffHeapStoredObject.HEADER_SIZE;
+    int maxTiny = FreeListManager.MAX_TINY-perObjectOverhead;
     int minHuge = maxTiny+1;
     int TOTAL_MEM = (maxTiny+perObjectOverhead)*BATCH_SIZE /*+ (maxBig+perObjectOverhead)*BATCH_SIZE*/ + round(TINY_MULTIPLE, minHuge+1+perObjectOverhead)*BATCH_SIZE + (TINY_MULTIPLE+perObjectOverhead)*BATCH_SIZE /*+ (MIN_BIG_SIZE+perObjectOverhead)*BATCH_SIZE*/ + round(TINY_MULTIPLE, minHuge+perObjectOverhead+1);
-    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);
+      StoredObject tinymc = ma.allocate(maxTiny);
       assertEquals(TOTAL_MEM-round(TINY_MULTIPLE, maxTiny+perObjectOverhead), ma.getFreeMemory());
       assertEquals(round(TINY_MULTIPLE, maxTiny+perObjectOverhead)*(BATCH_SIZE-1), ma.freeList.getFreeTinyMemory());
-      MemoryChunk hugemc = ma.allocate(minHuge);
+      StoredObject hugemc = ma.allocate(minHuge);
       assertEquals(TOTAL_MEM-round(TINY_MULTIPLE, minHuge+perObjectOverhead)/*-round(BIG_MULTIPLE, maxBig+perObjectOverhead)*/-round(TINY_MULTIPLE, maxTiny+perObjectOverhead), ma.getFreeMemory());
       long freeSlab = ma.freeList.getFreeFragmentMemory();
       long oldFreeHugeMemory = ma.freeList.getFreeHugeMemory();
@@ -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);
+        StoredObject hugemc2 = ma.allocate(minHuge);
         assertEquals(round(TINY_MULTIPLE, minHuge+perObjectOverhead)*(BATCH_SIZE-2), ma.freeList.getFreeHugeMemory());
         hugemc2.release();
         assertEquals(round(TINY_MULTIPLE, minHuge+perObjectOverhead)*(BATCH_SIZE-1), ma.freeList.getFreeHugeMemory());
@@ -265,15 +265,15 @@ 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);
       }
       bb.position(0);
-      ObjectChunk c = (ObjectChunk) ma.allocateAndInitialize(bb.array(), false, false);
+      OffHeapStoredObject c = (OffHeapStoredObject) ma.allocateAndInitialize(bb.array(), false, false);
       assertEquals(1024, c.getDataSize());
       if (!Arrays.equals(bb.array(), c.getRawBytes())) {
         fail("arrays are not equal. Expected " + Arrays.toString(bb.array()) + " but found: " + Arrays.toString(c.getRawBytes()));
@@ -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());
@@ -382,7 +382,7 @@ public class SimpleMemoryAllocatorJUnitTest {
         assertEquals(1024*1024, firstBlock.getBlockSize());
         assertEquals("N/A", firstBlock.getDataType());
         assertEquals(-1, firstBlock.getFreeListId());
-        assertTrue(firstBlock.getMemoryAddress() > 0);
+        assertTrue(firstBlock.getAddress() > 0);
         assertNull(firstBlock.getNextBlock());
         assertEquals(0, firstBlock.getRefCount());
         assertEquals(0, firstBlock.getSlabId());
@@ -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();
@@ -427,23 +427,23 @@ public class SimpleMemoryAllocatorJUnitTest {
   
   @Test
   public void testCompaction() {
-    final int perObjectOverhead = com.gemstone.gemfire.internal.offheap.ObjectChunk.OFF_HEAP_HEADER_SIZE;
+    final int perObjectOverhead = OffHeapStoredObject.HEADER_SIZE;
     final int BIG_ALLOC_SIZE = 150000;
     final int SMALL_ALLOC_SIZE = BIG_ALLOC_SIZE/2;
     final int TOTAL_MEM = BIG_ALLOC_SIZE;
-    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});
+      StoredObject bmc = ma.allocate(BIG_ALLOC_SIZE-perObjectOverhead);
       try {
-        MemoryChunk smc = ma.allocate(SMALL_ALLOC_SIZE-perObjectOverhead);
+        StoredObject 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);
+      StoredObject smc1 = ma.allocate(SMALL_ALLOC_SIZE-perObjectOverhead);
+      StoredObject smc2 = ma.allocate(SMALL_ALLOC_SIZE-perObjectOverhead);
       smc2.release();
       assertEquals(TOTAL_MEM-SMALL_ALLOC_SIZE, ma.freeList.getFreeMemory());
       try {
@@ -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<StoredObject> mcs = new ArrayList<StoredObject>();
       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);
+      StoredObject mc24 = ma.allocate(24);
       checkMcs(mcs);
       assertEquals((8+perObjectOverhead)*6 - (24+perObjectOverhead), ma.freeList.getFreeMemory());
       // At this point I should have 8*3 + perObjectOverhead*5 of free memory
-      MemoryChunk mc16 = ma.allocate(16);
+      StoredObject mc16 = ma.allocate(16);
       checkMcs(mcs);
       assertEquals((8+perObjectOverhead)*6 - (24+perObjectOverhead) - (16+perObjectOverhead), ma.freeList.getFreeMemory());
       // At this point I should have 8*1 + perObjectOverhead*4 of free memory
@@ -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);
+      StoredObject mcDO = ma.allocate(perObjectOverhead*2);
       checkMcs(mcs);
       // At this point I should have 8*0 + perObjectOverhead*0 of free memory
       assertEquals(0, ma.freeList.getFreeMemory());
@@ -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 (StoredObject mc: mcs) {
         mc.release();
         assertEquals(freeMem+(8+perObjectOverhead), ma.freeList.getFreeMemory());
         freeMem += (8+perObjectOverhead);
@@ -543,11 +543,11 @@ public class SimpleMemoryAllocatorJUnitTest {
   boolean memoryUsageEventReceived;
   @Test
   public void testUsageEventListener() {
-    final int perObjectOverhead = com.gemstone.gemfire.internal.offheap.ObjectChunk.OFF_HEAP_HEADER_SIZE;
+    final int perObjectOverhead = OffHeapStoredObject.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);
+      StoredObject smc = ma.allocate(SMALL_ALLOC_SIZE-perObjectOverhead);
       assertEquals(true, this.memoryUsageEventReceived);
       
       this.expectedMemoryUsage = SMALL_ALLOC_SIZE * 2;
@@ -588,19 +588,19 @@ public class SimpleMemoryAllocatorJUnitTest {
       SimpleMemoryAllocatorImpl.freeOffHeapMemory();
     }
   }
-  private void checkMcs(ArrayList<MemoryChunk> mcs) {
-    for (MemoryChunk mc: mcs) {
+  private void checkMcs(ArrayList<StoredObject> mcs) {
+    for (StoredObject mc: mcs) {
       assertEquals(8+8, mc.getSize());
     }
   }
   
   @Test
   public void testOutOfOffHeapMemory() {
-    final int perObjectOverhead = com.gemstone.gemfire.internal.offheap.ObjectChunk.OFF_HEAP_HEADER_SIZE;
+    final int perObjectOverhead = OffHeapStoredObject.HEADER_SIZE;
     final int BIG_ALLOC_SIZE = 150000;
     final int SMALL_ALLOC_SIZE = BIG_ALLOC_SIZE/2;
     final int TOTAL_MEM = BIG_ALLOC_SIZE;
-    final 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);
+      StoredObject 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);
+        StoredObject 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/3087c86f/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/StoredObjectTestSuite.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/StoredObjectTestSuite.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/StoredObjectTestSuite.java
index feb5de8..48a0e25 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/StoredObjectTestSuite.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/StoredObjectTestSuite.java
@@ -21,10 +21,10 @@ import org.junit.runner.RunWith;
 import org.junit.runners.Suite;
 
 @Suite.SuiteClasses({
-	DataAsAddressJUnitTest.class,
-	ObjectChunkJUnitTest.class,
-	ObjectChunkWithHeapFormJUnitTest.class,
-	ObjectChunkSliceJUnitTest.class,
+	TinyStoredObjectJUnitTest.class,
+	OffHeapStoredObjectJUnitTest.class,
+	OffHeapStoredObjectWithHeapFormJUnitTest.class,
+	OffHeapStoredObjectSliceJUnitTest.class,
 })
 @RunWith(Suite.class)
 public class StoredObjectTestSuite {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/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
deleted file mode 100644
index 3ae6159..0000000
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/SyncChunkStackJUnitTest.java
+++ /dev/null
@@ -1,289 +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 static org.mockito.Mockito.*;
-
-import org.junit.After;
-import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.mockito.listeners.InvocationListener;
-import org.mockito.listeners.MethodInvocationReport;
-
-import com.gemstone.gemfire.LogWriter;
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
-
-@Category(UnitTest.class)
-public class SyncChunkStackJUnitTest {
-  static {
-    ClassLoader.getSystemClassLoader().setDefaultAssertionStatus(true);
-  }
-
-  @BeforeClass
-  public static void setUpBeforeClass() throws Exception {
-  }
-
-  @AfterClass
-  public static void tearDownAfterClass() throws Exception {
-  }
-
-  @Before
-  public void setUp() throws Exception {
-  }
-
-  @After
-  public void tearDown() throws Exception {
-  }
-
-  @Test
-  public void addressZeroCausesStackToBeEmpty() {
-    SyncChunkStack stack = new SyncChunkStack(0L);
-    assertEquals(true, stack.isEmpty());
-  }
-
-  @Test
-  public void defaultStackIsEmpty() {
-    SyncChunkStack stack = new SyncChunkStack();
-    assertEquals(true, stack.isEmpty());
-  }
-
-  @Test
-  public void defaultStackReturnsZeroFromTop() {
-    SyncChunkStack stack = new SyncChunkStack();
-    assertEquals(0L, stack.getTopAddress());
-  }
-  
-  @Test
-  public void defaultStackReturnsZeroFromPoll() {
-    SyncChunkStack stack = new SyncChunkStack();
-    assertEquals(0L, stack.poll());
-  }
-  
-  @Test
-  public void defaultStackReturnsZeroFromClear() {
-    SyncChunkStack stack = new SyncChunkStack();
-    assertEquals(0L, stack.clear());
-    assertEquals(true, stack.isEmpty());
-  }
-  
-  @Test
-  public void defaultStackLogsNothing() {
-    SyncChunkStack stack = new SyncChunkStack();
-    LogWriter lw = mock(LogWriter.class, withSettings().invocationListeners(new InvocationListener() {
-      @Override
-      public void reportInvocation(MethodInvocationReport methodInvocationReport) {
-        fail("Unexpected invocation");
-      }
-    }));
-    stack.logSizes(lw, "should not be used");
-  }
-  
-  @Test
-  public void defaultStackComputeSizeIsZero() {
-    SyncChunkStack stack = new SyncChunkStack();
-    assertEquals(0L, stack.computeTotalSize());
-  }
-  
-  @Test
-  public void stackCreatedWithAddressIsNotEmpty() {
-    UnsafeMemoryChunk slab = new UnsafeMemoryChunk(1024);
-    try {
-      SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new UnsafeMemoryChunk[]{slab});
-      ObjectChunk chunk = (ObjectChunk) ma.allocate(100);
-
-      SyncChunkStack stack = new SyncChunkStack(chunk.getMemoryAddress());
-      assertEquals(false, stack.isEmpty());
-    } finally {
-      SimpleMemoryAllocatorImpl.freeOffHeapMemory();
-    }
-  }
-
-  @Test
-  public void stackWithChunkIsNotEmpty() {
-    UnsafeMemoryChunk slab = new UnsafeMemoryChunk(1024);
-    try {
-      SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new UnsafeMemoryChunk[]{slab});
-      ObjectChunk chunk = (ObjectChunk) ma.allocate(100);
-
-      SyncChunkStack stack = new SyncChunkStack();
-      stack.offer(chunk.getMemoryAddress());
-      assertEquals(false, stack.isEmpty());
-    } finally {
-      SimpleMemoryAllocatorImpl.freeOffHeapMemory();
-    }
-  }
-
-  @Test
-  public void stackWithChunkTopEqualsAddress() {
-    UnsafeMemoryChunk slab = new UnsafeMemoryChunk(1024);
-    try {
-      SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new UnsafeMemoryChunk[]{slab});
-      ObjectChunk chunk = (ObjectChunk) ma.allocate(100);
-
-      long addr = chunk.getMemoryAddress();
-      SyncChunkStack stack = new SyncChunkStack();
-      stack.offer(addr);
-      assertEquals(addr, stack.getTopAddress());
-    } finally {
-      SimpleMemoryAllocatorImpl.freeOffHeapMemory();
-    }
-  }
-
-  @Test
-  public void addressZeroOfferCausesFailedAssertion() {
-    SyncChunkStack stack = new SyncChunkStack(0L);
-    try {
-      stack.offer(0);
-      fail("expected AssertionError");
-    } catch (AssertionError expected) {
-    }
-  }
-
-
-  @Test
-  public void stackWithChunkClearReturnsAddressAndEmptiesStack() {
-    UnsafeMemoryChunk slab = new UnsafeMemoryChunk(1024);
-    try {
-      SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new UnsafeMemoryChunk[]{slab});
-      ObjectChunk chunk = (ObjectChunk) ma.allocate(100);
-
-      long addr = chunk.getMemoryAddress();
-      SyncChunkStack stack = new SyncChunkStack();
-      stack.offer(addr);
-      long clearAddr = stack.clear();
-      assertEquals(addr, clearAddr);
-      assertEquals(true, stack.isEmpty());
-    } finally {
-      SimpleMemoryAllocatorImpl.freeOffHeapMemory();
-    }
-  }
-
-  @Test
-  public void stackWithChunkPollReturnsAddressAndEmptiesStack() {
-    UnsafeMemoryChunk slab = new UnsafeMemoryChunk(1024);
-    try {
-      SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new UnsafeMemoryChunk[]{slab});
-      ObjectChunk chunk = (ObjectChunk) ma.allocate(100);
-
-      long addr = chunk.getMemoryAddress();
-      SyncChunkStack stack = new SyncChunkStack();
-      stack.offer(addr);
-      long pollAddr = stack.poll();
-      assertEquals(addr, pollAddr);
-      assertEquals(true, stack.isEmpty());
-    } finally {
-      SimpleMemoryAllocatorImpl.freeOffHeapMemory();
-    }
-  }
-
-  @Test
-  public void stackWithChunkTotalSizeIsChunkSize() {
-    UnsafeMemoryChunk slab = new UnsafeMemoryChunk(1024);
-    try {
-      SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new UnsafeMemoryChunk[]{slab});
-      ObjectChunk chunk = (ObjectChunk) ma.allocate(100);
-      int chunkSize = chunk.getSize();
-
-      long addr = chunk.getMemoryAddress();
-      SyncChunkStack stack = new SyncChunkStack();
-      stack.offer(addr);
-      assertEquals(chunkSize, stack.computeTotalSize());
-    } finally {
-      SimpleMemoryAllocatorImpl.freeOffHeapMemory();
-    }
-  }
-
-
-  @Test
-  public void stackWithChunkLogShowsMsgAndSize() {
-    UnsafeMemoryChunk slab = new UnsafeMemoryChunk(1024);
-    try {
-      SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new UnsafeMemoryChunk[]{slab});
-      ObjectChunk chunk = (ObjectChunk) ma.allocate(100);
-      int chunkSize = chunk.getSize();
-
-      long addr = chunk.getMemoryAddress();
-      SyncChunkStack stack = new SyncChunkStack();
-      stack.offer(addr);
-      LogWriter lw = mock(LogWriter.class);
-      stack.logSizes(lw, "foo");
-      verify(lw).info("foo"+chunkSize);
-    } finally {
-      SimpleMemoryAllocatorImpl.freeOffHeapMemory();
-    }
-  }
-  
-  private class TestableSyncChunkStack extends SyncChunkStack {
-    public boolean doConcurrentMod = true;
-    public int chunk2Size;
-    private SimpleMemoryAllocatorImpl ma;
-    TestableSyncChunkStack(SimpleMemoryAllocatorImpl ma) {
-      this.ma = ma;
-    }
-    @Override
-    protected void testHookDoConcurrentModification() {
-      if (doConcurrentMod) {
-        doConcurrentMod = false;
-        ObjectChunk chunk2 = (ObjectChunk) ma.allocate(50);
-        this.chunk2Size = chunk2.getSize();
-        this.offer(chunk2.getMemoryAddress());
-      }
-    }
-  }
-  @Test
-  public void stackWithChunkTotalSizeIsChunkSizeWithConcurrentMod() {
-    UnsafeMemoryChunk slab = new UnsafeMemoryChunk(1024);
-    try {
-      SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new UnsafeMemoryChunk[]{slab});
-      ObjectChunk chunk = (ObjectChunk) ma.allocate(100);
-      int chunkSize = chunk.getSize();
-
-      long addr = chunk.getMemoryAddress();
-      TestableSyncChunkStack stack = new TestableSyncChunkStack(ma);
-      stack.offer(addr);
-      long totalSize = stack.computeTotalSize();
-      assertEquals("chunkSize=" + chunkSize + " chunk2Size=" + stack.chunk2Size, chunkSize + stack.chunk2Size, totalSize);
-    } finally {
-      SimpleMemoryAllocatorImpl.freeOffHeapMemory();
-    }
-  }
-
-
-  @Test
-  public void stackWithChunkLogShowsMsgAndSizeWithConcurrentMod() {
-    UnsafeMemoryChunk slab = new UnsafeMemoryChunk(1024);
-    try {
-      SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new UnsafeMemoryChunk[]{slab});
-      ObjectChunk chunk = (ObjectChunk) ma.allocate(100);
-      int chunkSize = chunk.getSize();
-
-      long addr = chunk.getMemoryAddress();
-      TestableSyncChunkStack stack = new TestableSyncChunkStack(ma);
-      stack.offer(addr);
-      LogWriter lw = mock(LogWriter.class);
-      stack.logSizes(lw, "foo");
-      verify(lw).info("foo"+chunkSize);
-      verify(lw).info("foo"+stack.chunk2Size);
-    } finally {
-      SimpleMemoryAllocatorImpl.freeOffHeapMemory();
-    }
-  }
-}


[6/8] incubator-geode git commit: GEODE-982: refactor off-heap

Posted by ds...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/ObjectChunk.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/ObjectChunk.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/ObjectChunk.java
deleted file mode 100644
index 29e6956..0000000
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/ObjectChunk.java
+++ /dev/null
@@ -1,737 +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.io.DataOutput;
-import java.io.IOException;
-import java.lang.reflect.Constructor;
-import java.lang.reflect.InvocationTargetException;
-import java.lang.reflect.Method;
-import java.nio.ByteBuffer;
-
-import com.gemstone.gemfire.cache.Region;
-import com.gemstone.gemfire.internal.DSCODE;
-import com.gemstone.gemfire.internal.HeapDataOutputStream;
-import com.gemstone.gemfire.internal.InternalDataSerializer;
-import com.gemstone.gemfire.internal.cache.EntryEventImpl;
-import com.gemstone.gemfire.internal.cache.RegionEntry;
-import com.gemstone.gemfire.internal.cache.RegionEntryContext;
-import com.gemstone.gemfire.internal.offheap.annotations.Unretained;
-
-/**
-   * A chunk that stores a Java object.
-   * Currently the object stored in this chunk
-   * is always an entry value of a Region.
-   * Note: this class has a natural ordering that is inconsistent with equals.
-   * Instances of this class should have a short lifetime. We do not store references
-   * to it in the cache. Instead the memoryAddress is stored in a primitive field in
-   * the cache and if used it will then, if needed, create an instance of this class.
-   */
-  public class ObjectChunk extends OffHeapCachedDeserializable implements Comparable<ObjectChunk>, MemoryBlock {
-    /**
-     * The unsafe memory address of the first byte of this chunk
-     */
-    private final long memoryAddress;
-    
-    /**
-     * The useCount, chunkSize, dataSizeDelta, isSerialized, and isCompressed
-     * are all stored in off heap memory in a HEADER. This saves heap memory
-     * by using off heap.
-     */
-    public final static int OFF_HEAP_HEADER_SIZE = 4 + 4;
-    /**
-     * We need to smallest chunk to at least have enough room for a hdr
-     * and for an off heap ref (which is a long).
-     */
-    public final static int MIN_CHUNK_SIZE = OFF_HEAP_HEADER_SIZE + 8;
-    /**
-     * int field.
-     * The number of bytes in this chunk.
-     */
-    private final static int CHUNK_SIZE_OFFSET = 0;
-    /**
-     * Volatile int field
-     * The upper two bits are used for the isSerialized
-     * and isCompressed flags.
-     * The next three bits are unused.
-     * The lower 3 bits of the most significant byte contains a magic number to help us detect
-     * if we are changing the ref count of an object that has been released.
-     * The next byte contains the dataSizeDelta.
-     * The number of bytes of logical data in this chunk.
-     * Since the number of bytes of logical data is always <= chunkSize
-     * and since chunkSize never changes, we have dataSize be
-     * a delta whose max value would be HUGE_MULTIPLE-1.
-     * The lower two bytes contains the use count.
-     */
-    final static int REF_COUNT_OFFSET = 4;
-    /**
-     * The upper two bits are used for the isSerialized
-     * and isCompressed flags.
-     */
-    final static int IS_SERIALIZED_BIT =    0x80000000;
-    final static int IS_COMPRESSED_BIT =    0x40000000;
-    // UNUSED 0x38000000
-    final static int MAGIC_MASK = 0x07000000;
-    final static int MAGIC_NUMBER = 0x05000000;
-    final static int DATA_SIZE_DELTA_MASK = 0x00ff0000;
-    final static int DATA_SIZE_SHIFT = 16;
-    final static int REF_COUNT_MASK =       0x0000ffff;
-    final static int MAX_REF_COUNT = 0xFFFF;
-    final static long FILL_PATTERN = 0x3c3c3c3c3c3c3c3cL;
-    final static byte FILL_BYTE = 0x3c;
-    
-    protected ObjectChunk(long memoryAddress, int chunkSize) {
-      SimpleMemoryAllocatorImpl.validateAddressAndSize(memoryAddress, chunkSize);
-      this.memoryAddress = memoryAddress;
-      setSize(chunkSize);
-      UnsafeMemoryChunk.writeAbsoluteIntVolatile(getMemoryAddress()+REF_COUNT_OFFSET, MAGIC_NUMBER);
-    }
-    public void readyForFree() {
-      UnsafeMemoryChunk.writeAbsoluteIntVolatile(getMemoryAddress()+REF_COUNT_OFFSET, 0);
-    }
-    public void readyForAllocation() {
-      if (!UnsafeMemoryChunk.writeAbsoluteIntVolatile(getMemoryAddress()+REF_COUNT_OFFSET, 0, MAGIC_NUMBER)) {
-        throw new IllegalStateException("Expected 0 but found " + Integer.toHexString(UnsafeMemoryChunk.readAbsoluteIntVolatile(getMemoryAddress()+REF_COUNT_OFFSET)));
-      }
-    }
-    /**
-     * Should only be used by FakeChunk subclass
-     */
-    protected ObjectChunk() {
-      this.memoryAddress = 0L;
-    }
-    
-    /**
-     * Used to create a Chunk given an existing, already allocated,
-     * memoryAddress. The off heap header has already been initialized.
-     */
-    protected ObjectChunk(long memoryAddress) {
-      SimpleMemoryAllocatorImpl.validateAddress(memoryAddress);
-      this.memoryAddress = memoryAddress;
-    }
-    
-    protected ObjectChunk(ObjectChunk chunk) {
-      this.memoryAddress = chunk.memoryAddress;
-    }
-    
-    /**
-     * Throw an exception if this chunk is not allocated
-     */
-    public void checkIsAllocated() {
-      int originalBits = UnsafeMemoryChunk.readAbsoluteIntVolatile(this.memoryAddress+REF_COUNT_OFFSET);
-      if ((originalBits&MAGIC_MASK) != MAGIC_NUMBER) {
-        throw new IllegalStateException("It looks like this off heap memory was already freed. rawBits=" + Integer.toHexString(originalBits));
-      }
-    }
-    
-    public void incSize(int inc) {
-      setSize(getSize()+inc);
-    }
-    
-    protected void beforeReturningToAllocator() {
-      
-    }
-
-    @Override
-    public int getSize() {
-      return getSize(this.memoryAddress);
-    }
-
-    public void setSize(int size) {
-      setSize(this.memoryAddress, size);
-    }
-
-    public long getMemoryAddress() {
-      return this.memoryAddress;
-    }
-    
-    public int getDataSize() {
-      /*int dataSizeDelta = UnsafeMemoryChunk.readAbsoluteInt(this.memoryAddress+REF_COUNT_OFFSET);
-      dataSizeDelta &= DATA_SIZE_DELTA_MASK;
-      dataSizeDelta >>= DATA_SIZE_SHIFT;
-      return getSize() - dataSizeDelta;*/
-      return getDataSize(this.memoryAddress);
-    }
-    
-    protected static int getDataSize(long memoryAdress) {
-      int dataSizeDelta = UnsafeMemoryChunk.readAbsoluteInt(memoryAdress+REF_COUNT_OFFSET);
-      dataSizeDelta &= DATA_SIZE_DELTA_MASK;
-      dataSizeDelta >>= DATA_SIZE_SHIFT;
-      return getSize(memoryAdress) - dataSizeDelta;
-    }
-    
-    protected long getBaseDataAddress() {
-      return this.memoryAddress+OFF_HEAP_HEADER_SIZE;
-    }
-    protected int getBaseDataOffset() {
-      return 0;
-    }
-    
-    /**
-     * Creates and returns a direct ByteBuffer that contains the contents of this Chunk.
-     * Note that the returned ByteBuffer has a reference to this chunk's
-     * off-heap address so it can only be used while this Chunk is retained.
-     * @return the created direct byte buffer or null if it could not be created.
-     */
-    @Unretained
-    public ByteBuffer createDirectByteBuffer() {
-      return basicCreateDirectByteBuffer(getBaseDataAddress(), getDataSize());
-    }
-    @Override
-    public void sendTo(DataOutput out) throws IOException {
-      if (!this.isCompressed() && out instanceof HeapDataOutputStream) {
-        ByteBuffer bb = createDirectByteBuffer();
-        if (bb != null) {
-          HeapDataOutputStream hdos = (HeapDataOutputStream) out;
-          if (this.isSerialized()) {
-            hdos.write(bb);
-          } else {
-            hdos.writeByte(DSCODE.BYTE_ARRAY);
-            InternalDataSerializer.writeArrayLength(bb.remaining(), hdos);
-            hdos.write(bb);
-          }
-          return;
-        }
-      }
-      super.sendTo(out);
-    }
-    
-    @Override
-    public void sendAsByteArray(DataOutput out) throws IOException {
-      if (!isCompressed() && out instanceof HeapDataOutputStream) {
-        ByteBuffer bb = createDirectByteBuffer();
-        if (bb != null) {
-          HeapDataOutputStream hdos = (HeapDataOutputStream) out;
-          InternalDataSerializer.writeArrayLength(bb.remaining(), hdos);
-          hdos.write(bb);
-          return;
-        }
-      }
-      super.sendAsByteArray(out);
-    }
-       
-    private static volatile Class dbbClass = null;
-    private static volatile Constructor dbbCtor = null;
-    private static volatile boolean dbbCreateFailed = false;
-    
-    /**
-     * @return the created direct byte buffer or null if it could not be created.
-     */
-    private static ByteBuffer basicCreateDirectByteBuffer(long baseDataAddress, int dataSize) {
-      if (dbbCreateFailed) {
-        return null;
-      }
-      Constructor ctor = dbbCtor;
-      if (ctor == null) {
-        Class c = dbbClass;
-        if (c == null) {
-          try {
-            c = Class.forName("java.nio.DirectByteBuffer");
-          } catch (ClassNotFoundException e) {
-            //throw new IllegalStateException("Could not find java.nio.DirectByteBuffer", e);
-            dbbCreateFailed = true;
-            dbbAddressFailed = true;
-            return null;
-          }
-          dbbClass = c;
-        }
-        try {
-          ctor = c.getDeclaredConstructor(long.class, int.class);
-        } catch (NoSuchMethodException | SecurityException e) {
-          //throw new IllegalStateException("Could not get constructor DirectByteBuffer(long, int)", e);
-          dbbClass = null;
-          dbbCreateFailed = true;
-          return null;
-        }
-        ctor.setAccessible(true);
-        dbbCtor = ctor;
-      }
-      try {
-        return (ByteBuffer)ctor.newInstance(baseDataAddress, dataSize);
-      } catch (InstantiationException | IllegalAccessException | IllegalArgumentException | InvocationTargetException e) {
-        //throw new IllegalStateException("Could not create an instance using DirectByteBuffer(long, int)", e);
-        dbbClass = null;
-        dbbCtor = null;
-        dbbCreateFailed = true;
-        return null;
-      }
-    }
-    private static volatile Method dbbAddressMethod = null;
-    private static volatile boolean dbbAddressFailed = false;
-    
-    /**
-     * Returns the address of the Unsafe memory for the first byte of a direct ByteBuffer.
-     * If the buffer is not direct or the address can not be obtained return 0.
-     */
-    public static long getDirectByteBufferAddress(ByteBuffer bb) {
-      if (!bb.isDirect()) {
-        return 0L;
-      }
-      if (dbbAddressFailed) {
-        return 0L;
-      }
-      Method m = dbbAddressMethod;
-      if (m == null) {
-        Class c = dbbClass;
-        if (c == null) {
-          try {
-            c = Class.forName("java.nio.DirectByteBuffer");
-          } catch (ClassNotFoundException e) {
-            //throw new IllegalStateException("Could not find java.nio.DirectByteBuffer", e);
-            dbbCreateFailed = true;
-            dbbAddressFailed = true;
-            return 0L;
-          }
-          dbbClass = c;
-        }
-        try {
-          m = c.getDeclaredMethod("address");
-        } catch (NoSuchMethodException | SecurityException e) {
-          //throw new IllegalStateException("Could not get method DirectByteBuffer.address()", e);
-          dbbClass = null;
-          dbbAddressFailed = true;
-          return 0L;
-        }
-        m.setAccessible(true);
-        dbbAddressMethod = m;
-      }
-      try {
-        return (Long)m.invoke(bb);
-      } catch (IllegalAccessException | IllegalArgumentException | InvocationTargetException e) {
-        //throw new IllegalStateException("Could not create an invoke DirectByteBuffer.address()", e);
-        dbbClass = null;
-        dbbAddressMethod = null;
-        dbbAddressFailed = true;
-        return 0L;
-      }
-    }
-    /**
-     * Returns an address that can be used with unsafe apis to access this chunks memory.
-     * @param offset the offset from this chunk's first byte of the byte the returned address should point to. Must be >= 0.
-     * @param size the number of bytes that will be read using the returned address. Assertion will use this to verify that all the memory accessed belongs to this chunk. Must be > 0.
-     * @return a memory address that can be used with unsafe apis
-     */
-    public long getUnsafeAddress(int offset, int size) {
-      assert offset >= 0 && offset + size <= getDataSize(): "Offset=" + offset + ",size=" + size + ",dataSize=" + getDataSize() + ", chunkSize=" + getSize() + ", but offset + size must be <= " + getDataSize();
-      assert size > 0;
-      long result = getBaseDataAddress() + offset;
-      // validateAddressAndSizeWithinSlab(result, size);
-      return result;
-    }
-    
-    @Override
-    public byte readByte(int offset) {
-      assert offset < getDataSize();
-      return UnsafeMemoryChunk.readAbsoluteByte(getBaseDataAddress() + offset);
-    }
-
-    @Override
-    public void writeByte(int offset, byte value) {
-      assert offset < getDataSize();
-      UnsafeMemoryChunk.writeAbsoluteByte(getBaseDataAddress() + offset, value);
-    }
-
-    @Override
-    public void readBytes(int offset, byte[] bytes) {
-      readBytes(offset, bytes, 0, bytes.length);
-    }
-
-    @Override
-    public void writeBytes(int offset, byte[] bytes) {
-      writeBytes(offset, bytes, 0, bytes.length);
-    }
-
-    public long getAddressForReading(int offset, int size) {
-      //delegate to getUnsafeAddress - as both the methods does return the memory address from given offset
-      return getUnsafeAddress(offset, size);
-    }
-    
-    @Override
-    public void readBytes(int offset, byte[] bytes, int bytesOffset, int size) {
-      assert offset+size <= getDataSize();
-      UnsafeMemoryChunk.readAbsoluteBytes(getBaseDataAddress() + offset, bytes, bytesOffset, size);
-    }
-
-    @Override
-    public void writeBytes(int offset, byte[] bytes, int bytesOffset, int size) {
-      assert offset+size <= getDataSize();
-      UnsafeMemoryChunk.writeAbsoluteBytes(getBaseDataAddress() + offset, bytes, bytesOffset, size);
-    }
-    
-    @Override
-    public void release() {
-      release(this.memoryAddress);
-     }
-
-    @Override
-    public int compareTo(ObjectChunk o) {
-      int result = Integer.signum(getSize() - o.getSize());
-      if (result == 0) {
-        // For the same sized chunks we really don't care about their order
-        // but we need compareTo to only return 0 if the two chunks are identical
-        result = Long.signum(getMemoryAddress() - o.getMemoryAddress());
-      }
-      return result;
-    }
-    
-    @Override
-    public boolean equals(Object o) {
-      if (o instanceof ObjectChunk) {
-        return getMemoryAddress() == ((ObjectChunk) o).getMemoryAddress();
-      }
-      return false;
-    }
-    
-    @Override
-    public int hashCode() {
-      long value = this.getMemoryAddress();
-      return (int)(value ^ (value >>> 32));
-    }
-
-    // OffHeapCachedDeserializable methods 
-    
-    @Override
-    public void setSerializedValue(byte[] value) {
-      writeBytes(0, value);
-    }
-    
-    public byte[] getDecompressedBytes(RegionEntryContext context) {
-      byte[] result = getCompressedBytes();
-      long time = context.getCachePerfStats().startDecompression();
-      result = context.getCompressor().decompress(result);
-      context.getCachePerfStats().endDecompression(time);      
-      return result;
-    }
-    
-    /**
-     * Returns the raw possibly compressed bytes of this chunk
-     */
-    public byte[] getCompressedBytes() {
-      byte[] result = new byte[getDataSize()];
-      readBytes(0, result);
-      //debugLog("reading", true);
-      SimpleMemoryAllocatorImpl.getAllocator().getStats().incReads();
-      return result;
-    }
-    protected byte[] getRawBytes() {
-      byte[] result = getCompressedBytes();
-      // TODO OFFHEAP: change the following to assert !isCompressed();
-      if (isCompressed()) {
-        throw new UnsupportedOperationException();
-      }
-      return result;
-    }
-
-    @Override
-    public byte[] getSerializedValue() {
-      byte [] result = getRawBytes();
-      if (!isSerialized()) {
-        // The object is a byte[]. So we need to make it look like a serialized byte[] in our result
-        result = EntryEventImpl.serialize(result);
-      }
-      return result;
-    }
-    
-    @Override
-    public Object getDeserializedValue(Region r, RegionEntry re) {
-      if (isSerialized()) {
-        // TODO OFFHEAP: debug deserializeChunk
-        return EntryEventImpl.deserialize(getRawBytes());
-        //assert !isCompressed();
-        //return EntryEventImpl.deserializeChunk(this);
-      } else {
-        return getRawBytes();
-      }
-    }
-    
-    /**
-     * We want this to include memory overhead so use getSize() instead of getDataSize().
-     */
-    @Override
-    public int getSizeInBytes() {
-      // Calling getSize includes the off heap header size.
-      // We do not add anything to this since the size of the reference belongs to the region entry size
-      // not the size of this object.
-      return getSize();
-    }
-
-    @Override
-    public int getValueSizeInBytes() {
-      return getDataSize();
-    }
-
-    @Override
-    public void copyBytes(int src, int dst, int size) {
-      throw new UnsupportedOperationException("Implement if used");
-//      assert src+size <= getDataSize();
-//      assert dst+size < getDataSize();
-//      getSlabs()[this.getSlabIdx()].copyBytes(getBaseDataAddress()+src, getBaseDataAddress()+dst, size);
-    }
-
-    @Override
-    public boolean isSerialized() {
-      return (UnsafeMemoryChunk.readAbsoluteInt(this.memoryAddress+REF_COUNT_OFFSET) & IS_SERIALIZED_BIT) != 0;
-    }
-
-    @Override
-    public boolean isCompressed() {
-      return (UnsafeMemoryChunk.readAbsoluteInt(this.memoryAddress+REF_COUNT_OFFSET) & IS_COMPRESSED_BIT) != 0;
-    }
-
-    @Override
-    public boolean retain() {
-      return retain(this.memoryAddress);
-    }
-
-    @Override
-    public int getRefCount() {
-      return getRefCount(this.memoryAddress);
-    }
-
-    public static int getSize(long memAddr) {
-      SimpleMemoryAllocatorImpl.validateAddress(memAddr);
-      return UnsafeMemoryChunk.readAbsoluteInt(memAddr+CHUNK_SIZE_OFFSET);
-    }
-    public static void setSize(long memAddr, int size) {
-      SimpleMemoryAllocatorImpl.validateAddressAndSize(memAddr, size);
-      UnsafeMemoryChunk.writeAbsoluteInt(memAddr+CHUNK_SIZE_OFFSET, size);
-    }
-    public static long getNext(long memAddr) {
-      SimpleMemoryAllocatorImpl.validateAddress(memAddr);
-      return UnsafeMemoryChunk.readAbsoluteLong(memAddr+OFF_HEAP_HEADER_SIZE);
-    }
-    public static void setNext(long memAddr, long next) {
-      SimpleMemoryAllocatorImpl.validateAddress(memAddr);
-      UnsafeMemoryChunk.writeAbsoluteLong(memAddr+OFF_HEAP_HEADER_SIZE, next);
-    }
-    
-    /**
-     * Fills the chunk with a repeated byte fill pattern.
-     * @param baseAddress the starting address for a {@link ObjectChunk}.
-     */
-    public static void fill(long baseAddress) {
-      long startAddress = baseAddress + MIN_CHUNK_SIZE;
-      int size = getSize(baseAddress) - MIN_CHUNK_SIZE;
-      
-      UnsafeMemoryChunk.fill(startAddress, size, FILL_BYTE);
-    }
-    
-    /**
-     * Validates that the fill pattern for this chunk has not been disturbed.  This method
-     * assumes the TINY_MULTIPLE is 8 bytes.
-     * @throws IllegalStateException when the pattern has been violated.
-     */
-    public void validateFill() {
-      assert FreeListManager.TINY_MULTIPLE == 8;
-      
-      long startAddress = getMemoryAddress() + MIN_CHUNK_SIZE;
-      int size = getSize() - MIN_CHUNK_SIZE;
-      
-      for(int i = 0;i < size;i += FreeListManager.TINY_MULTIPLE) {
-        if(UnsafeMemoryChunk.readAbsoluteLong(startAddress + i) != FILL_PATTERN) {
-          throw new IllegalStateException("Fill pattern violated for chunk " + getMemoryAddress() + " with size " + getSize());
-        }        
-      }
-    }
-
-    public void setSerialized(boolean isSerialized) {
-      if (isSerialized) {
-        int bits;
-        int originalBits;
-        do {
-          originalBits = UnsafeMemoryChunk.readAbsoluteIntVolatile(this.memoryAddress+REF_COUNT_OFFSET);
-          if ((originalBits&MAGIC_MASK) != MAGIC_NUMBER) {
-            throw new IllegalStateException("It looks like this off heap memory was already freed. rawBits=" + Integer.toHexString(originalBits));
-          }
-          bits = originalBits | IS_SERIALIZED_BIT;
-        } while (!UnsafeMemoryChunk.writeAbsoluteIntVolatile(this.memoryAddress+REF_COUNT_OFFSET, originalBits, bits));
-      }
-    }
-    public void setCompressed(boolean isCompressed) {
-      if (isCompressed) {
-        int bits;
-        int originalBits;
-        do {
-          originalBits = UnsafeMemoryChunk.readAbsoluteIntVolatile(this.memoryAddress+REF_COUNT_OFFSET);
-          if ((originalBits&MAGIC_MASK) != MAGIC_NUMBER) {
-            throw new IllegalStateException("It looks like this off heap memory was already freed. rawBits=" + Integer.toHexString(originalBits));
-          }
-          bits = originalBits | IS_COMPRESSED_BIT;
-        } while (!UnsafeMemoryChunk.writeAbsoluteIntVolatile(this.memoryAddress+REF_COUNT_OFFSET, originalBits, bits));
-      }
-    }
-    public void setDataSize(int dataSize) { // KIRK
-      assert dataSize <= getSize();
-      int delta = getSize() - dataSize;
-      assert delta <= (DATA_SIZE_DELTA_MASK >> DATA_SIZE_SHIFT);
-      delta <<= DATA_SIZE_SHIFT;
-      int bits;
-      int originalBits;
-      do {
-        originalBits = UnsafeMemoryChunk.readAbsoluteIntVolatile(this.memoryAddress+REF_COUNT_OFFSET);
-        if ((originalBits&MAGIC_MASK) != MAGIC_NUMBER) {
-          throw new IllegalStateException("It looks like this off heap memory was already freed. rawBits=" + Integer.toHexString(originalBits));
-        }
-        bits = originalBits;
-        bits &= ~DATA_SIZE_DELTA_MASK; // clear the old dataSizeDelta bits
-        bits |= delta; // set the dataSizeDelta bits to the new delta value
-      } while (!UnsafeMemoryChunk.writeAbsoluteIntVolatile(this.memoryAddress+REF_COUNT_OFFSET, originalBits, bits));
-    }
-    
-    public void initializeUseCount() {
-      int rawBits;
-      do {
-        rawBits = UnsafeMemoryChunk.readAbsoluteIntVolatile(this.memoryAddress+REF_COUNT_OFFSET);
-        if ((rawBits&MAGIC_MASK) != MAGIC_NUMBER) {
-          throw new IllegalStateException("It looks like this off heap memory was already freed. rawBits=" + Integer.toHexString(rawBits));
-        }
-        int uc = rawBits & REF_COUNT_MASK;
-        if (uc != 0) {
-          throw new IllegalStateException("Expected use count to be zero but it was: " + uc + " rawBits=0x" + Integer.toHexString(rawBits));
-        }
-      } while (!UnsafeMemoryChunk.writeAbsoluteIntVolatile(this.memoryAddress+REF_COUNT_OFFSET, rawBits, rawBits+1));
-    }
-
-    public static int getRefCount(long memAddr) {
-      return UnsafeMemoryChunk.readAbsoluteInt(memAddr+REF_COUNT_OFFSET) & REF_COUNT_MASK;
-    }
-
-    public static boolean retain(long memAddr) {
-      SimpleMemoryAllocatorImpl.validateAddress(memAddr);
-      int uc;
-      int rawBits;
-      int retryCount = 0;
-      do {
-        rawBits = UnsafeMemoryChunk.readAbsoluteIntVolatile(memAddr+REF_COUNT_OFFSET);
-        if ((rawBits&MAGIC_MASK) != MAGIC_NUMBER) {
-          // same as uc == 0
-          // TODO MAGIC_NUMBER rethink its use and interaction with compactor fragments
-          return false;
-        }
-        uc = rawBits & REF_COUNT_MASK;
-        if (uc == MAX_REF_COUNT) {
-          throw new IllegalStateException("Maximum use count exceeded. rawBits=" + Integer.toHexString(rawBits));
-        } else if (uc == 0) {
-          return false;
-        }
-        retryCount++;
-        if (retryCount > 1000) {
-          throw new IllegalStateException("tried to write " + (rawBits+1) + " to @" + Long.toHexString(memAddr) + " 1,000 times.");
-        }
-      } while (!UnsafeMemoryChunk.writeAbsoluteIntVolatile(memAddr+REF_COUNT_OFFSET, rawBits, rawBits+1));
-      //debugLog("use inced ref count " + (uc+1) + " @" + Long.toHexString(memAddr), true);
-      if (ReferenceCountHelper.trackReferenceCounts()) {
-        ReferenceCountHelper.refCountChanged(memAddr, false, uc+1);
-      }
-
-      return true;
-    }
-    public static void release(final long memAddr) {
-      release(memAddr, null);
-    }
-    static void release(final long memAddr, FreeListManager freeListManager) {
-      SimpleMemoryAllocatorImpl.validateAddress(memAddr);
-      int newCount;
-      int rawBits;
-      boolean returnToAllocator;
-      do {
-        returnToAllocator = false;
-        rawBits = UnsafeMemoryChunk.readAbsoluteIntVolatile(memAddr+REF_COUNT_OFFSET);
-        if ((rawBits&MAGIC_MASK) != MAGIC_NUMBER) {
-          String msg = "It looks like off heap memory @" + Long.toHexString(memAddr) + " was already freed. rawBits=" + Integer.toHexString(rawBits) + " history=" + ReferenceCountHelper.getFreeRefCountInfo(memAddr);
-          //debugLog(msg, true);
-          throw new IllegalStateException(msg);
-        }
-        int curCount = rawBits&REF_COUNT_MASK;
-        if ((curCount) == 0) {
-          //debugLog("too many frees @" + Long.toHexString(memAddr), true);
-          throw new IllegalStateException("Memory has already been freed." + " history=" + ReferenceCountHelper.getFreeRefCountInfo(memAddr) /*+ System.identityHashCode(this)*/);
-        }
-        if (curCount == 1) {
-          newCount = 0; // clear the use count, bits, and the delta size since it will be freed.
-          returnToAllocator = true;
-        } else {
-          newCount = rawBits-1;
-        }
-      } while (!UnsafeMemoryChunk.writeAbsoluteIntVolatile(memAddr+REF_COUNT_OFFSET, rawBits, newCount));
-      //debugLog("free deced ref count " + (newCount&USE_COUNT_MASK) + " @" + Long.toHexString(memAddr), true);
-      if (returnToAllocator ) {
-       if (ReferenceCountHelper.trackReferenceCounts()) {
-          if (ReferenceCountHelper.trackFreedReferenceCounts()) {
-            ReferenceCountHelper.refCountChanged(memAddr, true, newCount&REF_COUNT_MASK);
-          }
-          ReferenceCountHelper.freeRefCountInfo(memAddr);
-        }
-        if (freeListManager == null) {
-          freeListManager = SimpleMemoryAllocatorImpl.getAllocator().getFreeListManager();
-        }
-        freeListManager.free(memAddr);
-      } else {
-        if (ReferenceCountHelper.trackReferenceCounts()) {
-          ReferenceCountHelper.refCountChanged(memAddr, true, newCount&REF_COUNT_MASK);
-        }
-      }
-    }
-    
-    @Override
-    public String toString() {
-      return toStringForOffHeapByteSource();
-      // This old impl is not safe because it calls getDeserializedForReading and we have code that call toString that does not inc the refcount.
-      // Also if this Chunk is compressed we don't know how to decompress it.
-      //return super.toString() + ":<dataSize=" + getDataSize() + " refCount=" + getRefCount() + " addr=" + getMemoryAddress() + " storedObject=" + getDeserializedForReading() + ">";
-    }
-    
-    protected String toStringForOffHeapByteSource() {
-      return super.toString() + ":<dataSize=" + getDataSize() + " refCount=" + getRefCount() + " addr=" + Long.toHexString(getMemoryAddress()) + ">";
-    }
-    
-    @Override
-    public State getState() {
-      if (getRefCount() > 0) {
-        return State.ALLOCATED;
-      } else {
-        return State.DEALLOCATED;
-      }
-    }
-    @Override
-    public MemoryBlock getNextBlock() {
-      throw new UnsupportedOperationException();
-    }
-    @Override
-    public int getBlockSize() {
-      return getSize();
-    }
-    @Override
-    public int getSlabId() {
-      throw new UnsupportedOperationException();
-    }
-    @Override
-    public int getFreeListId() {
-      return -1;
-    }
-    @Override
-    public String getDataType() {
-      return null;
-    }
-    @Override
-    public Object getDataValue() {
-      return null;
-    }
-    public ObjectChunk slice(int position, int limit) {
-      return new ObjectChunkSlice(this, position, limit);
-    }
-  }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/ObjectChunkSlice.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/ObjectChunkSlice.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/ObjectChunkSlice.java
deleted file mode 100644
index 3d6bf57..0000000
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/ObjectChunkSlice.java
+++ /dev/null
@@ -1,44 +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;
-
-/**
- * Represents a slice of an ObjectChunk.
- * A slice is a subsequence of the bytes stored in an ObjectChunk.
- */
-public class ObjectChunkSlice extends ObjectChunk {
-  private final int offset;
-  private final int capacity;
-  public ObjectChunkSlice(ObjectChunk objectChunk, int position, int limit) {
-    super(objectChunk);
-    this.offset = objectChunk.getBaseDataOffset() + position;
-    this.capacity = limit - position;
-  }
-  @Override
-  public int getDataSize() {
-    return this.capacity;
-  }
-  
-  @Override
-  protected long getBaseDataAddress() {
-    return super.getBaseDataAddress() + this.offset;
-  }
-  @Override
-  protected int getBaseDataOffset() {
-    return this.offset;
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/ObjectChunkWithHeapForm.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/ObjectChunkWithHeapForm.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/ObjectChunkWithHeapForm.java
deleted file mode 100644
index 5020c7a..0000000
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/ObjectChunkWithHeapForm.java
+++ /dev/null
@@ -1,40 +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;
-
-/**
- * Used to keep the heapForm around while an operation is still in progress.
- * This allows the operation to access the serialized heap form instead of copying
- * it from offheap. See bug 48135.
- */
-public class ObjectChunkWithHeapForm extends ObjectChunk {
-  private final byte[] heapForm;
-  
-  public ObjectChunkWithHeapForm(ObjectChunk chunk, byte[] heapForm) {
-    super(chunk);
-    this.heapForm = heapForm;
-  }
-
-  @Override
-  protected byte[] getRawBytes() {
-    return this.heapForm;
-  }
-  
-  public ObjectChunk getChunkWithoutHeapForm() {
-    return new ObjectChunk(this);
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapCachedDeserializable.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapCachedDeserializable.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapCachedDeserializable.java
deleted file mode 100644
index bd380e2..0000000
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapCachedDeserializable.java
+++ /dev/null
@@ -1,142 +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.cache.Region;
-import com.gemstone.gemfire.internal.DSCODE;
-import com.gemstone.gemfire.internal.cache.BytesAndBitsForCompactor;
-import com.gemstone.gemfire.internal.cache.EntryBits;
-import com.gemstone.gemfire.internal.cache.RegionEntry;
-import com.gemstone.gemfire.internal.offheap.annotations.Unretained;
-
-/**
- * This abstract class is intended to be used by {@link MemoryChunk} implementations that also want
- * to be a CachedDeserializable.
- * 
- * @author darrel
- * @since 9.0
- */
-public abstract class OffHeapCachedDeserializable extends AbstractStoredObject implements MemoryChunkWithRefCount {
-  public abstract void setSerializedValue(byte[] value);
-  @Override
-  public abstract byte[] getSerializedValue();
-  @Override
-  public abstract int getSizeInBytes();
-  @Override
-  public abstract int getValueSizeInBytes();
-  @Override
-  public abstract Object getDeserializedValue(Region r, RegionEntry re);
-
-  @Override
-  public void fillSerializedValue(BytesAndBitsForCompactor wrapper, byte userBits) {
-    if (isSerialized()) {
-      userBits = EntryBits.setSerialized(userBits, true);
-    }
-    wrapper.setChunkData((ObjectChunk) this, userBits);
-  }
-  
-  String getShortClassName() {
-    String cname = getClass().getName();
-    return cname.substring(getClass().getPackage().getName().length()+1);
-  }
-
-  @Override
-  public String toString() {
-    return getShortClassName()+"@"+this.hashCode();
-  }
-  public boolean checkDataEquals(@Unretained OffHeapCachedDeserializable other) {
-    if (this == other) {
-      return true;
-    }
-    if (isSerialized() != other.isSerialized()) {
-      return false;
-    }
-    int mySize = getValueSizeInBytes();
-    if (mySize != other.getValueSizeInBytes()) {
-      return false;
-    }
-    // We want to be able to do this operation without copying any of the data into the heap.
-    // Hopefully the jvm is smart enough to use our stack for this short lived array.
-    final byte[] dataCache1 = new byte[1024];
-    final byte[] dataCache2 = new byte[dataCache1.length];
-    // TODO OFFHEAP: no need to copy to heap. Just get the address of each and compare each byte.
-    int i;
-    // inc it twice since we are reading two different off-heap objects
-    SimpleMemoryAllocatorImpl.getAllocator().getStats().incReads();
-    SimpleMemoryAllocatorImpl.getAllocator().getStats().incReads();
-    for (i=0; i < mySize-(dataCache1.length-1); i+=dataCache1.length) {
-      this.readBytes(i, dataCache1);
-      other.readBytes(i, dataCache2);
-      for (int j=0; j < dataCache1.length; j++) {
-        if (dataCache1[j] != dataCache2[j]) {
-          return false;
-        }
-      }
-    }
-    int bytesToRead = mySize-i;
-    if (bytesToRead > 0) {
-      // need to do one more read which will be less than dataCache.length
-      this.readBytes(i, dataCache1, 0, bytesToRead);
-      other.readBytes(i, dataCache2, 0, bytesToRead);
-      for (int j=0; j < bytesToRead; j++) {
-        if (dataCache1[j] != dataCache2[j]) {
-          return false;
-        }
-      }
-    }
-    return true;
-  }
-  
-  public boolean isSerializedPdxInstance() {
-    byte dsCode = this.readByte(0);
-    return dsCode == DSCODE.PDX || dsCode == DSCODE.PDX_ENUM || dsCode == DSCODE.PDX_INLINE_ENUM;
-  }
-  
-  public boolean checkDataEquals(byte[] serializedObj) {
-    // caller was responsible for checking isSerialized
-    int mySize = getValueSizeInBytes();
-    if (mySize != serializedObj.length) {
-      return false;
-    }
-    // We want to be able to do this operation without copying any of the data into the heap.
-    // Hopefully the jvm is smart enough to use our stack for this short lived array.
-    // TODO OFFHEAP: compare as ByteBuffers?
-    final byte[] dataCache = new byte[1024];
-    int idx=0;
-    int i;
-    SimpleMemoryAllocatorImpl.getAllocator().getStats().incReads();
-    for (i=0; i < mySize-(dataCache.length-1); i+=dataCache.length) {
-      this.readBytes(i, dataCache);
-      for (int j=0; j < dataCache.length; j++) {
-        if (dataCache[j] != serializedObj[idx++]) {
-          return false;
-        }
-      }
-    }
-    int bytesToRead = mySize-i;
-    if (bytesToRead > 0) {
-      // need to do one more read which will be less than dataCache.length
-      this.readBytes(i, dataCache, 0, bytesToRead);
-      for (int j=0; j < bytesToRead; j++) {
-        if (dataCache[j] != serializedObj[idx++]) {
-          return false;
-        }
-      }
-    }
-    return true;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapHelper.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapHelper.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapHelper.java
index 4845931..8989293 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapHelper.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapHelper.java
@@ -60,7 +60,7 @@ public class OffHeapHelper {
         return ohv.getDeserializedForReading();
       }
       } finally {
-        ohv.release();
+        release(ohv);
       }
     } else {
       return v;
@@ -93,8 +93,8 @@ public class OffHeapHelper {
    * @return true if release was done
    */
   public static boolean release(@Released Object o) {
-    if (o instanceof MemoryChunkWithRefCount) {
-      ((MemoryChunkWithRefCount) o).release();
+    if (o instanceof StoredObject) {
+      ((StoredObject) o).release();
       return true;
     } else {
       return false;
@@ -105,9 +105,14 @@ public class OffHeapHelper {
    * @return true if release was done
    */
   public static boolean releaseWithNoTracking(@Released Object o) {
-    if (o instanceof MemoryChunkWithRefCount) {
+    if (o instanceof StoredObject) {
+      StoredObject so = (StoredObject) o;
+      if (!so.hasRefCount()) {
+        so.release();
+        return true;
+      }
       ReferenceCountHelper.skipRefCountTracking();
-      ((MemoryChunkWithRefCount) o).release();
+      so.release();
       ReferenceCountHelper.unskipRefCountTracking();
       return true;
     } else {
@@ -120,9 +125,14 @@ public class OffHeapHelper {
    * @return true if release was done
    */
   public static boolean releaseAndTrackOwner(@Released final Object o, final Object owner) {
-    if (o instanceof MemoryChunkWithRefCount) {
+    if (o instanceof StoredObject) {
+      StoredObject so = (StoredObject) o;
+      if (!so.hasRefCount()) {
+        so.release();
+        return true;
+      }
       ReferenceCountHelper.setReferenceCountOwner(owner);
-      ((MemoryChunkWithRefCount) o).release();
+      so.release();
       ReferenceCountHelper.setReferenceCountOwner(null);
       return true;
     } else {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapRegionEntryHelper.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapRegionEntryHelper.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapRegionEntryHelper.java
index b62d97a..a502418 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapRegionEntryHelper.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapRegionEntryHelper.java
@@ -63,8 +63,7 @@ public class OffHeapRegionEntryHelper {
   };
   
   private static long objectToAddress(@Unretained Object v) {
-    if (v instanceof ObjectChunk) return ((ObjectChunk) v).getMemoryAddress();
-    if (v instanceof DataAsAddress) return ((DataAsAddress) v).getEncodedAddress();
+    if (v instanceof StoredObject) return ((StoredObject) v).getAddress();
     if (v == null) return NULL_ADDRESS;
     if (v == Token.TOMBSTONE) return TOMBSTONE_ADDRESS;
     if (v == Token.INVALID) return INVALID_ADDRESS;
@@ -90,7 +89,7 @@ public class OffHeapRegionEntryHelper {
   @Unretained @Retained
   public static Object addressToObject(@Released @Retained long ohAddress, boolean decompress, RegionEntryContext context) {
     if (isOffHeap(ohAddress)) {
-      @Unretained ObjectChunk chunk =  new ObjectChunk(ohAddress);
+      @Unretained OffHeapStoredObject chunk =  new OffHeapStoredObject(ohAddress);
       @Unretained Object result = chunk;
       if (decompress && chunk.isCompressed()) {
         try {
@@ -113,7 +112,7 @@ public class OffHeapRegionEntryHelper {
       }
       return result;
     } else if ((ohAddress & ENCODED_BIT) != 0) {
-      DataAsAddress daa = new DataAsAddress(ohAddress);
+      TinyStoredObject daa = new TinyStoredObject(ohAddress);
       Object result = daa;
       if (decompress && daa.isCompressed()) {
         byte[] decompressedBytes = daa.getDecompressedBytes(context);
@@ -131,8 +130,8 @@ public class OffHeapRegionEntryHelper {
     }
   }
   
-  public static int getSerializedLengthFromDataAsAddress(DataAsAddress dataAsAddress) {
-    final long ohAddress = dataAsAddress.getEncodedAddress();
+  public static int getSerializedLengthFromDataAsAddress(TinyStoredObject dataAsAddress) {
+    final long ohAddress = dataAsAddress.getAddress();
     
      if ((ohAddress & ENCODED_BIT) != 0) {     
       boolean isLong = (ohAddress & LONG_BIT) != 0;     
@@ -160,7 +159,7 @@ public class OffHeapRegionEntryHelper {
 
   private static void releaseAddress(@Released long ohAddress) {
     if (isOffHeap(ohAddress)) {
-      ObjectChunk.release(ohAddress);
+      OffHeapStoredObject.release(ohAddress);
     }
   }
   
@@ -184,7 +183,7 @@ public class OffHeapRegionEntryHelper {
     setValue(re, Token.REMOVED_PHASE2);
   }
 
-  public static void releaseEntry(@Unretained OffHeapRegionEntry re, @Released MemoryChunkWithRefCount expectedValue) {
+  public static void releaseEntry(@Unretained OffHeapRegionEntry re, @Released StoredObject expectedValue) {
     long oldAddress = objectToAddress(expectedValue);
     final long newAddress = objectToAddress(Token.REMOVED_PHASE2);
     if (re.setAddress(oldAddress, newAddress) || re.getAddress() != newAddress) {
@@ -273,6 +272,15 @@ public class OffHeapRegionEntryHelper {
       }
   }
 
+  static int decodeAddressToDataSize(long addr) {
+    assert (addr & ENCODED_BIT) != 0;
+    boolean isLong = (addr & LONG_BIT) != 0;
+    if (isLong) {
+      return 9;
+    }
+    return (int) ((addr & SIZE_MASK) >> SIZE_SHIFT);
+  }
+  
   static byte[] decodeAddressToBytes(long addr, boolean decompress, boolean compressedOk) {
     assert (addr & ENCODED_BIT) != 0;
     boolean isCompressed = (addr & COMPRESSED_BIT) != 0;
@@ -354,11 +362,11 @@ public class OffHeapRegionEntryHelper {
     int retryCount = 0;
     @Retained long addr = re.getAddress();
     while (isOffHeap(addr)) {
-      if (ObjectChunk.retain(addr)) {
+      if (OffHeapStoredObject.retain(addr)) {
         @Unretained long addr2 = re.getAddress();
         if (addr != addr2) {
           retryCount = 0;
-          ObjectChunk.release(addr);
+          OffHeapStoredObject.release(addr);
           // spin around and try again.
           addr = addr2;
         } else {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapStoredObject.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapStoredObject.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapStoredObject.java
new file mode 100644
index 0000000..68c9bdd
--- /dev/null
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapStoredObject.java
@@ -0,0 +1,718 @@
+/*
+ * 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.io.DataOutput;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.internal.DSCODE;
+import com.gemstone.gemfire.internal.HeapDataOutputStream;
+import com.gemstone.gemfire.internal.InternalDataSerializer;
+import com.gemstone.gemfire.internal.cache.BytesAndBitsForCompactor;
+import com.gemstone.gemfire.internal.cache.EntryBits;
+import com.gemstone.gemfire.internal.cache.EntryEventImpl;
+import com.gemstone.gemfire.internal.cache.RegionEntry;
+import com.gemstone.gemfire.internal.cache.RegionEntryContext;
+import com.gemstone.gemfire.internal.offheap.annotations.Unretained;
+
+/**
+   * A class that stores a Java object in off-heap memory.
+   * See {@link AddressableMemoryManager} for how off-heap memory
+   * can be allocated, accessed, modified, and freed.
+   * Currently the object stored in this class
+   * is always an entry value of a Region.
+   * Note: this class has a natural ordering that is inconsistent with equals.
+   * Instances of this class should have a short lifetime. We do not store references
+   * to it in the cache. Instead the memoryAddress is stored in a primitive field in
+   * the cache and if used it will then, if needed, create an instance of this class.
+   */
+  public class OffHeapStoredObject extends AbstractStoredObject implements Comparable<OffHeapStoredObject>, MemoryBlock {
+    /**
+     * The memory address of the first byte of addressable memory that belongs to this object
+     */
+    private final long memoryAddress;
+    
+    /**
+     * The useCount, chunkSize, dataSizeDelta, isSerialized, and isCompressed
+     * are all stored in addressable memory in a HEADER. This saves heap memory
+     * by using off heap.
+     */
+    public final static int HEADER_SIZE = 4 + 4;
+    /**
+     * We need to smallest chunk to at least have enough room for a hdr
+     * and for an off heap ref (which is a long).
+     */
+    public final static int MIN_CHUNK_SIZE = HEADER_SIZE + 8;
+    /**
+     * int field.
+     * The number of bytes in this chunk.
+     */
+    private final static int CHUNK_SIZE_OFFSET = 0;
+    /**
+     * Volatile int field
+     * The upper two bits are used for the isSerialized
+     * and isCompressed flags.
+     * The next three bits are unused.
+     * The lower 3 bits of the most significant byte contains a magic number to help us detect
+     * if we are changing the ref count of an object that has been released.
+     * The next byte contains the dataSizeDelta.
+     * The number of bytes of logical data in this chunk.
+     * Since the number of bytes of logical data is always <= chunkSize
+     * and since chunkSize never changes, we have dataSize be
+     * a delta whose max value would be HUGE_MULTIPLE-1.
+     * The lower two bytes contains the use count.
+     */
+    final static int REF_COUNT_OFFSET = 4;
+    /**
+     * The upper two bits are used for the isSerialized
+     * and isCompressed flags.
+     */
+    final static int IS_SERIALIZED_BIT =    0x80000000;
+    final static int IS_COMPRESSED_BIT =    0x40000000;
+    // UNUSED 0x38000000
+    final static int MAGIC_MASK = 0x07000000;
+    final static int MAGIC_NUMBER = 0x05000000;
+    final static int DATA_SIZE_DELTA_MASK = 0x00ff0000;
+    final static int DATA_SIZE_SHIFT = 16;
+    final static int REF_COUNT_MASK =       0x0000ffff;
+    final static int MAX_REF_COUNT = 0xFFFF;
+    final static long FILL_PATTERN = 0x3c3c3c3c3c3c3c3cL;
+    final static byte FILL_BYTE = 0x3c;
+    
+    protected OffHeapStoredObject(long memoryAddress, int chunkSize) {
+      SimpleMemoryAllocatorImpl.validateAddressAndSize(memoryAddress, chunkSize);
+      this.memoryAddress = memoryAddress;
+      setSize(chunkSize);
+      AddressableMemoryManager.writeIntVolatile(getAddress()+REF_COUNT_OFFSET, MAGIC_NUMBER);
+    }
+    public void readyForFree() {
+      AddressableMemoryManager.writeIntVolatile(getAddress()+REF_COUNT_OFFSET, 0);
+    }
+    public void readyForAllocation() {
+      if (!AddressableMemoryManager.writeIntVolatile(getAddress()+REF_COUNT_OFFSET, 0, MAGIC_NUMBER)) {
+        throw new IllegalStateException("Expected 0 but found " + Integer.toHexString(AddressableMemoryManager.readIntVolatile(getAddress()+REF_COUNT_OFFSET)));
+      }
+    }
+    /**
+     * Should only be used by FakeChunk subclass
+     */
+    protected OffHeapStoredObject() {
+      this.memoryAddress = 0L;
+    }
+    
+    /**
+     * Used to create a Chunk given an existing, already allocated,
+     * memoryAddress. The off heap header has already been initialized.
+     */
+    protected OffHeapStoredObject(long memoryAddress) {
+      SimpleMemoryAllocatorImpl.validateAddress(memoryAddress);
+      this.memoryAddress = memoryAddress;
+    }
+    
+    protected OffHeapStoredObject(OffHeapStoredObject chunk) {
+      this.memoryAddress = chunk.memoryAddress;
+    }
+    
+    @Override
+    public void fillSerializedValue(BytesAndBitsForCompactor wrapper, byte userBits) {
+      if (isSerialized()) {
+        userBits = EntryBits.setSerialized(userBits, true);
+      }
+      wrapper.setOffHeapData(this, userBits);
+    }
+    
+    String getShortClassName() {
+      String cname = getClass().getName();
+      return cname.substring(getClass().getPackage().getName().length()+1);
+    }
+
+    @Override
+    public boolean checkDataEquals(@Unretained StoredObject so) {
+      if (this == so) {
+        return true;
+      }
+      if (isSerialized() != so.isSerialized()) {
+        return false;
+      }
+      int mySize = getValueSizeInBytes();
+      if (mySize != so.getValueSizeInBytes()) {
+        return false;
+      }
+      if (!(so instanceof OffHeapStoredObject)) {
+        return false;
+      }
+      OffHeapStoredObject other = (OffHeapStoredObject) so;
+      if (getAddress() == other.getAddress()) {
+        return true;
+      }
+      // We want to be able to do this operation without copying any of the data into the heap.
+      // Hopefully the jvm is smart enough to use our stack for this short lived array.
+      final byte[] dataCache1 = new byte[1024];
+      final byte[] dataCache2 = new byte[dataCache1.length];
+      // TODO OFFHEAP: no need to copy to heap. Just get the address of each and compare each byte. No need to call incReads when reading from address.
+      int i;
+      // inc it twice since we are reading two different objects
+      SimpleMemoryAllocatorImpl.getAllocator().getStats().incReads();
+      SimpleMemoryAllocatorImpl.getAllocator().getStats().incReads();
+      for (i=0; i < mySize-(dataCache1.length-1); i+=dataCache1.length) {
+        this.readDataBytes(i, dataCache1);
+        other.readDataBytes(i, dataCache2);
+        for (int j=0; j < dataCache1.length; j++) {
+          if (dataCache1[j] != dataCache2[j]) {
+            return false;
+          }
+        }
+      }
+      int bytesToRead = mySize-i;
+      if (bytesToRead > 0) {
+        // need to do one more read which will be less than dataCache.length
+        this.readDataBytes(i, dataCache1, 0, bytesToRead);
+        other.readDataBytes(i, dataCache2, 0, bytesToRead);
+        for (int j=0; j < bytesToRead; j++) {
+          if (dataCache1[j] != dataCache2[j]) {
+            return false;
+          }
+        }
+      }
+      return true;
+    }
+    
+    @Override
+    public boolean checkDataEquals(byte[] serializedObj) {
+      // caller was responsible for checking isSerialized
+      int mySize = getValueSizeInBytes();
+      if (mySize != serializedObj.length) {
+        return false;
+      }
+      // We want to be able to do this operation without copying any of the data into the heap.
+      // Hopefully the jvm is smart enough to use our stack for this short lived array.
+      // TODO OFFHEAP: no need to copy to heap. Just get the address of each and compare each byte. No need to call incReads when reading from address.
+      final byte[] dataCache = new byte[1024];
+      int idx=0;
+      int i;
+      SimpleMemoryAllocatorImpl.getAllocator().getStats().incReads();
+      for (i=0; i < mySize-(dataCache.length-1); i+=dataCache.length) {
+        this.readDataBytes(i, dataCache);
+        for (int j=0; j < dataCache.length; j++) {
+          if (dataCache[j] != serializedObj[idx++]) {
+            return false;
+          }
+        }
+      }
+      int bytesToRead = mySize-i;
+      if (bytesToRead > 0) {
+        // need to do one more read which will be less than dataCache.length
+        this.readDataBytes(i, dataCache, 0, bytesToRead);
+        for (int j=0; j < bytesToRead; j++) {
+          if (dataCache[j] != serializedObj[idx++]) {
+            return false;
+          }
+        }
+      }
+      return true;
+    }
+
+    
+    /**
+     * Throw an exception if this chunk is not allocated
+     */
+    public void checkIsAllocated() {
+      int originalBits = AddressableMemoryManager.readIntVolatile(this.memoryAddress+REF_COUNT_OFFSET);
+      if ((originalBits&MAGIC_MASK) != MAGIC_NUMBER) {
+        throw new IllegalStateException("It looks like this off heap memory was already freed. rawBits=" + Integer.toHexString(originalBits));
+      }
+    }
+    
+    public void incSize(int inc) {
+      setSize(getSize()+inc);
+    }
+    
+    protected void beforeReturningToAllocator() {
+      
+    }
+
+    @Override
+    public int getSize() {
+      return getSize(this.memoryAddress);
+    }
+
+    public void setSize(int size) {
+      setSize(this.memoryAddress, size);
+    }
+
+    @Override
+    public long getAddress() {
+      return this.memoryAddress;
+    }
+    
+    @Override
+    public int getDataSize() {
+      return getDataSize(this.memoryAddress);
+    }
+    
+    protected static int getDataSize(long memoryAdress) {
+      int dataSizeDelta = AddressableMemoryManager.readInt(memoryAdress+REF_COUNT_OFFSET);
+      dataSizeDelta &= DATA_SIZE_DELTA_MASK;
+      dataSizeDelta >>= DATA_SIZE_SHIFT;
+      return getSize(memoryAdress) - dataSizeDelta;
+    }
+    
+    protected long getBaseDataAddress() {
+      return this.memoryAddress+HEADER_SIZE;
+    }
+    protected int getBaseDataOffset() {
+      return 0;
+    }
+    
+    @Override
+    @Unretained
+    public ByteBuffer createDirectByteBuffer() {
+      return AddressableMemoryManager.createDirectByteBuffer(getBaseDataAddress(), getDataSize());
+    }
+    @Override
+    public void sendTo(DataOutput out) throws IOException {
+      if (!this.isCompressed() && out instanceof HeapDataOutputStream) {
+        ByteBuffer bb = createDirectByteBuffer();
+        if (bb != null) {
+          HeapDataOutputStream hdos = (HeapDataOutputStream) out;
+          if (this.isSerialized()) {
+            hdos.write(bb);
+          } else {
+            hdos.writeByte(DSCODE.BYTE_ARRAY);
+            InternalDataSerializer.writeArrayLength(bb.remaining(), hdos);
+            hdos.write(bb);
+          }
+          return;
+        }
+      }
+      super.sendTo(out);
+    }
+    
+    @Override
+    public void sendAsByteArray(DataOutput out) throws IOException {
+      if (!isCompressed() && out instanceof HeapDataOutputStream) {
+        ByteBuffer bb = createDirectByteBuffer();
+        if (bb != null) {
+          HeapDataOutputStream hdos = (HeapDataOutputStream) out;
+          InternalDataSerializer.writeArrayLength(bb.remaining(), hdos);
+          hdos.write(bb);
+          return;
+        }
+      }
+      super.sendAsByteArray(out);
+    }
+       
+    /**
+     * Returns an address that can be used with AddressableMemoryManager to access this object's data.
+     * @param offset the offset from this chunk's first byte of the byte the returned address should point to. Must be >= 0.
+     * @param size the number of bytes that will be read using the returned address. Assertion will use this to verify that all the memory accessed belongs to this chunk. Must be > 0.
+     * @return a memory address that can be used to access this object's data
+     */
+    @Override
+    public long getAddressForReadingData(int offset, int size) {
+      assert offset >= 0 && offset + size <= getDataSize(): "Offset=" + offset + ",size=" + size + ",dataSize=" + getDataSize() + ", chunkSize=" + getSize() + ", but offset + size must be <= " + getDataSize();
+      assert size > 0;
+      long result = getBaseDataAddress() + offset;
+      // validateAddressAndSizeWithinSlab(result, size);
+      return result;
+    }
+    
+    @Override
+    public byte readDataByte(int offset) {
+      assert offset < getDataSize();
+      return AddressableMemoryManager.readByte(getBaseDataAddress() + offset);
+    }
+
+    @Override
+    public void writeDataByte(int offset, byte value) {
+      assert offset < getDataSize();
+      AddressableMemoryManager.writeByte(getBaseDataAddress() + offset, value);
+    }
+
+    @Override
+    public void readDataBytes(int offset, byte[] bytes) {
+      readDataBytes(offset, bytes, 0, bytes.length);
+    }
+
+    @Override
+    public void writeDataBytes(int offset, byte[] bytes) {
+      writeDataBytes(offset, bytes, 0, bytes.length);
+    }
+
+    @Override
+    public void readDataBytes(int offset, byte[] bytes, int bytesOffset, int size) {
+      assert offset+size <= getDataSize();
+      AddressableMemoryManager.readBytes(getBaseDataAddress() + offset, bytes, bytesOffset, size);
+    }
+
+    @Override
+    public void writeDataBytes(int offset, byte[] bytes, int bytesOffset, int size) {
+      assert offset+size <= getDataSize();
+      AddressableMemoryManager.writeBytes(getBaseDataAddress() + offset, bytes, bytesOffset, size);
+    }
+    
+    @Override
+    public void release() {
+      release(this.memoryAddress);
+     }
+
+    @Override
+    public int compareTo(OffHeapStoredObject o) {
+      int result = Integer.signum(getSize() - o.getSize());
+      if (result == 0) {
+        // For the same sized chunks we really don't care about their order
+        // but we need compareTo to only return 0 if the two chunks are identical
+        result = Long.signum(getAddress() - o.getAddress());
+      }
+      return result;
+    }
+    
+    @Override
+    public boolean equals(Object o) {
+      if (o instanceof OffHeapStoredObject) {
+        return getAddress() == ((OffHeapStoredObject) o).getAddress();
+      }
+      return false;
+    }
+    
+    @Override
+    public int hashCode() {
+      long value = this.getAddress();
+      return (int)(value ^ (value >>> 32));
+    }
+
+    public void setSerializedValue(byte[] value) {
+      writeDataBytes(0, value);
+    }
+    
+    public byte[] getDecompressedBytes(RegionEntryContext context) {
+      byte[] result = getCompressedBytes();
+      long time = context.getCachePerfStats().startDecompression();
+      result = context.getCompressor().decompress(result);
+      context.getCachePerfStats().endDecompression(time);      
+      return result;
+    }
+    
+    /**
+     * Returns the raw possibly compressed bytes of this chunk
+     */
+    public byte[] getCompressedBytes() {
+      byte[] result = new byte[getDataSize()];
+      readDataBytes(0, result);
+      //debugLog("reading", true);
+      SimpleMemoryAllocatorImpl.getAllocator().getStats().incReads();
+      return result;
+    }
+    protected byte[] getRawBytes() {
+      byte[] result = getCompressedBytes();
+      // TODO OFFHEAP: change the following to assert !isCompressed();
+      if (isCompressed()) {
+        throw new UnsupportedOperationException();
+      }
+      return result;
+    }
+
+    @Override
+    public byte[] getSerializedValue() {
+      byte [] result = getRawBytes();
+      if (!isSerialized()) {
+        // The object is a byte[]. So we need to make it look like a serialized byte[] in our result
+        result = EntryEventImpl.serialize(result);
+      }
+      return result;
+    }
+    
+    @Override
+    public Object getDeserializedValue(Region r, RegionEntry re) {
+      if (isSerialized()) {
+        // TODO OFFHEAP: debug deserializeChunk
+        return EntryEventImpl.deserialize(getRawBytes());
+        //assert !isCompressed();
+        //return EntryEventImpl.deserializeChunk(this);
+      } else {
+        return getRawBytes();
+      }
+    }
+    
+    /**
+     * We want this to include memory overhead so use getSize() instead of getDataSize().
+     */
+    @Override
+    public int getSizeInBytes() {
+      // Calling getSize includes the off heap header size.
+      // We do not add anything to this since the size of the reference belongs to the region entry size
+      // not the size of this object.
+      return getSize();
+    }
+
+    @Override
+    public int getValueSizeInBytes() {
+      return getDataSize();
+    }
+
+    @Override
+    public boolean isSerialized() {
+      return (AddressableMemoryManager.readInt(this.memoryAddress+REF_COUNT_OFFSET) & IS_SERIALIZED_BIT) != 0;
+    }
+
+    @Override
+    public boolean isCompressed() {
+      return (AddressableMemoryManager.readInt(this.memoryAddress+REF_COUNT_OFFSET) & IS_COMPRESSED_BIT) != 0;
+    }
+
+    @Override
+    public boolean retain() {
+      return retain(this.memoryAddress);
+    }
+
+    @Override
+    public int getRefCount() {
+      return getRefCount(this.memoryAddress);
+    }
+
+    public static int getSize(long memAddr) {
+      SimpleMemoryAllocatorImpl.validateAddress(memAddr);
+      return AddressableMemoryManager.readInt(memAddr+CHUNK_SIZE_OFFSET);
+    }
+    public static void setSize(long memAddr, int size) {
+      SimpleMemoryAllocatorImpl.validateAddressAndSize(memAddr, size);
+      AddressableMemoryManager.writeInt(memAddr+CHUNK_SIZE_OFFSET, size);
+    }
+    public static long getNext(long memAddr) {
+      SimpleMemoryAllocatorImpl.validateAddress(memAddr);
+      return AddressableMemoryManager.readLong(memAddr+HEADER_SIZE);
+    }
+    public static void setNext(long memAddr, long next) {
+      SimpleMemoryAllocatorImpl.validateAddress(memAddr);
+      AddressableMemoryManager.writeLong(memAddr+HEADER_SIZE, next);
+    }
+    
+    /**
+     * Fills the chunk with a repeated byte fill pattern.
+     * @param baseAddress the starting address for a {@link OffHeapStoredObject}.
+     */
+    public static void fill(long baseAddress) {
+      long startAddress = baseAddress + MIN_CHUNK_SIZE;
+      int size = getSize(baseAddress) - MIN_CHUNK_SIZE;
+      
+      AddressableMemoryManager.fill(startAddress, size, FILL_BYTE);
+    }
+    
+    /**
+     * Validates that the fill pattern for this chunk has not been disturbed.  This method
+     * assumes the TINY_MULTIPLE is 8 bytes.
+     * @throws IllegalStateException when the pattern has been violated.
+     */
+    public void validateFill() {
+      assert FreeListManager.TINY_MULTIPLE == 8;
+      
+      long startAddress = getAddress() + MIN_CHUNK_SIZE;
+      int size = getSize() - MIN_CHUNK_SIZE;
+      
+      for(int i = 0;i < size;i += FreeListManager.TINY_MULTIPLE) {
+        if(AddressableMemoryManager.readLong(startAddress + i) != FILL_PATTERN) {
+          throw new IllegalStateException("Fill pattern violated for chunk " + getAddress() + " with size " + getSize());
+        }        
+      }
+    }
+
+    public void setSerialized(boolean isSerialized) {
+      if (isSerialized) {
+        int bits;
+        int originalBits;
+        do {
+          originalBits = AddressableMemoryManager.readIntVolatile(this.memoryAddress+REF_COUNT_OFFSET);
+          if ((originalBits&MAGIC_MASK) != MAGIC_NUMBER) {
+            throw new IllegalStateException("It looks like this off heap memory was already freed. rawBits=" + Integer.toHexString(originalBits));
+          }
+          bits = originalBits | IS_SERIALIZED_BIT;
+        } while (!AddressableMemoryManager.writeIntVolatile(this.memoryAddress+REF_COUNT_OFFSET, originalBits, bits));
+      }
+    }
+    public void setCompressed(boolean isCompressed) {
+      if (isCompressed) {
+        int bits;
+        int originalBits;
+        do {
+          originalBits = AddressableMemoryManager.readIntVolatile(this.memoryAddress+REF_COUNT_OFFSET);
+          if ((originalBits&MAGIC_MASK) != MAGIC_NUMBER) {
+            throw new IllegalStateException("It looks like this off heap memory was already freed. rawBits=" + Integer.toHexString(originalBits));
+          }
+          bits = originalBits | IS_COMPRESSED_BIT;
+        } while (!AddressableMemoryManager.writeIntVolatile(this.memoryAddress+REF_COUNT_OFFSET, originalBits, bits));
+      }
+    }
+    public void setDataSize(int dataSize) { // KIRK
+      assert dataSize <= getSize();
+      int delta = getSize() - dataSize;
+      assert delta <= (DATA_SIZE_DELTA_MASK >> DATA_SIZE_SHIFT);
+      delta <<= DATA_SIZE_SHIFT;
+      int bits;
+      int originalBits;
+      do {
+        originalBits = AddressableMemoryManager.readIntVolatile(this.memoryAddress+REF_COUNT_OFFSET);
+        if ((originalBits&MAGIC_MASK) != MAGIC_NUMBER) {
+          throw new IllegalStateException("It looks like this off heap memory was already freed. rawBits=" + Integer.toHexString(originalBits));
+        }
+        bits = originalBits;
+        bits &= ~DATA_SIZE_DELTA_MASK; // clear the old dataSizeDelta bits
+        bits |= delta; // set the dataSizeDelta bits to the new delta value
+      } while (!AddressableMemoryManager.writeIntVolatile(this.memoryAddress+REF_COUNT_OFFSET, originalBits, bits));
+    }
+    
+    public void initializeUseCount() {
+      int rawBits;
+      do {
+        rawBits = AddressableMemoryManager.readIntVolatile(this.memoryAddress+REF_COUNT_OFFSET);
+        if ((rawBits&MAGIC_MASK) != MAGIC_NUMBER) {
+          throw new IllegalStateException("It looks like this off heap memory was already freed. rawBits=" + Integer.toHexString(rawBits));
+        }
+        int uc = rawBits & REF_COUNT_MASK;
+        if (uc != 0) {
+          throw new IllegalStateException("Expected use count to be zero but it was: " + uc + " rawBits=0x" + Integer.toHexString(rawBits));
+        }
+      } while (!AddressableMemoryManager.writeIntVolatile(this.memoryAddress+REF_COUNT_OFFSET, rawBits, rawBits+1));
+    }
+
+    public static int getRefCount(long memAddr) {
+      return AddressableMemoryManager.readInt(memAddr+REF_COUNT_OFFSET) & REF_COUNT_MASK;
+    }
+
+    public static boolean retain(long memAddr) {
+      SimpleMemoryAllocatorImpl.validateAddress(memAddr);
+      int uc;
+      int rawBits;
+      int retryCount = 0;
+      do {
+        rawBits = AddressableMemoryManager.readIntVolatile(memAddr+REF_COUNT_OFFSET);
+        if ((rawBits&MAGIC_MASK) != MAGIC_NUMBER) {
+          // same as uc == 0
+          // TODO MAGIC_NUMBER rethink its use and interaction with compactor fragments
+          return false;
+        }
+        uc = rawBits & REF_COUNT_MASK;
+        if (uc == MAX_REF_COUNT) {
+          throw new IllegalStateException("Maximum use count exceeded. rawBits=" + Integer.toHexString(rawBits));
+        } else if (uc == 0) {
+          return false;
+        }
+        retryCount++;
+        if (retryCount > 1000) {
+          throw new IllegalStateException("tried to write " + (rawBits+1) + " to @" + Long.toHexString(memAddr) + " 1,000 times.");
+        }
+      } while (!AddressableMemoryManager.writeIntVolatile(memAddr+REF_COUNT_OFFSET, rawBits, rawBits+1));
+      //debugLog("use inced ref count " + (uc+1) + " @" + Long.toHexString(memAddr), true);
+      if (ReferenceCountHelper.trackReferenceCounts()) {
+        ReferenceCountHelper.refCountChanged(memAddr, false, uc+1);
+      }
+
+      return true;
+    }
+    public static void release(final long memAddr) {
+      release(memAddr, null);
+    }
+    static void release(final long memAddr, FreeListManager freeListManager) {
+      SimpleMemoryAllocatorImpl.validateAddress(memAddr);
+      int newCount;
+      int rawBits;
+      boolean returnToAllocator;
+      do {
+        returnToAllocator = false;
+        rawBits = AddressableMemoryManager.readIntVolatile(memAddr+REF_COUNT_OFFSET);
+        if ((rawBits&MAGIC_MASK) != MAGIC_NUMBER) {
+          String msg = "It looks like off heap memory @" + Long.toHexString(memAddr) + " was already freed. rawBits=" + Integer.toHexString(rawBits) + " history=" + ReferenceCountHelper.getFreeRefCountInfo(memAddr);
+          //debugLog(msg, true);
+          throw new IllegalStateException(msg);
+        }
+        int curCount = rawBits&REF_COUNT_MASK;
+        if ((curCount) == 0) {
+          //debugLog("too many frees @" + Long.toHexString(memAddr), true);
+          throw new IllegalStateException("Memory has already been freed." + " history=" + ReferenceCountHelper.getFreeRefCountInfo(memAddr) /*+ System.identityHashCode(this)*/);
+        }
+        if (curCount == 1) {
+          newCount = 0; // clear the use count, bits, and the delta size since it will be freed.
+          returnToAllocator = true;
+        } else {
+          newCount = rawBits-1;
+        }
+      } while (!AddressableMemoryManager.writeIntVolatile(memAddr+REF_COUNT_OFFSET, rawBits, newCount));
+      //debugLog("free deced ref count " + (newCount&USE_COUNT_MASK) + " @" + Long.toHexString(memAddr), true);
+      if (returnToAllocator ) {
+       if (ReferenceCountHelper.trackReferenceCounts()) {
+          if (ReferenceCountHelper.trackFreedReferenceCounts()) {
+            ReferenceCountHelper.refCountChanged(memAddr, true, newCount&REF_COUNT_MASK);
+          }
+          ReferenceCountHelper.freeRefCountInfo(memAddr);
+        }
+        if (freeListManager == null) {
+          freeListManager = SimpleMemoryAllocatorImpl.getAllocator().getFreeListManager();
+        }
+        freeListManager.free(memAddr);
+      } else {
+        if (ReferenceCountHelper.trackReferenceCounts()) {
+          ReferenceCountHelper.refCountChanged(memAddr, true, newCount&REF_COUNT_MASK);
+        }
+      }
+    }
+    
+    @Override
+    public String toString() {
+      return super.toString() + ":<dataSize=" + getDataSize() + " refCount=" + getRefCount() + " addr=" + Long.toHexString(getAddress()) + ">";
+    }
+    
+    @Override
+    public State getState() {
+      if (getRefCount() > 0) {
+        return State.ALLOCATED;
+      } else {
+        return State.DEALLOCATED;
+      }
+    }
+    @Override
+    public MemoryBlock getNextBlock() {
+      throw new UnsupportedOperationException();
+    }
+    @Override
+    public int getBlockSize() {
+      return getSize();
+    }
+    @Override
+    public int getSlabId() {
+      throw new UnsupportedOperationException();
+    }
+    @Override
+    public int getFreeListId() {
+      return -1;
+    }
+    @Override
+    public String getDataType() {
+      return null;
+    }
+    @Override
+    public Object getDataValue() {
+      return null;
+    }
+    public StoredObject slice(int position, int limit) {
+      return new OffHeapStoredObjectSlice(this, position, limit);
+    }
+    @Override
+    public boolean hasRefCount() {
+      return true;
+    }
+  }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapStoredObjectAddressStack.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapStoredObjectAddressStack.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapStoredObjectAddressStack.java
new file mode 100644
index 0000000..bde30e2
--- /dev/null
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapStoredObjectAddressStack.java
@@ -0,0 +1,141 @@
+/*
+ * 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.LogWriter;
+
+/**
+ * A "stack" of addresses of OffHeapStoredObject instances. The stored objects are not kept
+ * in java object form but instead each one is just an off-heap address.
+ * This class is used for each "tiny" free-list of the FreeListManager.
+ * This class is thread safe.
+ */
+public class OffHeapStoredObjectAddressStack {
+  // Ok to read without sync but must be synced on write
+  private volatile long topAddr;
+  
+  public OffHeapStoredObjectAddressStack(long addr) {
+    if (addr != 0L) SimpleMemoryAllocatorImpl.validateAddress(addr);
+    this.topAddr = addr;
+  }
+  public OffHeapStoredObjectAddressStack() {
+    this.topAddr = 0L;
+  }
+  public boolean isEmpty() {
+    return this.topAddr == 0L;
+  }
+  public void offer(long e) {
+    assert e != 0;
+    SimpleMemoryAllocatorImpl.validateAddress(e);
+    synchronized (this) {
+      OffHeapStoredObject.setNext(e, this.topAddr);
+      this.topAddr = e;
+    }
+  }
+  public long poll() {
+    long result;
+    synchronized (this) {
+      result = this.topAddr;
+      if (result != 0L) {
+        this.topAddr = OffHeapStoredObject.getNext(result);
+      }
+    }
+    return result;
+  }
+  /**
+   * Returns the address of the "top" item in this stack.
+   */
+  public long getTopAddress() {
+    return this.topAddr;
+  }
+  /**
+   * Removes all the addresses from this stack
+   * and returns the top address.
+   * The caller owns all the addresses after this call.
+   */
+  public long clear() {
+    long result;
+    synchronized (this) {
+      result = this.topAddr;
+      if (result != 0L) {
+        this.topAddr = 0L;
+      }
+    }
+    return result;
+  }
+  public void logSizes(LogWriter lw, String msg) {
+    long headAddr = this.topAddr;
+    long addr;
+    boolean concurrentModDetected;
+    do {
+      concurrentModDetected = false;
+      addr = headAddr;
+      while (addr != 0L) {
+        int curSize = OffHeapStoredObject.getSize(addr);
+        addr = OffHeapStoredObject.getNext(addr);
+        testHookDoConcurrentModification();
+        long curHead = this.topAddr;
+        if (curHead != headAddr) {
+          headAddr = curHead;
+          concurrentModDetected = true;
+          // Someone added or removed from the stack.
+          // So we break out of the inner loop and start
+          // again at the new head.
+          break;
+        }
+        // TODO construct a single log msg
+        // that gets reset when concurrentModDetected.
+        lw.info(msg + curSize);
+      }
+    } while (concurrentModDetected);
+  }
+  public long computeTotalSize() {
+    long result;
+    long headAddr = this.topAddr;
+    long addr;
+    boolean concurrentModDetected;
+    do {
+      concurrentModDetected = false;
+      result = 0;
+      addr = headAddr;
+      while (addr != 0L) {
+        result += OffHeapStoredObject.getSize(addr);
+        addr = OffHeapStoredObject.getNext(addr);
+        testHookDoConcurrentModification();
+        long curHead = this.topAddr;
+        if (curHead != headAddr) {
+          headAddr = curHead;
+          concurrentModDetected = true;
+          // Someone added or removed from the stack.
+          // So we break out of the inner loop and start
+          // again at the new head.
+          break;
+        }
+      }
+    } while (concurrentModDetected);
+    return result;
+  }
+  
+  /**
+   * This method allows tests to override it
+   * and do a concurrent modification to the stack.
+   * For production code it will be a noop.
+   */
+  protected void testHookDoConcurrentModification() {
+    // nothing needed in production code
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapStoredObjectSlice.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapStoredObjectSlice.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapStoredObjectSlice.java
new file mode 100644
index 0000000..4e4d3e4
--- /dev/null
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapStoredObjectSlice.java
@@ -0,0 +1,44 @@
+/*
+ * 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;
+
+/**
+ * Represents a slice of an OffHeapStoredObject.
+ * A slice is a subsequence of the bytes stored in an OffHeapStoredObject.
+ */
+public class OffHeapStoredObjectSlice extends OffHeapStoredObject {
+  private final int offset;
+  private final int capacity;
+  public OffHeapStoredObjectSlice(OffHeapStoredObject objectChunk, int position, int limit) {
+    super(objectChunk);
+    this.offset = objectChunk.getBaseDataOffset() + position;
+    this.capacity = limit - position;
+  }
+  @Override
+  public int getDataSize() {
+    return this.capacity;
+  }
+  
+  @Override
+  protected long getBaseDataAddress() {
+    return super.getBaseDataAddress() + this.offset;
+  }
+  @Override
+  protected int getBaseDataOffset() {
+    return this.offset;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3087c86f/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapStoredObjectWithHeapForm.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapStoredObjectWithHeapForm.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapStoredObjectWithHeapForm.java
new file mode 100644
index 0000000..aea2319
--- /dev/null
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapStoredObjectWithHeapForm.java
@@ -0,0 +1,41 @@
+/*
+ * 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;
+
+/**
+ * Used to keep the heapForm around while an operation is still in progress.
+ * This allows the operation to access the serialized heap form instead of copying
+ * it from offheap. See bug 48135.
+ */
+public class OffHeapStoredObjectWithHeapForm extends OffHeapStoredObject {
+  private final byte[] heapForm;
+  
+  public OffHeapStoredObjectWithHeapForm(OffHeapStoredObject chunk, byte[] heapForm) {
+    super(chunk);
+    this.heapForm = heapForm;
+  }
+
+  @Override
+  protected byte[] getRawBytes() {
+    return this.heapForm;
+  }
+  
+  @Override
+  public StoredObject getStoredObjectWithoutHeapForm() {
+    return new OffHeapStoredObject(this);
+  }
+}
\ No newline at end of file