You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by ds...@apache.org on 2016/03/05 01:51:07 UTC

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

Added Slab, SlabFactory, SlabImpl, and AddressableMemoryManager.


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

Branch: refs/heads/feature/GEODE-982
Commit: e2a126d4aab21e2e3e2d9d1df6dac068914d080a
Parents: 4e127b8
Author: Darrel Schneider <ds...@pivotal.io>
Authored: Fri Feb 26 14:58:33 2016 -0800
Committer: Darrel Schneider <ds...@pivotal.io>
Committed: Fri Feb 26 14:58:33 2016 -0800

----------------------------------------------------------------------
 .../gemfire/internal/cache/DiskEntry.java       |   8 +-
 .../internal/cache/tier/sockets/Part.java       |   8 +-
 .../offheap/AddressableMemoryChunk.java         |  29 --
 .../offheap/AddressableMemoryChunkFactory.java  |  27 --
 .../offheap/AddressableMemoryManager.java       | 154 ++++++++++
 .../gemfire/internal/offheap/DataAsAddress.java |   5 +
 .../gemfire/internal/offheap/Fragment.java      |   2 +-
 .../internal/offheap/FreeListManager.java       |  12 +-
 .../internal/offheap/MemoryAllocator.java       |   2 +-
 .../gemfire/internal/offheap/MemoryChunk.java   |  47 ---
 .../offheap/MemoryChunkWithRefCount.java        |  29 +-
 .../gemfire/internal/offheap/ObjectChunk.java   |  66 ++---
 .../offheap/SimpleMemoryAllocatorImpl.java      |  26 +-
 .../gemstone/gemfire/internal/offheap/Slab.java |  39 +++
 .../gemfire/internal/offheap/SlabFactory.java   |  27 ++
 .../gemfire/internal/offheap/SlabImpl.java      |  61 ++++
 .../gemfire/internal/offheap/StoredObject.java  |   4 +
 .../internal/offheap/UnsafeMemoryChunk.java     | 217 --------------
 .../internal/tcp/ByteBufferInputStream.java     |  42 +--
 .../gemfire/cache30/MultiVMRegionTestCase.java  |  14 +-
 .../cache/ChunkValueWrapperJUnitTest.java       |   4 +-
 .../cache/OldValueImporterTestBase.java         |  10 +-
 .../internal/offheap/FragmentJUnitTest.java     |  12 +-
 .../offheap/LifecycleListenerJUnitTest.java     |  24 +-
 .../offheap/MemoryChunkJUnitTestBase.java       | 290 -------------------
 .../internal/offheap/ObjectChunkJUnitTest.java  |   8 +-
 .../offheap/OffHeapHelperJUnitTest.java         |   2 +-
 .../internal/offheap/OffHeapRegionBase.java     |   6 +-
 .../OffHeapWriteObjectAsByteArrayJUnitTest.java |   4 +-
 ...moryAllocatorFillPatternIntegrationTest.java |   6 +-
 ...mpleMemoryAllocatorFillPatternJUnitTest.java |   6 +-
 .../offheap/SimpleMemoryAllocatorJUnitTest.java |  96 +++---
 .../offheap/SyncChunkStackJUnitTest.java        |  36 +--
 .../offheap/UnsafeMemoryChunkJUnitTest.java     |  87 ------
 .../gemfire/pdx/OffHeapByteSourceJUnitTest.java |   4 +-
 35 files changed, 510 insertions(+), 904 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/e2a126d4/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..5d4af90 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
@@ -40,11 +40,11 @@ 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.AddressableMemoryManager;
 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.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;
