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

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

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();
-}