You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by ab...@apache.org on 2015/11/25 00:30:17 UTC

[11/19] incubator-geode git commit: GEODE-580: cleanup off-heap code

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4b0925e3/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/SyncChunkStack.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/SyncChunkStack.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/SyncChunkStack.java
new file mode 100644
index 0000000..a615af0
--- /dev/null
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/SyncChunkStack.java
@@ -0,0 +1,130 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.gemstone.gemfire.internal.offheap;
+
+import com.gemstone.gemfire.LogWriter;
+
+/**
+ * A "stack" of "chunk" instances. The chunks are not kept
+ * in java object form but instead each "chunk" is just an
+ * off-heap address.
+ * This class is used for each "tiny" free-list of the off-heap memory allocator.
+ */
+public class SyncChunkStack {
+  // Ok to read without sync but must be synced on write
+  private volatile long topAddr;
+  
+  public SyncChunkStack(long addr) {
+    if (addr != 0L) SimpleMemoryAllocatorImpl.validateAddress(addr);
+    this.topAddr = addr;
+  }
+  public SyncChunkStack() {
+    this.topAddr = 0L;
+  }
+  public boolean isEmpty() {
+    return this.topAddr == 0L;
+  }
+  public void offer(long e) {
+    assert e != 0;
+    SimpleMemoryAllocatorImpl.validateAddress(e);
+    synchronized (this) {
+      Chunk.setNext(e, this.topAddr);
+      this.topAddr = e;
+    }
+  }
+  public long poll() {
+    long result;
+    synchronized (this) {
+      result = this.topAddr;
+      if (result != 0L) {
+        this.topAddr = Chunk.getNext(result);
+      }
+    }
+    return result;
+  }
+  /**
+   * Returns the address of the "top" item in this stack.
+   */
+  public long getTopAddress() {
+    return this.topAddr;
+  }
+  /**
+   * Removes all the Chunks from this stack
+   * and returns the address of the first chunk.
+   * The caller owns all the Chunks after this call.
+   */
+  public long clear() {
+    long result;
+    synchronized (this) {
+      result = this.topAddr;
+      if (result != 0L) {
+        this.topAddr = 0L;
+      }
+    }
+    return result;
+  }
+  public void logSizes(LogWriter lw, String msg) {
+    long headAddr = this.topAddr;
+    long addr;
+    boolean concurrentModDetected;
+    do {
+      concurrentModDetected = false;
+      addr = headAddr;
+      while (addr != 0L) {
+        int curSize = Chunk.getSize(addr);
+        addr = Chunk.getNext(addr);
+        long curHead = this.topAddr;
+        if (curHead != headAddr) {
+          headAddr = curHead;
+          concurrentModDetected = true;
+          // Someone added or removed from the stack.
+          // So we break out of the inner loop and start
+          // again at the new head.
+          break;
+        }
+        // TODO construct a single log msg
+        // that gets reset on the concurrent mad.
+        lw.info(msg + curSize);
+      }
+    } while (concurrentModDetected);
+  }
+  public long computeTotalSize() {
+    long result;
+    long headAddr = this.topAddr;
+    long addr;
+    boolean concurrentModDetected;
+    do {
+      concurrentModDetected = false;
+      result = 0;
+      addr = headAddr;
+      while (addr != 0L) {
+        result += Chunk.getSize(addr);
+        addr = Chunk.getNext(addr);
+        long curHead = this.topAddr;
+        if (curHead != headAddr) {
+          headAddr = curHead;
+          concurrentModDetected = true;
+          // Someone added or removed from the stack.
+          // So we break out of the inner loop and start
+          // again at the new head.
+          break;
+        }
+      }
+    } while (concurrentModDetected);
+    return result;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4b0925e3/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/ByteBufferInputStream.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/ByteBufferInputStream.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/ByteBufferInputStream.java
index 8577569..8a7d351 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/ByteBufferInputStream.java
+++ b/gemfire-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.Chunk;
 import com.gemstone.gemfire.internal.offheap.UnsafeMemoryChunk;
-import com.gemstone.gemfire.internal.offheap.SimpleMemoryAllocatorImpl.Chunk;
 
 /**
  * <p>

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4b0925e3/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/ImmutableByteBufferInputStream.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/ImmutableByteBufferInputStream.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/ImmutableByteBufferInputStream.java
index 4f16c8a..52f332f 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/ImmutableByteBufferInputStream.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/ImmutableByteBufferInputStream.java
@@ -18,7 +18,7 @@ package com.gemstone.gemfire.internal.tcp;
 
 import java.nio.ByteBuffer;
 
-import com.gemstone.gemfire.internal.offheap.SimpleMemoryAllocatorImpl.Chunk;
+import com.gemstone.gemfire.internal.offheap.Chunk;
 
 /**
  * You should only create an instance of this class if the bytes this buffer reads

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4b0925e3/gemfire-core/src/main/java/com/gemstone/gemfire/internal/util/BlobHelper.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/util/BlobHelper.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/util/BlobHelper.java
index 6665e55..7a4840e 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/util/BlobHelper.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/util/BlobHelper.java
@@ -27,7 +27,7 @@ import com.gemstone.gemfire.internal.DSCODE;
 import com.gemstone.gemfire.internal.HeapDataOutputStream;
 import com.gemstone.gemfire.internal.Version;
 import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
-import com.gemstone.gemfire.internal.offheap.SimpleMemoryAllocatorImpl.Chunk;
+import com.gemstone.gemfire.internal.offheap.Chunk;
 import com.gemstone.gemfire.internal.offheap.annotations.Unretained;
 import com.gemstone.gemfire.pdx.internal.PdxInputStream;
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4b0925e3/gemfire-core/src/main/java/com/gemstone/gemfire/pdx/internal/PdxInputStream.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/pdx/internal/PdxInputStream.java b/gemfire-core/src/main/java/com/gemstone/gemfire/pdx/internal/PdxInputStream.java
index 66d4887..85e078d 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/pdx/internal/PdxInputStream.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/pdx/internal/PdxInputStream.java
@@ -26,7 +26,7 @@ import java.util.Date;
 import com.gemstone.gemfire.DataSerializer;
 import com.gemstone.gemfire.InternalGemFireException;
 import com.gemstone.gemfire.pdx.PdxSerializationException;
-import com.gemstone.gemfire.internal.offheap.SimpleMemoryAllocatorImpl.Chunk;
+import com.gemstone.gemfire.internal.offheap.Chunk;
 import com.gemstone.gemfire.internal.tcp.ByteBufferInputStream;
 import com.gemstone.gemfire.internal.tcp.ImmutableByteBufferInputStream;
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4b0925e3/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ChunkValueWrapperJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ChunkValueWrapperJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ChunkValueWrapperJUnitTest.java
index 428b205..0606387 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ChunkValueWrapperJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ChunkValueWrapperJUnitTest.java
@@ -29,10 +29,10 @@ import org.junit.experimental.categories.Category;
 
 import com.gemstone.gemfire.internal.cache.DiskEntry.Helper.ChunkValueWrapper;
 import com.gemstone.gemfire.internal.cache.DiskEntry.Helper.Flushable;
+import com.gemstone.gemfire.internal.offheap.Chunk;
 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.SimpleMemoryAllocatorImpl.Chunk;
 import com.gemstone.gemfire.internal.offheap.UnsafeMemoryChunk;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4b0925e3/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/OffHeapTestUtil.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/OffHeapTestUtil.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/OffHeapTestUtil.java
index 2fc3c82..948c7f8 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/OffHeapTestUtil.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/OffHeapTestUtil.java
@@ -23,8 +23,9 @@ import junit.framework.Assert;
 
 import com.gemstone.gemfire.cache.CacheClosedException;
 import com.gemstone.gemfire.internal.offheap.MemoryBlock;
+import com.gemstone.gemfire.internal.offheap.RefCountChangeInfo;
+import com.gemstone.gemfire.internal.offheap.ReferenceCountHelper;
 import com.gemstone.gemfire.internal.offheap.SimpleMemoryAllocatorImpl;
-import com.gemstone.gemfire.internal.offheap.SimpleMemoryAllocatorImpl.RefCountChangeInfo;
 
 @SuppressWarnings("deprecation")
 public class OffHeapTestUtil {
@@ -51,7 +52,7 @@ public class OffHeapTestUtil {
     }
     
     if(orphans != null && ! orphans.isEmpty()) {
-      List<RefCountChangeInfo> info = SimpleMemoryAllocatorImpl.getRefCountInfo(orphans.get(0).getMemoryAddress());
+      List<RefCountChangeInfo> info = ReferenceCountHelper.getRefCountInfo(orphans.get(0).getMemoryAddress());
       System.out.println("FOUND ORPHAN!!");
       System.out.println("Sample orphan: " + orphans.get(0));
       System.out.println("Orphan info: " + info);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4b0925e3/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/OldValueImporterTestBase.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/OldValueImporterTestBase.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/OldValueImporterTestBase.java
index 727a59c..f7d0714 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/OldValueImporterTestBase.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/OldValueImporterTestBase.java
@@ -26,11 +26,11 @@ import org.junit.Test;
 
 import com.gemstone.gemfire.internal.HeapDataOutputStream;
 import com.gemstone.gemfire.internal.cache.EntryEventImpl.OldValueImporter;
+import com.gemstone.gemfire.internal.offheap.Chunk;
+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.SimpleMemoryAllocatorImpl.Chunk;
-import com.gemstone.gemfire.internal.offheap.SimpleMemoryAllocatorImpl.DataAsAddress;
 import com.gemstone.gemfire.internal.offheap.UnsafeMemoryChunk;
 import com.gemstone.gemfire.internal.util.BlobHelper;
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4b0925e3/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/ConcurrentBagJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/ConcurrentBagJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/ConcurrentBagJUnitTest.java
deleted file mode 100644
index 8bba8db..0000000
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/ConcurrentBagJUnitTest.java
+++ /dev/null
@@ -1,130 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package com.gemstone.gemfire.internal.offheap;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
-
-import java.util.Iterator;
-import java.util.NoSuchElementException;
-
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import com.gemstone.gemfire.internal.offheap.SimpleMemoryAllocatorImpl.ConcurrentBag.Node;
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
-
-@Category(UnitTest.class)
-public class ConcurrentBagJUnitTest {
-  
-  public static class IntNode implements SimpleMemoryAllocatorImpl.ConcurrentBag.Node {
-
-    private final int data;
-    
-    public IntNode(int i) {
-      this.data = i;
-    }
-    
-    public Integer getData() {
-      return this.data;
-    }
-    
-    private Node next;
-    @Override
-    public void setNextCBNode(Node next) {
-      this.next = next;
-    }
-    @Override
-    public Node getNextCBNode() {
-      return this.next;
-    }
-    
-  }
-  @Test
-  public void testBasicFreeList() {
-    SimpleMemoryAllocatorImpl.ConcurrentBag<IntNode> l = new SimpleMemoryAllocatorImpl.ConcurrentBag<IntNode>(5000);
-    assertEquals(false, l.iterator().hasNext());
-    try {
-      l.iterator().next();
-      fail("expected NoSuchElementException");
-    } catch (NoSuchElementException expected) {
-    }
-    assertEquals(null, l.poll());
-    
-    l.offer(new IntNode(1));
-    assertEquals(true, l.iterator().hasNext());
-    
-    assertEquals(Integer.valueOf(1), l.iterator().next().getData());
-    assertEquals(Integer.valueOf(1), l.poll().getData());
-    assertEquals(false, l.iterator().hasNext());
-    assertEquals(null, l.poll());
-    
-    try {
-      l.iterator().remove();
-      fail("expected UnsupportedOperationException");
-    } catch (UnsupportedOperationException expected) {
-    }
-//    {
-//      l.offer(new IntNode(1));
-//      l.offer(new IntNode(2));
-//      Iterator<IntNode> it = l.iterator();
-//      assertEquals(true, it.hasNext());
-//      assertEquals(Integer.valueOf(1), it.next().getData());
-//      assertEquals(true, it.hasNext());
-//      assertEquals(Integer.valueOf(2), it.next().getData());
-//      assertEquals(false, it.hasNext());
-//      
-//      it = l.iterator();
-//      try {
-//        it.remove();
-//        fail("expected IllegalStateException");
-//      } catch (IllegalStateException expected) {
-//      }
-//      it.next();
-//      it.remove();
-//      try {
-//        it.remove();
-//        fail("expected IllegalStateException");
-//      } catch (IllegalStateException expected) {
-//      }
-//      assertEquals(Integer.valueOf(2), it.next());
-//      assertEquals(false, it.hasNext());
-//      
-//      assertEquals(Integer.valueOf(2), l.poll());
-//      assertEquals(null, l.poll());
-//    }
-    
-    for (int i=1; i <= 3999; i++) {
-      l.offer(new IntNode(i));
-    }
-    {
-      Iterator<IntNode> it = l.iterator();
-//    for (int i=1; i <= 3999; i++) {
-      for (int i=3999; i >= 1; i--) {
-        assertEquals(true, it.hasNext());
-        assertEquals(Integer.valueOf(i), it.next().getData());
-      }
-      assertEquals(false, it.hasNext());
-    }
-//  for (int i=1; i <= 3999; i++) {
-    for (int i=3999; i >= 1; i--) {
-      assertEquals(Integer.valueOf(i), l.poll().getData());
-    }
-    assertEquals(null, l.poll());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4b0925e3/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/FreeListOffHeapRegionJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/FreeListOffHeapRegionJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/FreeListOffHeapRegionJUnitTest.java
index 2d38a26..93f2039 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/FreeListOffHeapRegionJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/FreeListOffHeapRegionJUnitTest.java
@@ -40,7 +40,7 @@ public class FreeListOffHeapRegionJUnitTest extends OffHeapRegionBase {
 
   @Override
   public int perObjectOverhead() {
-    return SimpleMemoryAllocatorImpl.Chunk.OFF_HEAP_HEADER_SIZE;
+    return Chunk.OFF_HEAP_HEADER_SIZE;
   }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4b0925e3/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapRegionBase.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapRegionBase.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapRegionBase.java
index 73fb51d..2f539d5 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapRegionBase.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapRegionBase.java
@@ -38,7 +38,6 @@ import com.gemstone.gemfire.compression.Compressor;
 import com.gemstone.gemfire.compression.SnappyCompressor;
 import com.gemstone.gemfire.internal.cache.EntryEventImpl;
 import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
-import com.gemstone.gemfire.internal.offheap.SimpleMemoryAllocatorImpl.Chunk;
 import com.gemstone.gemfire.internal.offheap.annotations.OffHeapIdentifier;
 import com.gemstone.gemfire.internal.offheap.annotations.Released;
 import com.gemstone.gemfire.internal.offheap.annotations.Retained;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4b0925e3/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapValidationJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapValidationJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapValidationJUnitTest.java
index a47d42d..b9e8456 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapValidationJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapValidationJUnitTest.java
@@ -61,7 +61,6 @@ import com.gemstone.gemfire.internal.HeapDataOutputStream;
 import com.gemstone.gemfire.internal.Version;
 import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
 import com.gemstone.gemfire.internal.cache.LocalRegion;
-import com.gemstone.gemfire.internal.offheap.SimpleMemoryAllocatorImpl.Chunk;
 import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4b0925e3/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapWriteObjectAsByteArrayJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapWriteObjectAsByteArrayJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapWriteObjectAsByteArrayJUnitTest.java
index e19f4cb..daebefa 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapWriteObjectAsByteArrayJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapWriteObjectAsByteArrayJUnitTest.java
@@ -34,8 +34,6 @@ import com.gemstone.gemfire.internal.cache.EntryEventImpl;
 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.SimpleMemoryAllocatorImpl.Chunk;
-import com.gemstone.gemfire.internal.offheap.SimpleMemoryAllocatorImpl.DataAsAddress;
 import com.gemstone.gemfire.internal.offheap.StoredObject;
 import com.gemstone.gemfire.internal.offheap.UnsafeMemoryChunk;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4b0925e3/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/OldFreeListOffHeapRegionJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/OldFreeListOffHeapRegionJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/OldFreeListOffHeapRegionJUnitTest.java
index 3636c79..6e26b2f 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/OldFreeListOffHeapRegionJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/OldFreeListOffHeapRegionJUnitTest.java
@@ -41,7 +41,7 @@ public class OldFreeListOffHeapRegionJUnitTest extends OffHeapRegionBase {
 
   @Override
   public int perObjectOverhead() {
-    return SimpleMemoryAllocatorImpl.Chunk.OFF_HEAP_HEADER_SIZE;
+    return Chunk.OFF_HEAP_HEADER_SIZE;
   }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4b0925e3/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/SimpleMemoryAllocatorFillPatternJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/SimpleMemoryAllocatorFillPatternJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/SimpleMemoryAllocatorFillPatternJUnitTest.java
index 745d2c0..c8b1834 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/SimpleMemoryAllocatorFillPatternJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/SimpleMemoryAllocatorFillPatternJUnitTest.java
@@ -29,10 +29,8 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import static org.junit.Assert.*;
-
 import junit.framework.TestCase;
 
-import com.gemstone.gemfire.internal.offheap.SimpleMemoryAllocatorImpl.Chunk;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4b0925e3/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/SimpleMemoryAllocatorJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/SimpleMemoryAllocatorJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/SimpleMemoryAllocatorJUnitTest.java
index ea90bdc..19dfebb 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/SimpleMemoryAllocatorJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/SimpleMemoryAllocatorJUnitTest.java
@@ -31,7 +31,6 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import com.gemstone.gemfire.OutOfOffHeapMemoryException;
-import com.gemstone.gemfire.internal.offheap.SimpleMemoryAllocatorImpl.Chunk;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
 @Category(UnitTest.class)
@@ -46,7 +45,7 @@ public class SimpleMemoryAllocatorJUnitTest {
     int TINY_MULTIPLE = com.gemstone.gemfire.internal.offheap.SimpleMemoryAllocatorImpl.TINY_MULTIPLE;
 //    int BIG_MULTIPLE = com.gemstone.gemfire.internal.offheap.SimpleMemoryAllocatorImpl.FreeListManager.BIG_MULTIPLE;
     int HUGE_MULTIPLE = com.gemstone.gemfire.internal.offheap.SimpleMemoryAllocatorImpl.HUGE_MULTIPLE;
-    int perObjectOverhead = com.gemstone.gemfire.internal.offheap.SimpleMemoryAllocatorImpl.Chunk.OFF_HEAP_HEADER_SIZE;
+    int perObjectOverhead = com.gemstone.gemfire.internal.offheap.Chunk.OFF_HEAP_HEADER_SIZE;
     int maxTiny = com.gemstone.gemfire.internal.offheap.SimpleMemoryAllocatorImpl.MAX_TINY-perObjectOverhead;
 //    int MIN_BIG_SIZE = round(BIG_MULTIPLE, maxTiny+perObjectOverhead+1)-perObjectOverhead;
 //    int maxBig = com.gemstone.gemfire.internal.offheap.SimpleMemoryAllocatorImpl.FreeListManager.MAX_BIG-perObjectOverhead;
@@ -167,7 +166,7 @@ public class SimpleMemoryAllocatorJUnitTest {
   
   @Test
   public void testCompaction() {
-    final int perObjectOverhead = com.gemstone.gemfire.internal.offheap.SimpleMemoryAllocatorImpl.Chunk.OFF_HEAP_HEADER_SIZE;
+    final int perObjectOverhead = com.gemstone.gemfire.internal.offheap.Chunk.OFF_HEAP_HEADER_SIZE;
     final int BIG_ALLOC_SIZE = 150000;
     final int SMALL_ALLOC_SIZE = BIG_ALLOC_SIZE/2;
     final int TOTAL_MEM = BIG_ALLOC_SIZE;
@@ -283,7 +282,7 @@ public class SimpleMemoryAllocatorJUnitTest {
   boolean memoryUsageEventReceived;
   @Test
   public void testUsageEventListener() {
-    final int perObjectOverhead = com.gemstone.gemfire.internal.offheap.SimpleMemoryAllocatorImpl.Chunk.OFF_HEAP_HEADER_SIZE;
+    final int perObjectOverhead = com.gemstone.gemfire.internal.offheap.Chunk.OFF_HEAP_HEADER_SIZE;
     final int SMALL_ALLOC_SIZE = 1000;
     UnsafeMemoryChunk slab = new UnsafeMemoryChunk(3000);
     try {
@@ -326,7 +325,7 @@ public class SimpleMemoryAllocatorJUnitTest {
   
   @Test
   public void testOutOfOffHeapMemory() {
-    final int perObjectOverhead = com.gemstone.gemfire.internal.offheap.SimpleMemoryAllocatorImpl.Chunk.OFF_HEAP_HEADER_SIZE;
+    final int perObjectOverhead = com.gemstone.gemfire.internal.offheap.Chunk.OFF_HEAP_HEADER_SIZE;
     final int BIG_ALLOC_SIZE = 150000;
     final int SMALL_ALLOC_SIZE = BIG_ALLOC_SIZE/2;
     final int TOTAL_MEM = BIG_ALLOC_SIZE;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4b0925e3/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/SimpleMemoryAllocatorLifecycleListenerJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/SimpleMemoryAllocatorLifecycleListenerJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/SimpleMemoryAllocatorLifecycleListenerJUnitTest.java
index 38f969e..2df8656 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/SimpleMemoryAllocatorLifecycleListenerJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/SimpleMemoryAllocatorLifecycleListenerJUnitTest.java
@@ -26,7 +26,6 @@ import org.junit.Assert;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import com.gemstone.gemfire.internal.offheap.SimpleMemoryAllocatorImpl.LifecycleListener;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
 /**
@@ -45,7 +44,7 @@ public class SimpleMemoryAllocatorLifecycleListenerJUnitTest {
   
   @After
   public void tearDown() throws Exception {
-    SimpleMemoryAllocatorImpl.removeLifecycleListener(this.listener);
+    LifecycleListener.removeLifecycleListener(this.listener);
     this.afterCreateCallbacks.clear();
     this.afterReuseCallbacks.clear();
     this.beforeCloseCallbacks.clear();
@@ -54,8 +53,8 @@ public class SimpleMemoryAllocatorLifecycleListenerJUnitTest {
 
   @Test
   public void testAddRemoveListener() {
-    SimpleMemoryAllocatorImpl.addLifecycleListener(this.listener);
-    SimpleMemoryAllocatorImpl.removeLifecycleListener(this.listener);
+    LifecycleListener.addLifecycleListener(this.listener);
+    LifecycleListener.removeLifecycleListener(this.listener);
 
     UnsafeMemoryChunk slab = new UnsafeMemoryChunk(1024); // 1k
     SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.create(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new UnsafeMemoryChunk[]{slab});
@@ -73,7 +72,7 @@ public class SimpleMemoryAllocatorLifecycleListenerJUnitTest {
   
   @Test
   public void testCallbacksAreCalledAfterCreate() {
-    SimpleMemoryAllocatorImpl.addLifecycleListener(this.listener);
+    LifecycleListener.addLifecycleListener(this.listener);
     
     UnsafeMemoryChunk slab = new UnsafeMemoryChunk(1024); // 1k
     SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.create(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new UnsafeMemoryChunk[]{slab});

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4b0925e3/gemfire-core/src/test/java/com/gemstone/gemfire/pdx/OffHeapByteBufferByteSourceJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/pdx/OffHeapByteBufferByteSourceJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/pdx/OffHeapByteBufferByteSourceJUnitTest.java
index b3a4ab5..c7c7b7b 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/pdx/OffHeapByteBufferByteSourceJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/pdx/OffHeapByteBufferByteSourceJUnitTest.java
@@ -22,8 +22,8 @@ import java.nio.ByteBuffer;
 
 import org.junit.experimental.categories.Category;
 
+import com.gemstone.gemfire.internal.offheap.Chunk;
 import com.gemstone.gemfire.internal.offheap.SimpleMemoryAllocatorImpl;
-import com.gemstone.gemfire.internal.offheap.SimpleMemoryAllocatorImpl.Chunk;
 import com.gemstone.gemfire.internal.offheap.StoredObject;
 import com.gemstone.gemfire.internal.tcp.ByteBufferInputStream.ByteSource;
 import com.gemstone.gemfire.internal.tcp.ByteBufferInputStream.ByteSourceFactory;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4b0925e3/gemfire-core/src/test/java/com/gemstone/gemfire/pdx/OffHeapByteSourceJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/pdx/OffHeapByteSourceJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/pdx/OffHeapByteSourceJUnitTest.java
index d0e1986..543ef94 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/pdx/OffHeapByteSourceJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/pdx/OffHeapByteSourceJUnitTest.java
@@ -20,12 +20,12 @@ import org.junit.After;
 import org.junit.Before;
 import org.junit.experimental.categories.Category;
 
+import com.gemstone.gemfire.internal.offheap.Chunk;
 import com.gemstone.gemfire.internal.offheap.NullOffHeapMemoryStats;
 import com.gemstone.gemfire.internal.offheap.NullOutOfOffHeapMemoryListener;
 import com.gemstone.gemfire.internal.offheap.SimpleMemoryAllocatorImpl;
 import com.gemstone.gemfire.internal.offheap.StoredObject;
 import com.gemstone.gemfire.internal.offheap.UnsafeMemoryChunk;
-import com.gemstone.gemfire.internal.offheap.SimpleMemoryAllocatorImpl.Chunk;
 import com.gemstone.gemfire.internal.tcp.ByteBufferInputStream.ByteSource;
 import com.gemstone.gemfire.internal.tcp.ByteBufferInputStream.ByteSourceFactory;
 import com.gemstone.gemfire.internal.tcp.ByteBufferInputStream.OffHeapByteSource;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4b0925e3/gemfire-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/excludedClasses.txt
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/excludedClasses.txt b/gemfire-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/excludedClasses.txt
index 060a5ec..ce49654 100644
--- a/gemfire-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/excludedClasses.txt
+++ b/gemfire-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/excludedClasses.txt
@@ -19,6 +19,7 @@ com/gemstone/gemfire/internal/logging/log4j/LocalizedMessage
 com/gemstone/gemfire/internal/logging/log4j/LogWriterAppenders
 com/gemstone/gemfire/internal/logging/log4j/LogWriterAppenders$Identifier
 com/gemstone/gemfire/internal/logging/log4j/LogWriterLogger
+com/gemstone/gemfire/internal/offheap/RefCountChangeInfo
 com/gemstone/gemfire/internal/process/BlockingProcessStreamReader
 com/gemstone/gemfire/internal/process/NonBlockingProcessStreamReader
 com/gemstone/gemfire/internal/process/ProcessStreamReader

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4b0925e3/gemfire-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/sanctionedSerializables.txt
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/sanctionedSerializables.txt b/gemfire-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/sanctionedSerializables.txt
index a9479e3..03e82d0 100644
--- a/gemfire-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/sanctionedSerializables.txt
+++ b/gemfire-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/sanctionedSerializables.txt
@@ -470,7 +470,6 @@ com/gemstone/gemfire/internal/memcached/commands/ClientError,true,-2426928000696
 com/gemstone/gemfire/internal/offheap/MemoryBlock$State,false
 com/gemstone/gemfire/internal/offheap/OffHeapStorage$1,false
 com/gemstone/gemfire/internal/offheap/OffHeapStorage$2,false
-com/gemstone/gemfire/internal/offheap/SimpleMemoryAllocatorImpl$RefCountChangeInfo,false,dupCount:int,owner:java/lang/Object,rc:int,stackTraceString:java/lang/String,threadName:java/lang/String
 com/gemstone/gemfire/internal/offheap/annotations/OffHeapIdentifier,false,id:java/lang/String
 com/gemstone/gemfire/internal/process/ConnectionFailedException,true,5622636452836752700
 com/gemstone/gemfire/internal/process/FileAlreadyExistsException,true,5471082555536094256