@@ -730,7 +730,7 @@ public interface DiskEntry extends RegionEntry {
           long addrToRead = this.chunk.getAddressForReading(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 +739,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);
           final long endAddr = addr + maxOffset;
           while (addr != endAddr) {
-            bb.put(UnsafeMemoryChunk.readAbsoluteByte(addr));
+            bb.put(AddressableMemoryManager.readByte(addr));
             addr++;
             if (!bb.hasRemaining()) {
               flushable.flush();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/e2a126d4/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..36c7cdc 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
@@ -19,9 +19,9 @@ 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.AddressableMemoryManager;
 import com.gemstone.gemfire.internal.offheap.DataAsAddress;
 import com.gemstone.gemfire.internal.offheap.StoredObject;
-import com.gemstone.gemfire.internal.offheap.UnsafeMemoryChunk;
 
 import java.io.*;
 import java.nio.*;
@@ -301,7 +301,7 @@ public class Part {
             if (buf.remaining() == 0) {
               HeapDataOutputStream.flushStream(out,  buf);
             }
-            buf.put(UnsafeMemoryChunk.readAbsoluteByte(addr));
+            buf.put(AddressableMemoryManager.readByte(addr));
             addr++;
             bytesToSend--;
           }
@@ -331,7 +331,7 @@ public class Part {
           int bytesToSend = c.getDataSize();
           long addr = c.getAddressForReading(0, bytesToSend);
           while (bytesToSend > 0) {
-            buf.put(UnsafeMemoryChunk.readAbsoluteByte(addr));
+            buf.put(AddressableMemoryManager.readByte(addr));
             addr++;
             bytesToSend--;
           }
@@ -392,7 +392,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/e2a126d4/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/e2a126d4/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/e2a126d4/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..0f2d007
--- /dev/null
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/AddressableMemoryManager.java
@@ -0,0 +1,154 @@
+/*
+ * 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;
+
+/**
+ * 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);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/e2a126d4/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
index 96957ac..c0797d2 100644
--- 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
@@ -128,4 +128,9 @@ public class DataAsAddress extends AbstractStoredObject {
   public void release() {
     // nothing needed
   }
+
+  @Override
+  public int getRefCount() {
+    return 1;
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/e2a126d4/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..351bb8c 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
@@ -115,7 +115,7 @@ 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

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/e2a126d4/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 a716f14..af53607 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,7 +40,7 @@ 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);
@@ -58,7 +58,7 @@ public class FreeListManager {
     }
     return result;
   }
-  private void getLiveChunks(AddressableMemoryChunk slab, List<ObjectChunk> result) {
+  private void getLiveChunks(Slab slab, List<ObjectChunk> result) {
     long addr = slab.getMemoryAddress();
     while (addr <= (slab.getMemoryAddress() + slab.getSize() - ObjectChunk.MIN_CHUNK_SIZE)) {
       Fragment f = isAddrInFragmentFreeSpace(addr);
@@ -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;
@@ -864,7 +864,7 @@ public class FreeListManager {
   
   void freeSlabs() {
     for (int i=0; i < slabs.length; i++) {
-      slabs[i].release();
+      slabs[i].free();
     }
   }
   /**
@@ -876,7 +876,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;
   }
   
@@ -885,7 +885,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/e2a126d4/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..cd710fa 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,7 +29,7 @@ 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

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/e2a126d4/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/e2a126d4/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
index e3ba6ab..f532a66 100644
--- 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
@@ -17,18 +17,35 @@
 package com.gemstone.gemfire.internal.offheap;
 
 /**
- * Adds a reference count to the basic MemoryChunk.
- * Also an Object can be stored in one of these.
+ * 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.
+ * A reference count is used to determine if the chunk is still allocated.
  * 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 {
-
+public interface MemoryChunkWithRefCount extends StoredObject {
+  
+  /**
+   * 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);
+  
   /**
-   * Returns the number of users of this memory.
+   * 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 int getRefCount();
+  public void copyBytes(int src, int dst, int size);
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/e2a126d4/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
index 878e87d..7e96763 100644
--- 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
@@ -100,14 +100,14 @@ import com.gemstone.gemfire.internal.offheap.annotations.Unretained;
       SimpleMemoryAllocatorImpl.validateAddressAndSize(memoryAddress, chunkSize);
       this.memoryAddress = memoryAddress;
       setSize(chunkSize);
-      UnsafeMemoryChunk.writeAbsoluteIntVolatile(getMemoryAddress()+REF_COUNT_OFFSET, MAGIC_NUMBER);
+      AddressableMemoryManager.writeIntVolatile(getMemoryAddress()+REF_COUNT_OFFSET, MAGIC_NUMBER);
     }
     public void readyForFree() {
-      UnsafeMemoryChunk.writeAbsoluteIntVolatile(getMemoryAddress()+REF_COUNT_OFFSET, 0);
+      AddressableMemoryManager.writeIntVolatile(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)));
+      if (!AddressableMemoryManager.writeIntVolatile(getMemoryAddress()+REF_COUNT_OFFSET, 0, MAGIC_NUMBER)) {
+        throw new IllegalStateException("Expected 0 but found " + Integer.toHexString(AddressableMemoryManager.readIntVolatile(getMemoryAddress()+REF_COUNT_OFFSET)));
       }
     }
     /**
@@ -230,7 +230,7 @@ import com.gemstone.gemfire.internal.offheap.annotations.Unretained;
      * Throw an exception if this chunk is not allocated
      */
     public void checkIsAllocated() {
-      int originalBits = UnsafeMemoryChunk.readAbsoluteIntVolatile(this.memoryAddress+REF_COUNT_OFFSET);
+      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));
       }
@@ -258,15 +258,11 @@ import com.gemstone.gemfire.internal.offheap.annotations.Unretained;
     }
     
     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);
+      int dataSizeDelta = AddressableMemoryManager.readInt(memoryAdress+REF_COUNT_OFFSET);
       dataSizeDelta &= DATA_SIZE_DELTA_MASK;
       dataSizeDelta >>= DATA_SIZE_SHIFT;
       return getSize(memoryAdress) - dataSizeDelta;
@@ -434,13 +430,13 @@ import com.gemstone.gemfire.internal.offheap.annotations.Unretained;
     @Override
     public byte readByte(int offset) {
       assert offset < getDataSize();
-      return UnsafeMemoryChunk.readAbsoluteByte(getBaseDataAddress() + offset);
+      return AddressableMemoryManager.readByte(getBaseDataAddress() + offset);
     }
 
     @Override
     public void writeByte(int offset, byte value) {
       assert offset < getDataSize();
-      UnsafeMemoryChunk.writeAbsoluteByte(getBaseDataAddress() + offset, value);
+      AddressableMemoryManager.writeByte(getBaseDataAddress() + offset, value);
     }
 
     @Override
@@ -461,13 +457,13 @@ import com.gemstone.gemfire.internal.offheap.annotations.Unretained;
     @Override
     public void readBytes(int offset, byte[] bytes, int bytesOffset, int size) {
       assert offset+size <= getDataSize();
-      UnsafeMemoryChunk.readAbsoluteBytes(getBaseDataAddress() + offset, bytes, bytesOffset, size);
+      AddressableMemoryManager.readBytes(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);
+      AddressableMemoryManager.writeBytes(getBaseDataAddress() + offset, bytes, bytesOffset, size);
     }
     
     @Override
@@ -579,12 +575,12 @@ import com.gemstone.gemfire.internal.offheap.annotations.Unretained;
 
     @Override
     public boolean isSerialized() {
-      return (UnsafeMemoryChunk.readAbsoluteInt(this.memoryAddress+REF_COUNT_OFFSET) & IS_SERIALIZED_BIT) != 0;
+      return (AddressableMemoryManager.readInt(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;
+      return (AddressableMemoryManager.readInt(this.memoryAddress+REF_COUNT_OFFSET) & IS_COMPRESSED_BIT) != 0;
     }
 
     @Override
@@ -599,19 +595,19 @@ import com.gemstone.gemfire.internal.offheap.annotations.Unretained;
 
     public static int getSize(long memAddr) {
       SimpleMemoryAllocatorImpl.validateAddress(memAddr);
-      return UnsafeMemoryChunk.readAbsoluteInt(memAddr+CHUNK_SIZE_OFFSET);
+      return AddressableMemoryManager.readInt(memAddr+CHUNK_SIZE_OFFSET);
     }
     public static void setSize(long memAddr, int size) {
       SimpleMemoryAllocatorImpl.validateAddressAndSize(memAddr, size);
-      UnsafeMemoryChunk.writeAbsoluteInt(memAddr+CHUNK_SIZE_OFFSET, size);
+      AddressableMemoryManager.writeInt(memAddr+CHUNK_SIZE_OFFSET, size);
     }
     public static long getNext(long memAddr) {
       SimpleMemoryAllocatorImpl.validateAddress(memAddr);
-      return UnsafeMemoryChunk.readAbsoluteLong(memAddr+OFF_HEAP_HEADER_SIZE);
+      return AddressableMemoryManager.readLong(memAddr+OFF_HEAP_HEADER_SIZE);
     }
     public static void setNext(long memAddr, long next) {
       SimpleMemoryAllocatorImpl.validateAddress(memAddr);
-      UnsafeMemoryChunk.writeAbsoluteLong(memAddr+OFF_HEAP_HEADER_SIZE, next);
+      AddressableMemoryManager.writeLong(memAddr+OFF_HEAP_HEADER_SIZE, next);
     }
     
     /**
@@ -622,7 +618,7 @@ import com.gemstone.gemfire.internal.offheap.annotations.Unretained;
       long startAddress = baseAddress + MIN_CHUNK_SIZE;
       int size = getSize(baseAddress) - MIN_CHUNK_SIZE;
       
-      UnsafeMemoryChunk.fill(startAddress, size, FILL_BYTE);
+      AddressableMemoryManager.fill(startAddress, size, FILL_BYTE);
     }
     
     /**
@@ -637,7 +633,7 @@ import com.gemstone.gemfire.internal.offheap.annotations.Unretained;
       int size = getSize() - MIN_CHUNK_SIZE;
       
       for(int i = 0;i < size;i += FreeListManager.TINY_MULTIPLE) {
-        if(UnsafeMemoryChunk.readAbsoluteLong(startAddress + i) != FILL_PATTERN) {
+        if(AddressableMemoryManager.readLong(startAddress + i) != FILL_PATTERN) {
           throw new IllegalStateException("Fill pattern violated for chunk " + getMemoryAddress() + " with size " + getSize());
         }        
       }
@@ -648,12 +644,12 @@ import com.gemstone.gemfire.internal.offheap.annotations.Unretained;
         int bits;
         int originalBits;
         do {
-          originalBits = UnsafeMemoryChunk.readAbsoluteIntVolatile(this.memoryAddress+REF_COUNT_OFFSET);
+          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 (!UnsafeMemoryChunk.writeAbsoluteIntVolatile(this.memoryAddress+REF_COUNT_OFFSET, originalBits, bits));
+        } while (!AddressableMemoryManager.writeIntVolatile(this.memoryAddress+REF_COUNT_OFFSET, originalBits, bits));
       }
     }
     public void setCompressed(boolean isCompressed) {
@@ -661,12 +657,12 @@ import com.gemstone.gemfire.internal.offheap.annotations.Unretained;
         int bits;
         int originalBits;
         do {
-          originalBits = UnsafeMemoryChunk.readAbsoluteIntVolatile(this.memoryAddress+REF_COUNT_OFFSET);
+          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 (!UnsafeMemoryChunk.writeAbsoluteIntVolatile(this.memoryAddress+REF_COUNT_OFFSET, originalBits, bits));
+        } while (!AddressableMemoryManager.writeIntVolatile(this.memoryAddress+REF_COUNT_OFFSET, originalBits, bits));
       }
     }
     public void setDataSize(int dataSize) { // KIRK
@@ -677,20 +673,20 @@ import com.gemstone.gemfire.internal.offheap.annotations.Unretained;
       int bits;
       int originalBits;
       do {
-        originalBits = UnsafeMemoryChunk.readAbsoluteIntVolatile(this.memoryAddress+REF_COUNT_OFFSET);
+        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 (!UnsafeMemoryChunk.writeAbsoluteIntVolatile(this.memoryAddress+REF_COUNT_OFFSET, originalBits, bits));
+      } while (!AddressableMemoryManager.writeIntVolatile(this.memoryAddress+REF_COUNT_OFFSET, originalBits, bits));
     }
     
     public void initializeUseCount() {
       int rawBits;
       do {
-        rawBits = UnsafeMemoryChunk.readAbsoluteIntVolatile(this.memoryAddress+REF_COUNT_OFFSET);
+        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));
         }
@@ -698,11 +694,11 @@ import com.gemstone.gemfire.internal.offheap.annotations.Unretained;
         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));
+      } while (!AddressableMemoryManager.writeIntVolatile(this.memoryAddress+REF_COUNT_OFFSET, rawBits, rawBits+1));
     }
 
     public static int getRefCount(long memAddr) {
-      return UnsafeMemoryChunk.readAbsoluteInt(memAddr+REF_COUNT_OFFSET) & REF_COUNT_MASK;
+      return AddressableMemoryManager.readInt(memAddr+REF_COUNT_OFFSET) & REF_COUNT_MASK;
     }
 
     public static boolean retain(long memAddr) {
@@ -711,7 +707,7 @@ import com.gemstone.gemfire.internal.offheap.annotations.Unretained;
       int rawBits;
       int retryCount = 0;
       do {
-        rawBits = UnsafeMemoryChunk.readAbsoluteIntVolatile(memAddr+REF_COUNT_OFFSET);
+        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
@@ -727,7 +723,7 @@ import com.gemstone.gemfire.internal.offheap.annotations.Unretained;
         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));
+      } 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);
@@ -745,7 +741,7 @@ import com.gemstone.gemfire.internal.offheap.annotations.Unretained;
       boolean returnToAllocator;
       do {
         returnToAllocator = false;
-        rawBits = UnsafeMemoryChunk.readAbsoluteIntVolatile(memAddr+REF_COUNT_OFFSET);
+        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);
@@ -762,7 +758,7 @@ import com.gemstone.gemfire.internal.offheap.annotations.Unretained;
         } else {
           newCount = rawBits-1;
         }
-      } while (!UnsafeMemoryChunk.writeAbsoluteIntVolatile(memAddr+REF_COUNT_OFFSET, rawBits, newCount));
+      } 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()) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/e2a126d4/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..b264d89 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");
     }
@@ -307,7 +307,7 @@ public class SimpleMemoryAllocatorImpl implements MemoryAllocator {
   }
   
   @Override
-  public MemoryChunk allocate(int size) {
+  public StoredObject allocate(int size) {
     //System.out.println("allocating " + size);
     ObjectChunk result = allocateChunk(size);
     //("allocated off heap object of size " + size + " @" + Long.toHexString(result.getMemoryAddress()), true);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/e2a126d4/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/e2a126d4/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/e2a126d4/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/e2a126d4/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..17c8f6f 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
@@ -44,6 +44,10 @@ public interface StoredObject extends Releasable, Sendable, CachedDeserializable
    */
   @Retained
   public boolean retain();
+  /**
+   * Returns the number of users of this memory.
+   */
+  public int getRefCount();
 
   /**
    * Returns true if the value stored in this memory chunk is a serialized object. Returns false if it is a byte array.

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/e2a126d4/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/e2a126d4/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..e2a37a0 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.AddressableMemoryManager;
 import com.gemstone.gemfire.internal.offheap.ObjectChunk;
-import com.gemstone.gemfire.internal.offheap.UnsafeMemoryChunk;
 
 /**
  * <p>
@@ -515,14 +515,14 @@ public class ByteBufferInputStream extends InputStream implements DataInput, jav
     private short basicGetShort(int pos) {
       long addr = this.chunk.getAddressForReading(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));
       }
     }
@@ -539,14 +539,14 @@ public class ByteBufferInputStream extends InputStream implements DataInput, jav
     private char basicGetChar(int pos) {
       long addr = this.chunk.getAddressForReading(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));
       }
     }
@@ -564,16 +564,16 @@ public class ByteBufferInputStream extends InputStream implements DataInput, jav
     private int basicGetInt(final int pos) {
       long addr = this.chunk.getAddressForReading(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);
       }
     }
@@ -590,20 +590,20 @@ public class ByteBufferInputStream extends InputStream implements DataInput, jav
     private long basicGetLong(final int pos) {
       long addr = this.chunk.getAddressForReading(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) +

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/e2a126d4/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/e2a126d4/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
index b69f82e..728219c 100644
--- 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
@@ -33,7 +33,7 @@ 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.internal.offheap.SlabImpl;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
 @Category(UnitTest.class)
@@ -46,7 +46,7 @@ public class ChunkValueWrapperJUnitTest {
 
   @Before
   public void setUp() throws Exception {
-    SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new UnsafeMemoryChunk[]{new UnsafeMemoryChunk(1024*1024)});
+    SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new SlabImpl[]{new SlabImpl(1024*1024)});
   }
 
   @After

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/e2a126d4/geode-core/src/test/java/com/gemstone/gemfire/internal/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..d9fc883 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
@@ -31,7 +31,7 @@ import com.gemstone.gemfire.internal.offheap.DataAsAddress;
 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,7 +110,7 @@ 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);
@@ -127,7 +127,7 @@ 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);
@@ -144,7 +144,7 @@ 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);
@@ -162,7 +162,7 @@ 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);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/e2a126d4/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..8ef7ee7 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();
     }
   }
   
@@ -213,7 +213,7 @@ public class FragmentJUnitTest {
     byte[] expectedBytes = new byte[(int)OffHeapStorage.MIN_SLAB_SIZE];
     Arrays.fill(expectedBytes, ObjectChunk.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/e2a126d4/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());