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:07 UTC

[01/19] incubator-geode git commit: fixing minor problem with a test configuration.

Repository: incubator-geode
Updated Branches:
  refs/heads/feature/GEODE-584 1e339735e -> e38a9425a


fixing minor problem with a test configuration.

This test was setting a bind-address but it did so after building
the distribution-config, so it wasn't used in the test.  That causes
it to fail on some machines.  Moving the bind-address setting one
line up in the file fixes the problem.


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

Branch: refs/heads/feature/GEODE-584
Commit: c9d877851034189f2a14d87fffee5cc58938e6de
Parents: bc12784
Author: Bruce Schuchardt <bs...@pivotal.io>
Authored: Fri Nov 20 14:03:21 2015 -0800
Committer: Bruce Schuchardt <bs...@pivotal.io>
Committed: Fri Nov 20 14:03:21 2015 -0800

----------------------------------------------------------------------
 .../membership/gms/locator/GMSLocatorRecoveryJUnitTest.java        | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c9d87785/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/locator/GMSLocatorRecoveryJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/locator/GMSLocatorRecoveryJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/locator/GMSLocatorRecoveryJUnitTest.java
index 90f6816..8484f7c 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/locator/GMSLocatorRecoveryJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/locator/GMSLocatorRecoveryJUnitTest.java
@@ -137,8 +137,8 @@ public class GMSLocatorRecoveryJUnitTest {
       nonDefault.put(DistributionConfig.LOG_FILE_NAME, "");
       nonDefault.put(DistributionConfig.LOG_LEVEL_NAME, "fine");
       nonDefault.put(DistributionConfig.LOCATORS_NAME, localHost.getHostName()+'['+port+']');
-      DistributionConfigImpl config = new DistributionConfigImpl(nonDefault);
       nonDefault.put(DistributionConfig.BIND_ADDRESS_NAME, localHost.getHostName());
+      DistributionConfigImpl config = new DistributionConfigImpl(nonDefault);
       RemoteTransportConfig transport = new RemoteTransportConfig(config,
           DistributionManager.NORMAL_DM_TYPE);
 


[06/19] incubator-geode git commit: GEODE-587: Include the lucence dependent jars on the classpath

Posted by ab...@apache.org.
GEODE-587: Include the lucence dependent jars on the classpath

The gemfire-assembly build creates the runtime classpath.  This is
a slight tweak to ensure the lucene jars are added to the classpath.


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

Branch: refs/heads/feature/GEODE-584
Commit: bbccb60563f5e1833b925342388bedc1813450e8
Parents: 452328f
Author: Anthony Baker <ab...@pivotal.io>
Authored: Mon Nov 23 12:41:06 2015 -0800
Committer: Anthony Baker <ab...@pivotal.io>
Committed: Mon Nov 23 12:41:06 2015 -0800

----------------------------------------------------------------------
 gemfire-assembly/build.gradle | 22 +++++++++++++++-------
 1 file changed, 15 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/bbccb605/gemfire-assembly/build.gradle
----------------------------------------------------------------------
diff --git a/gemfire-assembly/build.gradle b/gemfire-assembly/build.gradle
index 5f82905..8f803ae 100755
--- a/gemfire-assembly/build.gradle
+++ b/gemfire-assembly/build.gradle
@@ -74,10 +74,12 @@ task defaultCacheConfig(type: JavaExec, dependsOn: classes) {
 // This closure sets the gemfire classpath.  If we add another jar to the classpath it must
 // be included in the filter logic below.
 def cp = {
-  configurations.archives.dependencies.collect { it.dependencyProject }
+  def jars = configurations.archives.dependencies.collect { it.dependencyProject }
     .findAll { !it.name.contains('web') }
     .collect { it.jar.archiveName }
-    .join(' ') + ' ' +
+    .join(' ')
+
+  jars += ' ' + 
     project(':gemfire-core').configurations.runtime.collect { it.getName() }.findAll {
       it.contains('antlr') ||
       it.contains('commons-io') ||
@@ -105,10 +107,6 @@ def cp = {
       it.contains('log4j-jcl') ||
       it.contains('log4j-jul') ||
       it.contains('log4j-slf4j-impl') ||
-      it.contains('lucene-analyzers-common') ||
-      it.contains('lucene-core') ||
-      it.contains('lucene-queries') ||
-      it.contains('lucene-queryparser') ||
       it.contains('slf4j-api') ||
       it.contains('spring-core') ||
       it.contains('spring-shell') ||
@@ -116,7 +114,17 @@ def cp = {
       it.contains('hbase') ||
       it.contains('jgroups') ||
       it.contains('netty')
-    }.join(' ') 
+    }.join(' ')
+    
+  jars += ' ' +
+    project(':gemfire-lucene').configurations.runtime.collect { it.getName() }.findAll {
+      it.contains('lucene-analyzers-common') ||
+      it.contains('lucene-core') ||
+      it.contains('lucene-queries') ||
+      it.contains('lucene-queryparser')
+    }.join(' ')
+    
+  return jars
 }
 
 // Note: this dependency doesn't work if you change a library version from


[16/19] incubator-geode git commit: [GEODE-585]: Simplify hash index code Refactored hash index and hash index set Using modified versions of the fastutil methods for adding and finding index positions for objects Added hash index set unit tests Removes

Posted by ab...@apache.org.
[GEODE-585]: Simplify hash index code
Refactored hash index and hash index set
Using modified versions of the fastutil methods for adding and finding index positions for objects
Added hash index set unit tests
Removes PrimeFinder, ObjectProcedure, and HashIndexStrategy


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

Branch: refs/heads/feature/GEODE-584
Commit: abad018a944e870b418de81ebefad0524010f225
Parents: 4b0925e
Author: Jason Huynh <hu...@gmail.com>
Authored: Fri Nov 20 16:03:33 2015 -0800
Committer: Jason Huynh <hu...@gmail.com>
Committed: Tue Nov 24 10:18:31 2015 -0800

----------------------------------------------------------------------
 .../cache/query/internal/index/HashIndex.java   |  176 ++-
 .../query/internal/index/HashIndexSet.java      | 1086 ++++++------------
 .../query/internal/index/HashIndexStrategy.java |   90 --
 .../gemfire/internal/util/ObjectProcedure.java  |   30 -
 .../gemfire/internal/util/PrimeFinder.java      |  159 ---
 .../internal/index/HashIndexJUnitTest.java      |   23 +-
 .../internal/index/HashIndexSetJUnitTest.java   |  504 ++++++++
 7 files changed, 922 insertions(+), 1146 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/abad018a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/index/HashIndex.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/index/HashIndex.java b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/index/HashIndex.java
index 911fd6e..465c038 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/index/HashIndex.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/index/HashIndex.java
@@ -160,7 +160,7 @@ public class HashIndex extends AbstractIndex {
       }
     }
     
-    entriesSet = new HashIndexSet(entryToValuesMap, entryToOldKeysMap, internalIndexStats);
+    entriesSet = new HashIndexSet();
   }
 
   /**
@@ -233,17 +233,49 @@ public class HashIndex extends AbstractIndex {
             if (logger.isDebugEnabled()) { 
               logger.debug("A removed or invalid token was being added, and we had an old mapping.");
             }
-            entriesSet.remove(oldKey, entry, true);
+            removeFromEntriesSet(oldKey, entry, true);
           }
           return;
         }
       }
-      this.entriesSet.add(newKey, entry);
+      
+      // Before adding the entry with new value, remove it from reverse map and
+      // using the oldValue remove entry from the forward map.
+      // Reverse-map is used based on the system property
+      Object oldKey = getOldKey(entry);
+      
+      int indexSlot = this.entriesSet.add(newKey, entry);
+      
+      if (indexSlot >= 0) {
+        // Update the reverse map
+        if (IndexManager.isObjectModificationInplace()) {
+          this.entryToValuesMap.put(entry, newKey);
+        }
+        if (newKey != null && oldKey != null) {
+          removeFromEntriesSet(oldKey, entry, false, indexSlot);
+        }
+        // Update Stats after real addition
+        internalIndexStats.incNumValues(1);
+
+      }
     } catch (TypeMismatchException ex) {
       throw new IMQException("Could not add object of type "
           + key.getClass().getName(), ex);
     }
   }
+  
+  private Object getOldKey(RegionEntry entry) throws TypeMismatchException {
+    Object oldKey = null;
+    if (IndexManager.isObjectModificationInplace() && this.entryToValuesMap.containsKey(entry)) {
+      oldKey = this.entryToValuesMap.get(entry);
+    } else if (!IndexManager.isObjectModificationInplace() && this.entryToOldKeysMap != null) {
+      Map oldKeyMap = this.entryToOldKeysMap.get();
+      if (oldKeyMap != null) {
+        oldKey = TypeUtils.indexKeyFor(oldKeyMap.get(entry));
+      }
+    }
+    return oldKey;
+  }
 
   /**
    * @param opCode
@@ -293,15 +325,27 @@ public class HashIndex extends AbstractIndex {
     // space, but there is no way to ask an ArrayList what
     // it's current capacity is..so we trim after every
     // removal
-    boolean found = false;
     try {
       Object newKey = TypeUtils.indexKeyFor(key);
-      found = this.entriesSet.remove(newKey, entry, updateReverseMap);       
+      removeFromEntriesSet(newKey, entry, updateReverseMap);
     } catch (TypeMismatchException ex) {
       throw new IMQException("Could not add object of type "
           + key.getClass().getName(), ex);
     }
   }
+  
+  private void removeFromEntriesSet(Object newKey, RegionEntry entry, boolean updateReverseMap) {
+    removeFromEntriesSet(newKey, entry, updateReverseMap, -1);
+  }
+  
+  private void removeFromEntriesSet(Object newKey, RegionEntry entry, boolean updateReverseMap, int ignoreThisSlot) {
+    if (this.entriesSet.remove(newKey, entry, ignoreThisSlot)) {
+      if (updateReverseMap && IndexManager.isObjectModificationInplace()) {
+        entryToValuesMap.remove(entry);
+      }
+      internalIndexStats.incNumValues(-1);
+    }
+  }
 
   // // IndexProtocol interface implementation
   public boolean clear() throws QueryException {
@@ -628,8 +672,7 @@ public class HashIndex extends AbstractIndex {
   @Override
   void instantiateEvaluator(IndexCreationHelper ich) {
     this.evaluator = new IMQEvaluator(ich);
-    this.entriesSet.setHashIndexStrategy(new HashStrategy(
-        (IMQEvaluator) evaluator));
+    this.entriesSet.setEvaluator((HashIndex.IMQEvaluator)evaluator);
     this.comparator = ((IMQEvaluator) evaluator).comparator;
   }
 
@@ -672,9 +715,9 @@ public class HashIndex extends AbstractIndex {
       Object obj = entriesIter.next();
       Object key = null;
       if (obj != null && obj != HashIndexSet.REMOVED) {
-        
         RegionEntry re = (RegionEntry) obj;
         if (applyOrderBy) {
+          key = ((HashIndex.IMQEvaluator)evaluator).evaluateKey(obj);
           orderedKeys.add(new Object[]{key,i++});
           addValueToResultSet(re, orderedResults, iterOps, runtimeItr, context, projAttrib, intermediateResults, isIntersection, limit, observer, iteratorCreationTime);
         } else {
@@ -856,19 +899,22 @@ public class HashIndex extends AbstractIndex {
   void recreateIndexData() throws IMQException {
     // Mark the data maps to null & call the initialization code of index
     this.entriesSet.clear();
-	int numKeys = (int) this.internalIndexStats.getNumberOfKeys();
-	if (numKeys > 0) {
-		this.internalIndexStats.incNumKeys(-numKeys);
-	}
-	int numValues = (int) this.internalIndexStats.getNumberOfValues();
-	if (numValues > 0) {
-		this.internalIndexStats.incNumValues(-numValues);
-	}
-	int updates = (int) this.internalIndexStats.getNumUpdates();
-	if (updates > 0) {
-		this.internalIndexStats.incNumUpdates(updates);
-	}
-	this.initializeIndex(true);
+    if (IndexManager.isObjectModificationInplace()) {
+      entryToValuesMap.clear();
+    }
+    int numKeys = (int) this.internalIndexStats.getNumberOfKeys();
+    if (numKeys > 0) {
+      this.internalIndexStats.incNumKeys(-numKeys);
+    }
+    int numValues = (int) this.internalIndexStats.getNumberOfValues();
+    if (numValues > 0) {
+      this.internalIndexStats.incNumValues(-numValues);
+    }
+    int updates = (int) this.internalIndexStats.getNumUpdates();
+    if (updates > 0) {
+      this.internalIndexStats.incNumUpdates(updates);
+    }
+    this.initializeIndex(true);
   }
 
   public String dump() {
@@ -1435,7 +1481,7 @@ public class HashIndex extends AbstractIndex {
       return context;
     }
 
-    Object evaluateKey(Object object) {
+    public Object evaluateKey(Object object) {
       Object value = object;
       
       ExecutionContext newContext = null;
@@ -1459,6 +1505,10 @@ public class HashIndex extends AbstractIndex {
           logger.debug("Could not reevaluate key for hash index");
         }
       }
+      
+      if (key == null) {
+        key = IndexManager.NULL;
+      }
       return key;
     }
 
@@ -1471,8 +1521,8 @@ public class HashIndex extends AbstractIndex {
         //We check to see if an update was in progress.  If so (and is the way these turn to undefined),
         //the value is reevaluated and removed from the result set if it does not match the 
         //search criteria.  This occurs in addToResultsFromEntries()
-        Object key0 = ((Object[])arg0)[1];
-        Object key1 = ((Object[])arg1)[1];
+        Object key0 = ((Object[])arg0)[0];
+        Object key1 = ((Object[])arg1)[0];
       
         Comparable comp0 = (Comparable) key0;
         Comparable comp1 = (Comparable) key1;
@@ -1512,82 +1562,12 @@ public class HashIndex extends AbstractIndex {
       throws IMQException {
     // TODO Auto-generated method stub
   }
-
-  private class HashStrategy implements HashIndexStrategy {
-
-    private AttributeDescriptor attDesc;
-
-    private IMQEvaluator evaluator;
-    
-    public HashStrategy(IMQEvaluator evaluator) {
-      this.evaluator = evaluator;
-    }
-
-    public final int computeHashCode(Object o) {
-      return computeHashCode(o, false);
-    }
-
-    public final int computeHashCode(Object o, boolean reevaluateKey) {
-      if (reevaluateKey) {
-        return computeKey(o).hashCode();
-      }
-      return o.hashCode();
-    }
-
-    public final Object computeKey(Object o) {
-      Object key = evaluator.evaluateKey(o);
-      if (key == null) {
-        key = IndexManager.NULL;
-      }
-      return key;
-    }
-
-    public final boolean equalsOnAdd(Object o1, Object o2) {
-      if (o1 == null) {
-        return o2 == null;
-      }
-      try {
-        return TypeUtils.compare(o1, o2, OQLLexerTokenTypes.TOK_EQ).equals(Boolean.TRUE);
-      }
-      catch (TypeMismatchException e) {
-        return o1.equals(o2);
-      }
-    }
-
-    /*
-     * expects object o to be a region entry
-     */
-    public boolean equalsOnGet(Object indexKey, Object o) {
-      Object fieldValue = evaluator.evaluateKey(o);
-     
-      if (fieldValue == null && indexKey == IndexManager.NULL) {
-        return true;
-      } else {
-        try {
-          if (fieldValue instanceof PdxString) {
-           if (indexKey instanceof String) {
-             fieldValue = ((PdxString) fieldValue).toString(); 
-           }
-         }
-         else if (indexKey instanceof PdxString) {
-           if (fieldValue instanceof String) {
-             fieldValue = new PdxString((String)fieldValue);
-           }
-         }
-         return TypeUtils.compare(fieldValue, indexKey, OQLLexerTokenTypes.TOK_EQ).equals(Boolean.TRUE);
-        }
-        catch (TypeMismatchException e) {
-          return fieldValue.equals(indexKey);
-        }
-      }
-    }
-  }
-
+  
   public boolean isEmpty() {
     return entriesSet.isEmpty();
   }
   
-  public String printAll() {
-    return this.entriesSet.printAll();
-  }
+//  public String printAll() {
+//    return this.entriesSet.printAll();
+//  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/abad018a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/index/HashIndexSet.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/index/HashIndexSet.java b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/index/HashIndexSet.java
index 85887d4..2fa72d1 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/index/HashIndexSet.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/index/HashIndexSet.java
@@ -14,8 +14,29 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
+/*           
+ * insertionIndex(), index(), trimToSize() are based on code provided by fastutil
+ * They are based from add(), contains() and other methods from ObjectOpenHashSet
+ * We have used the traversing mechanism and the HashCommon.mix()
+ * Copyright (C) 2002-2014 Sebastiano Vigna 
+ *
+ * Licensed 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.cache.query.internal.index;
 
+import static it.unimi.dsi.fastutil.HashCommon.arraySize;
+import it.unimi.dsi.fastutil.HashCommon;
 import it.unimi.dsi.fastutil.objects.Object2ObjectOpenHashMap;
 
 import java.util.Collection;
@@ -24,27 +45,23 @@ import java.util.Iterator;
 import java.util.Map;
 import java.util.NoSuchElementException;
 import java.util.Set;
-import java.util.concurrent.ConcurrentMap;
 
 import com.gemstone.gemfire.cache.query.TypeMismatchException;
-import com.gemstone.gemfire.cache.query.internal.AttributeDescriptor;
 import com.gemstone.gemfire.cache.query.internal.index.AbstractIndex.InternalIndexStatistics;
+import com.gemstone.gemfire.cache.query.internal.parse.OQLLexerTokenTypes;
 import com.gemstone.gemfire.cache.query.internal.types.TypeUtils;
 import com.gemstone.gemfire.internal.cache.CachePerfStats;
 import com.gemstone.gemfire.internal.cache.CachedDeserializable;
 import com.gemstone.gemfire.internal.cache.RegionEntry;
 import com.gemstone.gemfire.internal.offheap.StoredObject;
-import com.gemstone.gemfire.internal.util.ObjectProcedure;
-import com.gemstone.gemfire.internal.util.PrimeFinder;
+import com.gemstone.gemfire.pdx.internal.PdxString;
 
 /**
- * An implementation of the <tt>Set</tt> interface that uses an open-addressed
- * hash table to store its contents.
- * 
- * On collisions, will store contents in an IndexElemArray and once a threshold
- * has been hit, will store in ConcurrentHashSets.
+ * An implementation of the <tt>Set</tt> interface for the HashIndex
+ * Not exactly a set as the hash keys can actually collide but will
+ * continue to look for an empty location to store the value
+ *
  */
-
 public class HashIndexSet implements Set {
 
   /**
@@ -53,113 +70,59 @@ public class HashIndexSet implements Set {
    */
   private transient CachePerfStats cacheStats;
 
-  /** the current number of occupied slots in the hash. */
+  /** the current number of entries in the set */
   protected transient int _size;
-  
-  /** the current number of free slots in the hash. */
+
+  /** the current number of open slots in the hash.
+   * Originally used when we collapsed collided keys into collections
+   * Not really used now */
   protected transient int _free;
-  
-  /** the current number of occupied slots in the hash. */
-  protected transient int _removedTokens;
+
+  /** number of removed tokens in the set, these are index positions that may be reused*/
+  transient int _removedTokens;
 
   /** the load above which rehashing occurs. */
   protected static final float DEFAULT_LOAD_FACTOR = 0.5f;
 
-  /**
-   * the default initial capacity for the hash table. This is one less than a
-   * prime value because one is added to it when searching for a prime capacity
-   * to account for the free slot required by open addressing. Thus, the real
-   * default capacity is 11.
-   */
-  protected static final int DEFAULT_INITIAL_CAPACITY = 100;
+  protected static final int DEFAULT_INITIAL_CAPACITY = 128;
 
-  /**
-   * Determines how full the internal table can become before rehashing is
-   * required. This must be a value in the range: 0.0 < loadFactor < 1.0. The
-   * default value is 0.5, which is about as large as you can get in open
-   * addressing without hurting performance. Cf. Knuth, Volume 3., Chapter 6.
-   */
   protected float _loadFactor;
+  
+  /** size of the backing table (-1)**/
+  protected int n;
+  
+  /** used for hashing into the table**/
+  protected int _mask;
 
   /**
-   * The maximum number of elements allowed without allocating more space.
+   * The maximum number of elements before rehashing
    */
   protected int _maxSize;
 
-  protected static final int CONDITIONAL_COMPACT_FACTOR = 2;
-  
-  //If after an update, the number of removed tokens X percent of the max size,
-  //we will compact and rehash to remove the tokens.
+  /** If after an update, the number of removed tokens X percent of the max size,
+    * we will compact and rehash to remove the tokens.
+    */
   protected static final float CONDITIONAL_REMOVED_TOKEN_REHASH_FACTOR = .7f;
 
   /** the set of Objects */
   protected transient Object[] _set;
 
-  /** the strategy used to hash objects in this collection. */
-  protected HashIndexStrategy _hashingStrategy;
+  protected HashIndex.IMQEvaluator _imqEvaluator;
 
+ /**
+  * The removed token
+  */
   protected static final Object REMOVED = new Object();
-  
+
   static boolean TEST_ALWAYS_REHASH = false;
 
   /**
-   * Map for RegionEntries=>value of indexedExpression (reverse map)
+   * This is used when inplace modification is off to detect old key
    */
-  private ConcurrentMap<Object, Object> entryToValuesMap;
-  protected ThreadLocal<Object2ObjectOpenHashMap> entryToOldKeysMap;
-  protected InternalIndexStatistics internalIndexStats;
-  
-  private AttributeDescriptor attDesc;
 
-  /**
-   * Creates a new <code>HashIndexSet</code> instance with the default capacity
-   * and load factor.
-   */
   public HashIndexSet() {
     this(DEFAULT_INITIAL_CAPACITY, DEFAULT_LOAD_FACTOR);
-  }
-
-  public HashIndexSet(ConcurrentMap reverseMap, ThreadLocal<Object2ObjectOpenHashMap> entryToOldKeysMap, InternalIndexStatistics internalIndexStats) {
-    this(DEFAULT_INITIAL_CAPACITY, DEFAULT_LOAD_FACTOR);
-    this.entryToValuesMap = reverseMap;
-    this.entryToOldKeysMap = entryToOldKeysMap;
-    this.internalIndexStats = internalIndexStats;
-  }
-  
-  /**
-   * Creates a new <code>HashIndexSet</code> instance with the default capacity
-   * and load factor.
-   * 
-   * @param strategy used to compute hash codes and to compare objects.
-   */
-  public HashIndexSet(HashIndexStrategy strategy) {
-    this(DEFAULT_INITIAL_CAPACITY, DEFAULT_LOAD_FACTOR);
-    this._hashingStrategy = strategy;
-  }
-
-  /**
-   * Creates a new <code>HashIndexSet</code> instance with a prime capacity
-   * equal to or greater than <tt>initialCapacity</tt> and with the default load
-   * factor.
-   * 
-   * @param initialCapacity
-   *          an <code>int</code> value
-   */
-  public HashIndexSet(int initialCapacity) {
-    this(initialCapacity, DEFAULT_LOAD_FACTOR);
-  }
 
-  /**
-   * Creates a new <code>HashIndexSet</code> instance with a prime capacity
-   * equal to or greater than <tt>initialCapacity</tt> and with the default load
-   * factor.
-   * 
-   * @param initialCapacity an <code>int</code> value
-   * @param strategy used to compute hash codes and to compare objects.
-   */
-  public HashIndexSet(int initialCapacity, HashIndexStrategy strategy) {
-    this(initialCapacity, DEFAULT_LOAD_FACTOR);
-    this._hashingStrategy = strategy;
   }
 
   /**
@@ -170,57 +133,12 @@ public class HashIndexSet implements Set {
    * @param initialCapacity an <code>int</code> value
    * @param loadFactor a <code>float</code> value
    */
-  public HashIndexSet(int initialCapacity, float loadFactor) {
-    _loadFactor = loadFactor;
-    setUp((int) Math.ceil(initialCapacity / loadFactor));
-  }
-
-  /**
-   * Creates a new <code>HashIndexSet</code> instance with a prime capacity
-   * equal to or greater than <tt>initialCapacity</tt> and with the specified
-   * load factor.
-   * 
-   * @param initialCapacity
-   *          an <code>int</code> value
-   * @param loadFactor
-   *          a <code>float</code> value
-   * @param strategy
-   *          used to compute hash codes and to compare objects.
-   */
-  public HashIndexSet(int initialCapacity, float loadFactor,
-      HashIndexStrategy strategy) {
-    this(initialCapacity, loadFactor);
-    this._hashingStrategy = strategy;
-  }
-
-  /**
-   * Creates a new <code>HashIndexSet</code> instance containing the elements of
-   * <tt>collection</tt>.
-   * 
-   * @param collection
-   *          a <code>Collection</code> value
-   */
-  public HashIndexSet(Collection collection) {
-    this(collection.size());
-    addAll(collection);
-  }
-
-  /**
-   * Creates a new <code>HashIndexSet</code> instance containing the elements of
-   * <tt>collection</tt>.
-   * 
-   * @param collection
-   *          a <code>Collection</code> value
-   * @param strategy
-   *          used to compute hash codes and to compare objects.
-   */
-  public HashIndexSet(Collection collection, HashIndexStrategy strategy) {
-    this(collection.size(), strategy);
-    addAll(collection);
+  private HashIndexSet(int initialCapacity, float loadFactor) {
+    setUp(initialCapacity , loadFactor);
   }
 
-  public void setHashIndexStrategy(HashIndexStrategy hashingStrategy) {
-    this._hashingStrategy = hashingStrategy;
+  public void setEvaluator(HashIndex.IMQEvaluator evaluator) {
+    this._imqEvaluator = evaluator;
   }
 
   /**
@@ -241,71 +159,62 @@ public class HashIndexSet implements Set {
   public boolean contains(Object obj) {
     return index(obj) >= 0;
   }
-  
+
   /**
-   * @param object can be either a region entry or index key
-   * @param recomputeKey
-              whether the object is a region entry and needs to have the key recomputed
+   * @param object is the index key
    * @return the hash key
    */
-  
-  private int computeHash(Object object, boolean recomputeKey) {
-    return _hashingStrategy.computeHashCode(object, recomputeKey) & 0x7fffffff;
+
+  private int computeHash(Object object) {
+    return object.hashCode();
   }
 
   /**
    * Locates the index of <tt>obj</tt>.
    * 
-   * @param obj an <code>Object</code> value, expected to be a
+   * @param obj an <code>Object</code> value, expected to be the value object
    * @return the index of <tt>obj</tt> or -1 if it isn't in the set.
    */
   protected int index(Object obj) {
-    return index(_hashingStrategy.computeKey(obj), obj);
+    return index(_imqEvaluator.evaluateKey(obj), obj);
   }
-  
+
   protected int index(Object key, Object obj) {
     return index(key, obj, -1);
   }
-  
+
   /**
    * Locates index slot of object using the provided key (in this case we are passing in old key)
+   * 
    * @param key
    * @param obj
    * @return the indexSlot of the given key/object combination
    */
   protected int index(Object key, Object obj, int ignoreThisSlot) {
-    int hash, probe, index, length;
+    int hash;
+    int pos;
     Object[] set;
-    Object cur;
-
+    int mask = _mask;
     set = _set;
-    length = set.length;
-    hash = computeHash(key, false);
-    index = hash % length;
-    cur = set[index];
-    
-    long start = -1L;
-    if (this.cacheStats != null) {
-      start = this.cacheStats.getStatTime();
-    }
-    //Find the correct collection that matches key of the object we are looking for
-    //if one exists, we then look to see if the element exists in the collection
-    //If a collection is not correct, we probe for the next collection until a null is found
-    while (cur != null) {
-      if (cur != REMOVED && index != ignoreThisSlot) {
-        if (cur instanceof RegionEntry) {
-          if (_hashingStrategy.equalsOnAdd(obj, cur)) {
-            return index;
-          }
-        }
-      }
-
-      probe = 1 + (hash % (length - 2));
-      index -= probe;
-      if (index < 0) {
-        index += length;
+    Object curr;
+    hash = computeHash(key);
+
+    /* Code originated from fastutils
+     * Copyright (C) 2002-2014 Sebastiano Vigna
+     * 
+     * Licensed 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 
+     */
+    if (!((curr = set[pos = (it.unimi.dsi.fastutil.HashCommon.mix(hash)) & mask]) == null || curr == REMOVED)) {
+      if (((curr).equals(obj) && pos != ignoreThisSlot))
+        return pos;
+      while (!((curr = set[pos = (pos + 1) & mask]) == null || curr == REMOVED)) {
+        if (((curr).equals(obj)) && pos != ignoreThisSlot)
+          return pos;
       }
-      cur = set[index];
     }
     return -1;
   }
@@ -313,18 +222,17 @@ public class HashIndexSet implements Set {
   public Iterator getAll() {
     return getAllNotMatching(Collections.EMPTY_LIST);
   }
-  
+
   public Iterator getAllNotMatching(Collection keysToRemove) {
     return new HashIndexSetIterator(keysToRemove, _set);
   }
-  
+
   /**
    * Locates the index of <tt>obj</tt>.
    * 
    * @param indexKey
    *          an <code>Object</code> value that represents the index key
-   * @return the a collection of objects that match the index key or an empty
-   *         collection if none match
+   * @return Iterator over a collection of objects that match the key
    */
   public Iterator get(Object indexKey) {
     return new HashIndexSetIterator(indexKey, _set);
@@ -332,7 +240,7 @@ public class HashIndexSet implements Set {
 
   /**
    * 
-   * @param set the array that all elements are stored in
+   * @param set represents the array that all elements are stored in
    * @param index
    *          the array index location to store the object. This should be
    *          calculated by one of the insertionIndex methods
@@ -342,60 +250,36 @@ public class HashIndexSet implements Set {
   private boolean addObjectToSet(Object[] set, int index, Object newObject) {
     boolean added = true;
     if (index < 0) {
-      throwObjectContractViolation(set[(-index - 1)], newObject);
+      throw new ArrayIndexOutOfBoundsException("Cannot add:" + newObject + " into array position:" + index);
     }
     Object oldObject = set[index];
     if (oldObject == null || oldObject == REMOVED) {
       set[index] = newObject;
-    } else if (oldObject instanceof RegionEntry) {
-      IndexElemArray elemArray = new IndexElemArray();
-      elemArray.add(oldObject);
-      elemArray.add(newObject);
-      set[index] = elemArray;
-    } else if (oldObject instanceof IndexConcurrentHashSet) {
-      added = ((IndexConcurrentHashSet) oldObject).add(newObject);
-    } else if (oldObject instanceof IndexElemArray) {
-      IndexElemArray elemArray = (IndexElemArray) oldObject;
-      if (elemArray.size() >= IndexManager.INDEX_ELEMARRAY_THRESHOLD) {
-        IndexConcurrentHashSet newSet = new IndexConcurrentHashSet(
-            IndexManager.INDEX_ELEMARRAY_THRESHOLD + 20, 0.75f, 1);
-        newSet.addAll(elemArray);
-        newSet.add(newObject);
-        set[index] = newSet;
-      } else {
-        elemArray.add(newObject);
-      }
-    }
+    } 
 
     return added;
   }
 
   /**
-   * Inserts a value into the set.
-   * 
-   * @param obj an <code>Object</code> value
-   * @return true if the set was modified by the add operation
+   * Unsupported, we do not use HashIndexSet as a general all purpose set
    */
-  public synchronized boolean add(Object obj){
-    throw new UnsupportedOperationException(
-        "add(Object) not supported, try add(Object key, Object obj) instead");
+  public synchronized boolean add(Object obj) {
+    throw new UnsupportedOperationException("add(Object) not supported, try add(Object key, Object obj) instead");
   }
 
-  public synchronized boolean add(Object indexKey, Object obj) throws TypeMismatchException {   
-    // Before adding the entry with new value, remove it from reverse map and
-    // using the oldValue remove entry from the forward map.
-    // Reverse-map is used based on the system property
-    Object oldKey = null;
-    if (IndexManager.isObjectModificationInplace() && this.entryToValuesMap.containsKey(obj)){
-        oldKey = this.entryToValuesMap.get(obj);
-    }
-    else if (!IndexManager.isObjectModificationInplace() && this.entryToOldKeysMap != null) {
-      Map oldKeyMap = this.entryToOldKeysMap.get();
-      if (oldKeyMap != null) {
-        oldKey = TypeUtils.indexKeyFor(oldKeyMap.get(obj));        
-      }
+  /**
+   * Add an object using the hash value of the provided indexKey
+   * 
+   * @param indexKey 
+   * @param obj the object to add
+   * @return true if object has been added
+   * @throws TypeMismatchException
+   */
+  public synchronized int add(Object indexKey, Object obj) throws TypeMismatchException {
+    if (indexKey == null) {
+      indexKey = IndexManager.NULL;
     }
-    // Note we cannot make the optimization for hash index.  Due to in place modification
+    // Note we cannot make the below optimization for hash index. Due to in place modification
     // where old key == new key (when no reverse map) we end up not updating to the correct slot in this case
     // If oldKey and the newKey are same there is no need to update the
     // index-maps.
@@ -403,132 +287,70 @@ public class HashIndexSet implements Set {
     // || indexKey.equals(oldKey)) {
     // return false;
     // }
-    
-    //grow/shrink capacity if needed
+
+    // grow/shrink capacity if needed
     preInsertHook();
     int indexSlot = insertionIndex(indexKey, obj, _set);
-    if (indexSlot < 0) {
-      return false; // already present in set, nothing to add
-    }
 
     Object old = _set[indexSlot];
-    boolean added = addObjectToSet(_set, indexSlot, obj);
-    
-    if (added) {
-      //Update the reverse map
-      if ( IndexManager.isObjectModificationInplace()) {
-        this.entryToValuesMap.put(obj, indexKey);
-      }
-      if (indexKey != null && oldKey != null) {
-        remove(oldKey, obj, false, indexSlot);
-      }
-      // Update Stats after real addition
-      internalIndexStats.incNumValues(1);
-    }
-    
+    addObjectToSet(_set, indexSlot, obj);
+
     // only call this now if we are adding to an actual empty slot, otherwise we
     // have reused
     // and inserted into a set or array
     if (old == null) {
       postInsertHook(true);
-    }
-    else {
+    } else {
       postInsertHook(false);
     }
-    return added; // yes, we added something
+    return indexSlot; // yes, we added something
   }
 
   /**
-   * Locates the index at which <tt>obj</tt> can be inserted. if there is
-   * already a value equal()ing <tt>obj</tt> in the set, returns that value's
-   * index as <tt>-index - 1</tt>.
+   * Locates the next available insertion index for the provided indexKey and set
    * 
    * @param obj
    *          an <code>Object</code> value
-   * @return the index of a FREE slot at which obj can be inserted or, if obj is
-   *         already stored in the hash, the negative value of that index, minus
-   *         1: -index -1.
-   */
-  protected int insertionIndex(Object obj) {
-    return insertionIndex(_hashingStrategy.computeKey(obj), obj, _set);
-  }
-  
-  protected int insertionIndex(Object obj, Object[] set) {
-    return insertionIndex(_hashingStrategy.computeKey(obj), obj, set);
-  }
-  
-  
-  /**
-   * Locates the index at which <tt>obj</tt> can be inserted. if there is
-   * already a value equal()ing <tt>obj</tt> in the set, returns that value's
-   * index as <tt>-index - 1</tt>.
-   * 
-   * @param obj
-   *          an <code>Object</code> value
-   * @return the index of a FREE slot at which obj can be inserted or, if obj is
-   *         already stored in the hash, the negative value of that index, minus
-   *         1: -index -1.
+   * @return the index of an open or resused position
    */
   protected int insertionIndex(Object indexKey, Object obj, Object[] set) {
-    int hash, probe, indexSlot, length;
-    Object cur;
-
-    length = set.length;
-    hash = computeHash(indexKey, false);
-    indexSlot = hash % length;
-
-    cur = set[indexSlot];
+    int hash;
+    int pos;
+    int mask = _mask;
+    Object curr;
+    final Object[] array = set;
+    hash = computeHash(indexKey);
 
-    if (cur == null) {
-      return indexSlot; // empty, all done
-    }
-
-    // Getting here means we have yet to find the correct key collection
-    // so we must find the double hash
     long start = -1L;
     if (this.cacheStats != null) {
       start = this.cacheStats.getStatTime();
       this.cacheStats.incQueryResultsHashCollisions();
     }
     try {
-
-      // compute the double hash
-      probe = 1 + (hash % (length - 2));
-      // if the slot we landed on is FULL (but not removed), probe
-      // until we find an empty slot, a REMOVED slot, or an element
-      // equal to the one we are trying to insert.
-      // finding an empty slot means that the value is not present
-      // and that we should use that slot as the insertion point;
-      // finding a REMOVED slot means that we need to keep searching,
-      // however we want to remember the offset of that REMOVED slot
-      // so we can reuse it in case a "new" insertion (i.e. not an update)
-      // is possible.
-      // finding a matching value means that we've found that our desired
-      // key is already in the table
-      if (cur != REMOVED) {
-        // starting at the natural offset, probe until we find an
-        // offset that isn't full.
-        do {
-
-          indexSlot -= probe;
-          if (indexSlot < 0) {
-            indexSlot += length;
-          }
-          cur = set[indexSlot];
-        } while (cur != null && cur != REMOVED);
+      /* Code originated from fastutils
+       * Copyright (C) 2002-2014 Sebastiano Vigna
+       * 
+       * Licensed 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 
+       */
+      if (!((curr = array[pos = (it.unimi.dsi.fastutil.HashCommon.mix(hash)) & mask]) == null || curr == REMOVED)) {
+        while (!((curr = array[pos = (pos + 1) & mask]) == null || curr == REMOVED)) {
+        }
       }
-      return indexSlot;
+      return pos;
     } finally {
       if (this.cacheStats != null) {
         this.cacheStats.endQueryResultsHashCollisionProbe(start);
       }
     }
   }
-
+  
   @Override
-  // GemStoneAddition
   public boolean equals(Object other) {
-    if (!(other instanceof Set)) {
+    if (!(other instanceof HashIndexSet)) {
       return false;
     }
     Set that = (Set) other;
@@ -539,164 +361,82 @@ public class HashIndexSet implements Set {
   }
 
   @Override
-  // GemStoneAddition
   public int hashCode() {
-    HashProcedure p = new HashProcedure();
-    forEach(p);
-    return p.getHashCode();
-  }
-
-  /**
-   * Executes <tt>procedure</tt> for each element in the set.
-   * 
-   * @param procedure
-   *          a <code>TObjectProcedure</code> value
-   * @return false if the loop over the set terminated because the procedure
-   *         returned false for some value.
-   */
-  public boolean forEach(ObjectProcedure procedure) {
+    int hash = 0;
     Object[] set = _set;
     for (int i = set.length; i-- > 0;) {
-      if (set[i] != null && set[i] != REMOVED && !procedure.executeWith(set[i])) {
-        return false;
+      if (set[i] != null && set[i] != REMOVED) {
+        hash += set[i].hashCode();
       }
     }
-    return true;
-  }
-
-  protected/* GemStoneAddition */final class HashProcedure implements
-      ObjectProcedure {
-    private int h = 0;
-
-    public int getHashCode() {
-      return h;
-    }
-
-    public final boolean executeWith(Object key) {
-      h += _hashingStrategy.computeHashCode(key);
-      return true;
-    }
+    return hash;
   }
 
   /**
-   * Expands the set to accomodate new values.
+   * Expands or contracts a set to the new specified n.
    * 
-   * @param newCapacity
-   *          an <code>int</code> value
+   * @param newN the expected size
    */
-  // GemStoneAddition
-  protected void rehash(int newCapacity) {
+  protected void rehash(int newN) {
     if (TEST_ALWAYS_REHASH) {
-        Thread.yield();
+      Thread.yield();
     }
     int oldCapacity = _set.length;
     Object[] oldSet = _set;
-    
-    Object[] newSet = new Object[newCapacity];
+
     _removedTokens = 0;
-    //adds/removes/rehash should all be synchronized by the hashindex
-    //we are ok to clear this map and repopulate
-    //we do not do this for _set because we could still be querying 
-    //but the reversemap is only used for adds/removes/rehash
-    if (IndexManager.isObjectModificationInplace()) {
-      entryToValuesMap.clear();
-    }
+    
+    n = newN;
+    _mask = n - 1;
+    _maxSize = computeMaxSize(n, _loadFactor);
+    _free = computeNumFree();
+    Object[] newSet = new Object[n + 1];
+    
     for (int i = oldCapacity; i-- > 0;) {
       if (oldSet[i] != null && oldSet[i] != REMOVED) {
         Object o = oldSet[i];
 
-        if (o instanceof RegionEntry) {
-          Object key = _hashingStrategy.computeKey(o);
-          if (key == null) {
-            key = IndexManager.NULL;
-          }
-          int index = insertionIndex(key, o, newSet);
-          if (index >= 0)
-            if (addObjectToSet(newSet, index, o)) {
-              updateReverseMap(o, key);
-            }
-        } 
+        Object key = _imqEvaluator.evaluateKey(o);
+        if (key == null) {
+          key = IndexManager.NULL;
+        }
+        int index = insertionIndex(key, o, newSet);
+        if (index >= 0) {
+          addObjectToSet(newSet, index, o);
+        }
       }
     }
     _set = newSet;
   }
-  
-  private void updateReverseMap(Object regionEntry, Object key) {
-    if (IndexManager.isObjectModificationInplace()) {
-      entryToValuesMap.put(regionEntry, key);
-    }
-  }
 
-  /**
-   * Convenience methods for subclasses to use in throwing exceptions about
-   * badly behaved user objects employed as keys. We have to throw an
-   * IllegalArgumentException with a rather verbose message telling the user
-   * that they need to fix their object implementation to conform to the general
-   * contract for java.lang.Object.
-   * 
-   * @param o1
-   *          the first of the equal elements with unequal hash codes.
-   * @param o2
-   *          the second of the equal elements with unequal hash codes.
-   * @exception IllegalArgumentException
-   *              the whole point of this method.
-   */
-  protected final void throwObjectContractViolation(Object o1, Object o2)
-      throws IllegalArgumentException {
-    throw new IllegalArgumentException(
-        "Equal objects must have equal hashcodes. "
-            + "During rehashing, Trove discovered that "
-            + "the following two objects claim to be "
-            + "equal (as in java.lang.Object.equals()) "
-            + "but their hashCodes (or those calculated by "
-            + "your HashIndexStrategy) are not equal."
-            + "This violates the general contract of "
-            + "java.lang.Object.hashCode().  See bullet point two "
-            + "in that method's documentation. " + "object #1 ="
-            + objToString(o1) + "; object #2 =" + objToString(o2));
-  }
-
-  private static String objToString(Object o) {
-    if (o instanceof Object[]) {
-      return java.util.Arrays.toString((Object[]) o);
-    } else {
-      return String.valueOf(o);
-    }
-  }
 
   /**
-   * Returns a new array containing the objects in the set.
+   * Unsupported as the hash index does not use this method call
    * 
    * @return an <code>Object[]</code> value
    */
   public Object[] toArray() {
-    throw new UnsupportedOperationException(
-        "toArray not yet supported");
+    throw new UnsupportedOperationException("toArray not yet supported");
   }
 
   /**
-   * Returns a typed array of the objects in the set.
-   * 
+   * Unsupported as the hash index does not use this method call
    * @param a
    *          an <code>Object[]</code> value
    * @return an <code>Object[]</code> value
    */
   public Object[] toArray(Object[] a) {
-    throw new UnsupportedOperationException(
-        "toArray(Object[] a) not yet supported");
+    throw new UnsupportedOperationException("toArray(Object[] a) not yet supported");
   }
 
   /**
    * Empties the set.
    */
-  // GemStoneAddition
   public void clear() {
     _size = 0;
     _free = capacity();
     _removedTokens = 0;
-    if (IndexManager.isObjectModificationInplace()) {
-      entryToValuesMap.clear();
-    }
+
     Object[] set = _set;
 
     for (int i = set.length; i-- > 0;) {
@@ -708,66 +448,58 @@ public class HashIndexSet implements Set {
     return _set.length;
   }
 
-  /**
-   * Removes <tt>obj</tt> from the set.  
-   * Currently not implemented correctly, use {@link HashIndexSet#remove(Object, Object, boolean)}
-   * 
-   * @param obj an <code>Object</code> value
-   * @return true if the set was modified by the remove operation.
-   */
-  public boolean remove(Object obj) {
 
-    throw new UnsupportedOperationException(
-        "remove(Object) not supported, try remove(Object key, Object obj) instead");
+  public boolean remove(Object obj) {
+    return remove(_imqEvaluator.evaluateKey(obj), obj);
   }
-  
-  
-  public synchronized boolean remove(Object key, Object obj, boolean updateReverseMap) {
-    return remove(key, obj, updateReverseMap, -1);
+
+  public synchronized boolean remove(Object key, Object obj) {
+    return remove(key, obj, -1);
   }
-  
+
   /**
    * 
    * @param key assumed to not be null, rather needs to be NULL token
    * @param obj
-   * @param updateReverseMap
    * @param newIndexSlot if inplace modification occurs with out having a reversemap
-   *  we end up scanning the entire index.  We want to remove the region entry from the index slot
-   *  but not the newly added (correct) slot.  Rather only the "old/wrong" slot
+   *          we end up scanning the entire index. We want to remove the region entry from the index slot
+   *          but not the newly added (correct) slot. Rather only the "old/wrong" slot
    * @return true if object was removed, false otherwise
    */
-  public synchronized boolean remove(Object key, Object obj, boolean updateReverseMap, int newIndexSlot) {
+  public synchronized boolean remove(Object key, Object obj, int newIndexSlot) {
     int indexSlot = index(key, obj, newIndexSlot);
     boolean removed = false;
-    //The check for newIndexSlot != indexSlot is incase of in place modification.
-    //When inplace occurs, oldkey == newkey and we end up wiping out the "new key" slow rather
-    //than the old key slot.  Instead let's get to the else portion
+    // The check for newIndexSlot != indexSlot is incase of in place modification.
+    // When inplace occurs, oldkey == newkey and we end up wiping out the "new key" slow rather
+    // than the old key slot. Instead let's get to the else portion
     if (indexSlot >= 0 && indexSlot != newIndexSlot) {
       removed = removeAt(indexSlot);
-      if (removed) {
-        if (updateReverseMap && IndexManager.isObjectModificationInplace()) {
-          entryToValuesMap.remove(obj);
-        }
-        internalIndexStats.incNumValues(-1);
-      }
       return removed;
-    }
-    else if (!IndexManager.isObjectModificationInplace()){
-      //object could not be found so it's possible there was an inplace modification
-        HashIndexSetIterator iterator = (HashIndexSetIterator)getAll();
-        while (iterator.hasNext()) {
-          Object indexedObject = iterator.next();
-          if (_hashingStrategy.equalsOnAdd(indexedObject, obj) && iterator.currentObjectIndex() != newIndexSlot) {
-            iterator.remove();
-            internalIndexStats.incNumValues(-1);
-            return true;
-          }
+    } else if (!IndexManager.isObjectModificationInplace()) {
+      // object could not be found so it's possible there was an inplace modification
+      HashIndexSetIterator iterator = (HashIndexSetIterator) getAll();
+      while (iterator.hasNext()) {
+        Object indexedObject = iterator.next();
+        if (areObjectsEqual(indexedObject, obj) && iterator.currentObjectIndex() != newIndexSlot) {
+          iterator.remove();
+          return true;
         }
+      }
     }
     return false;
   }
   
-  
+  public final boolean areObjectsEqual(Object o1, Object o2) {
+    if (o1 == null) {
+      return o2 == null;
+    }
+    try {
+      return TypeUtils.compare(o1, o2, OQLLexerTokenTypes.TOK_EQ).equals(Boolean.TRUE);
+    }
+    catch (TypeMismatchException e) {
+      return o1.equals(o2);
+    }
+  }
 
   /**
    * Creates an iterator over the values of the set. The iterator supports
@@ -780,11 +512,10 @@ public class HashIndexSet implements Set {
   }
 
   /**
-   * Tests the set to determine if all of the elements in <tt>collection</tt>
-   * are present.
+   * Determine if all of the elements in <tt>collection</tt> are present.
    * 
    * @param collection a <code>Collection</code> value
-   * @return true if all elements were present in the set.
+   * @return true if all elements are present.
    */
   public boolean containsAll(Collection collection) {
     for (Iterator i = collection.iterator(); i.hasNext();) {
@@ -796,25 +527,10 @@ public class HashIndexSet implements Set {
   }
 
   /**
-   * Adds all of the elements in <tt>collection</tt> to the set.
-   * 
-   * @param collection a <code>Collection</code> value
-   * @return true if the set was modified by the add all operation.
+   * Unsupported because type mismatch exception cannot be thrown from Set interface
    */
   public boolean addAll(Collection collection) {
-    boolean changed = false;
-    int size = collection.size();
-    Iterator it;
-
-    ensureCapacity(size);
-    it = collection.iterator();
-    while (size-- > 0) {
-      Object obj = it.next();
-      if (obj != null && add(obj)) {
-        changed = true;
-      }
-    }
-    return changed;
+    throw new UnsupportedOperationException("Add all not implemented");
   }
 
   /**
@@ -826,9 +542,8 @@ public class HashIndexSet implements Set {
   public boolean removeAll(Collection collection) {
     boolean changed = false;
     int size = collection.size();
-    Iterator it;
 
-    it = collection.iterator();
+    Iterator it = collection.iterator();
     while (size-- > 0) {
       if (remove(it.next())) {
         changed = true;
@@ -838,8 +553,7 @@ public class HashIndexSet implements Set {
   }
 
   /**
-   * Removes any values in the set which are not contained in
-   * <tt>collection</tt>.
+   * Removes any values in the set which are not contained in <tt>collection</tt>.
    * 
    * @param collection a <code>Collection</code> value
    * @return true if the set was modified by the retain all operation
@@ -847,11 +561,11 @@ public class HashIndexSet implements Set {
   public boolean retainAll(Collection collection) {
     boolean changed = false;
     int size = size();
-    Iterator it;
 
-    it = iterator();
-    while (size-- > 0) {
-      if (!collection.contains(it.next())) {
+    Iterator it = iterator();
+    while (it.hasNext()) {
+      Object object = it.next();
+      if (!collection.contains(object)) {
         it.remove();
         changed = true;
       }
@@ -859,133 +573,54 @@ public class HashIndexSet implements Set {
     return changed;
   }
 
-
   @Override
-  // GemStoneAddition
-
   /**
-   * Tells whether this set is currently holding any elements.
-   * 
-   * @return a <code>boolean</code> value
+   * return true if no elements exist in the array that are non null or REMOVED tokens
    */
   public boolean isEmpty() {
     return 0 == _size;
   }
 
   /**
-   * Returns the number of slots used in the backing array
+   * Returns the number of positions used in the backing array
    * Is not a true representation of the number of elements in the array
+   * as the array may contain REMOVED tokens
    * 
    * @return an <code>int</code> value
    */
   public int size() {
     return _size;
   }
-  
-  public int size(Object indexKey) {
-    int hash, probe, index, length;
-    Object[] set;
-    Object cur;
-    int size = 0;
 
-    //find the first array index location
-    set = _set;
-    length = set.length;
-    hash = computeHash(indexKey, false);
-    index = hash % length;
-    cur = set[index];
-
-    if (cur == null) {
-      // return
-      return 0;
-    }
-
-    while (cur != null) {
-      if (cur != REMOVED) {
-        if (cur instanceof RegionEntry) {
-          if (_hashingStrategy.equalsOnGet(indexKey, cur)) {
-            size++;
-          }
-        }
-        break;
-      }
-      //If this is not the correct collection, one that does not match the key
-      //we are looking for, then continue our search
-      probe = 1 + (hash % (length - 2));
-      index -= probe;
-      if (index < 0) {
-        index += length;
-      }
-      cur = set[index];
-    }
-    return size;
-  }
-  
   /**
-   * Ensure that this hashtable has sufficient capacity to hold
-   * <tt>desiredCapacity<tt> <b>additional</b> elements without
-   * requiring a rehash.  This is a tuning method you can call
-   * before doing a large insert.
-   * 
-   * @param desiredCapacity an <code>int</code> value
+   * only used for query optimization.  Instead of crawling the entire array doing matches
+   * let's just return the size of the array as that is the worst case size
    */
-  public void ensureCapacity(int desiredCapacity) {
-    if (desiredCapacity > (_maxSize - size())) {
-      rehash(PrimeFinder.nextPrime((int) Math.ceil(desiredCapacity + size()
-          / _loadFactor) + 1));
-      computeMaxSize(capacity());
-    }
+  public int size(Object indexKey) {
+    return _size;
   }
 
   /**
-   * Compresses the hashtable to the minimum prime size (as defined by
-   * PrimeFinder) that will hold all of the elements currently in the table. If
-   * you have done a lot of <tt>remove</tt> operations and plan to do a lot of
-   * queries or insertions or iteration, it is a good idea to invoke this
-   * method. Doing so will accomplish two things:
-   * 
-   * <ol>
-   * <li>You'll free memory allocated to the table but no longer needed because
-   * of the remove()s.</li>
-   * 
-   * <li>You'll get better query/insert/iterator performance because there won't
-   * be any <tt>REMOVED</tt> slots to skip over when probing for indices in the
-   * table.</li>
-   * </ol>
+   * Compress the backing array if possible
    */
   public void compact() {
-    // need at least one free spot for open addressing
-    rehash(PrimeFinder.nextPrime((int) Math.ceil(size() / _loadFactor) + 1));
-    computeMaxSize(capacity());
+    trimToSize(_size);
   }
-
-  // GemStoneAddition
-  /**
-   * Calls compact by taking next set expansion into account. The set is
-   * expanded based on the capacity and load factor (default .5) this method
-   * calls the compact if the size is well below next expansion.
-   */
-  public void conditionalCompact() {
-    if (_size < (capacity() * (_loadFactor / CONDITIONAL_COMPACT_FACTOR))) {
-      compact();
+  
+  public boolean trimToSize( final int n ) {
+    final int l = HashCommon.nextPowerOfTwo( (int)Math.ceil( n / _loadFactor ) );
+    if ( this.n <= l ) return true;
+    try {
+            rehash( l );
     }
-  }
-
-  /**
-   * This simply calls {@link #compact compact}. It is included for symmetry
-   * with other collection classes. Note that the name of this method is
-   * somewhat misleading (which is why we prefer <tt>compact</tt>) as the load
-   * factor may require capacity above and beyond the size of this collection.
-   * 
-   * @see #compact
-   */
-  public final void trimToSize() {
-    compact();
-  }
+    catch ( OutOfMemoryError cantDoIt ) {
+            return false;
+    }
+    return true;
+}
 
   /**
-   * Delete the record at <tt>index</tt>. Reduces the size of the collection by
-   * one.
+   * Remove the object at <tt>index</tt>.
    * 
    * @param index an <code>int</code> value
    */
@@ -994,210 +629,136 @@ public class HashIndexSet implements Set {
     cur = _set[index];
 
     if (cur == null || cur == REMOVED) {
-      //nothing removed
+      // nothing removed
       return false;
     } else {
       _set[index] = REMOVED;
       _size--;
-      _removedTokens ++;
+      _removedTokens++;
       return true;
-    } 
+    }
   }
 
   /**
-   * initializes the hashtable to a prime capacity which is at least
-   * <tt>initialCapacity + 1</tt>.
-   * 
-   * @param initialCapacity an <code>int</code> value
-   * @return the actual capacity chosen
+   * initializes this index set
    */
-  protected int setUp(int initialCapacity) {
-    int capacity;
-    capacity = PrimeFinder.nextPrime(initialCapacity);
-    computeMaxSize(capacity);
-    _set = new Object[capacity];
-    return capacity;
+  protected int setUp(final int expectedCapacity, final float loadFactor) {
+    n = arraySize( expectedCapacity, loadFactor );
+    this._loadFactor = loadFactor;
+    _maxSize = computeMaxSize(n, loadFactor);
+    _mask = n - 1;
+    _free = computeNumFree();
+    _set = new Object[n + 1];
+    return n;
   }
-
-  /**
-   * Computes the values of maxSize. There will always be at least one free slot
-   * required.
-   * 
-   * @param capacity an <code>int</code> value
-   */
-  private final void computeMaxSize(int capacity) {
-    // need at least one free slot for open addressing
-    _maxSize = Math.min(capacity - 1, (int) Math.floor(capacity * _loadFactor));
-    _free = capacity - _size; // reset the free element count
+  
+  private int computeNumFree() {
+    return n - _size;
+  }
+  
+  private int computeMaxSize(int n, float loadFactor) {
+    return Math.min( (int)Math.ceil( n * loadFactor ), n - 1 );
   }
 
   /**
-   * After an insert, this hook is called to adjust the size/free values of the
-   * set and to perform rehashing if necessary.
+   * After insert, allows for calculating metadata
    */
   protected final void postInsertHook(boolean usedFreeSlot) {
     if (usedFreeSlot) {
       _free--;
-    }
-    else {
-      //we used a removeToken
+    } else {
+      // we used a removeToken
       _removedTokens--;
     }
     _size++;
   }
-  
+
+  /**
+   * Before inserting we can ensure we have capacity
+   */
   protected final void preInsertHook() {
- // rehash whenever we exhaust the available space in the table
     if (_size > _maxSize || _free == 0 || TEST_ALWAYS_REHASH) {
-      // choose a new capacity suited to the new state of the table
-      // if we've grown beyond our maximum size, double capacity;
-      // if we've exhausted the free spots, rehash to the same capacity,
-      // which will free up any stale removed slots for reuse.
-      int newCapacity = _size > _maxSize ? PrimeFinder
-          .nextPrime(capacity() << 1) : capacity();
-      rehash(newCapacity);
-      computeMaxSize(capacity());
-    }
-    else if (_removedTokens > _maxSize * CONDITIONAL_REMOVED_TOKEN_REHASH_FACTOR) {
+      rehash(arraySize(_size + 1, _loadFactor));
+      computeMaxSize(capacity(), _loadFactor);
+      _free = computeNumFree();
+    } else if (_removedTokens > _maxSize * CONDITIONAL_REMOVED_TOKEN_REHASH_FACTOR) {
       compact();
     }
   }
 
-  final class ToObjectArrayProcedure implements ObjectProcedure {
-    private final Object[] target;
-    private int pos = 0;
-
-    public ToObjectArrayProcedure(final Object[] target) {
-      this.target = target;
-    }
-
-    public final boolean executeWith(Object value) {
-      target[pos++] = value;
-      return true;
-    }
-  } // ToObjectArrayProcedure
-
-  public String printAll() {
-    StringBuffer s = new StringBuffer();
-    for (int i = 0; i < _set.length; i++) {
-      Object object = _set[i];
-      if (object != null && object != REMOVED) {
-        s.append("\n slot[" + i + "]:");
-        if (object instanceof Collection) {
-          for (Object o : ((Collection) object)) {
-            if (o != null) {
-              RegionEntry re = (RegionEntry) o;
-              Object val = re._getValue(); // OFFHEAP _getValue ok
-              if (val instanceof StoredObject) {
-                // We don't have enough info here to deserialize an off-heap value
-                // so we can't call getDeserializedForReading.
-                // Also we can't call _getValueRetain because we do not
-                // know what region to pass in to it.
-                // So for now we just convert it to a String which all StoredObject
-                // impls can do without needing a refcount or to decompress.
-                val = val.toString();
-              }
-              if (val instanceof CachedDeserializable) {
-                val = ((CachedDeserializable) val).getDeserializedForReading();
-              }
-              s.append(re.getKey() + " =>  " + val + " # ");
-            }
-          }
-        } else {
-          RegionEntry re = (RegionEntry) object;
-          Object val = re._getValue(); // OFFHEAP _getValue ok
-          if (val instanceof StoredObject) {
-            // We don't have enough info here to deserialize an off-heap value
-            // so we can't call getDeserializedForReading.
-            // Also we can't call _getValueRetain because we do not
-            // know what region to pass in to it.
-            // So for now we just convert it to a String which all StoredObject
-            // impls can do without needing a refcount or to decompress.
-            val = val.toString();
-          }
-          if (val instanceof CachedDeserializable) {
-            val = ((CachedDeserializable) val).getDeserializedForReading();
-          }
-          s.append(re.getKey() + " =>  " + val);
-        }
-      }
-    }
-    return s.toString();
-     }
-     
-  
   private class HashIndexSetIterator implements Iterator {
     private Object keyToMatch;
-    //objects at time of iterator creation
+    // objects at time of iterator creation
     private final Object[] objects;
-    private int indexSlot;
+    private int pos;
+    private int prevPos;
     private Collection keysToRemove;
     private Object current;
     private int hash;
-    private int length;
+    private int mask;
     private int probe;
-    
-    private HashIndexSetIterator(Collection keysToRemove, Object[] objects ) {
+
+    private HashIndexSetIterator(Collection keysToRemove, Object[] objects) {
       this.keysToRemove = keysToRemove;
-      this.indexSlot = 0;
+      this.pos = 0;
+      this.prevPos = 0;
       this.objects = objects;
-      current = objects[indexSlot];
+      current = objects[pos];
     }
-    
+
     private HashIndexSetIterator(Object keyToMatch, Object[] objects) {
       this.keyToMatch = keyToMatch;
       this.objects = objects;
-      
-      length = objects.length;
-      hash = computeHash(keyToMatch, false);
-      probe = 1 + (hash % (length - 2));
-      indexSlot = hash % length;
-      current = objects[indexSlot];
+
+      mask = _mask;
+      hash = computeHash(keyToMatch);
+      pos = (it.unimi.dsi.fastutil.HashCommon.mix(hash)) & mask;
+      prevPos = pos;
+      current = objects[pos];
     }
     
+    private void setPos(int pos) {
+      this.prevPos = this.pos;
+      this.pos = pos;
+    }
+
     @Override
     public boolean hasNext() {
       // For Not Equals we need to look in the entire set
       if (keysToRemove != null) {
-        while (indexSlot < objects.length) {
-          current = objects[indexSlot];
+        while (pos < objects.length) {
+          current = objects[pos];
           if (current == null || current.equals(REMOVED)) {
-            //continue searching
-          }
-          else if (notMatchingAnyKeyToRemove(keysToRemove, current)) {
+            // continue searching
+          } else if (notMatchingAnyKeyToRemove(keysToRemove, current)) {
             return true;
           }
-     
-          indexSlot++;
+          setPos(pos+1);
         }
         return false;
       } else {
-
-        current = objects[indexSlot];
+        current = objects[pos];
         // For Equals query
         while (current != null) {
           if (current != REMOVED) {
-            if (_hashingStrategy.equalsOnGet(keyToMatch, current)) {
+            if (objectMatchesIndexKey(keyToMatch, current)) {
               return true;
             }
           }
           // If this is not the correct collection, one that does not match the
           // key we are looking for, then continue our search
-          indexSlot -= probe;
-          if (indexSlot < 0) {
-            indexSlot += length;
-          }
-          
-          current = objects[indexSlot];
-        } 
+          setPos((pos + 1) & mask);
+          current = objects[pos];
+        }
       }
       return false;
     }
+
     private boolean notMatchingAnyKeyToRemove(Collection keysToRemove, Object current) {
       Iterator keysToRemoveIterator = keysToRemove.iterator();
       while (keysToRemoveIterator.hasNext()) {
         Object keyToMatch = keysToRemoveIterator.next();
-        if (_hashingStrategy.equalsOnGet(keyToMatch, current)) {
+        if (objectMatchesIndexKey(keyToMatch, current)) {
           return false;
         }
       }
@@ -1206,43 +767,52 @@ public class HashIndexSet implements Set {
 
     @Override
     public Object next() throws NoSuchElementException {
-        Object obj = current;
-        if (keysToRemove != null) {
-          // for Not equals we need to continue looking
-          // so increment the index here
-          indexSlot++;
-        } else {
-          //advance the pointer
-          indexSlot -= probe;
-          if (indexSlot < 0) {
-            indexSlot += length;
-          }
-        }
-        return obj;
-    }
-
-    int currentObjectIndex() {
-      int indexToRemove = 0;
-      //Because we advanced on the next() call, we need to get the indexSlot prior to advancing
+      Object obj = current;
       if (keysToRemove != null) {
         // for Not equals we need to continue looking
         // so increment the index here
-        indexToRemove = indexSlot - 1;
+        setPos(pos+1);
       } else {
-        //move back the pointer
-        indexToRemove = indexSlot + probe;
-        if (indexSlot >= objects.length) {
-          indexToRemove = indexSlot - length;
-        }
+        // advance the pointer
+        setPos((pos + 1) & mask);
       }
-      return indexToRemove;
+      return obj;
     }
-    
+
+    int currentObjectIndex() {
+      return prevPos;
+    }
+
     @Override
     public void remove() {
       removeAt(currentObjectIndex());
     }
     
+    
+    public boolean objectMatchesIndexKey(Object indexKey, Object o) {
+      Object fieldValue = _imqEvaluator.evaluateKey(o);
+     
+      if (fieldValue == IndexManager.NULL && indexKey == IndexManager.NULL) {
+        return true;
+      } else {
+        try {
+          if (fieldValue instanceof PdxString) {
+           if (indexKey instanceof String) {
+             fieldValue = ((PdxString) fieldValue).toString(); 
+           }
+         }
+         else if (indexKey instanceof PdxString) {
+           if (fieldValue instanceof String) {
+             fieldValue = new PdxString((String)fieldValue);
+           }
+         }
+         return TypeUtils.compare(fieldValue, indexKey, OQLLexerTokenTypes.TOK_EQ).equals(Boolean.TRUE);
+        }
+        catch (TypeMismatchException e) {
+          return fieldValue.equals(indexKey);
+        }
+      }
+    }
   }
-} // HashIndexSet
+}
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/abad018a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/index/HashIndexStrategy.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/index/HashIndexStrategy.java b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/index/HashIndexStrategy.java
deleted file mode 100755
index 3ed0caf..0000000
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/index/HashIndexStrategy.java
+++ /dev/null
@@ -1,90 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-/*
- * IndexCreationHelper.java
- *
- * Created on March 16, 2005, 6:20 PM
- */
-package com.gemstone.gemfire.cache.query.internal.index;
-
-import com.gemstone.gemfire.cache.query.internal.ExecutionContext;
-
-
-/**
- * Interface to support plug-able hashing strategies in maps and sets.
- * Implementors can use this interface to make the hashing
- * algorithms use object values, values provided by the java runtime,
- * or a custom strategy when computing hash codes.
- *
- */
-
-public interface HashIndexStrategy {
-    
-    /**
-     * Computes a hash code for the specified object.  Implementors
-     * can use the object's own <tt>hashCode</tt> method, the Java
-     * runtime's <tt>identityHashCode</tt>, or a custom scheme.
-     * 
-     * @param o object for which the hashcode is to be computed
-     * @return the hashCode
-     */
-    public int computeHashCode(Object o);
-    
-
-    /**
-     * VMware Addition
-     * Computes a hash code for the specified object.  Implementors
-     * can use the object's own <tt>hashCode</tt> method, the Java
-     * runtime's <tt>identityHashCode</tt>, or a custom scheme.
-     * Used when resizing the internal set structure.  Due to not storing
-     * the indexKey, we have to recompute the indexKey from the object
-     * at this point.
-     * @param o object for which the hashcode is to be computed
-     * @param recomputeKey 
-     * @return the hashCode
-     */
-    public int computeHashCode(Object o, boolean recomputeKey);
-    
-    /**
-     * VMware Addition
-     * Computes the object's key
-     * @param o object for which the key is to be computed
-     * @return the key
-     */
-    public Object computeKey(Object o);
-
-    /**
-     * Compares o1 and o2 for equality.  Strategy implementors may use
-     * the objects' own equals() methods, compare object references,
-     * or implement some custom scheme.
-     *
-     * @param o1 an <code>Object</code> value
-     * @param o2 an <code>Object</code> value
-     * @return true if the objects are equal according to this strategy.
-     */
-    public boolean equalsOnAdd(Object o1, Object o2);
-    
-    /**
-     * Compares o1 and o2 for equality.  Strategy implementors may use
-     * the objects' own equals() methods, compare object references,
-     * or implement some custom scheme.
-     *
-     * @return true if the objects are equal according to this strategy.
-     */
-    public boolean equalsOnGet(Object getValue, Object o);
-    
-} 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/abad018a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/util/ObjectProcedure.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/util/ObjectProcedure.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/util/ObjectProcedure.java
deleted file mode 100755
index 421b6c8..0000000
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/util/ObjectProcedure.java
+++ /dev/null
@@ -1,30 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package com.gemstone.gemfire.internal.util;
-
-/**
- * Similar to the Trove TObjectProcedure, this is used in iterating over some
- * GemFire collections
- * 
- * @author bschuchardt
- *
- */
-public interface ObjectProcedure {
-
-  public boolean executeWith(Object entry);
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/abad018a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/util/PrimeFinder.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/util/PrimeFinder.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/util/PrimeFinder.java
deleted file mode 100644
index 529e5a2..0000000
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/util/PrimeFinder.java
+++ /dev/null
@@ -1,159 +0,0 @@
-//   Copyright (c) 1999 CERN - European Organization for Nuclear Research.
-//
-//   Permission to use, copy, modify, distribute and sell this software
-//   and its documentation for any purpose is hereby granted without fee,
-//   provided that the above copyright notice appear in all copies and
-//   that both that copyright notice and this permission notice appear in
-//   supporting documentation. CERN makes no representations about the
-//   suitability of this software for any purpose. It is provided "as is"
-//   without expressed or implied warranty.
-
-package com.gemstone.gemfire.internal.util;
-
-import java.util.Arrays;
-
-/*
- * Modified for Trove to use the java.util.Arrays sort/search
- * algorithms instead of those provided with colt.
- */
-
-/**
- * Used to keep hash table capacities prime numbers.
- * Not of interest for users; only for implementors of hashtables.
- *
- * <p>Choosing prime numbers as hash table capacities is a good idea
- * to keep them working fast, particularly under hash table
- * expansions.
- *
- * <p>However, JDK 1.2, JGL 3.1 and many other toolkits do nothing to
- * keep capacities prime.  This class provides efficient means to
- * choose prime capacities.
- *
- * <p>Choosing a prime is <tt>O(log 300)</tt> (binary search in a list
- * of 300 ints).  Memory requirements: 1 KB static memory.
- *
- * @author wolfgang.hoschek@cern.ch
- * @version 1.0, 09/24/99
- */
-public final class PrimeFinder {
-	/**
-	 * The largest prime this class can generate; currently equal to
-	 * <tt>Integer.MAX_VALUE</tt>.
-	 */
-	public static final int largestPrime = Integer.MAX_VALUE; //yes, it is prime.
-
-	/**
-	 * The prime number list consists of 11 chunks.
-     *
-	 * Each chunk contains prime numbers.
-     *
-	 * A chunk starts with a prime P1. The next element is a prime
-	 * P2. P2 is the smallest prime for which holds: P2 >= 2*P1.
-     *
-	 * The next element is P3, for which the same holds with respect
-	 * to P2, and so on.
-	 *
-	 * Chunks are chosen such that for any desired capacity >= 1000
-	 * the list includes a prime number <= desired capacity * 1.11.
-     *
-	 * Therefore, primes can be retrieved which are quite close to any
-	 * desired capacity, which in turn avoids wasting memory.
-     *
-	 * For example, the list includes
-	 * 1039,1117,1201,1277,1361,1439,1523,1597,1759,1907,2081.
-     *
-	 * So if you need a prime >= 1040, you will find a prime <=
-	 * 1040*1.11=1154.
-	 *	
-	 * Chunks are chosen such that they are optimized for a hashtable
-	 * growthfactor of 2.0;
-     *
-	 * If your hashtable has such a growthfactor then, after initially
-	 * "rounding to a prime" upon hashtable construction, it will
-	 * later expand to prime capacities such that there exist no
-	 * better primes.
-	 *
-	 * In total these are about 32*10=320 numbers -> 1 KB of static
-	 * memory needed.
-     *
-	 * If you are stingy, then delete every second or fourth chunk.
-	 */
-	
-	private static final int[] primeCapacities = {
-		//chunk #0
-		largestPrime,
-		
-		//chunk #1
-		5,11,23,47,97,197,397,797,1597,3203,6421,12853,25717,51437,102877,205759,
-        411527,823117,1646237,3292489,6584983,13169977,26339969,52679969,105359939,
-        210719881,421439783,842879579,1685759167,
-		  
-		//chunk #2
-		433,877,1759,3527,7057,14143,28289,56591,113189,226379,452759,905551,1811107,
-        3622219,7244441,14488931,28977863,57955739,115911563,231823147,463646329,927292699,
-        1854585413,
-		  
-		//chunk #3
-		953,1907,3821,7643,15287,30577,61169,122347,244703,489407,978821,1957651,3915341,
-        7830701,15661423,31322867,62645741,125291483,250582987,501165979,1002331963,
-        2004663929,
-		  
-		//chunk #4
-		1039,2081,4177,8363,16729,33461,66923,133853,267713,535481,1070981,2141977,4283963,
-        8567929,17135863,34271747,68543509,137087021,274174111,548348231,1096696463,
-		  
-		//chunk #5
-		31,67,137,277,557,1117,2237,4481,8963,17929,35863,71741,143483,286973,573953,
-        1147921,2295859,4591721,9183457,18366923,36733847,73467739,146935499,293871013,
-        587742049,1175484103,
-		  
-		//chunk #6
-		599,1201,2411,4831,9677,19373,38747,77509,155027,310081,620171,1240361,2480729,
-        4961459,9922933,19845871,39691759,79383533,158767069,317534141,635068283,1270136683,
-		  
-		//chunk #7
-		311,631,1277,2557,5119,10243,20507,41017,82037,164089,328213,656429,1312867,
-        2625761,5251529,10503061,21006137,42012281,84024581,168049163,336098327,672196673,
-        1344393353,
-		  
-		//chunk #8
-		3,7,17,37,79,163,331,673,1361,2729,5471,10949,21911,43853,87719,175447,350899,
-        701819,1403641,2807303,5614657,11229331,22458671,44917381,89834777,179669557,
-        359339171,718678369,1437356741,
-		  
-		//chunk #9
-		43,89,179,359,719,1439,2879,5779,11579,23159,46327,92657,185323,370661,741337,
-        1482707,2965421,5930887,11861791,23723597,47447201,94894427,189788857,379577741,
-        759155483,1518310967,
-		  
-		//chunk #10
-		379,761,1523,3049,6101,12203,24407,48817,97649,195311,390647,781301,1562611,
-        3125257,6250537,12501169,25002389,50004791,100009607,200019221,400038451,800076929,
-        1600153859
-    };
-
-	static { //initializer
-		// The above prime numbers are formatted for human readability.
-		// To find numbers fast, we sort them once and for all.
-		
-		Arrays.sort(primeCapacities);
-	}
-	
-    /**
-     * Returns a prime number which is <code>&gt;= desiredCapacity</code>
-     * and very close to <code>desiredCapacity</code> (within 11% if
-     * <code>desiredCapacity &gt;= 1000</code>).
-     *
-     * @param desiredCapacity the capacity desired by the user.
-     * @return the capacity which should be used for a hashtable.
-     */
-    public static final int nextPrime(int desiredCapacity) {
-        int i = Arrays.binarySearch(primeCapacities, desiredCapacity);
-        if (i<0) {
-            // desired capacity not found, choose next prime greater
-            // than desired capacity
-            i = -i -1; // remember the semantics of binarySearch...
-        }
-        return primeCapacities[i];
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/abad018a/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/internal/index/HashIndexJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/internal/index/HashIndexJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/internal/index/HashIndexJUnitTest.java
index e3247a7..e2a8643 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/internal/index/HashIndexJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/internal/index/HashIndexJUnitTest.java
@@ -189,6 +189,7 @@ public class HashIndexJUnitTest {
       }
       region.put("" + i, p);
     }
+    
     helpTestHashIndexForQuery("SELECT * FROM /portfolios p WHERE p.status = 'inactive'", "p.status", "/portfolios p");
     qs.removeIndexes();
     observer = new MyQueryObserverAdapter();
@@ -1382,17 +1383,17 @@ public class HashIndexJUnitTest {
   }
 
    
-  private void printIndex(Index index) {
-   if (index instanceof PartitionedIndex) {
-    Iterator it = ((PartitionedIndex)index).getBucketIndexes().iterator();
-    while (it.hasNext()) { 
-      ((HashIndex)it.next()).printAll();
-    }
-   }
-   else {
-     System.out.println(((HashIndex)index).printAll());
-   }
-  }
+//  private void printIndex(Index index) {
+//   if (index instanceof PartitionedIndex) {
+//    Iterator it = ((PartitionedIndex)index).getBucketIndexes().iterator();
+//    while (it.hasNext()) { 
+//      ((HashIndex)it.next()).printAll();
+//    }
+//   }
+//   else {
+//     System.out.println(((HashIndex)index).printAll());
+//   }
+//  }
   
   
   private class RelationshipKey implements Comparable {


[18/19] incubator-geode git commit: GEODE-592. Increased wait time

Posted by ab...@apache.org.
GEODE-592. Increased wait time


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

Branch: refs/heads/feature/GEODE-584
Commit: 52f0a1d59ea08de8cbff2be4ff96da3065ad5a05
Parents: 3588b02
Author: Hitesh Khamesra <hi...@yahoo.com>
Authored: Tue Nov 24 10:06:35 2015 -0800
Committer: Hitesh Khamesra <hi...@yahoo.com>
Committed: Tue Nov 24 12:19:33 2015 -0800

----------------------------------------------------------------------
 .../membership/gms/membership/fd/GMSHealthMonitorJUnitTest.java    | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/52f0a1d5/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/membership/fd/GMSHealthMonitorJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/membership/fd/GMSHealthMonitorJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/membership/fd/GMSHealthMonitorJUnitTest.java
index 1699068..1113f5f 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/membership/fd/GMSHealthMonitorJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/membership/fd/GMSHealthMonitorJUnitTest.java
@@ -149,7 +149,7 @@ public class GMSHealthMonitorJUnitTest {
 
     // allow the monitor to give up on the initial "next neighbor" and
     // move on to the one after it
-    long giveup = System.currentTimeMillis() + memberTimeout + 500;
+    long giveup = System.currentTimeMillis() + memberTimeout + 600;
     InternalDistributedMember expected = mockMembers.get(5);
     InternalDistributedMember neighbor = gmsHealthMonitor.getNextNeighbor();
     while (System.currentTimeMillis() < giveup && neighbor != expected) {


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

Posted by ab...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4b0925e3/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/SimpleMemoryAllocatorImpl.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/SimpleMemoryAllocatorImpl.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/SimpleMemoryAllocatorImpl.java
index c800335..f16253e 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/SimpleMemoryAllocatorImpl.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/SimpleMemoryAllocatorImpl.java
@@ -16,69 +16,30 @@
  */
 package com.gemstone.gemfire.internal.offheap;
 
-import java.io.ByteArrayOutputStream;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.io.PrintStream;
-import java.lang.reflect.Constructor;
-import java.lang.reflect.InvocationTargetException;
-import java.lang.reflect.Method;
-import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Collection;
 import java.util.Collections;
 import java.util.Comparator;
 import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
-import java.util.NavigableSet;
-import java.util.NoSuchElementException;
 import java.util.Set;
-import java.util.concurrent.ArrayBlockingQueue;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentLinkedQueue;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.ConcurrentSkipListSet;
-import java.util.concurrent.CopyOnWriteArrayList;
 import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicIntegerFieldUpdater;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.atomic.AtomicLongFieldUpdater;
 import java.util.concurrent.atomic.AtomicReference;
-import java.util.concurrent.atomic.AtomicReferenceArray;
-
 import org.apache.logging.log4j.Logger;
 
-import com.gemstone.gemfire.DataSerializer;
 import com.gemstone.gemfire.LogWriter;
-import com.gemstone.gemfire.OutOfOffHeapMemoryException;
 import com.gemstone.gemfire.cache.CacheClosedException;
 import com.gemstone.gemfire.cache.Region;
-import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
-import com.gemstone.gemfire.internal.DSCODE;
-import com.gemstone.gemfire.internal.DataSerializableFixedID;
-import com.gemstone.gemfire.internal.HeapDataOutputStream;
-import com.gemstone.gemfire.internal.InternalDataSerializer;
 import com.gemstone.gemfire.internal.cache.BucketRegion;
-import com.gemstone.gemfire.internal.cache.BytesAndBitsForCompactor;
-import com.gemstone.gemfire.internal.cache.CachedDeserializableFactory;
-import com.gemstone.gemfire.internal.cache.EntryBits;
-import com.gemstone.gemfire.internal.cache.EntryEventImpl;
 import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
 import com.gemstone.gemfire.internal.cache.LocalRegion;
 import com.gemstone.gemfire.internal.cache.PartitionedRegion;
 import com.gemstone.gemfire.internal.cache.PartitionedRegionDataStore;
 import com.gemstone.gemfire.internal.cache.RegionEntry;
-import com.gemstone.gemfire.internal.cache.RegionEntryContext;
-import com.gemstone.gemfire.internal.lang.StringUtils;
 import com.gemstone.gemfire.internal.logging.LogService;
-import com.gemstone.gemfire.internal.offheap.SimpleMemoryAllocatorImpl.Chunk;
-import com.gemstone.gemfire.internal.offheap.SimpleMemoryAllocatorImpl.ConcurrentBag.Node;
 import com.gemstone.gemfire.internal.offheap.annotations.OffHeapIdentifier;
 import com.gemstone.gemfire.internal.offheap.annotations.Unretained;
-import com.gemstone.gemfire.internal.shared.StringPrintWriter;
 
 /**
  * This allocator is somewhat like an Arena allocator.
@@ -95,7 +56,7 @@ import com.gemstone.gemfire.internal.shared.StringPrintWriter;
  */
 public final class SimpleMemoryAllocatorImpl implements MemoryAllocator, MemoryInspector {
 
-  private static final Logger logger = LogService.getLogger();
+  static final Logger logger = LogService.getLogger();
   
   public static final String FREE_OFF_HEAP_MEMORY_PROPERTY = "gemfire.free-off-heap-memory";
   
@@ -118,9 +79,9 @@ public final class SimpleMemoryAllocatorImpl implements MemoryAllocator, MemoryI
   public final static int MAX_TINY = TINY_MULTIPLE*TINY_FREE_LIST_COUNT;
   public final static int HUGE_MULTIPLE = 256;
   
-  private volatile OffHeapMemoryStats stats;
+  volatile OffHeapMemoryStats stats;
   
-  private volatile OutOfOffHeapMemoryListener ooohml;
+  volatile OutOfOffHeapMemoryListener ooohml;
   
   /** The MemoryChunks that this allocator is managing by allocating smaller chunks of them.
    * The contents of this array never change.
@@ -135,7 +96,7 @@ public final class SimpleMemoryAllocatorImpl implements MemoryAllocator, MemoryI
   
   private static SimpleMemoryAllocatorImpl singleton = null;
   private static final AtomicReference<Thread> asyncCleanupThread = new AtomicReference<Thread>();
-  private final ChunkFactory chunkFactory;
+  final ChunkFactory chunkFactory;
   
   public static SimpleMemoryAllocatorImpl getAllocator() {
     SimpleMemoryAllocatorImpl result = singleton;
@@ -157,7 +118,7 @@ public final class SimpleMemoryAllocatorImpl implements MemoryAllocator, MemoryI
       result.reuse(ooohml, lw, stats, offHeapMemorySize);
       lw.config("Reusing " + result.getTotalMemory() + " bytes of off-heap memory. The maximum size of a single off-heap object is " + result.largestSlab + " bytes.");
       created = true;
-      invokeAfterReuse(result);
+      LifecycleListener.invokeAfterReuse(result);
     } else {
       // allocate memory chunks
       //SimpleMemoryAllocatorImpl.cleanupPreviousAllocator();
@@ -189,7 +150,7 @@ public final class SimpleMemoryAllocatorImpl implements MemoryAllocator, MemoryI
       result = new SimpleMemoryAllocatorImpl(ooohml, stats, slabs);
       created = true;
       singleton = result;
-      invokeAfterCreate(result);
+      LifecycleListener.invokeAfterCreate(result);
     }
     } finally {
       if (!created) {
@@ -203,7 +164,7 @@ public final class SimpleMemoryAllocatorImpl implements MemoryAllocator, MemoryI
   public static SimpleMemoryAllocatorImpl create(OutOfOffHeapMemoryListener oooml, OffHeapMemoryStats stats, UnsafeMemoryChunk[] slabs) {
     SimpleMemoryAllocatorImpl result = new SimpleMemoryAllocatorImpl(oooml, stats, slabs);
     singleton = result;
-    invokeAfterCreate(result);
+    LifecycleListener.invokeAfterCreate(result);
     return result;
   }
   
@@ -323,7 +284,7 @@ public final class SimpleMemoryAllocatorImpl implements MemoryAllocator, MemoryI
     this.stats.incMaxMemory(this.totalSlabSize);
     this.stats.incFreeMemory(this.totalSlabSize);
     
-    this.freeList = new FreeListManager();
+    this.freeList = new FreeListManager(this);
   }
   
   public List<Chunk> getLostChunks() {
@@ -401,190 +362,12 @@ public final class SimpleMemoryAllocatorImpl implements MemoryAllocator, MemoryI
     //System.out.println("allocating " + size);
     Chunk result = this.freeList.allocate(size, chunkType);
     //("allocated off heap object of size " + size + " @" + Long.toHexString(result.getMemoryAddress()), true);
-    if (trackReferenceCounts()) {
-      refCountChanged(result.getMemoryAddress(), false, 1);
+    if (ReferenceCountHelper.trackReferenceCounts()) {
+      ReferenceCountHelper.refCountChanged(result.getMemoryAddress(), false, 1);
     }
     return result;
   }
   
-  /**
-   * Used to represent offheap addresses whose
-   * value encodes actual data instead a memory
-   * location.
-   * Instances of this class have a very short lifetime.
-   * 
-   * @author darrel
-   *
-   */
-  public static class DataAsAddress implements StoredObject {
-    private final long address;
-    
-    public DataAsAddress(long addr) {
-      this.address = addr;
-    }
-    
-    public long getEncodedAddress() {
-      return this.address;
-    }
-
-    @Override
-    public boolean equals(Object o) {
-      if (o instanceof DataAsAddress) {
-        return getEncodedAddress() == ((DataAsAddress) o).getEncodedAddress();
-      }
-      return false;
-    }
-    
-    @Override
-    public int hashCode() {
-      long value = getEncodedAddress();
-      return (int)(value ^ (value >>> 32));
-    }
-
-    @Override
-    public int getSizeInBytes() {
-      return 0;
-    }
-
-    public byte[] getDecompressedBytes(RegionEntryContext r) {
-      return OffHeapRegionEntryHelper.encodedAddressToBytes(this.address, true, r);
-    }
-
-    /**
-     * If we contain a byte[] return it.
-     * Otherwise return the serialize bytes in us in a byte array.
-     */
-    public byte[] getRawBytes() {
-      return OffHeapRegionEntryHelper.encodedAddressToRawBytes(this.address);
-    }
-    
-    @Override
-    public byte[] getSerializedValue() {
-      return OffHeapRegionEntryHelper.encodedAddressToBytes(this.address);
-    }
-
-    @Override
-    public Object getDeserializedValue(Region r, RegionEntry re) {
-      return OffHeapRegionEntryHelper.encodedAddressToObject(this.address);
-    }
-
-    @Override
-    public Object getDeserializedForReading() {
-      return getDeserializedValue(null,null);
-    }
-    
-    @Override
-    public Object getValueAsDeserializedHeapObject() {
-      return getDeserializedValue(null,null);
-    }
-    
-    @Override
-    public byte[] getValueAsHeapByteArray() {
-      if (isSerialized()) {
-        return getSerializedValue();
-      } else {
-        return (byte[])getDeserializedForReading();
-      }
-    }
-
-    @Override
-    public String getStringForm() {
-      try {
-        return StringUtils.forceToString(getDeserializedForReading());
-      } catch (RuntimeException ex) {
-        return "Could not convert object to string because " + ex;
-      }
-    }
-
-    @Override
-    public Object getDeserializedWritableCopy(Region r, RegionEntry re) {
-      return getDeserializedValue(null,null);
-    }
-
-    @Override
-    public Object getValue() {
-      if (isSerialized()) {
-        return getSerializedValue();
-      } else {
-        throw new IllegalStateException("Can not call getValue on StoredObject that is not serialized");
-      }
-    }
-
-    @Override
-    public void writeValueAsByteArray(DataOutput out) throws IOException {
-      DataSerializer.writeByteArray(getSerializedValue(), out);
-    }
-
-    @Override
-    public void fillSerializedValue(BytesAndBitsForCompactor wrapper,
-        byte userBits) {
-      byte[] value;
-      if (isSerialized()) {
-        value = getSerializedValue();
-        userBits = EntryBits.setSerialized(userBits, true);
-      } else {
-        value = (byte[]) getDeserializedForReading();
-      }
-      wrapper.setData(value, userBits, value.length, true);
-    }
-
-    @Override
-    public int getValueSizeInBytes() {
-      return 0;
-    }
-    
-    @Override
-    public void sendTo(DataOutput out) throws IOException {
-      if (isSerialized()) {
-        out.write(getSerializedValue());
-      } else {
-        Object objToSend = (byte[]) getDeserializedForReading(); // deserialized as a byte[]
-        DataSerializer.writeObject(objToSend, out);
-      }
-    }
-
-    @Override
-    public void sendAsByteArray(DataOutput out) throws IOException {
-      byte[] bytes;
-      if (isSerialized()) {
-        bytes = getSerializedValue();
-      } else {
-        bytes = (byte[]) getDeserializedForReading();
-      }
-      DataSerializer.writeByteArray(bytes, out);
-      
-    }
-    
-    @Override
-    public void sendAsCachedDeserializable(DataOutput out) throws IOException {
-      if (!isSerialized()) {
-        throw new IllegalStateException("sendAsCachedDeserializable can only be called on serialized StoredObjects");
-      }
-      InternalDataSerializer.writeDSFIDHeader(DataSerializableFixedID.VM_CACHED_DESERIALIZABLE, out);
-      sendAsByteArray(out);
-    }
-
-    @Override
-    public boolean isSerialized() {
-      return OffHeapRegionEntryHelper.isSerialized(this.address);
-    }
-
-    @Override
-    public boolean isCompressed() {
-      return OffHeapRegionEntryHelper.isCompressed(this.address);
-    }
-    
-    @Override
-    public boolean retain() {
-      // nothing needed
-      return true;
-    }
-    @Override
-    public void release() {
-      // nothing needed
-    }
-  }
-
   @SuppressWarnings("unused")
   public static void debugLog(String msg, boolean logStack) {
     if (logStack) {
@@ -607,8 +390,8 @@ public final class SimpleMemoryAllocatorImpl implements MemoryAllocator, MemoryI
     Chunk result = this.freeList.allocate(v.length, chunkType);
     //debugLog("allocated off heap object of size " + v.length + " @" + Long.toHexString(result.getMemoryAddress()), true);
     //debugLog("allocated off heap object of size " + v.length + " @" + Long.toHexString(result.getMemoryAddress()) +  "chunkSize=" + result.getSize() + " isSerialized=" + isSerialized + " v=" + Arrays.toString(v), true);
-    if (trackReferenceCounts()) {
-      refCountChanged(result.getMemoryAddress(), false, 1);
+    if (ReferenceCountHelper.trackReferenceCounts()) {
+      ReferenceCountHelper.refCountChanged(result.getMemoryAddress(), false, 1);
     }
     assert result.getChunkType() == chunkType: "chunkType=" + chunkType + " getChunkType()=" + result.getChunkType();
     result.setSerializedValue(v);
@@ -635,7 +418,7 @@ public final class SimpleMemoryAllocatorImpl implements MemoryAllocator, MemoryI
   @Override
   public void close() {
     try {
-      invokeBeforeClose(this);
+      LifecycleListener.invokeBeforeClose(this);
     } finally {
       this.ooohml.close();
       if (Boolean.getBoolean(FREE_OFF_HEAP_MEMORY_PROPERTY)) {
@@ -710,7 +493,7 @@ public final class SimpleMemoryAllocatorImpl implements MemoryAllocator, MemoryI
 //    asyncCleanupThread.set(t);    
   }
   
-  private void freeChunk(long addr) {
+  void freeChunk(long addr) {
     this.freeList.free(addr);
   }
   
@@ -772,7 +555,7 @@ public final class SimpleMemoryAllocatorImpl implements MemoryAllocator, MemoryI
     }
   }
   
-  private void notifyListeners() {
+  void notifyListeners() {
     final MemoryUsageListener[] savedListeners = this.memoryUsageListeners;
     
     if (savedListeners.length == 0) {
@@ -785,2391 +568,73 @@ public final class SimpleMemoryAllocatorImpl implements MemoryAllocator, MemoryI
     }
   }
   
-  public class FreeListManager {
-    private final AtomicReferenceArray<SyncChunkStack> tinyFreeLists = new AtomicReferenceArray<SyncChunkStack>(TINY_FREE_LIST_COUNT);
-    // Deadcoding the BIG stuff. Idea is to have a bigger TINY list by default
-//    /**
-//     * Every allocated chunk smaller than BIG_MULTIPLE*BIG_FREE_LIST_COUNT but that is not tiny will allocate a chunk of memory that is a multiple of this value.
-//     * Sizes are always rounded up to the next multiple of this constant
-//     * so internal fragmentation will be limited to BIG_MULTIPLE-1 bytes per allocation
-//     * and on average will be BIG_MULTIPLE/2 given a random distribution of size requests.
-//     */
-//    public final static int BIG_MULTIPLE = TINY_MULTIPLE*8;
-//    /**
-//     * Number of free lists to keep for big allocations.
-//     */
-//    private final static int BIG_FREE_LIST_COUNT = 2048;
-//    private final static int BIG_OFFSET = (MAX_TINY/BIG_MULTIPLE*BIG_MULTIPLE);
-//    public final static int MAX_BIG = (BIG_MULTIPLE*BIG_FREE_LIST_COUNT) + BIG_OFFSET;
-//    private final AtomicReferenceArray<ConcurrentChunkStack> bigFreeLists = new AtomicReferenceArray<ConcurrentChunkStack>(BIG_FREE_LIST_COUNT);
-    // hugeChunkSet is sorted by chunk size in ascending order. It will only contain chunks larger than MAX_TINY.
-    private final ConcurrentSkipListSet<Chunk> hugeChunkSet = new ConcurrentSkipListSet<Chunk>();
-    private final AtomicLong allocatedSize = new AtomicLong(0L);
-   
-    private int getNearestTinyMultiple(int size) {
-      return (size-1)/TINY_MULTIPLE;
-    }
-    public List<Chunk> getLiveChunks() {
-      ArrayList<Chunk> result = new ArrayList<Chunk>();
-      UnsafeMemoryChunk[] slabs = getSlabs();
-      for (int i=0; i < slabs.length; i++) {
-        getLiveChunks(slabs[i], result);
-      }
-      return result;
-    }
-    private void getLiveChunks(UnsafeMemoryChunk slab, List<Chunk> result) {
-      long addr = slab.getMemoryAddress();
-      while (addr <= (slab.getMemoryAddress() + slab.getSize() - Chunk.MIN_CHUNK_SIZE)) {
-        Fragment f = isAddrInFragmentFreeSpace(addr);
-        if (f != null) {
-          addr = f.getMemoryAddress() + f.getSize();
-        } else {
-          int curChunkSize = Chunk.getSize(addr);
-          int refCount = Chunk.getRefCount(addr);
-          if (refCount > 0) {
-            result.add(SimpleMemoryAllocatorImpl.this.chunkFactory.newChunk(addr));
-          }
-          addr += curChunkSize;
-        }
-      }
-    }
-    /**
-     * If addr is in the free space of a fragment then return that fragment; otherwise return null.
-     */
-    private Fragment isAddrInFragmentFreeSpace(long addr) {
-      for (Fragment f: this.fragmentList) {
-        if (addr >= (f.getMemoryAddress() + f.getFreeIndex()) && addr < (f.getMemoryAddress() + f.getSize())) {
-          return f;
-        }
-      }
-      return null;
-    }
-    public long getUsedMemory() {
-      return this.allocatedSize.get();
-    }
-    public long getFreeMemory() {
-      return getTotalMemory() - getUsedMemory();
-//      long result = getFreeFragmentMemory();
-//      result += getFreeTinyMemory();
-//      result += getFreeHugeMemory();
-//      return result;
-    }
-    public long getFreeFragmentMemory() {
-      long result = 0;
-      for (Fragment f: this.fragmentList) {
-        int freeSpace = f.freeSpace();
-        if (freeSpace >= Chunk.MIN_CHUNK_SIZE) {
-          result += freeSpace;
-        }
-      }
-      return result;
-    }
-    public long getFreeTinyMemory() {
-      long tinyFree = 0;
-      for (int i=0; i < this.tinyFreeLists.length(); i++) {
-        SyncChunkStack cl = this.tinyFreeLists.get(i);
-        if (cl != null) {
-          tinyFree += cl.computeTotalSize();
-        }
-      }
-      return tinyFree;
-    }
-//    public long getFreeBigMemory() {
-//      long bigFree = 0;
-//      for (int i=0; i < this.bigFreeLists.length(); i++) {
-//        ConcurrentChunkStack cl = this.bigFreeLists.get(i);
-//        if (cl != null) {
-//          bigFree += cl.computeTotalSize();
-//        }
-//      }
-//      return bigFree;
-//    }
-    public long getFreeHugeMemory() {
-      long hugeFree = 0;
-      for (Chunk c: this.hugeChunkSet) {
-        hugeFree += c.getSize();
-      }
-      return hugeFree;
-    }
-//    private int getNearestBigMultiple(int size) {
-//      return (size-1-BIG_OFFSET)/BIG_MULTIPLE;
-//    }
-
-    /**
-     * Each long in this array tells us how much of the corresponding slab is allocated.
-     */
-    //private final AtomicIntegerArray slabOffsets = new AtomicIntegerArray(getSlabs().length);
-    /**
-     * The slab id of the last slab we allocated from.
-     */
-    private final AtomicInteger lastFragmentAllocation = new AtomicInteger(0);
-
-    private final CopyOnWriteArrayList<Fragment> fragmentList;
-    public FreeListManager() {
-      UnsafeMemoryChunk[] slabs = getSlabs();
-      Fragment[] tmp = new Fragment[slabs.length];
-      for (int i=0; i < slabs.length; i++) {
-        tmp[i] = new Fragment(slabs[i].getMemoryAddress(), slabs[i].getSize());
-      }
-      this.fragmentList = new CopyOnWriteArrayList<Fragment>(tmp);
-      
-      if(validateMemoryWithFill) {
-        fillFragments();
-      }
-    }
-    
-    /**
-     * Fills all fragments with a fill used for data integrity validation.
-     */
-    private void fillFragments() {
-      for(Fragment fragment : this.fragmentList) {
-        fragment.fill();
-      }
-    }
-    
-    /**
-     * This is a bit of a hack. TODO add some timeout logic in case this thread never does another off heap allocation.
-     */
-//    private final ThreadLocal<Chunk> tlCache = new ThreadLocal<Chunk>();
-    
-    /**
-     * Allocate a chunk of memory of at least the given size.
-     * The basic algorithm is:
-     * 1. Look for a previously allocated and freed chunk close to the size requested.
-     * 2. See if the original chunk is big enough to split. If so do so.
-     * 3. Look for a previously allocated and freed chunk of any size larger than the one requested.
-     *    If we find one split it.
-     * <p>
-     * It might be better not to include step 3 since we expect and freed chunk to be reallocated in the future.
-     * Maybe it would be better for 3 to look for adjacent free blocks that can be merged together.
-     * For now we will just try 1 and 2 and then report out of mem.
-     * @param size minimum bytes the returned chunk must have.
-     * @param chunkType TODO
-     * @return the allocated chunk
-     * @throws IllegalStateException if a chunk can not be allocated.
-     */
-    @SuppressWarnings("synthetic-access")
-    public Chunk allocate(int size, ChunkType chunkType) {
-      Chunk result = null; /*tlCache.get();
-      
-      if (result != null && result.getDataSize() == size) {
-        tlCache.set(null);
-      } else */{
-        assert size > 0;
-        if (chunkType == null) {
-          chunkType = GemFireChunk.TYPE;
-        }
-        result = basicAllocate(size, true, chunkType);
-        result.setDataSize(size);
-      }
-      stats.incObjects(1);
-      int resultSize = result.getSize();
-      this.allocatedSize.addAndGet(resultSize);
-      stats.incUsedMemory(resultSize);
-      stats.incFreeMemory(-resultSize);
-      result.initializeUseCount();
-      notifyListeners();
-      
-      return result;
-    }
-    
-    private Chunk basicAllocate(int size, boolean useSlabs, ChunkType chunkType) {
-      if (useSlabs) {
-        // Every object stored off heap has a header so we need
-        // to adjust the size so that the header gets allocated.
-        // If useSlabs is false then the incoming size has already
-        // been adjusted.
-        size += Chunk.OFF_HEAP_HEADER_SIZE;
-      }
-      if (size <= MAX_TINY) {
-        return allocateTiny(size, useSlabs, chunkType);
-//      } else if (size <= MAX_BIG) {
-//        return allocateBig(size, useSlabs);
-      } else {
-        return allocateHuge(size, useSlabs, chunkType);
-      }
-    }
-    
-    private Chunk allocateFromFragments(int chunkSize, ChunkType chunkType) {
-      do {
-        final int lastAllocationId = this.lastFragmentAllocation.get();
-        for (int i=lastAllocationId; i < this.fragmentList.size(); i++) {
-          Chunk result = allocateFromFragment(i, chunkSize, chunkType);
-          if (result != null) {
-            return result;
-          }
-        }
-        for (int i=0; i < lastAllocationId; i++) {
-          Chunk result = allocateFromFragment(i, chunkSize, chunkType);
-          if (result != null) {
-            return result;
-          }
-        }
-      } while (compact(chunkSize));
-      // We tried all the fragments and didn't find any free memory.
-      logOffHeapState(chunkSize);
-      final OutOfOffHeapMemoryException failure = new OutOfOffHeapMemoryException("Out of off-heap memory. Could not allocate size of " + chunkSize);
-      try {
-        throw failure;
-      } finally {
-        SimpleMemoryAllocatorImpl.this.ooohml.outOfOffHeapMemory(failure);
-      }
-    }
-    
-    private void logOffHeapState(int chunkSize) {
-      if (InternalDistributedSystem.getAnyInstance() != null) {
-        LogWriter lw = InternalDistributedSystem.getAnyInstance().getLogWriter();
-        lw.info("OutOfOffHeapMemory allocating size of " + chunkSize + ". allocated=" + this.allocatedSize.get() + " compactions=" + this.compactCount.get() + " objects=" + stats.getObjects() + " free=" + stats.getFreeMemory() + " fragments=" + stats.getFragments() + " largestFragment=" + stats.getLargestFragment() + " fragmentation=" + stats.getFragmentation());
-        logFragmentState(lw);
-        logTinyState(lw);
-//        logBigState(lw);
-        logHugeState(lw);
-      }
-    }
-
-    private void logHugeState(LogWriter lw) {
-      for (Chunk c: this.hugeChunkSet) {
-        lw.info("Free huge of size " + c.getSize());
-      }
-    }
-//    private void logBigState(LogWriter lw) {
-//      for (int i=0; i < this.bigFreeLists.length(); i++) {
-//        ConcurrentChunkStack cl = this.bigFreeLists.get(i);
-//        if (cl != null) {
-//          cl.logSizes(lw, "Free big of size ");
-//        }
-//      }
-//    }
-    private void logTinyState(LogWriter lw) {
-      for (int i=0; i < this.tinyFreeLists.length(); i++) {
-        SyncChunkStack cl = this.tinyFreeLists.get(i);
-        if (cl != null) {
-          cl.logSizes(lw, "Free tiny of size ");
+  static void validateAddress(long addr) {
+    validateAddressAndSize(addr, -1);
+  }
+  
+  static void validateAddressAndSize(long addr, int size) {
+    // if the caller does not have a "size" to provide then use -1
+    if ((addr & 7) != 0) {
+      StringBuilder sb = new StringBuilder();
+      sb.append("address was not 8 byte aligned: 0x").append(Long.toString(addr, 16));
+      SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.singleton;
+      if (ma != null) {
+        sb.append(". Valid addresses must be in one of the following ranges: ");
+        for (int i=0; i < ma.slabs.length; i++) {
+          long startAddr = ma.slabs[i].getMemoryAddress();
+          long endAddr = startAddr + ma.slabs[i].getSize();
+          sb.append("[").append(Long.toString(startAddr, 16)).append("..").append(Long.toString(endAddr, 16)).append("] ");
         }
       }
+      throw new IllegalStateException(sb.toString());
     }
-    private void logFragmentState(LogWriter lw) {
-      for (Fragment f: this.fragmentList) {
-        int freeSpace = f.freeSpace();
-        if (freeSpace > 0) {
-          lw.info("Fragment at " + f.getMemoryAddress() + " of size " + f.getSize() + " has " + freeSpace + " bytes free.");
-        }
-      }
+    if (addr >= 0 && addr < 1024) {
+      throw new IllegalStateException("addr was smaller than expected 0x" + addr);
     }
+    validateAddressAndSizeWithinSlab(addr, size);
+  }
 
-    private final AtomicInteger compactCount = new AtomicInteger();
-    /**
-     * Compacts memory and returns true if enough memory to allocate chunkSize
-     * is freed. Otherwise returns false;
-     * TODO OFFHEAP: what should be done about contiguous chunks that end up being bigger than 2G?
-     * Currently if we are given slabs bigger than 2G or that just happen to be contiguous and add
-     * up to 2G then the compactor may unify them together into a single Chunk and our 32-bit chunkSize
-     * field will overflow. This code needs to detect this and just create a chunk of 2G and then start
-     * a new one.
-     * Or to prevent it from happening we could just check the incoming slabs and throw away a few bytes
-     * to keep them from being contiguous.
-     */
-    private boolean compact(int chunkSize) {
-      final long startCompactionTime = getStats().startCompaction();
-      final int countPreSync = this.compactCount.get();
-      try {
-        synchronized (this) {
-          if (this.compactCount.get() != countPreSync) {
-            // someone else did a compaction while we waited on the sync.
-            // So just return true causing the caller to retry the allocation.
-            return true;
-          }
-          ArrayList<SyncChunkStack> freeChunks = new ArrayList<SyncChunkStack>();
-          collectFreeChunks(freeChunks);
-          final int SORT_ARRAY_BLOCK_SIZE = 128;
-          long[] sorted = new long[SORT_ARRAY_BLOCK_SIZE];
-          int sortedSize = 0;
-          boolean result = false;
-          int largestFragment = 0;
-          for (SyncChunkStack l: freeChunks) {
-            long addr = l.poll();
-            while (addr != 0) {
-              int idx = Arrays.binarySearch(sorted, 0, sortedSize, addr);
-              //System.out.println("DEBUG addr=" + addr + " size=" + Chunk.getSize(addr) + " idx="+idx + " sortedSize=" + sortedSize);
-              if (idx >= 0) {
-                throw new IllegalStateException("duplicate memory address found during compaction!");
-              }
-              idx = -idx;
-              idx--;
-              if (idx == sortedSize) {
-                // addr is > everything in the array
-                if (sortedSize == 0) {
-                  // nothing was in the array
-                  sorted[0] = addr;
-                  sortedSize++;
-                } else {
-                  // see if we can conflate into sorted[idx]
-                  long lowAddr = sorted[idx-1];
-                  int lowSize = Chunk.getSize(lowAddr);
-                  if (lowAddr + lowSize == addr) {
-                    // append the addr chunk to lowAddr
-                    Chunk.setSize(lowAddr, lowSize + Chunk.getSize(addr));
-                  } else {
-                    if (sortedSize >= sorted.length) {
-                      long[] newSorted = new long[sorted.length+SORT_ARRAY_BLOCK_SIZE];
-                      System.arraycopy(sorted, 0, newSorted, 0, sorted.length);
-                      sorted = newSorted;
-                    }
-                    sortedSize++;
-                    sorted[idx] = addr;
-                  }
-                }
-              } else {
-                int addrSize = Chunk.getSize(addr);
-                long highAddr = sorted[idx];
-                if (addr + addrSize == highAddr) {
-                  // append highAddr chunk to addr
-                  Chunk.setSize(addr, addrSize + Chunk.getSize(highAddr));
-                  sorted[idx] = addr;
-                } else {
-                  boolean insert = idx==0;
-                  if (!insert) {
-                    long lowAddr = sorted[idx-1];
-  //                  if (lowAddr == 0L) {
-  //                    long[] tmp = Arrays.copyOf(sorted, sortedSize);
-  //                    throw new IllegalStateException("addr was zero at idx=" + (idx-1) + " sorted="+ Arrays.toString(tmp));
-  //                  }
-                    int lowSize = Chunk.getSize(lowAddr);
-                    if (lowAddr + lowSize == addr) {
-                      // append the addr chunk to lowAddr
-                      Chunk.setSize(lowAddr, lowSize + addrSize);
-                    } else {
-                      insert = true;
-                    }
-                  }
-                  if (insert) {
-                    if (sortedSize >= sorted.length) {
-                      long[] newSorted = new long[sorted.length+SORT_ARRAY_BLOCK_SIZE];
-                      System.arraycopy(sorted, 0, newSorted, 0, idx);
-                      newSorted[idx] = addr;
-                      System.arraycopy(sorted, idx, newSorted, idx+1, sortedSize-idx);
-                      sorted = newSorted;
-                    } else {
-                      System.arraycopy(sorted, idx, sorted, idx+1, sortedSize-idx);
-                      sorted[idx] = addr;
-                    }
-                    sortedSize++;
-                  }
-                }
+  static void validateAddressAndSizeWithinSlab(long addr, int size) {
+    if (DO_EXPENSIVE_VALIDATION) {
+      SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.singleton;
+      if (ma != null) {
+        for (int i=0; i < ma.slabs.length; i++) {
+          if (ma.slabs[i].getMemoryAddress() <= addr && addr < (ma.slabs[i].getMemoryAddress() + ma.slabs[i].getSize())) {
+            // validate addr + size is within the same slab
+            if (size != -1) { // skip this check if size is -1
+              if (!(ma.slabs[i].getMemoryAddress() <= (addr+size-1) && (addr+size-1) < (ma.slabs[i].getMemoryAddress() + ma.slabs[i].getSize()))) {
+                throw new IllegalStateException(" address 0x" + Long.toString(addr+size-1, 16) + " does not address the original slab memory");
               }
-              addr = l.poll();
-            }
-          }
-          for (int i=sortedSize-1; i > 0; i--) {
-            long addr = sorted[i];
-            long lowAddr = sorted[i-1];
-            int lowSize = Chunk.getSize(lowAddr);
-            if (lowAddr + lowSize == addr) {
-              // append addr chunk to lowAddr
-              Chunk.setSize(lowAddr, lowSize + Chunk.getSize(addr));
-              sorted[i] = 0L;
-            }
-          }
-          this.lastFragmentAllocation.set(0);
-          ArrayList<Fragment> tmp = new ArrayList<Fragment>();
-          for (int i=sortedSize-1; i >= 0; i--) {
-            long addr = sorted[i];
-            if (addr == 0L) continue;
-            int addrSize = Chunk.getSize(addr);
-            Fragment f = new Fragment(addr, addrSize);
-            if (addrSize >= chunkSize) {
-              result = true;
-            }
-            if (addrSize > largestFragment) {
-              largestFragment = addrSize;
-              // TODO it might be better to sort them biggest first
-              tmp.add(0, f);
-            } else {
-              tmp.add(f);
-            }
-          }
-          this.fragmentList.addAll(tmp);
-          
-          // Reinitialize fragments with fill pattern data
-          if(validateMemoryWithFill) {
-            fillFragments();
-          }
-          
-          // Signal any waiters that a compaction happened.
-          this.compactCount.incrementAndGet();
-          
-          getStats().setLargestFragment(largestFragment);
-          getStats().setFragments(tmp.size());        
-          updateFragmentation();
-          
-          return result;
-        } // sync
-      } finally {
-        getStats().endCompaction(startCompactionTime);
-      }
-    }
-    
-    private void updateFragmentation() {      
-      long freeSize = getStats().getFreeMemory();
-
-      // Calculate free space fragmentation only if there is free space available.
-      if(freeSize > 0) {
-        long largestFragment = getStats().getLargestFragment();
-        long numerator = freeSize - largestFragment;
-        
-        double percentage = (double) numerator / (double) freeSize;
-        percentage *= 100d;
-        
-        int wholePercentage = (int) Math.rint(percentage);
-        getStats().setFragmentation(wholePercentage);
-      } else {
-        // No free space? Then we have no free space fragmentation.
-        getStats().setFragmentation(0);
-      }
-    }
-    
-    private void collectFreeChunks(List<SyncChunkStack> l) {
-      collectFreeFragmentChunks(l);
-      collectFreeHugeChunks(l);
-//      collectFreeBigChunks(l);
-      collectFreeTinyChunks(l);
-    }
-    private void collectFreeFragmentChunks(List<SyncChunkStack> l) {
-      if (this.fragmentList.size() == 0) return;
-      SyncChunkStack result = new SyncChunkStack();
-      for (Fragment f: this.fragmentList) {
-        int offset;
-        int diff;
-        do {
-          offset = f.getFreeIndex();
-          diff = f.getSize() - offset;
-        } while (diff >= Chunk.MIN_CHUNK_SIZE && !f.allocate(offset, offset+diff));
-        if (diff < Chunk.MIN_CHUNK_SIZE) {
-          if (diff > 0) {
-            logger.debug("Lost memory of size {}", diff);
-          }
-          // fragment is too small to turn into a chunk
-          // TODO we need to make sure this never happens
-          // by keeping sizes rounded. I think I did this
-          // by introducing MIN_CHUNK_SIZE and by rounding
-          // the size of huge allocations.
-          continue;
-        }
-        long chunkAddr = f.getMemoryAddress()+offset;
-        Chunk.setSize(chunkAddr, diff);
-        result.offer(chunkAddr);
-      }
-      // All the fragments have been turned in to chunks so now clear them
-      // The compaction will create new fragments.
-      this.fragmentList.clear();
-      if (!result.isEmpty()) {
-        l.add(result);
-      }
-    }
-    private void collectFreeTinyChunks(List<SyncChunkStack> l) {
-      for (int i=0; i < this.tinyFreeLists.length(); i++) {
-        SyncChunkStack cl = this.tinyFreeLists.get(i);
-        if (cl != null) {
-          long head = cl.clear();
-          if (head != 0L) {
-            l.add(new SyncChunkStack(head));
-          }
-        }
-      }
-    }
-//    private void collectFreeBigChunks(List<ConcurrentChunkStack> l) {
-//      for (int i=0; i < this.bigFreeLists.length(); i++) {
-//        ConcurrentChunkStack cl = this.bigFreeLists.get(i);
-//        if (cl != null) {
-//          long head = cl.clear();
-//          if (head != 0L) {
-//            l.add(new ConcurrentChunkStack(head));
-//          }
-//        }
-//      }
-//    }
-    public void collectFreeHugeChunks(List<SyncChunkStack> l) {
-      Chunk c = this.hugeChunkSet.pollFirst();
-      SyncChunkStack result = null;
-      while (c != null) {
-        if (result == null) {
-          result = new SyncChunkStack();
-          l.add(result);
-        }
-        result.offer(c.getMemoryAddress());
-        c = this.hugeChunkSet.pollFirst();
-      }
-    }
-    
-    private Chunk allocateFromFragment(final int fragIdx, final int chunkSize, ChunkType chunkType) {
-      if (fragIdx >= this.fragmentList.size()) return null;
-      final Fragment fragment;
-      try {
-        fragment = this.fragmentList.get(fragIdx);
-      } catch (IndexOutOfBoundsException ignore) {
-        // A concurrent compaction can cause this.
-        return null;
-      }
-      boolean retryFragment;
-      do {
-        retryFragment = false;
-        int oldOffset = fragment.getFreeIndex();
-        int fragmentSize = fragment.getSize();
-        int fragmentFreeSize = fragmentSize - oldOffset;
-        if (fragmentFreeSize >= chunkSize) {
-          // this fragment has room
-          // Try to allocate up to BATCH_SIZE more chunks from it
-          int allocSize = chunkSize * BATCH_SIZE;
-          if (allocSize > fragmentFreeSize) {
-            allocSize = (fragmentFreeSize / chunkSize) * chunkSize;
-          }
-          int newOffset = oldOffset + allocSize;
-          int extraSize = fragmentSize - newOffset;
-          if (extraSize < Chunk.MIN_CHUNK_SIZE) {
-            // include these last few bytes of the fragment in the allocation.
-            // If we don't then they will be lost forever.
-            // The extraSize bytes only apply to the first chunk we allocate (not the batch ones).
-            newOffset += extraSize;
-          } else {
-            extraSize = 0;
-          }
-          if (fragment.allocate(oldOffset, newOffset)) {
-            // We did the allocate!
-            this.lastFragmentAllocation.set(fragIdx);
-            Chunk result = chunkFactory.newChunk(fragment.getMemoryAddress()+oldOffset, chunkSize+extraSize, chunkType);
-            allocSize -= chunkSize+extraSize;
-            oldOffset += extraSize;
-            while (allocSize > 0) {
-              oldOffset += chunkSize;
-              // we add the batch ones immediately to the freelist
-              result.readyForFree();
-              free(result.getMemoryAddress(), false);
-              result = chunkFactory.newChunk(fragment.getMemoryAddress()+oldOffset, chunkSize, chunkType);
-              allocSize -= chunkSize;
-            }
-            
-            if(validateMemoryWithFill) {
-              result.validateFill();
             }
-            
-            return result;
-          } else {
-            // TODO OFFHEAP: if batch allocations are disabled should we not call basicAllocate here?
-            // Since we know another thread did a concurrent alloc
-            // that possibly did a batch check the free list again.
-            Chunk result = basicAllocate(chunkSize, false, chunkType);
-            if (result != null) {
-              return result;
-            }
-            retryFragment = true;
-          }
-        }
-      } while (retryFragment);
-      return null; // did not find enough free space in this fragment
-    }
-
-    private int round(int multiple, int value) {
-      return (int) ((((long)value + (multiple-1)) / multiple) * multiple);
-    }
-    private Chunk allocateTiny(int size, boolean useFragments, ChunkType chunkType) {
-      return basicAllocate(getNearestTinyMultiple(size), TINY_MULTIPLE, 0, this.tinyFreeLists, useFragments, chunkType);
-    }
-//    private Chunk allocateBig(int size, boolean useFragments) {
-//      return basicAllocate(getNearestBigMultiple(size), BIG_MULTIPLE, BIG_OFFSET, this.bigFreeLists, useFragments);
-//    }
-    private Chunk basicAllocate(int idx, int multiple, int offset, AtomicReferenceArray<SyncChunkStack> freeLists, boolean useFragments, ChunkType chunkType) {
-      SyncChunkStack clq = freeLists.get(idx);
-      if (clq != null) {
-        long memAddr = clq.poll();
-        if (memAddr != 0) {
-          Chunk result = SimpleMemoryAllocatorImpl.this.chunkFactory.newChunk(memAddr, chunkType);
-          
-          // Data integrity check.
-          if(validateMemoryWithFill) {          
-            result.validateFill();
-          }
-          
-          result.readyForAllocation(chunkType);
-          return result;
-        }
-      }
-      if (useFragments) {
-        return allocateFromFragments(((idx+1)*multiple)+offset, chunkType);
-      } else {
-        return null;
-      }
-    }
-    private Chunk allocateHuge(int size, boolean useFragments, ChunkType chunkType) {
-      // sizeHolder is a fake Chunk used to search our sorted hugeChunkSet.
-      Chunk sizeHolder = newFakeChunk(size);
-      NavigableSet<Chunk> ts = this.hugeChunkSet.tailSet(sizeHolder);
-      Chunk result = ts.pollFirst();
-      if (result != null) {
-        if (result.getSize() - (HUGE_MULTIPLE - Chunk.OFF_HEAP_HEADER_SIZE) < size) {
-          // close enough to the requested size; just return it.
-          
-          // Data integrity check.
-          if(validateMemoryWithFill) {          
-            result.validateFill();
-          }
-          if (chunkType.getSrcType() != Chunk.getSrcType(result.getMemoryAddress())) {
-            // The java wrapper class that was cached in the huge chunk list is the wrong type.
-            // So allocate a new one and garbage collect the old one.
-            result = SimpleMemoryAllocatorImpl.this.chunkFactory.newChunk(result.getMemoryAddress(), chunkType);
+            return;
           }
-          result.readyForAllocation(chunkType);
-          return result;
-        } else {
-          this.hugeChunkSet.add(result);
-        }
-      }
-      if (useFragments) {
-        // We round it up to the next multiple of TINY_MULTIPLE to make
-        // sure we always have chunks allocated on an 8 byte boundary.
-        return allocateFromFragments(round(TINY_MULTIPLE, size), chunkType);
-      } else {
-        return null;
-      }
-    }
-    
-    @SuppressWarnings("synthetic-access")
-    public void free(long addr) {
-      free(addr, true);
-    }
-    
-    private void free(long addr, boolean updateStats) {
-      int cSize = Chunk.getSize(addr);
-      if (updateStats) {
-        stats.incObjects(-1);
-        this.allocatedSize.addAndGet(-cSize);
-        stats.incUsedMemory(-cSize);
-        stats.incFreeMemory(cSize);
-        notifyListeners();
-      }
-      /*Chunk oldTlChunk = this.tlCache.get();
-      this.tlCache.set(c);
-      if (oldTlChunk != null) {
-        int oldTlcSize = oldTlChunk.getSize();
-        if (oldTlcSize <= MAX_TINY) {
-          freeTiny(oldTlChunk);
-        } else if (oldTlcSize <= MAX_BIG) {
-          freeBig(oldTlChunk);
-        } else {
-          freeHuge(oldTlChunk);
-        }
-      }*/
-      if (cSize <= MAX_TINY) {
-        freeTiny(addr, cSize);
-//      } else if (cSize <= MAX_BIG) {
-//        freeBig(addr, cSize);
-      } else {
-        freeHuge(addr, cSize);
-      }
-    }
-    private void freeTiny(long addr, int cSize) {
-      basicFree(addr, getNearestTinyMultiple(cSize), this.tinyFreeLists);
-    }
-//    private void freeBig(long addr, int cSize) {
-//      basicFree(addr, getNearestBigMultiple(cSize), this.bigFreeLists);
-//    }
-    private void basicFree(long addr, int idx, AtomicReferenceArray<SyncChunkStack> freeLists) {
-      SyncChunkStack clq = freeLists.get(idx);
-      if (clq != null) {
-        clq.offer(addr);
-      } else {
-        clq = new SyncChunkStack();
-        clq.offer(addr);
-        if (!freeLists.compareAndSet(idx, null, clq)) {
-          clq = freeLists.get(idx);
-          clq.offer(addr);
         }
+        throw new IllegalStateException(" address 0x" + Long.toString(addr, 16) + " does not address the original slab memory");
       }
-      
-    }
-    private void freeHuge(long addr, int cSize) {
-      this.hugeChunkSet.add(SimpleMemoryAllocatorImpl.this.chunkFactory.newChunk(addr)); // TODO make this a collection of longs
     }
   }
   
+  /** The inspection snapshot for MemoryInspector */
+  private List<MemoryBlock> memoryBlocks;
   
-  
-  
-  /*private Chunk newChunk(long addr, int chunkSize) {
-    return this.chunkFactory.newChunk(addr, chunkSize);
-  }*/
-  
-  private Chunk newFakeChunk(int chunkSize) {
-    return new FakeChunk(chunkSize);
+  @Override
+  public MemoryInspector getMemoryInspector() {
+    return this;
   }
   
-  
-  public static interface ChunkFactory  {
-    /**
-     * Create a new chunk of the given size and type at the given address.
-     */
-    Chunk newChunk(long address, int chunkSize, ChunkType chunkType);
-    /**
-     * Create a new chunk for a block of memory (identified by address)
-     * that has already been allocated.
-     * The size and type are derived from the existing object header.
-     */
-    Chunk newChunk(long address);
-    /**
-     * Create a new chunk of the given type for a block of memory (identified by address)
-     * that has already been allocated.
-     * The size is derived from the existing object header.
-     */
-    Chunk newChunk(long address, ChunkType chunkType);
-    /**
-     * Given the address of an existing chunk return its ChunkType.
-     */
-    ChunkType getChunkTypeForAddress(long address);
-    /**
-     * Given the rawBits from the object header of an existing chunk
-     * return its ChunkType.
-     */
-    ChunkType getChunkTypeForRawBits(int bits);
+  @Override
+  public synchronized void clearInspectionSnapshot() {
+    this.memoryBlocks = null;
   }
   
-  private static class GemFireChunkFactory implements ChunkFactory {
-    @Override
-    public Chunk newChunk(long address, int chunkSize, ChunkType chunkType) {
-      assert chunkType.equals(GemFireChunk.TYPE);
-      return new GemFireChunk(address,chunkSize);
-    }
-
-    @Override
-    public Chunk newChunk(long address) {
-      return new GemFireChunk(address);
+  @Override
+  public synchronized void createInspectionSnapshot() {
+    List<MemoryBlock> value = this.memoryBlocks;
+    if (value == null) {
+      value = getOrderedBlocks();
+      this.memoryBlocks = value;
     }
-
-    @Override
-    public Chunk newChunk(long address, ChunkType chunkType) {
-      assert chunkType.equals(GemFireChunk.TYPE);
-      return new GemFireChunk(address);
-    }
-
-    @Override
-    public ChunkType getChunkTypeForAddress(long address) {
-      assert Chunk.getSrcType(address) == Chunk.SRC_TYPE_GFE;
-      return GemFireChunk.TYPE;
-    }
-
-    @Override
-    public ChunkType getChunkTypeForRawBits(int bits) {
-      assert Chunk.getSrcTypeFromRawBits(bits) == Chunk.SRC_TYPE_GFE;
-      return GemFireChunk.TYPE;
-    }
-  }
-  
-  
-  /**
-   * Used to keep the heapForm around while an operation is still in progress.
-   * This allows the operation to access the serialized heap form instead of copying
-   * it from offheap. See bug 48135.
-   * 
-   * @author darrel
-   *
-   */
-  public static class ChunkWithHeapForm extends GemFireChunk {
-    private final byte[] heapForm;
-    
-    public ChunkWithHeapForm(GemFireChunk chunk, byte[] heapForm) {
-      super(chunk);
-      this.heapForm = heapForm;
-    }
-
-    @Override
-    protected byte[] getRawBytes() {
-      return this.heapForm;
-    }
-    
-    public Chunk getChunkWithoutHeapForm() {
-      return new GemFireChunk(this);
-    }
-  }
-  
-  public static abstract class ChunkType {
-    public abstract int getSrcType();
-    public abstract Chunk newChunk(long memoryAddress);
-    public abstract Chunk newChunk(long memoryAddress, int chunkSize);
-  }
-  
-  public static class GemFireChunkType extends ChunkType {
-    private static final GemFireChunkType singleton = new GemFireChunkType();
-    public static GemFireChunkType singleton() { return singleton; }
-    
-    private GemFireChunkType() {}
-
-    @Override
-    public int getSrcType() {
-      return Chunk.SRC_TYPE_GFE;
-    }
-
-    @Override
-    public Chunk newChunk(long memoryAddress) {      
-      return new GemFireChunk(memoryAddress);
-    }
-
-    @Override
-    public Chunk newChunk(long memoryAddress, int chunkSize) {     
-      return new GemFireChunk(memoryAddress, chunkSize);
-    }
-  }
-  public static class GemFireChunk extends Chunk {
-    public static final ChunkType TYPE = new ChunkType() {
-      @Override
-      public int getSrcType() {
-        return Chunk.SRC_TYPE_GFE;
-      }
-      @Override
-      public Chunk newChunk(long memoryAddress) {
-        return new GemFireChunk(memoryAddress);
-      }
-      @Override
-      public Chunk newChunk(long memoryAddress, int chunkSize) {
-        return new GemFireChunk(memoryAddress, chunkSize);
-      }
-    };
-    public GemFireChunk(long memoryAddress, int chunkSize) {
-      super(memoryAddress, chunkSize, TYPE);
-    }
-
-    public GemFireChunk(long memoryAddress) {
-      super(memoryAddress);
-      // chunkType may be set by caller when it calls readyForAllocation
-    }
-    public GemFireChunk(GemFireChunk chunk) {
-      super(chunk);
-    }
-    @Override
-    public Chunk slice(int position, int limit) {
-      return new GemFireChunkSlice(this, position, limit);
-    }
-  }
-  public static class GemFireChunkSlice extends GemFireChunk {
-    private final int offset;
-    private final int capacity;
-    public GemFireChunkSlice(GemFireChunk gemFireChunk, int position, int limit) {
-      super(gemFireChunk);
-      this.offset = gemFireChunk.getBaseDataOffset() + position;
-      this.capacity = limit - position;
-    }
-    @Override
-    public int getDataSize() {
-      return this.capacity;
-    }
-    
-    @Override
-    protected long getBaseDataAddress() {
-      return super.getBaseDataAddress() + this.offset;
-    }
-    @Override
-    protected int getBaseDataOffset() {
-      return this.offset;
-    }
-  }
-  /**
-   * Note: this class has a natural ordering that is inconsistent with equals.
-   * Instances of this class should have a short lifetime. We do not store references
-   * to it in the cache. Instead the memoryAddress is stored in a primitive field in
-   * the cache and if used it will then, if needed, create an instance of this class.
-   */
-  public static abstract class Chunk extends OffHeapCachedDeserializable implements Comparable<Chunk>, ConcurrentBag.Node, MemoryBlock {
-    /**
-     * The unsafe memory address of the first byte of this chunk
-     */
-    private final long memoryAddress;
-    
-    /**
-     * The useCount, chunkSize, dataSizeDelta, isSerialized, and isCompressed
-     * are all stored in off heap memory in a HEADER. This saves heap memory
-     * by using off heap.
-     */
-    public final static int OFF_HEAP_HEADER_SIZE = 4 + 4;
-    /**
-     * We need to smallest chunk to at least have enough room for a hdr
-     * and for an off heap ref (which is a long).
-     */
-    public final static int MIN_CHUNK_SIZE = OFF_HEAP_HEADER_SIZE + 8;
-    /**
-     * int field.
-     * The number of bytes in this chunk.
-     */
-    private final static int CHUNK_SIZE_OFFSET = 0;
-    /**
-     * Volatile int field
-     * The upper two bits are used for the isSerialized
-     * and isCompressed flags.
-     * The next three bits are used to encode the SRC_TYPE enum.
-     * The lower 3 bits of the most significant byte contains a magic number to help us detect
-     * if we are changing the ref count of an object that has been released.
-     * The next byte contains the dataSizeDelta.
-     * The number of bytes of logical data in this chunk.
-     * Since the number of bytes of logical data is always <= chunkSize
-     * and since chunkSize never changes, we have dataSize be
-     * a delta whose max value would be HUGE_MULTIPLE-1.
-     * The lower two bytes contains the use count.
-     */
-    private final static int REF_COUNT_OFFSET = 4;
-    /**
-     * The upper two bits are used for the isSerialized
-     * and isCompressed flags.
-     */
-    private final static int IS_SERIALIZED_BIT =    0x80000000;
-    private final static int IS_COMPRESSED_BIT =    0x40000000;
-    private final static int SRC_TYPE_MASK = 0x38000000;
-    private final static int SRC_TYPE_SHIFT = 16/*refCount*/+8/*dataSize*/+3/*magicSize*/;
-    private final static int MAGIC_MASK = 0x07000000;
-    private final static int MAGIC_NUMBER = 0x05000000;
-    private final static int DATA_SIZE_DELTA_MASK = 0x00ff0000;
-    private final static int DATA_SIZE_SHIFT = 16;
-    private final static int REF_COUNT_MASK =       0x0000ffff;
-    private final static int MAX_REF_COUNT = 0xFFFF;
-    final static long FILL_PATTERN = 0x3c3c3c3c3c3c3c3cL;
-    final static byte FILL_BYTE = 0x3c;
-    
-    // The 8 bits reserved for SRC_TYPE are basically no longer used.
-    // So we could free up these 8 bits for some other use or we could
-    // keep them for future extensions.
-    // If we ever want to allocate other "types" into a chunk of off-heap
-    // memory then the SRC_TYPE would be the way to go.
-    // For example we may want to allocate the memory for the off-heap
-    // RegionEntry in off-heap memory without it being of type GFE.
-    // When it is of type GFE then it either needs to be the bytes
-    // of a byte array or it needs to be a serialized java object.
-    // For the RegionEntry we may want all the primitive fields of
-    // the entry at certain offsets in the off-heap memory so we could
-    // access them directly in native byte format (i.e. no serialization).
-    // Note that for every SRC_TYPE we should have a ChunkType subclass.
-    public final static int SRC_TYPE_UNUSED0 = 0 << SRC_TYPE_SHIFT;
-    public final static int SRC_TYPE_UNUSED1 = 1 << SRC_TYPE_SHIFT;
-    public final static int SRC_TYPE_UNUSED2 = 2 << SRC_TYPE_SHIFT;
-    public final static int SRC_TYPE_UNUSED3 = 3 << SRC_TYPE_SHIFT;
-    public final static int SRC_TYPE_GFE = 4 << SRC_TYPE_SHIFT;
-    public final static int SRC_TYPE_UNUSED5 = 5 << SRC_TYPE_SHIFT;
-    public final static int SRC_TYPE_UNUSED6 = 6 << SRC_TYPE_SHIFT;
-    public final static int SRC_TYPE_UNUSED7 = 7 << SRC_TYPE_SHIFT;
-    
-    protected Chunk(long memoryAddress, int chunkSize, ChunkType chunkType) {
-      validateAddressAndSize(memoryAddress, chunkSize);
-      this.memoryAddress = memoryAddress;
-      setSize(chunkSize);
-      UnsafeMemoryChunk.writeAbsoluteIntVolatile(getMemoryAddress()+REF_COUNT_OFFSET, MAGIC_NUMBER|chunkType.getSrcType());
-    }
-    public void readyForFree() {
-      UnsafeMemoryChunk.writeAbsoluteIntVolatile(getMemoryAddress()+REF_COUNT_OFFSET, 0);
-    }
-    public void readyForAllocation(ChunkType chunkType) {
-      if (!UnsafeMemoryChunk.writeAbsoluteIntVolatile(getMemoryAddress()+REF_COUNT_OFFSET, 0, MAGIC_NUMBER|chunkType.getSrcType())) {
-        throw new IllegalStateException("Expected 0 but found " + Integer.toHexString(UnsafeMemoryChunk.readAbsoluteIntVolatile(getMemoryAddress()+REF_COUNT_OFFSET)));
-      }
-    }
-    /**
-     * Should only be used by FakeChunk subclass
-     */
-    protected Chunk() {
-      this.memoryAddress = 0L;
-    }
-    
-    /**
-     * Used to create a Chunk given an existing, already allocated,
-     * memoryAddress. The off heap header has already been initialized.
-     */
-    protected Chunk(long memoryAddress) {
-      validateAddress(memoryAddress);
-      this.memoryAddress = memoryAddress;
-    }
-    
-    protected Chunk(Chunk chunk) {
-      this.memoryAddress = chunk.memoryAddress;
-    }
-    
-    /**
-     * Throw an exception if this chunk is not allocated
-     */
-    public void checkIsAllocated() {
-      int originalBits = UnsafeMemoryChunk.readAbsoluteIntVolatile(this.memoryAddress+REF_COUNT_OFFSET);
-      if ((originalBits&MAGIC_MASK) != MAGIC_NUMBER) {
-        throw new IllegalStateException("It looks like this off heap memory was already freed. rawBits=" + Integer.toHexString(originalBits));
-      }
-    }
-    
-    public void incSize(int inc) {
-      setSize(getSize()+inc);
-    }
-    
-    protected void beforeReturningToAllocator() {
-      
-    }
-
-    @Override
-    public int getSize() {
-      return getSize(this.memoryAddress);
-    }
-
-    public void setSize(int size) {
-      setSize(this.memoryAddress, size);
-    }
-
-    public long getMemoryAddress() {
-      return this.memoryAddress;
-    }
-    
-    public int getDataSize() {
-      /*int dataSizeDelta = UnsafeMemoryChunk.readAbsoluteInt(this.memoryAddress+REF_COUNT_OFFSET);
-      dataSizeDelta &= DATA_SIZE_DELTA_MASK;
-      dataSizeDelta >>= DATA_SIZE_SHIFT;
-      return getSize() - dataSizeDelta;*/
-      return getDataSize(this.memoryAddress);
-    }
-    
-    protected static int getDataSize(long memoryAdress) {
-      int dataSizeDelta = UnsafeMemoryChunk.readAbsoluteInt(memoryAdress+REF_COUNT_OFFSET);
-      dataSizeDelta &= DATA_SIZE_DELTA_MASK;
-      dataSizeDelta >>= DATA_SIZE_SHIFT;
-      return getSize(memoryAdress) - dataSizeDelta;
-    }
-    
-    protected long getBaseDataAddress() {
-      return this.memoryAddress+OFF_HEAP_HEADER_SIZE;
-    }
-    protected int getBaseDataOffset() {
-      return 0;
-    }
-    
-    /**
-     * Creates and returns a direct ByteBuffer that contains the contents of this Chunk.
-     * Note that the returned ByteBuffer has a reference to this chunk's
-     * off-heap address so it can only be used while this Chunk is retained.
-     * @return the created direct byte buffer or null if it could not be created.
-     */
-    @Unretained
-    public ByteBuffer createDirectByteBuffer() {
-      return basicCreateDirectByteBuffer(getBaseDataAddress(), getDataSize());
-    }
-    @Override
-    public void sendTo(DataOutput out) throws IOException {
-      if (!this.isCompressed() && out instanceof HeapDataOutputStream) {
-        ByteBuffer bb = createDirectByteBuffer();
-        if (bb != null) {
-          HeapDataOutputStream hdos = (HeapDataOutputStream) out;
-          if (this.isSerialized()) {
-            hdos.write(bb);
-          } else {
-            hdos.writeByte(DSCODE.BYTE_ARRAY);
-            InternalDataSerializer.writeArrayLength(bb.remaining(), hdos);
-            hdos.write(bb);
-          }
-          return;
-        }
-      }
-      super.sendTo(out);
-    }
-    
-    @Override
-    public void sendAsByteArray(DataOutput out) throws IOException {
-      if (!isCompressed() && out instanceof HeapDataOutputStream) {
-        ByteBuffer bb = createDirectByteBuffer();
-        if (bb != null) {
-          HeapDataOutputStream hdos = (HeapDataOutputStream) out;
-          InternalDataSerializer.writeArrayLength(bb.remaining(), hdos);
-          hdos.write(bb);
-          return;
-        }
-      }
-      super.sendAsByteArray(out);
-    }
-       
-    private static volatile Class dbbClass = null;
-    private static volatile Constructor dbbCtor = null;
-    private static volatile boolean dbbCreateFailed = false;
-    
-    /**
-     * @return the created direct byte buffer or null if it could not be created.
-     */
-    private static ByteBuffer basicCreateDirectByteBuffer(long baseDataAddress, int dataSize) {
-      if (dbbCreateFailed) {
-        return null;
-      }
-      Constructor ctor = dbbCtor;
-      if (ctor == null) {
-        Class c = dbbClass;
-        if (c == null) {
-          try {
-            c = Class.forName("java.nio.DirectByteBuffer");
-          } catch (ClassNotFoundException e) {
-            //throw new IllegalStateException("Could not find java.nio.DirectByteBuffer", e);
-            dbbCreateFailed = true;
-            dbbAddressFailed = true;
-            return null;
-          }
-          dbbClass = c;
-        }
-        try {
-          ctor = c.getDeclaredConstructor(long.class, int.class);
-        } catch (NoSuchMethodException | SecurityException e) {
-          //throw new IllegalStateException("Could not get constructor DirectByteBuffer(long, int)", e);
-          dbbClass = null;
-          dbbCreateFailed = true;
-          return null;
-        }
-        ctor.setAccessible(true);
-        dbbCtor = ctor;
-      }
-      try {
-        return (ByteBuffer)ctor.newInstance(baseDataAddress, dataSize);
-      } catch (InstantiationException | IllegalAccessException | IllegalArgumentException | InvocationTargetException e) {
-        //throw new IllegalStateException("Could not create an instance using DirectByteBuffer(long, int)", e);
-        dbbClass = null;
-        dbbCtor = null;
-        dbbCreateFailed = true;
-        return null;
-      }
-    }
-    private static volatile Method dbbAddressMethod = null;
-    private static volatile boolean dbbAddressFailed = false;
-    
-    /**
-     * Returns the address of the Unsafe memory for the first byte of a direct ByteBuffer.
-     * If the buffer is not direct or the address can not be obtained return 0.
-     */
-    public static long getDirectByteBufferAddress(ByteBuffer bb) {
-      if (!bb.isDirect()) {
-        return 0L;
-      }
-      if (dbbAddressFailed) {
-        return 0L;
-      }
-      Method m = dbbAddressMethod;
-      if (m == null) {
-        Class c = dbbClass;
-        if (c == null) {
-          try {
-            c = Class.forName("java.nio.DirectByteBuffer");
-          } catch (ClassNotFoundException e) {
-            //throw new IllegalStateException("Could not find java.nio.DirectByteBuffer", e);
-            dbbCreateFailed = true;
-            dbbAddressFailed = true;
-            return 0L;
-          }
-          dbbClass = c;
-        }
-        try {
-          m = c.getDeclaredMethod("address");
-        } catch (NoSuchMethodException | SecurityException e) {
-          //throw new IllegalStateException("Could not get method DirectByteBuffer.address()", e);
-          dbbClass = null;
-          dbbAddressFailed = true;
-          return 0L;
-        }
-        m.setAccessible(true);
-        dbbAddressMethod = m;
-      }
-      try {
-        return (Long)m.invoke(bb);
-      } catch (IllegalAccessException | IllegalArgumentException | InvocationTargetException e) {
-        //throw new IllegalStateException("Could not create an invoke DirectByteBuffer.address()", e);
-        dbbClass = null;
-        dbbAddressMethod = null;
-        dbbAddressFailed = true;
-        return 0L;
-      }
-    }
-    /**
-     * Returns an address that can be used with unsafe apis to access this chunks memory.
-     * @param offset the offset from this chunk's first byte of the byte the returned address should point to. Must be >= 0.
-     * @param size the number of bytes that will be read using the returned address. Assertion will use this to verify that all the memory accessed belongs to this chunk. Must be > 0.
-     * @return a memory address that can be used with unsafe apis
-     */
-    public long getUnsafeAddress(int offset, int size) {
-      assert offset >= 0 && offset + size <= getDataSize(): "Offset=" + offset + ",size=" + size + ",dataSize=" + getDataSize() + ", chunkSize=" + getSize() + ", but offset + size must be <= " + getDataSize();
-      assert size > 0;
-      long result = getBaseDataAddress() + offset;
-      // validateAddressAndSizeWithinSlab(result, size);
-      return result;
-    }
-    
-    @Override
-    public byte readByte(int offset) {
-      assert offset < getDataSize();
-      return UnsafeMemoryChunk.readAbsoluteByte(getBaseDataAddress() + offset);
-    }
-
-    @Override
-    public void writeByte(int offset, byte value) {
-      assert offset < getDataSize();
-      UnsafeMemoryChunk.writeAbsoluteByte(getBaseDataAddress() + offset, value);
-    }
-
-    @Override
-    public void readBytes(int offset, byte[] bytes) {
-      readBytes(offset, bytes, 0, bytes.length);
-    }
-
-    @Override
-    public void writeBytes(int offset, byte[] bytes) {
-      writeBytes(offset, bytes, 0, bytes.length);
-    }
-
-    public long getAddressForReading(int offset, int size) {
-      assert offset+size <= getDataSize();
-      return getBaseDataAddress() + offset;
-    }
-    
-    @Override
-    public void readBytes(int offset, byte[] bytes, int bytesOffset, int size) {
-      assert offset+size <= getDataSize();
-      UnsafeMemoryChunk.readAbsoluteBytes(getBaseDataAddress() + offset, bytes, bytesOffset, size);
-    }
-
-    @Override
-    public void writeBytes(int offset, byte[] bytes, int bytesOffset, int size) {
-      assert offset+size <= getDataSize();
-      validateAddressAndSizeWithinSlab(getBaseDataAddress() + offset, size);
-      UnsafeMemoryChunk.writeAbsoluteBytes(getBaseDataAddress() + offset, bytes, bytesOffset, size);
-    }
-    
-    @Override
-    public void release() {
-      release(this.memoryAddress, true);
-     }
-
-    @Override
-    public int compareTo(Chunk o) {
-      int result = Integer.signum(getSize() - o.getSize());
-      if (result == 0) {
-        // For the same sized chunks we really don't care about their order
-        // but we need compareTo to only return 0 if the two chunks are identical
-        result = Long.signum(getMemoryAddress() - o.getMemoryAddress());
-      }
-      return result;
-    }
-    
-    @Override
-    public boolean equals(Object o) {
-      if (o instanceof Chunk) {
-        return getMemoryAddress() == ((Chunk) o).getMemoryAddress();
-      }
-      return false;
-    }
-    
-    @Override
-    public int hashCode() {
-      long value = this.getMemoryAddress();
-      return (int)(value ^ (value >>> 32));
-    }
-
-    // OffHeapCachedDeserializable methods 
-    
-    @Override
-    public void setSerializedValue(byte[] value) {
-      writeBytes(0, value);
-    }
-    
-    public byte[] getDecompressedBytes(RegionEntryContext context) {
-      byte[] result = getCompressedBytes();
-      long time = context.getCachePerfStats().startDecompression();
-      result = context.getCompressor().decompress(result);
-      context.getCachePerfStats().endDecompression(time);      
-      return result;
-    }
-    
-    /**
-     * Returns the raw possibly compressed bytes of this chunk
-     */
-    public byte[] getCompressedBytes() {
-      byte[] result = new byte[getDataSize()];
-      readBytes(0, result);
-      //debugLog("reading", true);
-      getAllocator().getStats().incReads();
-      return result;
-    }
-    protected byte[] getRawBytes() {
-      byte[] result = getCompressedBytes();
-      // TODO OFFHEAP: change the following to assert !isCompressed();
-      if (isCompressed()) {
-        throw new UnsupportedOperationException();
-      }
-      return result;
-    }
-
-    @Override
-    public byte[] getSerializedValue() {
-      byte [] result = getRawBytes();
-      if (!isSerialized()) {
-        // The object is a byte[]. So we need to make it look like a serialized byte[] in our result
-        result = EntryEventImpl.serialize(result);
-      }
-      return result;
-    }
-    
-    @Override
-    public Object getDeserializedValue(Region r, RegionEntry re) {
-      if (isSerialized()) {
-        // TODO OFFHEAP: debug deserializeChunk
-        return EntryEventImpl.deserialize(getRawBytes());
-        //assert !isCompressed();
-        //return EntryEventImpl.deserializeChunk(this);
-      } else {
-        return getRawBytes();
-      }
-    }
-    
-    /**
-     * We want this to include memory overhead so use getSize() instead of getDataSize().
-     */
-    @Override
-    public int getSizeInBytes() {
-      // Calling getSize includes the off heap header size.
-      // We do not add anything to this since the size of the reference belongs to the region entry size
-      // not the size of this object.
-      return getSize();
-    }
-
-    @Override
-    public int getValueSizeInBytes() {
-      return getDataSize();
-    }
-
-    @Override
-    public void copyBytes(int src, int dst, int size) {
-      throw new UnsupportedOperationException("Implement if used");
-//      assert src+size <= getDataSize();
-//      assert dst+size < getDataSize();
-//      getSlabs()[this.getSlabIdx()].copyBytes(getBaseDataAddress()+src, getBaseDataAddress()+dst, size);
-    }
-
-    @Override
-    public boolean isSerialized() {
-      return (UnsafeMemoryChunk.readAbsoluteInt(this.memoryAddress+REF_COUNT_OFFSET) & IS_SERIALIZED_BIT) != 0;
-    }
-
-    @Override
-    public boolean isCompressed() {
-      return (UnsafeMemoryChunk.readAbsoluteInt(this.memoryAddress+REF_COUNT_OFFSET) & IS_COMPRESSED_BIT) != 0;
-    }
-
-    @Override
-    public boolean retain() {
-      return retain(this.memoryAddress);
-    }
-
-    @Override
-    public int getRefCount() {
-      return getRefCount(this.memoryAddress);
-    }
-
-    // By adding this one object ref to Chunk we are able to have free lists that only have memory overhead of a single objref per free item.
-    //private Node cbNodeNext;
-    @Override
-    public void setNextCBNode(Node next) {
-      throw new UnsupportedOperationException();
-      //this.cbNodeNext = next;
-    }
-
-    @Override
-    public Node getNextCBNode() {
-      throw new UnsupportedOperationException();
-      //return this.cbNodeNext;
-    }
-    public static int getSize(long memAddr) {
-      validateAddress(memAddr);
-      return UnsafeMemoryChunk.readAbsoluteInt(memAddr+CHUNK_SIZE_OFFSET);
-    }
-    public static void setSize(long memAddr, int size) {
-      validateAddressAndSize(memAddr, size);
-      UnsafeMemoryChunk.writeAbsoluteInt(memAddr+CHUNK_SIZE_OFFSET, size);
-    }
-    public static long getNext(long memAddr) {
-      validateAddress(memAddr);
-      return UnsafeMemoryChunk.readAbsoluteLong(memAddr+OFF_HEAP_HEADER_SIZE);
-    }
-    public static void setNext(long memAddr, long next) {
-      validateAddress(memAddr);
-      UnsafeMemoryChunk.writeAbsoluteLong(memAddr+OFF_HEAP_HEADER_SIZE, next);
-    }
-    @Override
-    public ChunkType getChunkType() {
-      return getAllocator().getChunkFactory().getChunkTypeForAddress(getMemoryAddress());
-    }
-    public static int getSrcTypeOrdinal(long memAddr) {
-      return getSrcType(memAddr) >> SRC_TYPE_SHIFT;
-    }
-    public static int getSrcType(long memAddr) {
-      return getSrcTypeFromRawBits(UnsafeMemoryChunk.readAbsoluteInt(memAddr+REF_COUNT_OFFSET));
-    }
-    public static int getSrcTypeFromRawBits(int rawBits) {
-      return rawBits & SRC_TYPE_MASK;
-    }
-    public static int getSrcTypeOrdinalFromRawBits(int rawBits) {
-      return getSrcTypeFromRawBits(rawBits) >> SRC_TYPE_SHIFT;
-    }
-    
-    /**
-     * Fills the chunk with a repeated byte fill pattern.
-     * @param baseAddress the starting address for a {@link Chunk}.
-     */
-    public static void fill(long baseAddress) {
-      long startAddress = baseAddress + MIN_CHUNK_SIZE;
-      int size = getSize(baseAddress) - MIN_CHUNK_SIZE;
-      
-      UnsafeMemoryChunk.fill(startAddress, size, FILL_BYTE);
-    }
-    
-    /**
-     * Validates that the fill pattern for this chunk has not been disturbed.  This method
-     * assumes the TINY_MULTIPLE is 8 bytes.
-     * @throws IllegalStateException when the pattern has been violated.
-     */
-    public void validateFill() {
-      assert TINY_MULTIPLE == 8;
-      
-      long startAddress = getMemoryAddress() + MIN_CHUNK_SIZE;
-      int size = getSize() - MIN_CHUNK_SIZE;
-      
-      for(int i = 0;i < size;i += TINY_MULTIPLE) {
-        if(UnsafeMemoryChunk.readAbsoluteLong(startAddress + i) != FILL_PATTERN) {
-          throw new IllegalStateException("Fill pattern violated for chunk " + getMemoryAddress() + " with size " + getSize());
-        }        
-      }
-    }
-
-    public void setSerialized(boolean isSerialized) {
-      if (isSerialized) {
-        int bits;
-        int originalBits;
-        do {
-          originalBits = UnsafeMemoryChunk.readAbsoluteIntVolatile(this.memoryAddress+REF_COUNT_OFFSET);
-          if ((originalBits&MAGIC_MASK) != MAGIC_NUMBER) {
-            throw new IllegalStateException("It looks like this off heap memory was already freed. rawBits=" + Integer.toHexString(originalBits));
-          }
-          bits = originalBits | IS_SERIALIZED_BIT;
-        } while (!UnsafeMemoryChunk.writeAbsoluteIntVolatile(this.memoryAddress+REF_COUNT_OFFSET, originalBits, bits));
-      }
-    }
-    public void setCompressed(boolean isCompressed) {
-      if (isCompressed) {
-        int bits;
-        int originalBits;
-        do {
-          originalBits = UnsafeMemoryChunk.readAbsoluteIntVolatile(this.memoryAddress+REF_COUNT_OFFSET);
-          if ((originalBits&MAGIC_MASK) != MAGIC_NUMBER) {
-            throw new IllegalStateException("It looks like this off heap memory was already freed. rawBits=" + Integer.toHexString(originalBits));
-          }
-          bits = originalBits | IS_COMPRESSED_BIT;
-        } while (!UnsafeMemoryChunk.writeAbsoluteIntVolatile(this.memoryAddress+REF_COUNT_OFFSET, originalBits, bits));
-      }
-    }
-    public void setDataSize(int dataSize) { // KIRK
-      assert dataSize <= getSize();
-      int delta = getSize() - dataSize;
-      assert delta <= (DATA_SIZE_DELTA_MASK >> DATA_SIZE_SHIFT);
-      delta <<= DATA_SIZE_SHIFT;
-      int bits;
-      int originalBits;
-      do {
-        originalBits = UnsafeMemoryChunk.readAbsoluteIntVolatile(this.memoryAddress+REF_COUNT_OFFSET);
-        if ((originalBits&MAGIC_MASK) != MAGIC_NUMBER) {
-          throw new IllegalStateException("It looks like this off heap memory was already freed. rawBits=" + Integer.toHexString(originalBits));
-        }
-        bits = originalBits;
-        bits &= ~DATA_SIZE_DELTA_MASK; // clear the old dataSizeDelta bits
-        bits |= delta; // set the dataSizeDelta bits to the new delta value
-      } while (!UnsafeMemoryChunk.writeAbsoluteIntVolatile(this.memoryAddress+REF_COUNT_OFFSET, originalBits, bits));
-    }
-    
-    public void initializeUseCount() {
-      int rawBits;
-      do {
-        rawBits = UnsafeMemoryChunk.readAbsoluteIntVolatile(this.memoryAddress+REF_COUNT_OFFSET);
-        if ((rawBits&MAGIC_MASK) != MAGIC_NUMBER) {
-          throw new IllegalStateException("It looks like this off heap memory was already freed. rawBits=" + Integer.toHexString(rawBits));
-        }
-        int uc = rawBits & REF_COUNT_MASK;
-        if (uc != 0) {
-          throw new IllegalStateException("Expected use count to be zero but it was: " + uc + " rawBits=0x" + Integer.toHexString(rawBits));
-        }
-      } while (!UnsafeMemoryChunk.writeAbsoluteIntVolatile(this.memoryAddress+REF_COUNT_OFFSET, rawBits, rawBits+1));
-    }
-
-    public static int getRefCount(long memAddr) {
-      return UnsafeMemoryChunk.readAbsoluteInt(memAddr+REF_COUNT_OFFSET) & REF_COUNT_MASK;
-    }
-
-    public static boolean retain(long memAddr) {
-      validateAddress(memAddr);
-      int uc;
-      int rawBits;
-      int retryCount = 0;
-      do {
-        rawBits = UnsafeMemoryChunk.readAbsoluteIntVolatile(memAddr+REF_COUNT_OFFSET);
-        if ((rawBits&MAGIC_MASK) != MAGIC_NUMBER) {
-          // same as uc == 0
-          // TODO MAGIC_NUMBER rethink its use and interaction with compactor fragments
-          return false;
-        }
-        uc = rawBits & REF_COUNT_MASK;
-        if (uc == MAX_REF_COUNT) {
-          throw new IllegalStateException("Maximum use count exceeded. rawBits=" + Integer.toHexString(rawBits));
-        } else if (uc == 0) {
-          return false;
-        }
-        retryCount++;
-        if (retryCount > 1000) {
-          throw new IllegalStateException("tried to write " + (rawBits+1) + " to @" + Long.toHexString(memAddr) + " 1,000 times.");
-        }
-      } while (!UnsafeMemoryChunk.writeAbsoluteIntVolatile(memAddr+REF_COUNT_OFFSET, rawBits, rawBits+1));
-      //debugLog("use inced ref count " + (uc+1) + " @" + Long.toHexString(memAddr), true);
-      if (trackReferenceCounts()) {
-        refCountChanged(memAddr, false, uc+1);
-      }
-
-      return true;
-    }
-    public static void release(final long memAddr, boolean issueOnReturnCallback) {
-      validateAddress(memAddr);
-      int newCount;
-      int rawBits;
-      boolean returnToAllocator;
-      do {
-        returnToAllocator = false;
-        rawBits = UnsafeMemoryChunk.readAbsoluteIntVolatile(memAddr+REF_COUNT_OFFSET);
-        if ((rawBits&MAGIC_MASK) != MAGIC_NUMBER) {
-          String msg = "It looks like off heap memory @" + Long.toHexString(memAddr) + " was already freed. rawBits=" + Integer.toHexString(rawBits) + " history=" + getFreeRefCountInfo(memAddr);
-          //debugLog(msg, true);
-          throw new IllegalStateException(msg);
-        }
-        int curCount = rawBits&REF_COUNT_MASK;
-        if ((curCount) == 0) {
-          //debugLog("too many frees @" + Long.toHexString(memAddr), true);
-          throw new IllegalStateException("Memory has already been freed." + " history=" + getFreeRefCountInfo(memAddr) /*+ System.identityHashCode(this)*/);
-        }
-        if (curCount == 1) {
-          newCount = 0; // clear the use count, bits, and the delta size since it will be freed.
-          returnToAllocator = true;
-        } else {
-          newCount = rawBits-1;
-        }
-      } while (!UnsafeMemoryChunk.writeAbsoluteIntVolatile(memAddr+REF_COUNT_OFFSET, rawBits, newCount));
-      //debugLog("free deced ref count " + (newCount&USE_COUNT_MASK) + " @" + Long.toHexString(memAddr), true);
-      if (returnToAllocator ) {
-        /*
-        if(issueOnReturnCallback) {
-         final GemFireCacheImpl.StaticSystemCallbacks sysCb =
-              GemFireCacheImpl.FactoryStatics.systemCallbacks;
-          if(sysCb != null ) {
-            ChunkType ct = SimpleMemoryAllocatorImpl.getAllocator().getChunkFactory().getChunkTypeForRawBits(rawBits);
-            int dataSizeDelta = computeDataSizeDelta(rawBits);
-            sysCb.beforeReturningOffHeapMemoryToAllocator(memAddr, ct, dataSizeDelta);
-          }
-        }
-        */
-       
-        if (trackReferenceCounts()) {
-          if (trackFreedReferenceCounts()) {
-            refCountChanged(memAddr, true, newCount&REF_COUNT_MASK);
-          }
-          freeRefCountInfo(memAddr);
-        }
-        
-        // Use fill pattern for free list data integrity check.
-        if(SimpleMemoryAllocatorImpl.getAllocator().validateMemoryWithFill) {
-          fill(memAddr);
-        }
-        
-        getAllocator().freeChunk(memAddr);
-      } else {
-        if (trackReferenceCounts()) {
-          refCountChanged(memAddr, true, newCount&REF_COUNT_MASK);
-        }
-      }
-    }
-    
-    private static int computeDataSizeDelta(int rawBits) {
-      int dataSizeDelta = rawBits;
-      dataSizeDelta &= DATA_SIZE_DELTA_MASK;
-      dataSizeDelta >>= DATA_SIZE_SHIFT;
-      return dataSizeDelta;
-    }
-    
-    @Override
-    public String toString() {
-      return toStringForOffHeapByteSource();
-      // This old impl is not safe because it calls getDeserializedForReading and we have code that call toString that does not inc the refcount.
-      // Also if this Chunk is compressed we don't know how to decompress it.
-      //return super.toString() + ":<dataSize=" + getDataSize() + " refCount=" + getRefCount() + " addr=" + getMemoryAddress() + " storedObject=" + getDeserializedForReading() + ">";
-    }
-    
-    protected String toStringForOffHeapByteSource() {
-      return super.toString() + ":<dataSize=" + getDataSize() + " refCount=" + getRefCount() + " addr=" + Long.toHexString(getMemoryAddress()) + ">";
-    }
-    
-    @Override
-    public State getState() {
-      if (getRefCount() > 0) {
-        return State.ALLOCATED;
-      } else {
-        return State.DEALLOCATED;
-      }
-    }
-    @Override
-    public MemoryBlock getNextBlock() {
-      throw new UnsupportedOperationException();
-    }
-    @Override
-    public int getBlockSize() {
-      return getSize();
-    }
-    @Override
-    public int getSlabId() {
-      throw new UnsupportedOperationException();
-    }
-    @Override
-    public int getFreeListId() {
-      return -1;
-    }
-    @Override
-    public String getDataType() {
-      return null;
-    }
-    @Override
-    public Object getDataValue() {
-      return null;
-    }
-    public Chunk slice(int position, int limit) {
-      throw new UnsupportedOperationException();
-    }
-  }
-  public static class FakeChunk extends Chunk {
-    private final int size;
-    public FakeChunk(int size) {
-      super();
-      this.size = size;
-    }
-    @Override
-    public int getSize() {
-      return this.size;
-    }
-  }
-  /**
-   * Simple stack structure.
-   * The chunk in the top of this stack is pointed to by topAddr.
-   * Each next chunk is found be reading a long from the data in the previous chunk.
-   * An address of 0L means it is then end of the stack.
-   * This class has a subtle race condition in it between
-   * one thread doing a poll, allocating data into the chunk returned by poll,
-   * and then offering it back. Meanwhile another thread did a poll of the same head chunk,
-   * read some of the allocating data as the "next" address and then did the compareAndSet
-   * call and it worked because the first thread had already put it back in.
-   * So this class should not be used. Instead use SyncChunkStack.
-   * 
-   * @author darrel
-   *
-   */
-  public static class BuggyConcurrentChunkStack {
-    // all uses of topAddr are done using topAddrUpdater
-    @SuppressWarnings("unused")
-    private volatile long topAddr;
-    private static final AtomicLongFieldUpdater<BuggyConcurrentChunkStack> topAddrUpdater = AtomicLongFieldUpdater.newUpdater(BuggyConcurrentChunkStack.class, "topAddr");
-    
-    public BuggyConcurrentChunkStack(long addr) {
-      if (addr != 0L) validateAddress(addr);
-      this.topAddr = addr;
-    }
-    public BuggyConcurrentChunkStack() {
-      this.topAddr = 0L;
-    }
-    public boolean isEmpty() {
-      return topAddrUpdater.get(this) == 0L;
-    }
-    public void offer(long e) {
-      assert e != 0;
-      validateAddress(e);
-      long curHead;
-      do {
-        curHead = topAddrUpdater.get(this);
-        Chunk.setNext(e, curHead);
-      } while (!topAddrUpdater.compareAndSet(this, curHead, e));
-    }
-    public long poll() {
-      long result;
-      long newHead;
-      do {
-        result = topAddrUpdater.get(this);
-        if (result == 0L) return 0L;
-        newHead = Chunk.getNext(result);
-        
-      } while (!topAddrUpdater.compareAndSet(this, result, newHead));
-      if (newHead != 0L) validateAddress(newHead);
-      return result;
-    }
-    /**
-     * 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;
-      do {
-        result = topAddrUpdater.get(this);
-        if (result == 0L) return 0L;
-      } while (!topAddrUpdater.compareAndSet(this, result, 0L));
-      return result;
-    }
-    public void logSizes(LogWriter lw, String msg) {
-      long headAddr = topAddrUpdater.get(this);
-      long addr;
-      boolean concurrentModDetected;
-      do {
-        concurrentModDetected = false;
-        addr = headAddr;
-        while (addr != 0L) {
-          int curSize = Chunk.getSize(addr);
-          addr = Chunk.getNext(addr);
-          long curHead = topAddrUpdater.get(this);
-          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 = topAddrUpdater.get(this);
-      long addr;
-      boolean concurrentModDetected;
-      do {
-        concurrentModDetected = false;
-        result = 0;
-        addr = headAddr;
-        while (addr != 0L) {
-          result += Chunk.getSize(addr);
-          addr = Chunk.getNext(addr);
-          long curHead = topAddrUpdater.get(this);
-          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;
-    }
-  }
-  public static class SyncChunkStack {
-    // Ok to read without sync but must be synced on write
-    private volatile long topAddr;
-    
-    public SyncChunkStack(long addr) {
-      if (addr != 0L) 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;
-      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;
-    }
-    /**
-     * 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;
-    }
-  }
-  
-  private static void validateAddress(long addr) {
-    validateAddressAndSize(addr, -1);
-  }
-  
-  private static void validateAddressAndSize(long addr, int size) {
-    // if the caller does not have a "size" to provide then use -1
-    if ((addr & 7) != 0) {
-      StringBuilder sb = new StringBuilder();
-      sb.append("address was not 8 byte aligned: 0x").append(Long.toString(addr, 16));
-      SimpleMem

<TRUNCATED>


[15/19] incubator-geode git commit: [GEODE-585]: Simplify hash index code Refactored hash index and hash index set Using modified versions of the fastutil methods for adding and finding index positions for objects Added hash index set unit tests Removes

Posted by ab...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/abad018a/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/internal/index/HashIndexSetJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/internal/index/HashIndexSetJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/internal/index/HashIndexSetJUnitTest.java
new file mode 100644
index 0000000..de5cebd
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/internal/index/HashIndexSetJUnitTest.java
@@ -0,0 +1,504 @@
+/*
+ * 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.cache.query.internal.index;
+
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.IntStream;
+
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+import com.gemstone.gemfire.cache.query.TypeMismatchException;
+import com.gemstone.gemfire.cache.query.data.Portfolio;
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+
+@Category(UnitTest.class)
+public class HashIndexSetJUnitTest {
+  
+  Map<Integer, Portfolio> portfoliosMap;
+  Set<Portfolio> portfolioSet;
+  HashIndexSet his;
+  
+  public void setupHashIndexSet(int numEntries) {
+    his = createHashIndexSet();
+    portfoliosMap = createPortfolioObjects(numEntries, 0);
+    portfolioSet = new HashSet<Portfolio>(portfoliosMap.values());
+    addPortfoliosToHashIndexSet(portfoliosMap, his);
+  }
+  
+  private void addPortfoliosToHashIndexSet(Map<Integer, Portfolio> portfoliosMap, HashIndexSet hashIndexSet) {
+    portfoliosMap.forEach((k,v) -> {
+      try {
+        hashIndexSet.add(k, v);
+      }
+      catch (TypeMismatchException exception) {
+        throw new Error(exception);
+      }
+    });
+  }
+  
+  private HashIndexSet createHashIndexSet() {
+    HashIndexSet his = new HashIndexSet();  
+    HashIndex.IMQEvaluator mockEvaluator = mock(HashIndex.IMQEvaluator.class);
+    when(mockEvaluator.evaluateKey(any(Object.class))).thenAnswer(new EvaluateKeyAnswer());
+    his.setEvaluator(mockEvaluator);
+    return his;
+  }
+  
+  /**
+   * we are "indexed" on indexKey.  Equality of portfolios is based on ID
+   * indexKeys are based on 0 -> numEntries
+   * IDs are startID -> startID + numEntries
+   * @param numToCreate how many portfolios to create
+   * @param startID the ID value to start incrementing from
+   * @return
+   */
+  public Map<Integer, Portfolio> createPortfolioObjects(int numToCreate, int startID) {
+    Map<Integer, Portfolio> portfoliosMap = new HashMap<>();
+    IntStream.range(0, numToCreate).forEach(e -> {
+        Portfolio p = new Portfolio(e + startID);
+        p.indexKey = e;
+        portfoliosMap.put(p.indexKey, p);
+    });
+    return portfoliosMap;
+  }
+  
+  @Test
+  public void testHashIndexSetAdd() throws Exception {
+    int numEntries = 100;
+    setupHashIndexSet(numEntries);
+    
+    Assert.assertEquals(numEntries, his.size());
+    his.iterator().forEachRemaining((e ->portfolioSet.remove(e)));
+    Assert.assertTrue(portfolioSet.isEmpty());  
+  }
+  
+  @Test
+  public void testHashIndexSetAddWithNullKey() throws Exception {
+    int numEntries = 100;
+    setupHashIndexSet(numEntries);
+    
+    Assert.assertEquals(numEntries, his.size());
+    his.add(null, new Portfolio(numEntries + 1));
+    Assert.assertEquals(numEntries + 1, his.size());
+  }
+  
+  //we have to be sure that we dont cause a compaction or growth or else
+  //removed tokens will be removed and a new backing array created
+  @Test
+  public void testHashIndexSetAddUseRemoveTokenSlot() throws Exception {
+    int numEntries = 20;
+    setupHashIndexSet(numEntries);
+    
+    Assert.assertEquals(numEntries, his.size());
+    his.removeAll(portfolioSet);
+    Assert.assertEquals(numEntries, his._removedTokens);
+    Assert.assertEquals(0, his.size());
+    addPortfoliosToHashIndexSet(portfoliosMap, his);
+    
+    Assert.assertEquals(0, his._removedTokens);
+    Assert.assertEquals(numEntries, his.size());
+  }
+  
+  @Test
+  public void testCompactDueToTooManyRemoveTokens() throws Exception {
+    int numEntries = 100;    
+    setupHashIndexSet(numEntries);
+    
+    Assert.assertEquals(numEntries, his.size());
+    his.removeAll(portfolioSet);
+    Assert.assertEquals(numEntries, his._removedTokens);
+    
+    Assert.assertEquals(0, his.size());
+    
+    //Very very bad but we fake out the number of removed tokens
+    his._removedTokens = his._maxSize;
+    addPortfoliosToHashIndexSet(portfoliosMap, his);
+    
+    //compaction should have occured, removed tokens should now be gone
+    Assert.assertEquals(0, his._removedTokens);
+    Assert.assertEquals(numEntries, his.size());
+  }
+  
+  @Test
+  public void testRehashRetainsAllValues() throws Exception {
+    int numEntries = 80;
+    setupHashIndexSet(numEntries);
+    
+    Assert.assertEquals(numEntries, his.size());
+    his.rehash(1000);
+    Assert.assertEquals(numEntries, his.size());
+    his.iterator().forEachRemaining((e ->portfolioSet.remove(e)));
+    Assert.assertTrue(portfolioSet.isEmpty());  
+  }
+  
+  @Test
+  public void testShrinkByRehashRetainsAllValues() throws Exception {
+    int numEntries = 20;
+    setupHashIndexSet(numEntries);
+
+    Assert.assertEquals(numEntries, his.size());
+    his.rehash(64);
+    Assert.assertEquals(numEntries, his.size());
+    his.iterator().forEachRemaining((e ->portfolioSet.remove(e)));
+    Assert.assertTrue(portfolioSet.isEmpty());  
+  }
+  
+  @Test
+  public void testGetByKey() throws Exception {
+    int numEntries = 20;
+    setupHashIndexSet(numEntries);
+
+    Assert.assertEquals(numEntries, his.size());
+    his.get(1).forEachRemaining((e ->portfolioSet.remove(e)));
+    Assert.assertEquals(numEntries - 1, portfolioSet.size());  
+  }
+  
+  @Test
+  public void testGetByKeyMultipleCollisions() throws Exception {
+    int numEntries = 20;
+    int keyToLookup = 1;
+    his = createHashIndexSet();
+    Map<Integer, Portfolio> collectionOfPorts1 = this.createPortfolioObjects(numEntries, 0);
+    Map<Integer, Portfolio> collectionOfPorts2 = this.createPortfolioObjects(numEntries, numEntries);
+    
+    addPortfoliosToHashIndexSet(collectionOfPorts1, his);
+    addPortfoliosToHashIndexSet(collectionOfPorts2, his);
+    
+    Assert.assertEquals(numEntries * 2, his.size());
+    Iterator iterator = his.get(keyToLookup);
+    int numIterated = 0;
+    while (iterator.hasNext()) {
+      numIterated ++;
+      //verify that the returned values match what we lookedup
+      Assert.assertEquals(keyToLookup, ((Portfolio)iterator.next()).indexKey);
+    }
+    Assert.assertEquals(2, numIterated);  
+  }
+  
+  @Test
+  public void testGetByKeyLocatesAfterMultipleColiisionsAndRemoveToken() throws Exception {
+    int numEntries = 20;
+    int keyToLookup = 1;
+    his = createHashIndexSet();
+    Map<Integer, Portfolio> collectionOfPorts1 = this.createPortfolioObjects(numEntries, 0);
+    Map<Integer, Portfolio> collectionOfPorts2 = this.createPortfolioObjects(numEntries, numEntries);
+    Map<Integer, Portfolio> collectionOfPorts3 = this.createPortfolioObjects(numEntries, numEntries * 2);
+    
+    addPortfoliosToHashIndexSet(collectionOfPorts1, his);
+    addPortfoliosToHashIndexSet(collectionOfPorts2, his);
+    addPortfoliosToHashIndexSet(collectionOfPorts3, his);
+    
+    Assert.assertEquals(numEntries * 3, his.size());
+    Iterator iterator = his.get(keyToLookup);
+    int numIterated = 0;
+    while (iterator.hasNext()) {
+      numIterated ++;
+      //verify that the returned values match what we lookedup
+      Assert.assertEquals(keyToLookup, ((Portfolio)iterator.next()).indexKey);
+    }
+    Assert.assertEquals(3, numIterated);  
+    
+    //let's remove the second collision
+    his.remove(keyToLookup, collectionOfPorts2.get(keyToLookup));
+    
+    iterator = his.get(keyToLookup);
+    numIterated = 0;
+    while (iterator.hasNext()) {
+      numIterated ++;
+      //verify that the returned values match what we lookedup
+      Assert.assertEquals(keyToLookup, ((Portfolio)iterator.next()).indexKey);
+    }
+    Assert.assertEquals(2, numIterated);  
+    
+    //Add it back in and make sure we can iterate all 3 again
+    his.add(keyToLookup, collectionOfPorts2.get(keyToLookup));
+    iterator = his.get(keyToLookup);
+    numIterated = 0;
+    while (iterator.hasNext()) {
+      numIterated ++;
+      //verify that the returned values match what we lookedup
+      Assert.assertEquals(keyToLookup, ((Portfolio)iterator.next()).indexKey);
+    }
+    Assert.assertEquals(3, numIterated);  
+
+  }
+  
+  @Test
+  public void testGetAllNotMatching() throws Exception {
+    int numEntries = 20;
+    his = createHashIndexSet();
+    Map<Integer, Portfolio> collectionOfPorts1 = this.createPortfolioObjects(numEntries, 0);
+    Map<Integer, Portfolio> collectionOfPorts2 = this.createPortfolioObjects(numEntries, numEntries);
+    
+    addPortfoliosToHashIndexSet(collectionOfPorts1, his);
+    addPortfoliosToHashIndexSet(collectionOfPorts2, his);
+    
+    Assert.assertEquals(numEntries * 2, his.size());
+    List<Integer> keysNotToMatch = new LinkedList<>();
+    keysNotToMatch.add(3);
+    keysNotToMatch.add(4);
+    Iterator iterator = his.getAllNotMatching(keysNotToMatch);
+    int numIterated = 0;
+    while (iterator.hasNext()) {
+      numIterated ++;
+      int idFound = ((Portfolio)iterator.next()).indexKey;
+      Assert.assertTrue(idFound != 3 && idFound != 4);
+    }
+    //Make sure we iterated all the entries minus the entries that we decided not to match
+    Assert.assertEquals(numEntries * 2 - 4, numIterated);  
+  }
+  
+  @Test
+  public void testIndexOfObject() throws Exception {
+    int numEntries = 10;
+    his = createHashIndexSet();
+    portfoliosMap = createPortfolioObjects(numEntries, 0);
+    portfoliosMap.forEach((k,v) -> {
+      try {
+        int index = his.add(k, portfoliosMap.get(k));
+        int foundIndex = his.index(portfoliosMap.get(k));
+        Assert.assertEquals(index, foundIndex);
+      }
+      catch (TypeMismatchException ex) {
+        throw new Error(ex);
+      }
+    });
+  }
+  
+  //Add multiple portfolios with the same id
+  //they should collide, we should then be able to look up each one correctly
+  @Test
+  public void testIndexOfObjectWithCollision() throws Exception {
+    int numEntries = 10;
+    his = createHashIndexSet();
+    Map<Integer, Portfolio> portfoliosMap1 = createPortfolioObjects(numEntries, 0);
+    Map<Integer, Portfolio> portfoliosMap2 = createPortfolioObjects(numEntries, numEntries);
+
+    portfoliosMap1.forEach((k,v) -> {
+      try {
+        int index = his.add(k, portfoliosMap1.get(k));
+        int foundIndex = his.index(portfoliosMap1.get(k));
+        Assert.assertEquals(index, foundIndex);
+      }
+      catch (TypeMismatchException ex) {
+        throw new Error(ex);
+      }
+    });
+    portfoliosMap2.forEach((k,v) -> {
+      try {
+        int index = his.add(k, portfoliosMap2.get(k));
+        int foundIndex = his.index(portfoliosMap2.get(k));
+        Assert.assertEquals(index, foundIndex);
+      }
+      catch (TypeMismatchException ex) {
+        throw new Error(ex);
+      }
+    });
+  }
+  
+  
+  @Test
+  public void testIndexWhenObjectNotInSet() {
+    int numEntries = 10;
+    his = createHashIndexSet();
+    portfoliosMap = createPortfolioObjects(numEntries, 0);
+    Assert.assertEquals(-1, his.index(portfoliosMap.get(1)));
+  }
+  
+  @Test
+  public void testIndexWhenObjectNotInSetWhenPopulated() {
+    int numEntries = 10;
+    this.setupHashIndexSet(numEntries);
+    Assert.assertEquals(-1, his.index(new Portfolio(numEntries+1)));
+  }
+  
+  
+  @Test
+  public void testRemove() throws Exception {
+    int numEntries = 20;
+    setupHashIndexSet(numEntries);
+
+    Assert.assertEquals(numEntries, his.size());
+    portfoliosMap.forEach((k,v) -> his.remove(k, v));
+    Assert.assertEquals(0, his.size());
+  }
+  
+  //Test remove where we look for an instance that is not at the specified index slot
+  @Test
+  public void testRemoveIgnoreSlot() throws Exception {
+    int numEntries = 20;
+    setupHashIndexSet(numEntries);
+
+    Assert.assertEquals(numEntries, his.size());
+    portfoliosMap.forEach((k,v) -> his.remove(k, v, his.index(v)));
+    Assert.assertEquals(numEntries, his.size());
+  }
+  
+  @Test
+  public void testRemoveAtWithNull() throws Exception {
+    his = createHashIndexSet();
+    Assert.assertTrue(his.isEmpty());
+    Assert.assertFalse(his.removeAt(0));
+  }
+  
+  @Test
+  public void testRemoveAtWithRemoveToken() throws Exception {
+    his = createHashIndexSet();
+    int index = his.add(1, new Portfolio(1));
+    Assert.assertTrue(his.removeAt(index));
+    Assert.assertFalse(his.removeAt(index));
+  }
+  
+  @Test
+  public void testHashIndexRemoveAll() throws Exception {
+    int numEntries = 100;
+    setupHashIndexSet(numEntries);
+    
+    Assert.assertEquals(numEntries, his.size());
+    his.removeAll(portfolioSet);
+    Assert.assertTrue(his.isEmpty());  
+  }
+  
+  //Remove all should still remove all portfolios provided, even if there are more provided then contained
+  @Test
+  public void testHashIndexRemoveAllWithAdditionalPortfolios() throws Exception {
+    int numEntries = 100;
+    setupHashIndexSet(numEntries);
+    
+    Assert.assertEquals(numEntries, his.size());
+    portfolioSet.add(new Portfolio(numEntries + 1));
+    his.removeAll(portfolioSet);
+    Assert.assertTrue(his.isEmpty());  
+  }
+  
+  @Test
+  public void testHashIndexContainsAll() throws Exception {
+    int numEntries = 100;
+    setupHashIndexSet(numEntries);
+    
+    Assert.assertEquals(numEntries, his.size());
+    Assert.assertTrue(his.containsAll(portfolioSet));
+  }
+  
+  @Test
+  public void testHashIndexRetainAll() throws Exception {
+    int numEntries = 10;
+    setupHashIndexSet(numEntries);
+    Set subset = new HashSet();
+    portfolioSet.forEach(e -> {if (e.indexKey % 2 == 0) {subset.add(e);}});
+    Assert.assertEquals(numEntries, his.size());
+    his.retainAll(subset);
+    his.iterator().forEachRemaining((e ->subset.remove(e)));
+    Assert.assertTrue(subset.isEmpty()); 
+    Assert.assertEquals(numEntries/2, his.size());
+  }
+  
+  @Test
+  public void testHashIndexContainsAllShouldReturnFalse() throws Exception {
+    int numEntries = 100;
+    setupHashIndexSet(numEntries);
+    
+    Assert.assertEquals(numEntries, his.size());
+    portfolioSet.add(new Portfolio(numEntries + 1));
+    Assert.assertFalse(his.containsAll(portfolioSet));
+  }
+  
+  @Test
+  public void testClear() throws Exception {
+    int numEntries = 100;
+    setupHashIndexSet(numEntries);
+    
+    Assert.assertEquals(numEntries, his.size());
+    his.clear();
+    Assert.assertTrue(his.isEmpty());
+    Assert.assertTrue(his._removedTokens == 0);
+  }
+  
+  @Test
+  public void testAreNullObjectsEqual() throws Exception {
+    his = createHashIndexSet();
+    Assert.assertTrue(his.areObjectsEqual(null, null));
+  }
+  
+  @Test
+  public void testAreIndexeSetsEqualAndHashCodeSame() throws Exception {
+    Map<Integer, Portfolio> portfolioMap = createPortfolioObjects(100, 0);
+    HashIndexSet indexSet1 = createHashIndexSet();
+    HashIndexSet indexSet2 = createHashIndexSet();
+
+    addPortfoliosToHashIndexSet(portfolioMap, indexSet1);
+    addPortfoliosToHashIndexSet(portfolioMap, indexSet2);
+ 
+    Assert.assertTrue(indexSet1.equals(indexSet2));
+    Assert.assertTrue(indexSet2.equals(indexSet1));
+    Assert.assertEquals(indexSet1.hashCode(), indexSet2.hashCode());
+  }
+  
+  @Test
+  public void testAreIndexeSetsNotEqualAndHashCodeDifferent() throws Exception {
+    Map<Integer, Portfolio> portfolioMap = createPortfolioObjects(100, 0);
+    HashIndexSet indexSet1 = createHashIndexSet();
+    HashIndexSet indexSet2 = createHashIndexSet();
+
+    addPortfoliosToHashIndexSet(portfolioMap, indexSet1);
+
+    indexSet2.add(1, portfolioMap.get(1));
+    Assert.assertFalse(indexSet2.equals(indexSet1));
+    Assert.assertFalse(indexSet1.equals(indexSet2));
+    Assert.assertNotEquals(indexSet1.hashCode(), indexSet2.hashCode()); 
+  }
+  
+  @Test
+  public void testIndexSetNotEqualsOtherObjectType() {
+    HashIndexSet indexSet = createHashIndexSet();
+    Assert.assertFalse(indexSet.equals("Other type"));
+    Assert.assertFalse(indexSet.equals(new Object()));
+  }
+ 
+  private static class EvaluateKeyAnswer implements Answer {
+
+    @Override
+    public Object answer(InvocationOnMock invocation) throws Throwable {
+      Object evalOn = invocation.getArgumentAt(0, Object.class);
+      if (evalOn instanceof Portfolio) {
+        Portfolio p = (Portfolio) evalOn;
+        return p.indexKey;
+      }
+      return null;
+    }
+    
+  }
+  
+  
+}


[10/19] incubator-geode git commit: fixing GEODE-588 CI Failure: LocatorDUnitTest.testMultipleMcastLocators

Posted by ab...@apache.org.
fixing GEODE-588 CI Failure: LocatorDUnitTest.testMultipleMcastLocators

forcing JGroups to use an IPv4 address for multicast unless the
PreferIPv6Addresses property is set to true.  Use of multicast is
also restricted to cache operations on regions with multicast enabled.
This avoids problems when multicast isn't properly set up and a user
enables use of multicast in the DistributedSystem config.


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

Branch: refs/heads/feature/GEODE-584
Commit: e3d5ebe11422f6245e3f4ee4a86ce76e60c57bef
Parents: c0d396a
Author: Bruce Schuchardt <bs...@pivotal.io>
Authored: Tue Nov 24 09:00:22 2015 -0800
Committer: Bruce Schuchardt <bs...@pivotal.io>
Committed: Tue Nov 24 09:00:22 2015 -0800

----------------------------------------------------------------------
 .../gemfire/distributed/internal/DistributionManager.java   | 2 +-
 .../distributed/internal/InternalDistributedSystem.java     | 8 +++++---
 .../internal/membership/gms/messenger/JGroupsMessenger.java | 9 ++++-----
 .../gemstone/gemfire/internal/cache/GemFireCacheImpl.java   | 4 ----
 4 files changed, 10 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/e3d5ebe1/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionManager.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionManager.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionManager.java
index 247775d..837194c 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionManager.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionManager.java
@@ -3383,7 +3383,7 @@ public class DistributionManager
     allOthers.remove(getDistributionManagerId());
 //    ReplyProcessor21 rp = new ReplyProcessor21(this, allOthers);
 //    m.setProcessorId(rp.getProcessorId());
-    m.setMulticast(system.getConfig().getMcastPort() != 0);
+//    m.setMulticast(system.getConfig().getMcastPort() != 0);
     m.setRecipients(allOthers);
 
     //Address recipient = (Address) m.getRecipient();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/e3d5ebe1/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/InternalDistributedSystem.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/InternalDistributedSystem.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/InternalDistributedSystem.java
index 956fe8b..261b8a9 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/InternalDistributedSystem.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/InternalDistributedSystem.java
@@ -2578,10 +2578,12 @@ public final class InternalDistributedSystem
     
 //    logger.info("reconnecting IDS@"+System.identityHashCode(this));
 
-    if (isDebugEnabled) {
-      logger.debug("changing thread name to ReconnectThread");
+    if (Thread.currentThread().getName().equals("DisconnectThread")) {
+      if (isDebugEnabled) {
+        logger.debug("changing thread name to ReconnectThread");
+      }
+      Thread.currentThread().setName("ReconnectThread");
     }
-    Thread.currentThread().setName("ReconnectThread");
     
     // get the membership manager for quorum checks
     MembershipManager mbrMgr = this.dm.getMembershipManager();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/e3d5ebe1/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/JGroupsMessenger.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/JGroupsMessenger.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/JGroupsMessenger.java
index 8ac8f77..889795d 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/JGroupsMessenger.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/JGroupsMessenger.java
@@ -276,7 +276,7 @@ public class JGroupsMessenger implements Messenger {
       else {
         logger.debug("JGroups configuration: {}", properties);
         
-        checkForWindowsIPv6();
+        checkForIPv6();
         InputStream is = new ByteArrayInputStream(properties.getBytes("UTF-8"));
         myChannel = new JChannel(is);
       }
@@ -320,11 +320,10 @@ public class JGroupsMessenger implements Messenger {
    * and preferIPv6Addresses is not set or is true.  We want it to use an
    * IPv4 address for a dual-IP stack so that both IPv4 and IPv6 messaging work
    */
-  private void checkForWindowsIPv6() throws Exception {
-    boolean isWindows = ((String)System.getProperty("os.name")).indexOf("Windows") >= 0;
+  private void checkForIPv6() throws Exception {
     boolean preferIpV6Addr = Boolean.getBoolean("java.net.preferIPv6Addresses");
-    if (isWindows && !preferIpV6Addr) {
-      logger.debug("Windows detected - forcing JGroups to think IPv4 is being used so it will choose an IPv4 address");
+    if (!preferIpV6Addr) {
+      logger.debug("forcing JGroups to think IPv4 is being used so it will choose an IPv4 address");
       Field m = org.jgroups.util.Util.class.getDeclaredField("ip_stack_type");
       m.setAccessible(true);
       m.set(null, org.jgroups.util.StackType.IPv4);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/e3d5ebe1/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/GemFireCacheImpl.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/GemFireCacheImpl.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/GemFireCacheImpl.java
index cd7376d..27bb813 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/GemFireCacheImpl.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/GemFireCacheImpl.java
@@ -2205,10 +2205,6 @@ public class GemFireCacheImpl implements InternalCache, ClientCache, HasCachePer
             Set otherMembers = dm.getOtherDistributionManagerIds();
             ReplyProcessor21 processor = new ReplyProcessor21(system, otherMembers);
             CloseCacheMessage msg = new CloseCacheMessage();
-            // [bruce] if multicast is available, use it to send the message to
-            // avoid race conditions with cache content operations that might
-            // also be multicast
-            msg.setMulticast(system.getConfig().getMcastPort() != 0);
             msg.setRecipients(otherMembers);
             msg.setProcessorId(processor.getProcessorId());
             dm.putOutgoing(msg);


[09/19] incubator-geode git commit: fix GEODE-598 CI Failure: ReconnectDUnitTest.testReconnectALocator

Posted by ab...@apache.org.
fix GEODE-598 CI Failure: ReconnectDUnitTest.testReconnectALocator

changed the criterion used by the test to determine whether the Locator
has started to reconnect.


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

Branch: refs/heads/feature/GEODE-584
Commit: c0d396ab16ac4cb598acdd29d1c57d19735ad305
Parents: dc780a8
Author: Bruce Schuchardt <bs...@pivotal.io>
Authored: Mon Nov 23 15:33:54 2015 -0800
Committer: Bruce Schuchardt <bs...@pivotal.io>
Committed: Mon Nov 23 15:33:54 2015 -0800

----------------------------------------------------------------------
 .../java/com/gemstone/gemfire/cache30/ReconnectDUnitTest.java    | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c0d396ab/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/ReconnectDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/ReconnectDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/ReconnectDUnitTest.java
index 5b53159..dd6f1fa 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/ReconnectDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/ReconnectDUnitTest.java
@@ -1099,10 +1099,10 @@ public class ReconnectDUnitTest extends CacheTestCase
         if (oldLocator != null) {
           WaitCriterion wc = new WaitCriterion() {
             public boolean done() {
-              return ((InternalLocator)oldLocator).isStopped();
+              return msys.isReconnecting();
             }
             public String description() {
-              return "waiting for locator to stop: " + oldLocator;
+              return "waiting for locator to start reconnecting: " + oldLocator;
             }
           };
           waitForCriterion(wc, 10000, 50, true);


[03/19] incubator-geode git commit: GEODE-587: Publish lucene jar to distribution and maven

Posted by ab...@apache.org.
GEODE-587: Publish lucene jar to distribution and maven


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

Branch: refs/heads/feature/GEODE-584
Commit: 25c8f0cc48e831d6afb459767a42c1c98fb47996
Parents: ac0b68e
Author: Anthony Baker <ab...@pivotal.io>
Authored: Fri Nov 20 16:23:57 2015 -0800
Committer: Anthony Baker <ab...@pivotal.io>
Committed: Fri Nov 20 21:03:20 2015 -0800

----------------------------------------------------------------------
 gemfire-assembly/build.gradle | 7 ++++++-
 gemfire-lucene/build.gradle   | 4 ++--
 2 files changed, 8 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/25c8f0cc/gemfire-assembly/build.gradle
----------------------------------------------------------------------
diff --git a/gemfire-assembly/build.gradle b/gemfire-assembly/build.gradle
index dd7b9fa..5f82905 100755
--- a/gemfire-assembly/build.gradle
+++ b/gemfire-assembly/build.gradle
@@ -25,6 +25,7 @@ dependencies {
   archives project(':gemfire-json')  
   archives project(':gemfire-joptsimple')  
   archives project(':gemfire-core')  
+  archives project(':gemfire-lucene')
   archives project(':gemfire-web')
   archives project(':gemfire-web-api')
 
@@ -181,6 +182,9 @@ distributions {
         }
         from project(":gemfire-core").configurations.archives.allArtifacts.files
 
+        from project(":gemfire-lucene").configurations.runtime
+        from project(":gemfire-lucene").configurations.archives.allArtifacts.files
+
         // include this jar        
         from project(":gemfire-web-api").jar.outputs.files.getFiles()
         
@@ -216,7 +220,8 @@ def commonJar = [publicationName:'commonJar', project:project(":gemfire-common")
 def coreJar = [publicationName:'coreJar', project:project(":gemfire-core").name]
 def jsonJar = [publicationName:'jsonJar', project:project(":gemfire-json").name]
 def joptsimpleJar = [publicationName:'joptsimpleJar', project:project(":gemfire-joptsimple").name]
-def MavenJars = [ coreJar, jsonJar, joptsimpleJar, commonJar ]
+def luceneJar = [publicationName:'luceneJar', project:project(":gemfire-lucene").name]
+def MavenJars = [ coreJar, jsonJar, joptsimpleJar, commonJar, luceneJar ]
 
 afterEvaluate {
   publishing {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/25c8f0cc/gemfire-lucene/build.gradle
----------------------------------------------------------------------
diff --git a/gemfire-lucene/build.gradle b/gemfire-lucene/build.gradle
index 6c3adda..ff47748 100644
--- a/gemfire-lucene/build.gradle
+++ b/gemfire-lucene/build.gradle
@@ -1,6 +1,6 @@
 dependencies {
-    provided project(':gemfire-core')
-    provided project(':gemfire-common')
+    compile project(':gemfire-core')
+    compile project(':gemfire-common')
 
     compile 'org.apache.lucene:lucene-analyzers-common:' + project.'lucene.version'
     compile 'org.apache.lucene:lucene-core:' + project.'lucene.version'


[07/19] incubator-geode git commit: fix GEODE-594: CI Failure in BackwardCompatibilitySerializationDUnitTest

Posted by ab...@apache.org.
fix GEODE-594: CI Failure in BackwardCompatibilitySerializationDUnitTest

changed the test to be a CacheTestCase and use getCache()


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

Branch: refs/heads/feature/GEODE-584
Commit: 59f208734481f8f1da77790e5807e846442b9b7c
Parents: bbccb60
Author: Bruce Schuchardt <bs...@pivotal.io>
Authored: Mon Nov 23 14:32:05 2015 -0800
Committer: Bruce Schuchardt <bs...@pivotal.io>
Committed: Mon Nov 23 14:33:22 2015 -0800

----------------------------------------------------------------------
 .../BackwardCompatibilitySerializationDUnitTest.java   | 13 +++----------
 1 file changed, 3 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/59f20873/gemfire-core/src/test/java/com/gemstone/gemfire/internal/BackwardCompatibilitySerializationDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/BackwardCompatibilitySerializationDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/BackwardCompatibilitySerializationDUnitTest.java
index d730b9e..35269f7 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/BackwardCompatibilitySerializationDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/BackwardCompatibilitySerializationDUnitTest.java
@@ -16,10 +16,6 @@
  */
 package com.gemstone.gemfire.internal;
 
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.DataInput;
@@ -34,14 +30,11 @@ import java.util.ArrayList;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
-import org.junit.experimental.categories.Category;
 
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.CacheFactory;
+import com.gemstone.gemfire.cache30.CacheTestCase;
 import com.gemstone.gemfire.internal.cache.DistributedPutAllOperation.EntryVersionsList;
-import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
-
-import dunit.DistributedTestCase;
 
 /**
  * Test the DSFID serialization framework added for rolling upgrades in 7.1
@@ -50,7 +43,7 @@ import dunit.DistributedTestCase;
  * 
  * 
  */
-public class BackwardCompatibilitySerializationDUnitTest extends DistributedTestCase {
+public class BackwardCompatibilitySerializationDUnitTest extends CacheTestCase {
 
   private transient ByteArrayOutputStream baos;
   private transient ByteArrayInputStream bais;
@@ -206,7 +199,7 @@ public class BackwardCompatibilitySerializationDUnitTest extends DistributedTest
     }
     
     // some msgs require distributed system
-    Cache c = new CacheFactory().set("locators", "localhost["+getDUnitLocatorPort()+"]").create();
+    Cache c = getCache();
     for (Object o : DSFIDFactory.getDsfidmap2().values()) {
       Constructor<?> cons = (Constructor<?>) o;
       if (cons != null) {


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

Posted by ab...@apache.org.
GEODE-580: cleanup off-heap code

Unused dead code and classes have been removed.
The following inner classes have been moved to their own class files:
LifeCycleListener, Chunk, ChunkFactory, GemFireChunkFactory,
GemFireChunk, GemFireChunkSlice, ChunkWithHeapForm,
DataAsAddress, Fragment, RefCountChangeInfo, MemoryBlockNode,
SyncChunkStack, and FreeListManager.

The reference count debugging code has all been moved
to a new class named ReferenceCountHelper.


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

Branch: refs/heads/feature/GEODE-584
Commit: 4b0925e30a2b22173bfb974d8fe60c569663be9d
Parents: e3d5ebe
Author: Darrel Schneider <ds...@pivotal.io>
Authored: Thu Nov 19 15:13:01 2015 -0800
Committer: Darrel Schneider <ds...@pivotal.io>
Committed: Tue Nov 24 09:57:05 2015 -0800

----------------------------------------------------------------------
 .../internal/GetOperationContextImpl.java       |    2 +-
 .../query/internal/index/AbstractIndex.java     |    2 +-
 .../query/internal/index/DummyQRegion.java      |    2 +-
 .../cache/query/internal/index/HashIndex.java   |    2 +-
 .../internal/cache/AbstractRegionEntry.java     |   29 +-
 .../internal/cache/AbstractRegionMap.java       |   16 +-
 .../cache/BytesAndBitsForCompactor.java         |    2 +-
 .../gemfire/internal/cache/DiskEntry.java       |    8 +-
 .../internal/cache/DistributedRegion.java       |    2 +-
 .../gemfire/internal/cache/EntryEventImpl.java  |   45 +-
 .../gemfire/internal/cache/LocalRegion.java     |   10 +-
 .../gemstone/gemfire/internal/cache/Oplog.java  |    6 +-
 .../internal/cache/PartitionedRegion.java       |    2 +-
 .../internal/cache/VMThinRegionEntry.java       |    5 -
 .../internal/cache/tier/sockets/Part.java       |    4 +-
 .../cache/wan/GatewaySenderEventImpl.java       |   10 +-
 .../gemfire/internal/offheap/Chunk.java         |  793 +++++
 .../gemfire/internal/offheap/ChunkFactory.java  |   51 +
 .../gemfire/internal/offheap/ChunkType.java     |   26 +
 .../internal/offheap/ChunkWithHeapForm.java     |   40 +
 .../gemfire/internal/offheap/DataAsAddress.java |  205 ++
 .../gemfire/internal/offheap/Fragment.java      |  125 +
 .../internal/offheap/FreeListManager.java       |  807 +++++
 .../gemfire/internal/offheap/GemFireChunk.java  |   55 +
 .../internal/offheap/GemFireChunkFactory.java   |   52 +
 .../internal/offheap/GemFireChunkSlice.java     |   44 +
 .../internal/offheap/LifecycleListener.java     |   98 +
 .../internal/offheap/MemoryAllocator.java       |    1 -
 .../gemfire/internal/offheap/MemoryBlock.java   |    1 -
 .../internal/offheap/MemoryBlockNode.java       |  158 +
 .../offheap/OffHeapCachedDeserializable.java    |    1 -
 .../gemfire/internal/offheap/OffHeapHelper.java |    8 +-
 .../offheap/OffHeapRegionEntryHelper.java       |    8 +-
 .../internal/offheap/RefCountChangeInfo.java    |  112 +
 .../internal/offheap/ReferenceCountHelper.java  |  235 ++
 .../offheap/SimpleMemoryAllocatorImpl.java      | 3242 +-----------------
 .../internal/offheap/SyncChunkStack.java        |  130 +
 .../internal/tcp/ByteBufferInputStream.java     |    2 +-
 .../tcp/ImmutableByteBufferInputStream.java     |    2 +-
 .../gemfire/internal/util/BlobHelper.java       |    2 +-
 .../gemfire/pdx/internal/PdxInputStream.java    |    2 +-
 .../cache/ChunkValueWrapperJUnitTest.java       |    2 +-
 .../gemfire/internal/cache/OffHeapTestUtil.java |    5 +-
 .../cache/OldValueImporterTestBase.java         |    4 +-
 .../offheap/ConcurrentBagJUnitTest.java         |  130 -
 .../offheap/FreeListOffHeapRegionJUnitTest.java |    2 +-
 .../internal/offheap/OffHeapRegionBase.java     |    1 -
 .../offheap/OffHeapValidationJUnitTest.java     |    1 -
 .../OffHeapWriteObjectAsByteArrayJUnitTest.java |    2 -
 .../OldFreeListOffHeapRegionJUnitTest.java      |    2 +-
 ...mpleMemoryAllocatorFillPatternJUnitTest.java |    2 -
 .../offheap/SimpleMemoryAllocatorJUnitTest.java |    9 +-
 ...moryAllocatorLifecycleListenerJUnitTest.java |    9 +-
 .../OffHeapByteBufferByteSourceJUnitTest.java   |    2 +-
 .../gemfire/pdx/OffHeapByteSourceJUnitTest.java |    2 +-
 .../gemfire/codeAnalysis/excludedClasses.txt    |    1 +
 .../codeAnalysis/sanctionedSerializables.txt    |    1 -
 57 files changed, 3100 insertions(+), 3422 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4b0925e3/gemfire-core/src/main/java/com/gemstone/gemfire/cache/operations/internal/GetOperationContextImpl.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/operations/internal/GetOperationContextImpl.java b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/operations/internal/GetOperationContextImpl.java
index fd2a944..d973cd7 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/operations/internal/GetOperationContextImpl.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/operations/internal/GetOperationContextImpl.java
@@ -18,8 +18,8 @@ package com.gemstone.gemfire.cache.operations.internal;
 
 import com.gemstone.gemfire.SerializationException;
 import com.gemstone.gemfire.cache.operations.GetOperationContext;
+import com.gemstone.gemfire.internal.offheap.Chunk;
 import com.gemstone.gemfire.internal.offheap.Releasable;
-import com.gemstone.gemfire.internal.offheap.SimpleMemoryAllocatorImpl.Chunk;
 import com.gemstone.gemfire.internal.offheap.StoredObject;
 import com.gemstone.gemfire.internal.offheap.annotations.Unretained;
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4b0925e3/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/index/AbstractIndex.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/index/AbstractIndex.java b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/index/AbstractIndex.java
index 7ddacde..b65dc7e 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/index/AbstractIndex.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/index/AbstractIndex.java
@@ -74,7 +74,7 @@ import com.gemstone.gemfire.internal.cache.RegionEntry;
 import com.gemstone.gemfire.internal.cache.persistence.query.CloseableIterator;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 import com.gemstone.gemfire.internal.logging.LogService;
-import com.gemstone.gemfire.internal.offheap.SimpleMemoryAllocatorImpl.Chunk;
+import com.gemstone.gemfire.internal.offheap.Chunk;
 import com.gemstone.gemfire.internal.offheap.annotations.Released;
 import com.gemstone.gemfire.internal.offheap.annotations.Retained;
 import com.gemstone.gemfire.pdx.PdxInstance;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4b0925e3/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/index/DummyQRegion.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/index/DummyQRegion.java b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/index/DummyQRegion.java
index 1e1da4b..3577e38 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/index/DummyQRegion.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/index/DummyQRegion.java
@@ -41,7 +41,7 @@ import com.gemstone.gemfire.internal.cache.LocalRegion;
 import com.gemstone.gemfire.internal.cache.RegionEntry;
 import com.gemstone.gemfire.internal.cache.RegionEntryContext;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
-import com.gemstone.gemfire.internal.offheap.SimpleMemoryAllocatorImpl.Chunk;
+import com.gemstone.gemfire.internal.offheap.Chunk;
 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/main/java/com/gemstone/gemfire/cache/query/internal/index/HashIndex.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/index/HashIndex.java b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/index/HashIndex.java
index dd9cffd..911fd6e 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/index/HashIndex.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/index/HashIndex.java
@@ -79,7 +79,7 @@ import com.gemstone.gemfire.internal.cache.Token;
 import com.gemstone.gemfire.internal.cache.persistence.query.CloseableIterator;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 import com.gemstone.gemfire.internal.logging.LogService;
-import com.gemstone.gemfire.internal.offheap.SimpleMemoryAllocatorImpl.Chunk;
+import com.gemstone.gemfire.internal.offheap.Chunk;
 import com.gemstone.gemfire.internal.offheap.annotations.Released;
 import com.gemstone.gemfire.internal.offheap.annotations.Retained;
 import com.gemstone.gemfire.pdx.internal.PdxString;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4b0925e3/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/AbstractRegionEntry.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/AbstractRegionEntry.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/AbstractRegionEntry.java
index f250bd2..dd33b15 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/AbstractRegionEntry.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/AbstractRegionEntry.java
@@ -21,6 +21,7 @@ import java.io.IOException;
 import java.util.Arrays;
 
 import org.apache.logging.log4j.Logger;
+
 import static com.gemstone.gemfire.internal.offheap.annotations.OffHeapIdentifier.ABSTRACT_REGION_ENTRY_FILL_IN_VALUE;
 import static com.gemstone.gemfire.internal.offheap.annotations.OffHeapIdentifier.ABSTRACT_REGION_ENTRY_PREPARE_VALUE_FOR_CACHE;
 
@@ -60,14 +61,14 @@ import com.gemstone.gemfire.internal.lang.StringUtils;
 import com.gemstone.gemfire.internal.logging.LogService;
 import com.gemstone.gemfire.internal.logging.log4j.LocalizedMessage;
 import com.gemstone.gemfire.internal.logging.log4j.LogMarker;
+import com.gemstone.gemfire.internal.offheap.Chunk;
+import com.gemstone.gemfire.internal.offheap.ChunkWithHeapForm;
+import com.gemstone.gemfire.internal.offheap.GemFireChunk;
 import com.gemstone.gemfire.internal.offheap.MemoryAllocator;
 import com.gemstone.gemfire.internal.offheap.OffHeapCachedDeserializable;
 import com.gemstone.gemfire.internal.offheap.OffHeapHelper;
+import com.gemstone.gemfire.internal.offheap.ReferenceCountHelper;
 import com.gemstone.gemfire.internal.offheap.SimpleMemoryAllocatorImpl;
-import com.gemstone.gemfire.internal.offheap.SimpleMemoryAllocatorImpl.Chunk;
-import com.gemstone.gemfire.internal.offheap.SimpleMemoryAllocatorImpl.ChunkType;
-import com.gemstone.gemfire.internal.offheap.SimpleMemoryAllocatorImpl.GemFireChunk;
-import com.gemstone.gemfire.internal.offheap.SimpleMemoryAllocatorImpl.GemFireChunkType;
 import com.gemstone.gemfire.internal.offheap.StoredObject;
 import com.gemstone.gemfire.internal.offheap.annotations.Released;
 import com.gemstone.gemfire.internal.offheap.annotations.Retained;
@@ -415,7 +416,7 @@ public abstract class AbstractRegionEntry implements RegionEntry,
 
   @Override
   public Object getValue(RegionEntryContext context) {
-    SimpleMemoryAllocatorImpl.createReferenceCountOwner();
+    ReferenceCountHelper.createReferenceCountOwner();
     @Retained Object result = _getValueRetain(context, true);
     //Asif: If the thread is an Index Creation Thread & the value obtained is 
     //Token.REMOVED , we can skip  synchronization block. This is required to prevent
@@ -431,11 +432,11 @@ public abstract class AbstractRegionEntry implements RegionEntry,
 //    }
     
     if (Token.isRemoved(result)) {
-      SimpleMemoryAllocatorImpl.setReferenceCountOwner(null);
+      ReferenceCountHelper.setReferenceCountOwner(null);
       return null;
     } else {
       result = OffHeapHelper.copyAndReleaseIfNeeded(result); // sqlf does not dec ref count in this call
-      SimpleMemoryAllocatorImpl.setReferenceCountOwner(null);
+      ReferenceCountHelper.setReferenceCountOwner(null);
       setRecentlyUsed();
       return result;
     }
@@ -564,14 +565,14 @@ public abstract class AbstractRegionEntry implements RegionEntry,
   
   @Retained
   public final Object getValueInVM(RegionEntryContext context) {
-    SimpleMemoryAllocatorImpl.createReferenceCountOwner();
+    ReferenceCountHelper.createReferenceCountOwner();
     @Retained Object v = _getValueRetain(context, true);
     
     if (v == null) { // should only be possible if disk entry
       v = Token.NOT_AVAILABLE;
     }
     @Retained Object result = OffHeapHelper.copyAndReleaseIfNeeded(v); // TODO OFFHEAP keep it offheap?
-    SimpleMemoryAllocatorImpl.setReferenceCountOwner(null);
+    ReferenceCountHelper.setReferenceCountOwner(null);
     return result;
   }
   
@@ -758,9 +759,9 @@ public abstract class AbstractRegionEntry implements RegionEntry,
     // :ezoerner:20080814 We also read old value from disk or buffer
     // in the case where there is a non-null expectedOldValue
     // see PartitionedRegion#remove(Object key, Object value)
-    SimpleMemoryAllocatorImpl.skipRefCountTracking();
+    ReferenceCountHelper.skipRefCountTracking();
     @Retained @Released Object curValue = _getValueRetain(region, true);
-    SimpleMemoryAllocatorImpl.unskipRefCountTracking();
+    ReferenceCountHelper.unskipRefCountTracking();
     try {
     if (curValue == null) curValue = Token.NOT_AVAILABLE;
     
@@ -1343,12 +1344,12 @@ public abstract class AbstractRegionEntry implements RegionEntry,
         }
         byte[] compressedData = compressBytes(r, data);
         boolean isCompressed = compressedData != data;
-        SimpleMemoryAllocatorImpl.setReferenceCountOwner(this);
+        ReferenceCountHelper.setReferenceCountOwner(this);
         MemoryAllocator ma = SimpleMemoryAllocatorImpl.getAllocator(); // fix for bug 47875
         val = ma.allocateAndInitialize(compressedData, isSerialized, isCompressed, GemFireChunk.TYPE); // TODO:KIRK:48068 race happens right after this line
-        SimpleMemoryAllocatorImpl.setReferenceCountOwner(null);
+        ReferenceCountHelper.setReferenceCountOwner(null);
         if (val instanceof GemFireChunk) {
-          val = new com.gemstone.gemfire.internal.offheap.SimpleMemoryAllocatorImpl.ChunkWithHeapForm((GemFireChunk)val, data);
+          val = new com.gemstone.gemfire.internal.offheap.ChunkWithHeapForm((GemFireChunk)val, data);
         }
 //        if (val instanceof Chunk && r instanceof LocalRegion) {
 //          Chunk c = (Chunk) val;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4b0925e3/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/AbstractRegionMap.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/AbstractRegionMap.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/AbstractRegionMap.java
index 80b55d5..6d49d74 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/AbstractRegionMap.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/AbstractRegionMap.java
@@ -74,10 +74,10 @@ import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 import com.gemstone.gemfire.internal.logging.LogService;
 import com.gemstone.gemfire.internal.logging.log4j.LocalizedMessage;
 import com.gemstone.gemfire.internal.logging.log4j.LogMarker;
+import com.gemstone.gemfire.internal.offheap.Chunk;
 import com.gemstone.gemfire.internal.offheap.OffHeapHelper;
 import com.gemstone.gemfire.internal.offheap.OffHeapRegionEntryHelper;
-import com.gemstone.gemfire.internal.offheap.SimpleMemoryAllocatorImpl;
-import com.gemstone.gemfire.internal.offheap.SimpleMemoryAllocatorImpl.Chunk;
+import com.gemstone.gemfire.internal.offheap.ReferenceCountHelper;
 import com.gemstone.gemfire.internal.offheap.annotations.Released;
 import com.gemstone.gemfire.internal.offheap.annotations.Retained;
 import com.gemstone.gemfire.internal.offheap.annotations.Unretained;
@@ -3357,11 +3357,11 @@ RETRY_LOOP:
     // replace is propagated to server, so no need to check
     // satisfiesOldValue on client
     if (expectedOldValue != null && !replaceOnClient) {
-      SimpleMemoryAllocatorImpl.skipRefCountTracking();
+      ReferenceCountHelper.skipRefCountTracking();
       
       @Retained @Released Object v = re._getValueRetain(event.getLocalRegion(), true);
       
-      SimpleMemoryAllocatorImpl.unskipRefCountTracking();
+      ReferenceCountHelper.unskipRefCountTracking();
       try {
         if (!AbstractRegionEntry.checkExpectedOldValue(expectedOldValue, v, event.getLocalRegion())) {
           return false;
@@ -3392,9 +3392,9 @@ RETRY_LOOP:
       if (event.hasDelta() || event.getOperation().guaranteesOldValue()
           || GemFireCacheImpl.sqlfSystem()) {
         // In these cases we want to even get the old value from disk if it is not in memory
-        SimpleMemoryAllocatorImpl.skipRefCountTracking();
+        ReferenceCountHelper.skipRefCountTracking();
         @Released Object oldValueInVMOrDisk = re.getValueOffHeapOrDiskWithoutFaultIn(event.getLocalRegion());
-        SimpleMemoryAllocatorImpl.unskipRefCountTracking();
+        ReferenceCountHelper.unskipRefCountTracking();
         try {
           event.setOldValue(oldValueInVMOrDisk, requireOldValue
               || GemFireCacheImpl.sqlfSystem());
@@ -3403,11 +3403,11 @@ RETRY_LOOP:
         }
       } else {
         // In these cases only need the old value if it is in memory
-        SimpleMemoryAllocatorImpl.skipRefCountTracking();
+        ReferenceCountHelper.skipRefCountTracking();
         
         @Retained @Released Object oldValueInVM = re._getValueRetain(event.getLocalRegion(), true); // OFFHEAP: re synced so can use its ref.
         
-        SimpleMemoryAllocatorImpl.unskipRefCountTracking();
+        ReferenceCountHelper.unskipRefCountTracking();
         try {
           event.setOldValue(oldValueInVM,
               requireOldValue || GemFireCacheImpl.sqlfSystem());

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4b0925e3/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/BytesAndBitsForCompactor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/BytesAndBitsForCompactor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/BytesAndBitsForCompactor.java
index 8e46605..3a3b5a1 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/BytesAndBitsForCompactor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/BytesAndBitsForCompactor.java
@@ -16,7 +16,7 @@
  */
 package com.gemstone.gemfire.internal.cache;
 
-import com.gemstone.gemfire.internal.offheap.SimpleMemoryAllocatorImpl.Chunk;
+import com.gemstone.gemfire.internal.offheap.Chunk;
 import com.gemstone.gemfire.internal.offheap.annotations.Unretained;
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4b0925e3/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/DiskEntry.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/DiskEntry.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/DiskEntry.java
index 3973556..c855cca 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/DiskEntry.java
+++ b/gemfire-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.Chunk;
 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.SimpleMemoryAllocatorImpl;
 import com.gemstone.gemfire.internal.offheap.UnsafeMemoryChunk;
-import com.gemstone.gemfire.internal.offheap.SimpleMemoryAllocatorImpl.Chunk;
 import com.gemstone.gemfire.internal.offheap.StoredObject;
 import com.gemstone.gemfire.internal.offheap.annotations.Released;
 import com.gemstone.gemfire.internal.offheap.annotations.Retained;
@@ -340,9 +340,9 @@ public interface DiskEntry extends RegionEntry {
       synchronized (syncObj) {
         entry.setLastModified(mgr, de.getLastModified());
                               
-        SimpleMemoryAllocatorImpl.setReferenceCountOwner(entry);
+        ReferenceCountHelper.setReferenceCountOwner(entry);
         v = de._getValueRetain(context, true); // OFFHEAP copied to heap entry; todo allow entry to refer to offheap since it will be copied to network.
-        SimpleMemoryAllocatorImpl.setReferenceCountOwner(null);
+        ReferenceCountHelper.setReferenceCountOwner(null);
         if (v == null) {
           if (did == null) {
             // fix for bug 41449

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4b0925e3/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/DistributedRegion.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/DistributedRegion.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/DistributedRegion.java
index daf4c8d..5d263a6 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/DistributedRegion.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/DistributedRegion.java
@@ -121,8 +121,8 @@ import com.gemstone.gemfire.internal.cache.wan.parallel.ConcurrentParallelGatewa
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 import com.gemstone.gemfire.internal.logging.LogService;
 import com.gemstone.gemfire.internal.logging.log4j.LocalizedMessage;
+import com.gemstone.gemfire.internal.offheap.Chunk;
 import com.gemstone.gemfire.internal.offheap.OffHeapHelper;
-import com.gemstone.gemfire.internal.offheap.SimpleMemoryAllocatorImpl.Chunk;
 import com.gemstone.gemfire.internal.offheap.annotations.Released;
 import com.gemstone.gemfire.internal.offheap.annotations.Retained;
 import com.gemstone.gemfire.internal.sequencelog.RegionLogger;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4b0925e3/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/EntryEventImpl.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/EntryEventImpl.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/EntryEventImpl.java
index 0786a69..9cf2f13 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/EntryEventImpl.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/EntryEventImpl.java
@@ -22,6 +22,7 @@ import java.io.DataInput;
 import java.io.DataInputStream;
 import java.io.DataOutput;
 import java.io.IOException;
+
 import org.apache.logging.log4j.Logger;
 
 import com.gemstone.gemfire.CopyHelper;
@@ -72,11 +73,11 @@ import com.gemstone.gemfire.internal.lang.StringUtils;
 import com.gemstone.gemfire.internal.logging.LogService;
 import com.gemstone.gemfire.internal.logging.log4j.LocalizedMessage;
 import com.gemstone.gemfire.internal.logging.log4j.LogMarker;
+import com.gemstone.gemfire.internal.offheap.Chunk;
 import com.gemstone.gemfire.internal.offheap.OffHeapHelper;
 import com.gemstone.gemfire.internal.offheap.OffHeapRegionEntryHelper;
+import com.gemstone.gemfire.internal.offheap.ReferenceCountHelper;
 import com.gemstone.gemfire.internal.offheap.Releasable;
-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.offheap.annotations.Released;
 import com.gemstone.gemfire.internal.offheap.annotations.Retained;
@@ -930,13 +931,13 @@ public class EntryEventImpl
       OffHeapHelper.releaseAndTrackOwner(this.newValue, this);
     }
     if (v instanceof Chunk) {
-      SimpleMemoryAllocatorImpl.setReferenceCountOwner(this);
+      ReferenceCountHelper.setReferenceCountOwner(this);
       if (!((Chunk) v).retain()) {
-        SimpleMemoryAllocatorImpl.setReferenceCountOwner(null);
+        ReferenceCountHelper.setReferenceCountOwner(null);
         this.newValue = null;
         return;
       }
-      SimpleMemoryAllocatorImpl.setReferenceCountOwner(null);
+      ReferenceCountHelper.setReferenceCountOwner(null);
     }
     this.newValue = v;
     this.cachedSerializedNewValue = null;
@@ -992,7 +993,7 @@ public class EntryEventImpl
     if (v == curOldValue) return;
     if (this.offHeapOk) {
       if (curOldValue instanceof Chunk) {
-        if (SimpleMemoryAllocatorImpl.trackReferenceCounts()) {
+        if (ReferenceCountHelper.trackReferenceCounts()) {
           OffHeapHelper.releaseAndTrackOwner(curOldValue, new OldValueOwner());
         } else {
           OffHeapHelper.release(curOldValue);
@@ -1008,10 +1009,10 @@ public class EntryEventImpl
     if (v == this.oldValue) return;
     
     if (v instanceof Chunk) {
-      if (SimpleMemoryAllocatorImpl.trackReferenceCounts()) {
-        SimpleMemoryAllocatorImpl.setReferenceCountOwner(new OldValueOwner());
+      if (ReferenceCountHelper.trackReferenceCounts()) {
+        ReferenceCountHelper.setReferenceCountOwner(new OldValueOwner());
         boolean couldNotRetain = (!((Chunk) v).retain());
-        SimpleMemoryAllocatorImpl.setReferenceCountOwner(null);
+        ReferenceCountHelper.setReferenceCountOwner(null);
         if (couldNotRetain) {
           this.oldValue = null;
           return;
@@ -1737,14 +1738,14 @@ public class EntryEventImpl
             || GemFireCacheImpl.sqlfSystem()
             ) {
           @Retained Object ov;
-          if (SimpleMemoryAllocatorImpl.trackReferenceCounts()) {
-            SimpleMemoryAllocatorImpl.setReferenceCountOwner(new OldValueOwner());
+          if (ReferenceCountHelper.trackReferenceCounts()) {
+            ReferenceCountHelper.setReferenceCountOwner(new OldValueOwner());
             if (GemFireCacheImpl.sqlfSystem()) {
               ov = reentry.getValueOffHeapOrDiskWithoutFaultIn(this.region);
             } else {
               ov = reentry._getValueRetain(owner, true);
             }
-            SimpleMemoryAllocatorImpl.setReferenceCountOwner(null);
+            ReferenceCountHelper.setReferenceCountOwner(null);
           } else {
             if (GemFireCacheImpl.sqlfSystem()) {
               ov = reentry.getValueOffHeapOrDiskWithoutFaultIn(this.region);
@@ -2087,9 +2088,9 @@ public class EntryEventImpl
     try {
       RegionEntry re = this.region.getRegionEntry(getKey());
       if (re == null) return false;
-      SimpleMemoryAllocatorImpl.skipRefCountTracking();
+      ReferenceCountHelper.skipRefCountTracking();
       Object v = re._getValueRetain(this.region, true);
-      SimpleMemoryAllocatorImpl.unskipRefCountTracking();
+      ReferenceCountHelper.unskipRefCountTracking();
       try {
         return setOldValue(v);
       } finally {
@@ -3054,10 +3055,10 @@ public class EntryEventImpl
     
     if (ov instanceof Chunk) {
       //this.region.getCache().getLogger().info("DEBUG freeing ref to old value on " + System.identityHashCode(ov));
-      if (SimpleMemoryAllocatorImpl.trackReferenceCounts()) {
-        SimpleMemoryAllocatorImpl.setReferenceCountOwner(new OldValueOwner());
+      if (ReferenceCountHelper.trackReferenceCounts()) {
+        ReferenceCountHelper.setReferenceCountOwner(new OldValueOwner());
         ((Chunk) ov).release();
-        SimpleMemoryAllocatorImpl.setReferenceCountOwner(null);
+        ReferenceCountHelper.setReferenceCountOwner(null);
       } else {
         ((Chunk) ov).release();
       }
@@ -3085,19 +3086,19 @@ public class EntryEventImpl
   public void copyOffHeapToHeap() {
     Object ov = basicGetOldValue();
     if (ov instanceof Chunk) {
-      if (SimpleMemoryAllocatorImpl.trackReferenceCounts()) {
-        SimpleMemoryAllocatorImpl.setReferenceCountOwner(new OldValueOwner());
+      if (ReferenceCountHelper.trackReferenceCounts()) {
+        ReferenceCountHelper.setReferenceCountOwner(new OldValueOwner());
         this.oldValue = OffHeapHelper.copyAndReleaseIfNeeded(ov);
-        SimpleMemoryAllocatorImpl.setReferenceCountOwner(null);
+        ReferenceCountHelper.setReferenceCountOwner(null);
       } else {
         this.oldValue = OffHeapHelper.copyAndReleaseIfNeeded(ov);
       }
     }
     Object nv = basicGetNewValue();
     if (nv instanceof Chunk) {
-      SimpleMemoryAllocatorImpl.setReferenceCountOwner(this);
+      ReferenceCountHelper.setReferenceCountOwner(this);
       this.newValue = OffHeapHelper.copyAndReleaseIfNeeded(nv);
-      SimpleMemoryAllocatorImpl.setReferenceCountOwner(null);
+      ReferenceCountHelper.setReferenceCountOwner(null);
     }
     if (this.newValue instanceof Chunk || this.oldValue instanceof Chunk) {
       throw new IllegalStateException("event's old/new value still off-heap after calling copyOffHeapToHeap");

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4b0925e3/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/LocalRegion.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/LocalRegion.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/LocalRegion.java
index c17caa2..caf07ce 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/LocalRegion.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/LocalRegion.java
@@ -203,9 +203,9 @@ import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 import com.gemstone.gemfire.internal.logging.LogService;
 import com.gemstone.gemfire.internal.logging.log4j.LocalizedMessage;
 import com.gemstone.gemfire.internal.logging.log4j.LogMarker;
+import com.gemstone.gemfire.internal.offheap.Chunk;
 import com.gemstone.gemfire.internal.offheap.OffHeapHelper;
-import com.gemstone.gemfire.internal.offheap.SimpleMemoryAllocatorImpl;
-import com.gemstone.gemfire.internal.offheap.SimpleMemoryAllocatorImpl.Chunk;
+import com.gemstone.gemfire.internal.offheap.ReferenceCountHelper;
 import com.gemstone.gemfire.internal.offheap.StoredObject;
 import com.gemstone.gemfire.internal.offheap.annotations.Retained;
 import com.gemstone.gemfire.internal.offheap.annotations.Unretained;
@@ -2178,14 +2178,14 @@ public class LocalRegion extends AbstractRegion
       RegionEntry entry = this.entries.getEntry(keyInfo.getKey());
       boolean result = entry != null;
       if (result) {
-        SimpleMemoryAllocatorImpl.skipRefCountTracking();
+        ReferenceCountHelper.skipRefCountTracking();
         Object val = entry.getTransformedValue(); // no need to decompress since we only want to know if we have an existing value 
         if (val instanceof StoredObject) {
           OffHeapHelper.release(val);
-          SimpleMemoryAllocatorImpl.unskipRefCountTracking();
+          ReferenceCountHelper.unskipRefCountTracking();
           return true;
         }
-        SimpleMemoryAllocatorImpl.unskipRefCountTracking();
+        ReferenceCountHelper.unskipRefCountTracking();
         // No need to to check CachedDeserializable because of Bruce's fix in r30960 for bug 42162. See bug 42732.
         // this works because INVALID and LOCAL_INVALID will never be faulted out of mem
         // If val is NOT_AVAILABLE that means we have a valid value on disk.

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4b0925e3/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/Oplog.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/Oplog.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/Oplog.java
index cd197f2..3a2d1ed 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/Oplog.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/Oplog.java
@@ -108,7 +108,7 @@ import com.gemstone.gemfire.internal.logging.LogService;
 import com.gemstone.gemfire.internal.logging.log4j.LocalizedMessage;
 import com.gemstone.gemfire.internal.logging.log4j.LogMarker;
 import com.gemstone.gemfire.internal.offheap.OffHeapHelper;
-import com.gemstone.gemfire.internal.offheap.SimpleMemoryAllocatorImpl;
+import com.gemstone.gemfire.internal.offheap.ReferenceCountHelper;
 import com.gemstone.gemfire.internal.offheap.StoredObject;
 import com.gemstone.gemfire.internal.offheap.annotations.Released;
 import com.gemstone.gemfire.internal.offheap.annotations.Retained;
@@ -4310,10 +4310,10 @@ public final class Oplog implements CompactableOplog, Flushable {
     DiskId did = entry.getDiskId();
     byte userBits = 0;
     long oplogOffset = did.getOffsetInOplog();
-    SimpleMemoryAllocatorImpl.skipRefCountTracking();
+    ReferenceCountHelper.skipRefCountTracking();
     // TODO OFFHEAP: no need to retain. We just use it while we have the RegionEntry synced.
     @Retained @Released Object value = entry._getValueRetain(dr, true);
-    SimpleMemoryAllocatorImpl.unskipRefCountTracking();
+    ReferenceCountHelper.unskipRefCountTracking();
     // TODO:KIRK:OK Object value = entry.getValueWithContext(dr);
     boolean foundData = false;
     if (value == null) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4b0925e3/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/PartitionedRegion.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/PartitionedRegion.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/PartitionedRegion.java
index 399700d..48f4787 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/PartitionedRegion.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/PartitionedRegion.java
@@ -252,7 +252,7 @@ import com.gemstone.gemfire.internal.logging.LogService;
 import com.gemstone.gemfire.internal.logging.LoggingThreadGroup;
 import com.gemstone.gemfire.internal.logging.log4j.LocalizedMessage;
 import com.gemstone.gemfire.internal.logging.log4j.LogMarker;
-import com.gemstone.gemfire.internal.offheap.SimpleMemoryAllocatorImpl.Chunk;
+import com.gemstone.gemfire.internal.offheap.Chunk;
 import com.gemstone.gemfire.internal.offheap.annotations.Unretained;
 import com.gemstone.gemfire.internal.sequencelog.RegionLogger;
 import com.gemstone.gemfire.internal.util.TransformUtils;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4b0925e3/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/VMThinRegionEntry.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/VMThinRegionEntry.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/VMThinRegionEntry.java
index 6317fbc..bdc05c3 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/VMThinRegionEntry.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/VMThinRegionEntry.java
@@ -17,11 +17,6 @@
 
 package com.gemstone.gemfire.internal.cache;
 
-import java.util.UUID;
-
-import com.gemstone.gemfire.internal.offheap.SimpleMemoryAllocatorImpl.ConcurrentBag;
-
-
 /**
  * Implementation class of RegionEntry interface.
  * VM -> entries stored in VM memory

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4b0925e3/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/Part.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/Part.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/Part.java
index 6af005e..5418c68 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/Part.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/Part.java
@@ -18,8 +18,8 @@ package com.gemstone.gemfire.internal.cache.tier.sockets;
 
 import com.gemstone.gemfire.internal.*;
 import com.gemstone.gemfire.internal.cache.CachedDeserializable;
-import com.gemstone.gemfire.internal.offheap.SimpleMemoryAllocatorImpl.Chunk;
-import com.gemstone.gemfire.internal.offheap.SimpleMemoryAllocatorImpl.DataAsAddress;
+import com.gemstone.gemfire.internal.offheap.Chunk;
+import com.gemstone.gemfire.internal.offheap.DataAsAddress;
 import com.gemstone.gemfire.internal.offheap.StoredObject;
 import com.gemstone.gemfire.internal.offheap.UnsafeMemoryChunk;
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4b0925e3/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/wan/GatewaySenderEventImpl.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/wan/GatewaySenderEventImpl.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/wan/GatewaySenderEventImpl.java
index c4f0c87..4df8f35 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/wan/GatewaySenderEventImpl.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/wan/GatewaySenderEventImpl.java
@@ -50,11 +50,11 @@ import com.gemstone.gemfire.internal.cache.WrappedCallbackArgument;
 import com.gemstone.gemfire.internal.cache.lru.Sizeable;
 import com.gemstone.gemfire.internal.cache.tier.sockets.CacheServerHelper;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
+import com.gemstone.gemfire.internal.offheap.Chunk;
+import com.gemstone.gemfire.internal.offheap.ChunkWithHeapForm;
 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.SimpleMemoryAllocatorImpl;
-import com.gemstone.gemfire.internal.offheap.SimpleMemoryAllocatorImpl.Chunk;
-import com.gemstone.gemfire.internal.offheap.SimpleMemoryAllocatorImpl.ChunkWithHeapForm;
 import com.gemstone.gemfire.internal.offheap.StoredObject;
 import com.gemstone.gemfire.internal.offheap.annotations.OffHeapIdentifier;
 import com.gemstone.gemfire.internal.offheap.annotations.Released;
@@ -935,9 +935,9 @@ public class GatewaySenderEventImpl implements
     if (event.hasDelta()) {
       this.valueIsObject = 0x02;
     } else {
-      SimpleMemoryAllocatorImpl.setReferenceCountOwner(this);
+      ReferenceCountHelper.setReferenceCountOwner(this);
       so = event.getOffHeapNewValue();
-      SimpleMemoryAllocatorImpl.setReferenceCountOwner(null);      
+      ReferenceCountHelper.setReferenceCountOwner(null);      
         // TODO OFFHEAP MERGE: check for a cached serialized value first
         // so we can use it instead of reading offheap
         // If we do read offheap then add the serialize new value to the event cache

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4b0925e3/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/Chunk.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/Chunk.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/Chunk.java
new file mode 100644
index 0000000..e32a1c6
--- /dev/null
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/Chunk.java
@@ -0,0 +1,793 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.gemstone.gemfire.internal.offheap;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.nio.ByteBuffer;
+
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.internal.DSCODE;
+import com.gemstone.gemfire.internal.HeapDataOutputStream;
+import com.gemstone.gemfire.internal.InternalDataSerializer;
+import com.gemstone.gemfire.internal.cache.EntryEventImpl;
+import com.gemstone.gemfire.internal.cache.RegionEntry;
+import com.gemstone.gemfire.internal.cache.RegionEntryContext;
+import com.gemstone.gemfire.internal.offheap.annotations.Unretained;
+
+/**
+   * Note: this class has a natural ordering that is inconsistent with equals.
+   * Instances of this class should have a short lifetime. We do not store references
+   * to it in the cache. Instead the memoryAddress is stored in a primitive field in
+   * the cache and if used it will then, if needed, create an instance of this class.
+   */
+  public abstract class Chunk extends OffHeapCachedDeserializable implements Comparable<Chunk>, MemoryBlock {
+    /**
+     * The unsafe memory address of the first byte of this chunk
+     */
+    private final long memoryAddress;
+    
+    /**
+     * The useCount, chunkSize, dataSizeDelta, isSerialized, and isCompressed
+     * are all stored in off heap memory in a HEADER. This saves heap memory
+     * by using off heap.
+     */
+    public final static int OFF_HEAP_HEADER_SIZE = 4 + 4;
+    /**
+     * We need to smallest chunk to at least have enough room for a hdr
+     * and for an off heap ref (which is a long).
+     */
+    public final static int MIN_CHUNK_SIZE = OFF_HEAP_HEADER_SIZE + 8;
+    /**
+     * int field.
+     * The number of bytes in this chunk.
+     */
+    private final static int CHUNK_SIZE_OFFSET = 0;
+    /**
+     * Volatile int field
+     * The upper two bits are used for the isSerialized
+     * and isCompressed flags.
+     * The next three bits are used to encode the SRC_TYPE enum.
+     * The lower 3 bits of the most significant byte contains a magic number to help us detect
+     * if we are changing the ref count of an object that has been released.
+     * The next byte contains the dataSizeDelta.
+     * The number of bytes of logical data in this chunk.
+     * Since the number of bytes of logical data is always <= chunkSize
+     * and since chunkSize never changes, we have dataSize be
+     * a delta whose max value would be HUGE_MULTIPLE-1.
+     * The lower two bytes contains the use count.
+     */
+    private final static int REF_COUNT_OFFSET = 4;
+    /**
+     * The upper two bits are used for the isSerialized
+     * and isCompressed flags.
+     */
+    private final static int IS_SERIALIZED_BIT =    0x80000000;
+    private final static int IS_COMPRESSED_BIT =    0x40000000;
+    private final static int SRC_TYPE_MASK = 0x38000000;
+    private final static int SRC_TYPE_SHIFT = 16/*refCount*/+8/*dataSize*/+3/*magicSize*/;
+    private final static int MAGIC_MASK = 0x07000000;
+    private final static int MAGIC_NUMBER = 0x05000000;
+    private final static int DATA_SIZE_DELTA_MASK = 0x00ff0000;
+    private final static int DATA_SIZE_SHIFT = 16;
+    private final static int REF_COUNT_MASK =       0x0000ffff;
+    private final static int MAX_REF_COUNT = 0xFFFF;
+    final static long FILL_PATTERN = 0x3c3c3c3c3c3c3c3cL;
+    final static byte FILL_BYTE = 0x3c;
+    
+    // The 8 bits reserved for SRC_TYPE are basically no longer used.
+    // So we could free up these 8 bits for some other use or we could
+    // keep them for future extensions.
+    // If we ever want to allocate other "types" into a chunk of off-heap
+    // memory then the SRC_TYPE would be the way to go.
+    // For example we may want to allocate the memory for the off-heap
+    // RegionEntry in off-heap memory without it being of type GFE.
+    // When it is of type GFE then it either needs to be the bytes
+    // of a byte array or it needs to be a serialized java object.
+    // For the RegionEntry we may want all the primitive fields of
+    // the entry at certain offsets in the off-heap memory so we could
+    // access them directly in native byte format (i.e. no serialization).
+    // Note that for every SRC_TYPE we should have a ChunkType subclass.
+    public final static int SRC_TYPE_UNUSED0 = 0 << SRC_TYPE_SHIFT;
+    public final static int SRC_TYPE_UNUSED1 = 1 << SRC_TYPE_SHIFT;
+    public final static int SRC_TYPE_UNUSED2 = 2 << SRC_TYPE_SHIFT;
+    public final static int SRC_TYPE_UNUSED3 = 3 << SRC_TYPE_SHIFT;
+    public final static int SRC_TYPE_GFE = 4 << SRC_TYPE_SHIFT;
+    public final static int SRC_TYPE_UNUSED5 = 5 << SRC_TYPE_SHIFT;
+    public final static int SRC_TYPE_UNUSED6 = 6 << SRC_TYPE_SHIFT;
+    public final static int SRC_TYPE_UNUSED7 = 7 << SRC_TYPE_SHIFT;
+    
+    protected Chunk(long memoryAddress, int chunkSize, ChunkType chunkType) {
+      SimpleMemoryAllocatorImpl.validateAddressAndSize(memoryAddress, chunkSize);
+      this.memoryAddress = memoryAddress;
+      setSize(chunkSize);
+      UnsafeMemoryChunk.writeAbsoluteIntVolatile(getMemoryAddress()+REF_COUNT_OFFSET, MAGIC_NUMBER|chunkType.getSrcType());
+    }
+    public void readyForFree() {
+      UnsafeMemoryChunk.writeAbsoluteIntVolatile(getMemoryAddress()+REF_COUNT_OFFSET, 0);
+    }
+    public void readyForAllocation(ChunkType chunkType) {
+      if (!UnsafeMemoryChunk.writeAbsoluteIntVolatile(getMemoryAddress()+REF_COUNT_OFFSET, 0, MAGIC_NUMBER|chunkType.getSrcType())) {
+        throw new IllegalStateException("Expected 0 but found " + Integer.toHexString(UnsafeMemoryChunk.readAbsoluteIntVolatile(getMemoryAddress()+REF_COUNT_OFFSET)));
+      }
+    }
+    /**
+     * Should only be used by FakeChunk subclass
+     */
+    protected Chunk() {
+      this.memoryAddress = 0L;
+    }
+    
+    /**
+     * Used to create a Chunk given an existing, already allocated,
+     * memoryAddress. The off heap header has already been initialized.
+     */
+    protected Chunk(long memoryAddress) {
+      SimpleMemoryAllocatorImpl.validateAddress(memoryAddress);
+      this.memoryAddress = memoryAddress;
+    }
+    
+    protected Chunk(Chunk chunk) {
+      this.memoryAddress = chunk.memoryAddress;
+    }
+    
+    /**
+     * Throw an exception if this chunk is not allocated
+     */
+    public void checkIsAllocated() {
+      int originalBits = UnsafeMemoryChunk.readAbsoluteIntVolatile(this.memoryAddress+REF_COUNT_OFFSET);
+      if ((originalBits&MAGIC_MASK) != MAGIC_NUMBER) {
+        throw new IllegalStateException("It looks like this off heap memory was already freed. rawBits=" + Integer.toHexString(originalBits));
+      }
+    }
+    
+    public void incSize(int inc) {
+      setSize(getSize()+inc);
+    }
+    
+    protected void beforeReturningToAllocator() {
+      
+    }
+
+    @Override
+    public int getSize() {
+      return getSize(this.memoryAddress);
+    }
+
+    public void setSize(int size) {
+      setSize(this.memoryAddress, size);
+    }
+
+    public long getMemoryAddress() {
+      return this.memoryAddress;
+    }
+    
+    public int getDataSize() {
+      /*int dataSizeDelta = UnsafeMemoryChunk.readAbsoluteInt(this.memoryAddress+REF_COUNT_OFFSET);
+      dataSizeDelta &= DATA_SIZE_DELTA_MASK;
+      dataSizeDelta >>= DATA_SIZE_SHIFT;
+      return getSize() - dataSizeDelta;*/
+      return getDataSize(this.memoryAddress);
+    }
+    
+    protected static int getDataSize(long memoryAdress) {
+      int dataSizeDelta = UnsafeMemoryChunk.readAbsoluteInt(memoryAdress+REF_COUNT_OFFSET);
+      dataSizeDelta &= DATA_SIZE_DELTA_MASK;
+      dataSizeDelta >>= DATA_SIZE_SHIFT;
+      return getSize(memoryAdress) - dataSizeDelta;
+    }
+    
+    protected long getBaseDataAddress() {
+      return this.memoryAddress+OFF_HEAP_HEADER_SIZE;
+    }
+    protected int getBaseDataOffset() {
+      return 0;
+    }
+    
+    /**
+     * Creates and returns a direct ByteBuffer that contains the contents of this Chunk.
+     * Note that the returned ByteBuffer has a reference to this chunk's
+     * off-heap address so it can only be used while this Chunk is retained.
+     * @return the created direct byte buffer or null if it could not be created.
+     */
+    @Unretained
+    public ByteBuffer createDirectByteBuffer() {
+      return basicCreateDirectByteBuffer(getBaseDataAddress(), getDataSize());
+    }
+    @Override
+    public void sendTo(DataOutput out) throws IOException {
+      if (!this.isCompressed() && out instanceof HeapDataOutputStream) {
+        ByteBuffer bb = createDirectByteBuffer();
+        if (bb != null) {
+          HeapDataOutputStream hdos = (HeapDataOutputStream) out;
+          if (this.isSerialized()) {
+            hdos.write(bb);
+          } else {
+            hdos.writeByte(DSCODE.BYTE_ARRAY);
+            InternalDataSerializer.writeArrayLength(bb.remaining(), hdos);
+            hdos.write(bb);
+          }
+          return;
+        }
+      }
+      super.sendTo(out);
+    }
+    
+    @Override
+    public void sendAsByteArray(DataOutput out) throws IOException {
+      if (!isCompressed() && out instanceof HeapDataOutputStream) {
+        ByteBuffer bb = createDirectByteBuffer();
+        if (bb != null) {
+          HeapDataOutputStream hdos = (HeapDataOutputStream) out;
+          InternalDataSerializer.writeArrayLength(bb.remaining(), hdos);
+          hdos.write(bb);
+          return;
+        }
+      }
+      super.sendAsByteArray(out);
+    }
+       
+    private static volatile Class dbbClass = null;
+    private static volatile Constructor dbbCtor = null;
+    private static volatile boolean dbbCreateFailed = false;
+    
+    /**
+     * @return the created direct byte buffer or null if it could not be created.
+     */
+    private static ByteBuffer basicCreateDirectByteBuffer(long baseDataAddress, int dataSize) {
+      if (dbbCreateFailed) {
+        return null;
+      }
+      Constructor ctor = dbbCtor;
+      if (ctor == null) {
+        Class c = dbbClass;
+        if (c == null) {
+          try {
+            c = Class.forName("java.nio.DirectByteBuffer");
+          } catch (ClassNotFoundException e) {
+            //throw new IllegalStateException("Could not find java.nio.DirectByteBuffer", e);
+            dbbCreateFailed = true;
+            dbbAddressFailed = true;
+            return null;
+          }
+          dbbClass = c;
+        }
+        try {
+          ctor = c.getDeclaredConstructor(long.class, int.class);
+        } catch (NoSuchMethodException | SecurityException e) {
+          //throw new IllegalStateException("Could not get constructor DirectByteBuffer(long, int)", e);
+          dbbClass = null;
+          dbbCreateFailed = true;
+          return null;
+        }
+        ctor.setAccessible(true);
+        dbbCtor = ctor;
+      }
+      try {
+        return (ByteBuffer)ctor.newInstance(baseDataAddress, dataSize);
+      } catch (InstantiationException | IllegalAccessException | IllegalArgumentException | InvocationTargetException e) {
+        //throw new IllegalStateException("Could not create an instance using DirectByteBuffer(long, int)", e);
+        dbbClass = null;
+        dbbCtor = null;
+        dbbCreateFailed = true;
+        return null;
+      }
+    }
+    private static volatile Method dbbAddressMethod = null;
+    private static volatile boolean dbbAddressFailed = false;
+    
+    /**
+     * Returns the address of the Unsafe memory for the first byte of a direct ByteBuffer.
+     * If the buffer is not direct or the address can not be obtained return 0.
+     */
+    public static long getDirectByteBufferAddress(ByteBuffer bb) {
+      if (!bb.isDirect()) {
+        return 0L;
+      }
+      if (dbbAddressFailed) {
+        return 0L;
+      }
+      Method m = dbbAddressMethod;
+      if (m == null) {
+        Class c = dbbClass;
+        if (c == null) {
+          try {
+            c = Class.forName("java.nio.DirectByteBuffer");
+          } catch (ClassNotFoundException e) {
+            //throw new IllegalStateException("Could not find java.nio.DirectByteBuffer", e);
+            dbbCreateFailed = true;
+            dbbAddressFailed = true;
+            return 0L;
+          }
+          dbbClass = c;
+        }
+        try {
+          m = c.getDeclaredMethod("address");
+        } catch (NoSuchMethodException | SecurityException e) {
+          //throw new IllegalStateException("Could not get method DirectByteBuffer.address()", e);
+          dbbClass = null;
+          dbbAddressFailed = true;
+          return 0L;
+        }
+        m.setAccessible(true);
+        dbbAddressMethod = m;
+      }
+      try {
+        return (Long)m.invoke(bb);
+      } catch (IllegalAccessException | IllegalArgumentException | InvocationTargetException e) {
+        //throw new IllegalStateException("Could not create an invoke DirectByteBuffer.address()", e);
+        dbbClass = null;
+        dbbAddressMethod = null;
+        dbbAddressFailed = true;
+        return 0L;
+      }
+    }
+    /**
+     * Returns an address that can be used with unsafe apis to access this chunks memory.
+     * @param offset the offset from this chunk's first byte of the byte the returned address should point to. Must be >= 0.
+     * @param size the number of bytes that will be read using the returned address. Assertion will use this to verify that all the memory accessed belongs to this chunk. Must be > 0.
+     * @return a memory address that can be used with unsafe apis
+     */
+    public long getUnsafeAddress(int offset, int size) {
+      assert offset >= 0 && offset + size <= getDataSize(): "Offset=" + offset + ",size=" + size + ",dataSize=" + getDataSize() + ", chunkSize=" + getSize() + ", but offset + size must be <= " + getDataSize();
+      assert size > 0;
+      long result = getBaseDataAddress() + offset;
+      // validateAddressAndSizeWithinSlab(result, size);
+      return result;
+    }
+    
+    @Override
+    public byte readByte(int offset) {
+      assert offset < getDataSize();
+      return UnsafeMemoryChunk.readAbsoluteByte(getBaseDataAddress() + offset);
+    }
+
+    @Override
+    public void writeByte(int offset, byte value) {
+      assert offset < getDataSize();
+      UnsafeMemoryChunk.writeAbsoluteByte(getBaseDataAddress() + offset, value);
+    }
+
+    @Override
+    public void readBytes(int offset, byte[] bytes) {
+      readBytes(offset, bytes, 0, bytes.length);
+    }
+
+    @Override
+    public void writeBytes(int offset, byte[] bytes) {
+      writeBytes(offset, bytes, 0, bytes.length);
+    }
+
+    public long getAddressForReading(int offset, int size) {
+      assert offset+size <= getDataSize();
+      return getBaseDataAddress() + offset;
+    }
+    
+    @Override
+    public void readBytes(int offset, byte[] bytes, int bytesOffset, int size) {
+      assert offset+size <= getDataSize();
+      UnsafeMemoryChunk.readAbsoluteBytes(getBaseDataAddress() + offset, bytes, bytesOffset, size);
+    }
+
+    @Override
+    public void writeBytes(int offset, byte[] bytes, int bytesOffset, int size) {
+      assert offset+size <= getDataSize();
+      SimpleMemoryAllocatorImpl.validateAddressAndSizeWithinSlab(getBaseDataAddress() + offset, size);
+      UnsafeMemoryChunk.writeAbsoluteBytes(getBaseDataAddress() + offset, bytes, bytesOffset, size);
+    }
+    
+    @Override
+    public void release() {
+      release(this.memoryAddress, true);
+     }
+
+    @Override
+    public int compareTo(Chunk o) {
+      int result = Integer.signum(getSize() - o.getSize());
+      if (result == 0) {
+        // For the same sized chunks we really don't care about their order
+        // but we need compareTo to only return 0 if the two chunks are identical
+        result = Long.signum(getMemoryAddress() - o.getMemoryAddress());
+      }
+      return result;
+    }
+    
+    @Override
+    public boolean equals(Object o) {
+      if (o instanceof Chunk) {
+        return getMemoryAddress() == ((Chunk) o).getMemoryAddress();
+      }
+      return false;
+    }
+    
+    @Override
+    public int hashCode() {
+      long value = this.getMemoryAddress();
+      return (int)(value ^ (value >>> 32));
+    }
+
+    // OffHeapCachedDeserializable methods 
+    
+    @Override
+    public void setSerializedValue(byte[] value) {
+      writeBytes(0, value);
+    }
+    
+    public byte[] getDecompressedBytes(RegionEntryContext context) {
+      byte[] result = getCompressedBytes();
+      long time = context.getCachePerfStats().startDecompression();
+      result = context.getCompressor().decompress(result);
+      context.getCachePerfStats().endDecompression(time);      
+      return result;
+    }
+    
+    /**
+     * Returns the raw possibly compressed bytes of this chunk
+     */
+    public byte[] getCompressedBytes() {
+      byte[] result = new byte[getDataSize()];
+      readBytes(0, result);
+      //debugLog("reading", true);
+      SimpleMemoryAllocatorImpl.getAllocator().getStats().incReads();
+      return result;
+    }
+    protected byte[] getRawBytes() {
+      byte[] result = getCompressedBytes();
+      // TODO OFFHEAP: change the following to assert !isCompressed();
+      if (isCompressed()) {
+        throw new UnsupportedOperationException();
+      }
+      return result;
+    }
+
+    @Override
+    public byte[] getSerializedValue() {
+      byte [] result = getRawBytes();
+      if (!isSerialized()) {
+        // The object is a byte[]. So we need to make it look like a serialized byte[] in our result
+        result = EntryEventImpl.serialize(result);
+      }
+      return result;
+    }
+    
+    @Override
+    public Object getDeserializedValue(Region r, RegionEntry re) {
+      if (isSerialized()) {
+        // TODO OFFHEAP: debug deserializeChunk
+        return EntryEventImpl.deserialize(getRawBytes());
+        //assert !isCompressed();
+        //return EntryEventImpl.deserializeChunk(this);
+      } else {
+        return getRawBytes();
+      }
+    }
+    
+    /**
+     * We want this to include memory overhead so use getSize() instead of getDataSize().
+     */
+    @Override
+    public int getSizeInBytes() {
+      // Calling getSize includes the off heap header size.
+      // We do not add anything to this since the size of the reference belongs to the region entry size
+      // not the size of this object.
+      return getSize();
+    }
+
+    @Override
+    public int getValueSizeInBytes() {
+      return getDataSize();
+    }
+
+    @Override
+    public void copyBytes(int src, int dst, int size) {
+      throw new UnsupportedOperationException("Implement if used");
+//      assert src+size <= getDataSize();
+//      assert dst+size < getDataSize();
+//      getSlabs()[this.getSlabIdx()].copyBytes(getBaseDataAddress()+src, getBaseDataAddress()+dst, size);
+    }
+
+    @Override
+    public boolean isSerialized() {
+      return (UnsafeMemoryChunk.readAbsoluteInt(this.memoryAddress+REF_COUNT_OFFSET) & IS_SERIALIZED_BIT) != 0;
+    }
+
+    @Override
+    public boolean isCompressed() {
+      return (UnsafeMemoryChunk.readAbsoluteInt(this.memoryAddress+REF_COUNT_OFFSET) & IS_COMPRESSED_BIT) != 0;
+    }
+
+    @Override
+    public boolean retain() {
+      return retain(this.memoryAddress);
+    }
+
+    @Override
+    public int getRefCount() {
+      return getRefCount(this.memoryAddress);
+    }
+
+    public static int getSize(long memAddr) {
+      SimpleMemoryAllocatorImpl.validateAddress(memAddr);
+      return UnsafeMemoryChunk.readAbsoluteInt(memAddr+CHUNK_SIZE_OFFSET);
+    }
+    public static void setSize(long memAddr, int size) {
+      SimpleMemoryAllocatorImpl.validateAddressAndSize(memAddr, size);
+      UnsafeMemoryChunk.writeAbsoluteInt(memAddr+CHUNK_SIZE_OFFSET, size);
+    }
+    public static long getNext(long memAddr) {
+      SimpleMemoryAllocatorImpl.validateAddress(memAddr);
+      return UnsafeMemoryChunk.readAbsoluteLong(memAddr+OFF_HEAP_HEADER_SIZE);
+    }
+    public static void setNext(long memAddr, long next) {
+      SimpleMemoryAllocatorImpl.validateAddress(memAddr);
+      UnsafeMemoryChunk.writeAbsoluteLong(memAddr+OFF_HEAP_HEADER_SIZE, next);
+    }
+    @Override
+    public ChunkType getChunkType() {
+      return SimpleMemoryAllocatorImpl.getAllocator().getChunkFactory().getChunkTypeForAddress(getMemoryAddress());
+    }
+    public static int getSrcTypeOrdinal(long memAddr) {
+      return getSrcType(memAddr) >> SRC_TYPE_SHIFT;
+    }
+    public static int getSrcType(long memAddr) {
+      return getSrcTypeFromRawBits(UnsafeMemoryChunk.readAbsoluteInt(memAddr+REF_COUNT_OFFSET));
+    }
+    public static int getSrcTypeFromRawBits(int rawBits) {
+      return rawBits & SRC_TYPE_MASK;
+    }
+    public static int getSrcTypeOrdinalFromRawBits(int rawBits) {
+      return getSrcTypeFromRawBits(rawBits) >> SRC_TYPE_SHIFT;
+    }
+    
+    /**
+     * Fills the chunk with a repeated byte fill pattern.
+     * @param baseAddress the starting address for a {@link Chunk}.
+     */
+    public static void fill(long baseAddress) {
+      long startAddress = baseAddress + MIN_CHUNK_SIZE;
+      int size = getSize(baseAddress) - MIN_CHUNK_SIZE;
+      
+      UnsafeMemoryChunk.fill(startAddress, size, FILL_BYTE);
+    }
+    
+    /**
+     * Validates that the fill pattern for this chunk has not been disturbed.  This method
+     * assumes the TINY_MULTIPLE is 8 bytes.
+     * @throws IllegalStateException when the pattern has been violated.
+     */
+    public void validateFill() {
+      assert SimpleMemoryAllocatorImpl.TINY_MULTIPLE == 8;
+      
+      long startAddress = getMemoryAddress() + MIN_CHUNK_SIZE;
+      int size = getSize() - MIN_CHUNK_SIZE;
+      
+      for(int i = 0;i < size;i += SimpleMemoryAllocatorImpl.TINY_MULTIPLE) {
+        if(UnsafeMemoryChunk.readAbsoluteLong(startAddress + i) != FILL_PATTERN) {
+          throw new IllegalStateException("Fill pattern violated for chunk " + getMemoryAddress() + " with size " + getSize());
+        }        
+      }
+    }
+
+    public void setSerialized(boolean isSerialized) {
+      if (isSerialized) {
+        int bits;
+        int originalBits;
+        do {
+          originalBits = UnsafeMemoryChunk.readAbsoluteIntVolatile(this.memoryAddress+REF_COUNT_OFFSET);
+          if ((originalBits&MAGIC_MASK) != MAGIC_NUMBER) {
+            throw new IllegalStateException("It looks like this off heap memory was already freed. rawBits=" + Integer.toHexString(originalBits));
+          }
+          bits = originalBits | IS_SERIALIZED_BIT;
+        } while (!UnsafeMemoryChunk.writeAbsoluteIntVolatile(this.memoryAddress+REF_COUNT_OFFSET, originalBits, bits));
+      }
+    }
+    public void setCompressed(boolean isCompressed) {
+      if (isCompressed) {
+        int bits;
+        int originalBits;
+        do {
+          originalBits = UnsafeMemoryChunk.readAbsoluteIntVolatile(this.memoryAddress+REF_COUNT_OFFSET);
+          if ((originalBits&MAGIC_MASK) != MAGIC_NUMBER) {
+            throw new IllegalStateException("It looks like this off heap memory was already freed. rawBits=" + Integer.toHexString(originalBits));
+          }
+          bits = originalBits | IS_COMPRESSED_BIT;
+        } while (!UnsafeMemoryChunk.writeAbsoluteIntVolatile(this.memoryAddress+REF_COUNT_OFFSET, originalBits, bits));
+      }
+    }
+    public void setDataSize(int dataSize) { // KIRK
+      assert dataSize <= getSize();
+      int delta = getSize() - dataSize;
+      assert delta <= (DATA_SIZE_DELTA_MASK >> DATA_SIZE_SHIFT);
+      delta <<= DATA_SIZE_SHIFT;
+      int bits;
+      int originalBits;
+      do {
+        originalBits = UnsafeMemoryChunk.readAbsoluteIntVolatile(this.memoryAddress+REF_COUNT_OFFSET);
+        if ((originalBits&MAGIC_MASK) != MAGIC_NUMBER) {
+          throw new IllegalStateException("It looks like this off heap memory was already freed. rawBits=" + Integer.toHexString(originalBits));
+        }
+        bits = originalBits;
+        bits &= ~DATA_SIZE_DELTA_MASK; // clear the old dataSizeDelta bits
+        bits |= delta; // set the dataSizeDelta bits to the new delta value
+      } while (!UnsafeMemoryChunk.writeAbsoluteIntVolatile(this.memoryAddress+REF_COUNT_OFFSET, originalBits, bits));
+    }
+    
+    public void initializeUseCount() {
+      int rawBits;
+      do {
+        rawBits = UnsafeMemoryChunk.readAbsoluteIntVolatile(this.memoryAddress+REF_COUNT_OFFSET);
+        if ((rawBits&MAGIC_MASK) != MAGIC_NUMBER) {
+          throw new IllegalStateException("It looks like this off heap memory was already freed. rawBits=" + Integer.toHexString(rawBits));
+        }
+        int uc = rawBits & REF_COUNT_MASK;
+        if (uc != 0) {
+          throw new IllegalStateException("Expected use count to be zero but it was: " + uc + " rawBits=0x" + Integer.toHexString(rawBits));
+        }
+      } while (!UnsafeMemoryChunk.writeAbsoluteIntVolatile(this.memoryAddress+REF_COUNT_OFFSET, rawBits, rawBits+1));
+    }
+
+    public static int getRefCount(long memAddr) {
+      return UnsafeMemoryChunk.readAbsoluteInt(memAddr+REF_COUNT_OFFSET) & REF_COUNT_MASK;
+    }
+
+    public static boolean retain(long memAddr) {
+      SimpleMemoryAllocatorImpl.validateAddress(memAddr);
+      int uc;
+      int rawBits;
+      int retryCount = 0;
+      do {
+        rawBits = UnsafeMemoryChunk.readAbsoluteIntVolatile(memAddr+REF_COUNT_OFFSET);
+        if ((rawBits&MAGIC_MASK) != MAGIC_NUMBER) {
+          // same as uc == 0
+          // TODO MAGIC_NUMBER rethink its use and interaction with compactor fragments
+          return false;
+        }
+        uc = rawBits & REF_COUNT_MASK;
+        if (uc == MAX_REF_COUNT) {
+          throw new IllegalStateException("Maximum use count exceeded. rawBits=" + Integer.toHexString(rawBits));
+        } else if (uc == 0) {
+          return false;
+        }
+        retryCount++;
+        if (retryCount > 1000) {
+          throw new IllegalStateException("tried to write " + (rawBits+1) + " to @" + Long.toHexString(memAddr) + " 1,000 times.");
+        }
+      } while (!UnsafeMemoryChunk.writeAbsoluteIntVolatile(memAddr+REF_COUNT_OFFSET, rawBits, rawBits+1));
+      //debugLog("use inced ref count " + (uc+1) + " @" + Long.toHexString(memAddr), true);
+      if (ReferenceCountHelper.trackReferenceCounts()) {
+        ReferenceCountHelper.refCountChanged(memAddr, false, uc+1);
+      }
+
+      return true;
+    }
+    public static void release(final long memAddr, boolean issueOnReturnCallback) {
+      SimpleMemoryAllocatorImpl.validateAddress(memAddr);
+      int newCount;
+      int rawBits;
+      boolean returnToAllocator;
+      do {
+        returnToAllocator = false;
+        rawBits = UnsafeMemoryChunk.readAbsoluteIntVolatile(memAddr+REF_COUNT_OFFSET);
+        if ((rawBits&MAGIC_MASK) != MAGIC_NUMBER) {
+          String msg = "It looks like off heap memory @" + Long.toHexString(memAddr) + " was already freed. rawBits=" + Integer.toHexString(rawBits) + " history=" + ReferenceCountHelper.getFreeRefCountInfo(memAddr);
+          //debugLog(msg, true);
+          throw new IllegalStateException(msg);
+        }
+        int curCount = rawBits&REF_COUNT_MASK;
+        if ((curCount) == 0) {
+          //debugLog("too many frees @" + Long.toHexString(memAddr), true);
+          throw new IllegalStateException("Memory has already been freed." + " history=" + ReferenceCountHelper.getFreeRefCountInfo(memAddr) /*+ System.identityHashCode(this)*/);
+        }
+        if (curCount == 1) {
+          newCount = 0; // clear the use count, bits, and the delta size since it will be freed.
+          returnToAllocator = true;
+        } else {
+          newCount = rawBits-1;
+        }
+      } while (!UnsafeMemoryChunk.writeAbsoluteIntVolatile(memAddr+REF_COUNT_OFFSET, rawBits, newCount));
+      //debugLog("free deced ref count " + (newCount&USE_COUNT_MASK) + " @" + Long.toHexString(memAddr), true);
+      if (returnToAllocator ) {
+        /*
+        if(issueOnReturnCallback) {
+         final GemFireCacheImpl.StaticSystemCallbacks sysCb =
+              GemFireCacheImpl.FactoryStatics.systemCallbacks;
+          if(sysCb != null ) {
+            ChunkType ct = SimpleMemoryAllocatorImpl.getAllocator().getChunkFactory().getChunkTypeForRawBits(rawBits);
+            int dataSizeDelta = computeDataSizeDelta(rawBits);
+            sysCb.beforeReturningOffHeapMemoryToAllocator(memAddr, ct, dataSizeDelta);
+          }
+        }
+        */
+       
+        if (ReferenceCountHelper.trackReferenceCounts()) {
+          if (ReferenceCountHelper.trackFreedReferenceCounts()) {
+            ReferenceCountHelper.refCountChanged(memAddr, true, newCount&REF_COUNT_MASK);
+          }
+          ReferenceCountHelper.freeRefCountInfo(memAddr);
+        }
+        
+        // Use fill pattern for free list data integrity check.
+        if(SimpleMemoryAllocatorImpl.getAllocator().validateMemoryWithFill) {
+          fill(memAddr);
+        }
+        
+        SimpleMemoryAllocatorImpl.getAllocator().freeChunk(memAddr);
+      } else {
+        if (ReferenceCountHelper.trackReferenceCounts()) {
+          ReferenceCountHelper.refCountChanged(memAddr, true, newCount&REF_COUNT_MASK);
+        }
+      }
+    }
+    
+    private static int computeDataSizeDelta(int rawBits) {
+      int dataSizeDelta = rawBits;
+      dataSizeDelta &= DATA_SIZE_DELTA_MASK;
+      dataSizeDelta >>= DATA_SIZE_SHIFT;
+      return dataSizeDelta;
+    }
+    
+    @Override
+    public String toString() {
+      return toStringForOffHeapByteSource();
+      // This old impl is not safe because it calls getDeserializedForReading and we have code that call toString that does not inc the refcount.
+      // Also if this Chunk is compressed we don't know how to decompress it.
+      //return super.toString() + ":<dataSize=" + getDataSize() + " refCount=" + getRefCount() + " addr=" + getMemoryAddress() + " storedObject=" + getDeserializedForReading() + ">";
+    }
+    
+    protected String toStringForOffHeapByteSource() {
+      return super.toString() + ":<dataSize=" + getDataSize() + " refCount=" + getRefCount() + " addr=" + Long.toHexString(getMemoryAddress()) + ">";
+    }
+    
+    @Override
+    public State getState() {
+      if (getRefCount() > 0) {
+        return State.ALLOCATED;
+      } else {
+        return State.DEALLOCATED;
+      }
+    }
+    @Override
+    public MemoryBlock getNextBlock() {
+      throw new UnsupportedOperationException();
+    }
+    @Override
+    public int getBlockSize() {
+      return getSize();
+    }
+    @Override
+    public int getSlabId() {
+      throw new UnsupportedOperationException();
+    }
+    @Override
+    public int getFreeListId() {
+      return -1;
+    }
+    @Override
+    public String getDataType() {
+      return null;
+    }
+    @Override
+    public Object getDataValue() {
+      return null;
+    }
+    public Chunk slice(int position, int limit) {
+      throw new UnsupportedOperationException();
+    }
+  }
\ 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/offheap/ChunkFactory.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/ChunkFactory.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/ChunkFactory.java
new file mode 100644
index 0000000..f7d4ba8
--- /dev/null
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/ChunkFactory.java
@@ -0,0 +1,51 @@
+/*
+ * 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;
+
+
+/**
+ * ChunkFactory can be used to create Chunk instances.
+ * It can also be used to determine the ChunkType given a Chunk address
+ * or the object header bits from an existing Chunk.
+ */
+public interface ChunkFactory  {
+  /**
+   * Create a new chunk of the given size and type at the given address.
+   */
+  Chunk newChunk(long address, int chunkSize, ChunkType chunkType);
+  /**
+   * Create a new chunk for a block of memory (identified by address)
+   * that has already been allocated.
+   * The size and type are derived from the existing object header.
+   */
+  Chunk newChunk(long address);
+  /**
+   * Create a new chunk of the given type for a block of memory (identified by address)
+   * that has already been allocated.
+   * The size is derived from the existing object header.
+   */
+  Chunk newChunk(long address, ChunkType chunkType);
+  /**
+   * Given the address of an existing chunk return its ChunkType.
+   */
+  ChunkType getChunkTypeForAddress(long address);
+  /**
+   * Given the rawBits from the object header of an existing chunk
+   * return its ChunkType.
+   */
+  ChunkType getChunkTypeForRawBits(int bits);
+}
\ 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/offheap/ChunkType.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/ChunkType.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/ChunkType.java
new file mode 100644
index 0000000..9841368
--- /dev/null
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/ChunkType.java
@@ -0,0 +1,26 @@
+/*
+ * 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 new chunks of a certain type.
+ */
+public abstract class ChunkType {
+  public abstract int getSrcType();
+  public abstract Chunk newChunk(long memoryAddress);
+  public abstract Chunk newChunk(long memoryAddress, int chunkSize);
+}
\ 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/offheap/ChunkWithHeapForm.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/ChunkWithHeapForm.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/ChunkWithHeapForm.java
new file mode 100644
index 0000000..d7e65f7
--- /dev/null
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/ChunkWithHeapForm.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.gemstone.gemfire.internal.offheap;
+
+/**
+ * Used to keep the heapForm around while an operation is still in progress.
+ * This allows the operation to access the serialized heap form instead of copying
+ * it from offheap. See bug 48135.
+ */
+public class ChunkWithHeapForm extends GemFireChunk {
+  private final byte[] heapForm;
+  
+  public ChunkWithHeapForm(GemFireChunk chunk, byte[] heapForm) {
+    super(chunk);
+    this.heapForm = heapForm;
+  }
+
+  @Override
+  protected byte[] getRawBytes() {
+    return this.heapForm;
+  }
+  
+  public Chunk getChunkWithoutHeapForm() {
+    return new GemFireChunk(this);
+  }
+}
\ 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/offheap/DataAsAddress.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/DataAsAddress.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/DataAsAddress.java
new file mode 100644
index 0000000..61204ba
--- /dev/null
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/DataAsAddress.java
@@ -0,0 +1,205 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.gemstone.gemfire.internal.offheap;
+
+import java.io.DataOutput;
+import java.io.IOException;
+
+import com.gemstone.gemfire.DataSerializer;
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.internal.DataSerializableFixedID;
+import com.gemstone.gemfire.internal.InternalDataSerializer;
+import com.gemstone.gemfire.internal.cache.BytesAndBitsForCompactor;
+import com.gemstone.gemfire.internal.cache.EntryBits;
+import com.gemstone.gemfire.internal.cache.RegionEntry;
+import com.gemstone.gemfire.internal.cache.RegionEntryContext;
+import com.gemstone.gemfire.internal.lang.StringUtils;
+
+/**
+ * Used to represent offheap addresses whose
+ * value encodes actual data instead a memory
+ * location.
+ * Instances of this class have a very short lifetime.
+ */
+public class DataAsAddress implements StoredObject {
+  private final long address;
+  
+  public DataAsAddress(long addr) {
+    this.address = addr;
+  }
+  
+  public long getEncodedAddress() {
+    return this.address;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (o instanceof DataAsAddress) {
+      return getEncodedAddress() == ((DataAsAddress) o).getEncodedAddress();
+    }
+    return false;
+  }
+  
+  @Override
+  public int hashCode() {
+    long value = getEncodedAddress();
+    return (int)(value ^ (value >>> 32));
+  }
+
+  @Override
+  public int getSizeInBytes() {
+    return 0;
+  }
+
+  public byte[] getDecompressedBytes(RegionEntryContext r) {
+    return OffHeapRegionEntryHelper.encodedAddressToBytes(this.address, true, r);
+  }
+
+  /**
+   * If we contain a byte[] return it.
+   * Otherwise return the serialize bytes in us in a byte array.
+   */
+  public byte[] getRawBytes() {
+    return OffHeapRegionEntryHelper.encodedAddressToRawBytes(this.address);
+  }
+  
+  @Override
+  public byte[] getSerializedValue() {
+    return OffHeapRegionEntryHelper.encodedAddressToBytes(this.address);
+  }
+
+  @Override
+  public Object getDeserializedValue(Region r, RegionEntry re) {
+    return OffHeapRegionEntryHelper.encodedAddressToObject(this.address);
+  }
+
+  @Override
+  public Object getDeserializedForReading() {
+    return getDeserializedValue(null,null);
+  }
+  
+  @Override
+  public Object getValueAsDeserializedHeapObject() {
+    return getDeserializedValue(null,null);
+  }
+  
+  @Override
+  public byte[] getValueAsHeapByteArray() {
+    if (isSerialized()) {
+      return getSerializedValue();
+    } else {
+      return (byte[])getDeserializedForReading();
+    }
+  }
+
+  @Override
+  public String getStringForm() {
+    try {
+      return StringUtils.forceToString(getDeserializedForReading());
+    } catch (RuntimeException ex) {
+      return "Could not convert object to string because " + ex;
+    }
+  }
+
+  @Override
+  public Object getDeserializedWritableCopy(Region r, RegionEntry re) {
+    return getDeserializedValue(null,null);
+  }
+
+  @Override
+  public Object getValue() {
+    if (isSerialized()) {
+      return getSerializedValue();
+    } else {
+      throw new IllegalStateException("Can not call getValue on StoredObject that is not serialized");
+    }
+  }
+
+  @Override
+  public void writeValueAsByteArray(DataOutput out) throws IOException {
+    DataSerializer.writeByteArray(getSerializedValue(), out);
+  }
+
+  @Override
+  public void fillSerializedValue(BytesAndBitsForCompactor wrapper,
+      byte userBits) {
+    byte[] value;
+    if (isSerialized()) {
+      value = getSerializedValue();
+      userBits = EntryBits.setSerialized(userBits, true);
+    } else {
+      value = (byte[]) getDeserializedForReading();
+    }
+    wrapper.setData(value, userBits, value.length, true);
+  }
+
+  @Override
+  public int getValueSizeInBytes() {
+    return 0;
+  }
+  
+  @Override
+  public void sendTo(DataOutput out) throws IOException {
+    if (isSerialized()) {
+      out.write(getSerializedValue());
+    } else {
+      Object objToSend = (byte[]) getDeserializedForReading(); // deserialized as a byte[]
+      DataSerializer.writeObject(objToSend, out);
+    }
+  }
+
+  @Override
+  public void sendAsByteArray(DataOutput out) throws IOException {
+    byte[] bytes;
+    if (isSerialized()) {
+      bytes = getSerializedValue();
+    } else {
+      bytes = (byte[]) getDeserializedForReading();
+    }
+    DataSerializer.writeByteArray(bytes, out);
+    
+  }
+  
+  @Override
+  public void sendAsCachedDeserializable(DataOutput out) throws IOException {
+    if (!isSerialized()) {
+      throw new IllegalStateException("sendAsCachedDeserializable can only be called on serialized StoredObjects");
+    }
+    InternalDataSerializer.writeDSFIDHeader(DataSerializableFixedID.VM_CACHED_DESERIALIZABLE, out);
+    sendAsByteArray(out);
+  }
+
+  @Override
+  public boolean isSerialized() {
+    return OffHeapRegionEntryHelper.isSerialized(this.address);
+  }
+
+  @Override
+  public boolean isCompressed() {
+    return OffHeapRegionEntryHelper.isCompressed(this.address);
+  }
+  
+  @Override
+  public boolean retain() {
+    // nothing needed
+    return true;
+  }
+  @Override
+  public void release() {
+    // nothing needed
+  }
+}
\ No newline at end of file


[08/19] incubator-geode git commit: fix CI Failure: ClientMembershipDUnitTest.testGetConnectedClients

Posted by ab...@apache.org.
fix CI Failure: ClientMembershipDUnitTest.testGetConnectedClients

increased the wait-time for clients to connect to the server from
10 seconds to 60 seconds.


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

Branch: refs/heads/feature/GEODE-584
Commit: dc780a89cad684c4b62ca306683674212a80b5ac
Parents: 59f2087
Author: Bruce Schuchardt <bs...@pivotal.io>
Authored: Mon Nov 23 14:51:34 2015 -0800
Committer: Bruce Schuchardt <bs...@pivotal.io>
Committed: Mon Nov 23 14:51:34 2015 -0800

----------------------------------------------------------------------
 .../com/gemstone/gemfire/cache30/ClientMembershipDUnitTest.java | 5 +++--
 1 file changed, 3 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/dc780a89/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/ClientMembershipDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/ClientMembershipDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/ClientMembershipDUnitTest.java
index 4f06619..d9e9f4c 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/ClientMembershipDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/ClientMembershipDUnitTest.java
@@ -1342,6 +1342,7 @@ public class ClientMembershipDUnitTest extends ClientServerTestCase {
     
     for (int i = 0; i < host.getVMCount(); i++) { 
       final VM vm = Host.getHost(0).getVM(i);
+      System.out.println("creating pool in vm_"+i);
       vm.invoke(createPool);
       clientMemberIdArray[i] =  String.valueOf(vm.invoke(
         ClientMembershipDUnitTest.class, "getMemberId"));
@@ -1365,7 +1366,7 @@ public class ClientMembershipDUnitTest extends ClientServerTestCase {
           return true;
         }
       };
-      waitForCriterion(wc, 10000, 100, false);
+      waitForCriterion(wc, 30000, 100, false);
     }
     
     Map connectedClients = InternalClientMembership.getConnectedClients(false);
@@ -1465,7 +1466,7 @@ public class ClientMembershipDUnitTest extends ClientServerTestCase {
           return true;
         }
       };
-      waitForCriterion(wc, 10000, 100, false);
+      waitForCriterion(wc, 60000, 100, false);
     }
 
     {


[17/19] incubator-geode git commit: GEODE-587: Improve classpath generation

Posted by ab...@apache.org.
GEODE-587: Improve classpath generation

Improve classpath generation for the *-dependencies jar files.  All
dependent projects are now automatically analyzed for matching jar
names.


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

Branch: refs/heads/feature/GEODE-584
Commit: 3588b029a23c0dd4fd3576a7f25efe0c57581386
Parents: abad018
Author: Anthony Baker <ab...@pivotal.io>
Authored: Mon Nov 23 17:30:06 2015 -0800
Committer: Anthony Baker <ab...@pivotal.io>
Committed: Tue Nov 24 11:33:52 2015 -0800

----------------------------------------------------------------------
 gemfire-assembly/build.gradle | 20 +++++++++++---------
 1 file changed, 11 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3588b029/gemfire-assembly/build.gradle
----------------------------------------------------------------------
diff --git a/gemfire-assembly/build.gradle b/gemfire-assembly/build.gradle
index 8f803ae..1324cb9 100755
--- a/gemfire-assembly/build.gradle
+++ b/gemfire-assembly/build.gradle
@@ -74,13 +74,16 @@ task defaultCacheConfig(type: JavaExec, dependsOn: classes) {
 // This closure sets the gemfire classpath.  If we add another jar to the classpath it must
 // be included in the filter logic below.
 def cp = {
+  // first add all the dependent project jars
   def jars = configurations.archives.dependencies.collect { it.dependencyProject }
     .findAll { !it.name.contains('web') }
     .collect { it.jar.archiveName }
     .join(' ')
 
-  jars += ' ' + 
-    project(':gemfire-core').configurations.runtime.collect { it.getName() }.findAll {
+  // then add all the dependencies of the dependent jars
+  jars += ' ' + configurations.archives.dependencies.collect { 
+    it.dependencyProject.configurations.runtime.collect { it.getName() }.findAll {
+      // depedencies from gemfire-core
       it.contains('antlr') ||
       it.contains('commons-io') ||
       it.contains('commons-logging') ||
@@ -113,17 +116,16 @@ def cp = {
       it.contains('snappy-java') ||
       it.contains('hbase') ||
       it.contains('jgroups') ||
-      it.contains('netty')
-    }.join(' ')
-    
-  jars += ' ' +
-    project(':gemfire-lucene').configurations.runtime.collect { it.getName() }.findAll {
+      it.contains('netty') ||
+      
+      // dependencies from gemfire-lucene
       it.contains('lucene-analyzers-common') ||
       it.contains('lucene-core') ||
       it.contains('lucene-queries') ||
       it.contains('lucene-queryparser')
-    }.join(' ')
-    
+    }
+  }.flatten().unique().join(' ')
+
   return jars
 }
 


[04/19] incubator-geode git commit: fixing compilation problem for Eclipse v4.4.1 in new unit test

Posted by ab...@apache.org.
fixing compilation problem for Eclipse v4.4.1 in new unit test

reviewed by Dan


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

Branch: refs/heads/feature/GEODE-584
Commit: 1276cc80033c4e8c642137d2d7e53291c4b4fc71
Parents: 25c8f0c
Author: Bruce Schuchardt <bs...@pivotal.io>
Authored: Mon Nov 23 12:25:37 2015 -0800
Committer: Bruce Schuchardt <bs...@pivotal.io>
Committed: Mon Nov 23 12:25:37 2015 -0800

----------------------------------------------------------------------
 .../lucene/internal/PartitionedRepositoryManagerJUnitTest.java   | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1276cc80/gemfire-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/PartitionedRepositoryManagerJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/PartitionedRepositoryManagerJUnitTest.java b/gemfire-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/PartitionedRepositoryManagerJUnitTest.java
index 41376f5..bf09f99 100644
--- a/gemfire-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/PartitionedRepositoryManagerJUnitTest.java
+++ b/gemfire-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/PartitionedRepositoryManagerJUnitTest.java
@@ -171,7 +171,7 @@ public class PartitionedRepositoryManagerJUnitTest {
 
     Set<Integer> buckets = new LinkedHashSet<Integer>(Arrays.asList(0, 1));
     InternalRegionFunctionContext ctx = Mockito.mock(InternalRegionFunctionContext.class);
-    Mockito.when(ctx.getLocalBucketSet((any(Region.class)))).thenReturn(buckets);
+    Mockito.when(ctx.getLocalBucketSet((any()))).thenReturn(buckets);
     Collection<IndexRepository> repos = repoManager.getRepositories(ctx);
     assertEquals(2, repos.size());
 
@@ -199,7 +199,7 @@ public class PartitionedRepositoryManagerJUnitTest {
     Set<Integer> buckets = new LinkedHashSet<Integer>(Arrays.asList(0, 1));
 
     InternalRegionFunctionContext ctx = Mockito.mock(InternalRegionFunctionContext.class);
-    Mockito.when(ctx.getLocalBucketSet((any(Region.class)))).thenReturn(buckets);
+    Mockito.when(ctx.getLocalBucketSet((any()))).thenReturn(buckets);
     repoManager.getRepositories(ctx);
   }
   


[05/19] incubator-geode git commit: fixing GEODE-590: CI failure in GMSLocatorRecoveryJUnitTest

Posted by ab...@apache.org.
fixing GEODE-590: CI failure in GMSLocatorRecoveryJUnitTest

GMSHealthMonitorJUnitTest was leaving a system property set and poisoning
subsequent unit tests by overriding the bind-address setting in the
junit JVM.

reviewed by Jianxia


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

Branch: refs/heads/feature/GEODE-584
Commit: 452328f5c71de8c60ff757bd33a6f96e6ee8d403
Parents: 1276cc8
Author: Bruce Schuchardt <bs...@pivotal.io>
Authored: Mon Nov 23 12:29:52 2015 -0800
Committer: Bruce Schuchardt <bs...@pivotal.io>
Committed: Mon Nov 23 12:29:52 2015 -0800

----------------------------------------------------------------------
 .../membership/gms/membership/fd/GMSHealthMonitorJUnitTest.java     | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/452328f5/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/membership/fd/GMSHealthMonitorJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/membership/fd/GMSHealthMonitorJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/membership/fd/GMSHealthMonitorJUnitTest.java
index 697e899..1699068 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/membership/fd/GMSHealthMonitorJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/membership/fd/GMSHealthMonitorJUnitTest.java
@@ -100,6 +100,7 @@ public class GMSHealthMonitorJUnitTest {
   @After
   public void tearDown() {
     gmsHealthMonitor.stop();
+    System.getProperties().remove("gemfire.bind-address");
   }
 
   @Test


[19/19] incubator-geode git commit: Merge branch 'develop' into feature/GEODE-584

Posted by ab...@apache.org.
Merge branch 'develop' into feature/GEODE-584


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

Branch: refs/heads/feature/GEODE-584
Commit: e38a9425a5ef193759139a4933079bdb728c6489
Parents: 1e33973 52f0a1d
Author: Anthony Baker <ab...@pivotal.io>
Authored: Tue Nov 24 12:26:05 2015 -0800
Committer: Anthony Baker <ab...@pivotal.io>
Committed: Tue Nov 24 12:26:05 2015 -0800

----------------------------------------------------------------------
 gemfire-assembly/build.gradle                   |   35 +-
 .../internal/GetOperationContextImpl.java       |    2 +-
 .../query/internal/index/AbstractIndex.java     |    2 +-
 .../query/internal/index/DummyQRegion.java      |    2 +-
 .../cache/query/internal/index/HashIndex.java   |  178 +-
 .../query/internal/index/HashIndexSet.java      | 1086 ++----
 .../query/internal/index/HashIndexStrategy.java |   90 -
 .../internal/DistributionManager.java           |    2 +-
 .../internal/InternalDistributedSystem.java     |    8 +-
 .../gms/messenger/JGroupsMessenger.java         |    9 +-
 .../internal/cache/AbstractRegionEntry.java     |   29 +-
 .../internal/cache/AbstractRegionMap.java       |   16 +-
 .../cache/BytesAndBitsForCompactor.java         |    2 +-
 .../gemfire/internal/cache/DiskEntry.java       |    8 +-
 .../internal/cache/DistributedRegion.java       |    2 +-
 .../gemfire/internal/cache/EntryEventImpl.java  |   45 +-
 .../internal/cache/GemFireCacheImpl.java        |    4 -
 .../gemfire/internal/cache/LocalRegion.java     |   10 +-
 .../gemstone/gemfire/internal/cache/Oplog.java  |    6 +-
 .../internal/cache/PartitionedRegion.java       |    2 +-
 .../internal/cache/VMThinRegionEntry.java       |    5 -
 .../internal/cache/tier/sockets/Part.java       |    4 +-
 .../cache/wan/GatewaySenderEventImpl.java       |   10 +-
 .../gemfire/internal/offheap/Chunk.java         |  793 +++++
 .../gemfire/internal/offheap/ChunkFactory.java  |   51 +
 .../gemfire/internal/offheap/ChunkType.java     |   26 +
 .../internal/offheap/ChunkWithHeapForm.java     |   40 +
 .../gemfire/internal/offheap/DataAsAddress.java |  205 ++
 .../gemfire/internal/offheap/Fragment.java      |  125 +
 .../internal/offheap/FreeListManager.java       |  807 +++++
 .../gemfire/internal/offheap/GemFireChunk.java  |   55 +
 .../internal/offheap/GemFireChunkFactory.java   |   52 +
 .../internal/offheap/GemFireChunkSlice.java     |   44 +
 .../internal/offheap/LifecycleListener.java     |   98 +
 .../internal/offheap/MemoryAllocator.java       |    1 -
 .../gemfire/internal/offheap/MemoryBlock.java   |    1 -
 .../internal/offheap/MemoryBlockNode.java       |  158 +
 .../offheap/OffHeapCachedDeserializable.java    |    1 -
 .../gemfire/internal/offheap/OffHeapHelper.java |    8 +-
 .../offheap/OffHeapRegionEntryHelper.java       |    8 +-
 .../internal/offheap/RefCountChangeInfo.java    |  112 +
 .../internal/offheap/ReferenceCountHelper.java  |  235 ++
 .../offheap/SimpleMemoryAllocatorImpl.java      | 3242 +-----------------
 .../internal/offheap/SyncChunkStack.java        |  130 +
 .../internal/tcp/ByteBufferInputStream.java     |    2 +-
 .../tcp/ImmutableByteBufferInputStream.java     |    2 +-
 .../gemfire/internal/util/BlobHelper.java       |    2 +-
 .../gemfire/internal/util/ObjectProcedure.java  |   30 -
 .../gemfire/internal/util/PrimeFinder.java      |  159 -
 .../gemfire/pdx/internal/PdxInputStream.java    |    2 +-
 .../internal/index/HashIndexJUnitTest.java      |   23 +-
 .../internal/index/HashIndexSetJUnitTest.java   |  504 +++
 .../cache30/ClientMembershipDUnitTest.java      |    5 +-
 .../gemfire/cache30/ReconnectDUnitTest.java     |    4 +-
 .../locator/GMSLocatorRecoveryJUnitTest.java    |    4 +-
 .../fd/GMSHealthMonitorJUnitTest.java           |    3 +-
 ...wardCompatibilitySerializationDUnitTest.java |   13 +-
 .../cache/ChunkValueWrapperJUnitTest.java       |    2 +-
 .../gemfire/internal/cache/OffHeapTestUtil.java |    5 +-
 .../cache/OldValueImporterTestBase.java         |    4 +-
 .../offheap/ConcurrentBagJUnitTest.java         |  130 -
 .../offheap/FreeListOffHeapRegionJUnitTest.java |    2 +-
 .../internal/offheap/OffHeapRegionBase.java     |    1 -
 .../offheap/OffHeapValidationJUnitTest.java     |    1 -
 .../OffHeapWriteObjectAsByteArrayJUnitTest.java |    2 -
 .../OldFreeListOffHeapRegionJUnitTest.java      |    2 +-
 ...mpleMemoryAllocatorFillPatternJUnitTest.java |    2 -
 .../offheap/SimpleMemoryAllocatorJUnitTest.java |    9 +-
 ...moryAllocatorLifecycleListenerJUnitTest.java |    9 +-
 .../OffHeapByteBufferByteSourceJUnitTest.java   |    2 +-
 .../gemfire/pdx/OffHeapByteSourceJUnitTest.java |    2 +-
 .../gemfire/codeAnalysis/excludedClasses.txt    |    1 +
 .../codeAnalysis/sanctionedSerializables.txt    |    1 -
 gemfire-lucene/build.gradle                     |    4 +-
 .../PartitionedRepositoryManagerJUnitTest.java  |    4 +-
 75 files changed, 4073 insertions(+), 4612 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/e38a9425/gemfire-assembly/build.gradle
----------------------------------------------------------------------
diff --cc gemfire-assembly/build.gradle
index de8b584,1324cb9..1f1b798
--- a/gemfire-assembly/build.gradle
+++ b/gemfire-assembly/build.gradle
@@@ -73,14 -74,19 +74,19 @@@ task defaultCacheConfig(type: JavaExec
  // This closure sets the gemfire classpath.  If we add another jar to the classpath it must
  // be included in the filter logic below.
  def cp = {
-   configurations.archives.dependencies.collect { it.dependencyProject }
+   // first add all the dependent project jars
+   def jars = configurations.archives.dependencies.collect { it.dependencyProject }
      .findAll { !it.name.contains('web') }
      .collect { it.jar.archiveName }
-     .join(' ') + ' ' +
-     project(':gemfire-core').configurations.runtime.collect { it.getName() }.findAll {
+     .join(' ')
+ 
+   // then add all the dependencies of the dependent jars
+   jars += ' ' + configurations.archives.dependencies.collect { 
+     it.dependencyProject.configurations.runtime.collect { it.getName() }.findAll {
+       // depedencies from gemfire-core
        it.contains('antlr') ||
        it.contains('commons-io') ||
 -      it.contains('commons-logging') ||
 +      it.contains('commons-lang') ||
        it.contains('fastutil') ||
        it.contains('jackson-annotations') ||
        it.contains('jackson-core') ||


[02/19] incubator-geode git commit: another attempt to fix this test's configuration

Posted by ab...@apache.org.
another attempt to fix this test's configuration

For some reason one of the build machines is causing a problem with this
test.  It should be using the same ip address for both bind-address and
locator-name but something is messing with it.  I've changed it to use
host addresses instead.  The test passes on the build machine with or
without this change when run as a single test.


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

Branch: refs/heads/feature/GEODE-584
Commit: ac0b68ecd3a85a4b8c1bf70e8e9d242429a7ea67
Parents: c9d8778
Author: Bruce Schuchardt <bs...@pivotal.io>
Authored: Fri Nov 20 14:50:38 2015 -0800
Committer: Bruce Schuchardt <bs...@pivotal.io>
Committed: Fri Nov 20 14:51:32 2015 -0800

----------------------------------------------------------------------
 .../membership/gms/locator/GMSLocatorRecoveryJUnitTest.java      | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ac0b68ec/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/locator/GMSLocatorRecoveryJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/locator/GMSLocatorRecoveryJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/locator/GMSLocatorRecoveryJUnitTest.java
index 8484f7c..49e1c48 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/locator/GMSLocatorRecoveryJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/locator/GMSLocatorRecoveryJUnitTest.java
@@ -136,8 +136,8 @@ public class GMSLocatorRecoveryJUnitTest {
       nonDefault.put(DistributionConfig.MCAST_PORT_NAME, "0");
       nonDefault.put(DistributionConfig.LOG_FILE_NAME, "");
       nonDefault.put(DistributionConfig.LOG_LEVEL_NAME, "fine");
-      nonDefault.put(DistributionConfig.LOCATORS_NAME, localHost.getHostName()+'['+port+']');
-      nonDefault.put(DistributionConfig.BIND_ADDRESS_NAME, localHost.getHostName());
+      nonDefault.put(DistributionConfig.LOCATORS_NAME, localHost.getHostAddress()+'['+port+']');
+      nonDefault.put(DistributionConfig.BIND_ADDRESS_NAME, localHost.getHostAddress());
       DistributionConfigImpl config = new DistributionConfigImpl(nonDefault);
       RemoteTransportConfig transport = new RemoteTransportConfig(config,
           DistributionManager.NORMAL_DM_TYPE);


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

Posted by ab...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4b0925e3/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/Fragment.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/Fragment.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/Fragment.java
new file mode 100644
index 0000000..bd05ddb
--- /dev/null
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/Fragment.java
@@ -0,0 +1,125 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.gemstone.gemfire.internal.offheap;
+
+import java.util.concurrent.atomic.AtomicIntegerFieldUpdater;
+
+/**
+ * A fragment is a block of memory that can have chunks allocated from it.
+ * The allocations are always from the front so the free memory is always
+ * at the end. The freeIdx keeps track of the first byte of free memory in
+ * the fragment.
+ * The base memory address and the total size of a fragment never change.
+ * During compaction fragments go away and are recreated.
+ * 
+ * @author darrel
+ *
+ */
+public class Fragment implements MemoryBlock {
+  private static final byte FILL_BYTE = Chunk.FILL_BYTE;
+  private final long baseAddr;
+  private final int size;
+  @SuppressWarnings("unused")
+  private volatile int freeIdx;
+  private static AtomicIntegerFieldUpdater<Fragment> freeIdxUpdater = AtomicIntegerFieldUpdater.newUpdater(Fragment.class, "freeIdx");
+  
+  public Fragment(long addr, int size) {
+    SimpleMemoryAllocatorImpl.validateAddress(addr);
+    this.baseAddr = addr;
+    this.size = size;
+    freeIdxUpdater.set(this, 0);
+  }
+  
+  public int freeSpace() {
+    return getSize() - getFreeIndex();
+  }
+
+  public boolean allocate(int oldOffset, int newOffset) {
+    return freeIdxUpdater.compareAndSet(this, oldOffset, newOffset);
+  }
+
+  public int getFreeIndex() {
+    return freeIdxUpdater.get(this);
+  }
+
+  public int getSize() {
+    return this.size;
+  }
+
+  public long getMemoryAddress() {
+    return this.baseAddr;
+  }
+
+  @Override
+  public State getState() {
+    return State.UNUSED;
+  }
+
+  @Override
+  public MemoryBlock getNextBlock() {
+    throw new UnsupportedOperationException();
+  }
+  
+  @Override
+  public int getBlockSize() {
+    return freeSpace();
+  }
+  
+  @Override
+  public int getSlabId() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public int getFreeListId() {
+    return -1;
+  }
+
+  @Override
+  public int getRefCount() {
+    return 0;
+  }
+
+  @Override
+  public String getDataType() {
+    return "N/A";
+  }
+
+  @Override
+  public boolean isSerialized() {
+    return false;
+  }
+
+  @Override
+  public boolean isCompressed() {
+    return false;
+  }
+
+  @Override
+  public Object getDataValue() {
+    return null;
+  }
+  
+  public void fill() {
+    UnsafeMemoryChunk.fill(this.baseAddr, this.size, FILL_BYTE);
+  }
+
+  @Override
+  public ChunkType getChunkType() {
+    return null;
+  }
+}
\ 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/offheap/FreeListManager.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/FreeListManager.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/FreeListManager.java
new file mode 100644
index 0000000..48a0756
--- /dev/null
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/FreeListManager.java
@@ -0,0 +1,807 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.gemstone.gemfire.internal.offheap;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.NavigableSet;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReferenceArray;
+
+import com.gemstone.gemfire.LogWriter;
+import com.gemstone.gemfire.OutOfOffHeapMemoryException;
+import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
+import com.gemstone.gemfire.internal.offheap.MemoryBlock.State;
+
+/**
+ * Manages the free lists for a SimpleMemoryAllocatorImpl
+ */
+public class FreeListManager {
+  final private AtomicReferenceArray<SyncChunkStack> tinyFreeLists = new AtomicReferenceArray<SyncChunkStack>(SimpleMemoryAllocatorImpl.TINY_FREE_LIST_COUNT);
+  // hugeChunkSet is sorted by chunk size in ascending order. It will only contain chunks larger than MAX_TINY.
+  private final ConcurrentSkipListSet<Chunk> hugeChunkSet = new ConcurrentSkipListSet<Chunk>();
+  private final AtomicLong allocatedSize = new AtomicLong(0L);
+
+  private int getNearestTinyMultiple(int size) {
+    return (size-1)/SimpleMemoryAllocatorImpl.TINY_MULTIPLE;
+  }
+  List<Chunk> getLiveChunks() {
+    ArrayList<Chunk> result = new ArrayList<Chunk>();
+    UnsafeMemoryChunk[] slabs = this.ma.getSlabs();
+    for (int i=0; i < slabs.length; i++) {
+      getLiveChunks(slabs[i], result);
+    }
+    return result;
+  }
+  private void getLiveChunks(UnsafeMemoryChunk slab, List<Chunk> result) {
+    long addr = slab.getMemoryAddress();
+    while (addr <= (slab.getMemoryAddress() + slab.getSize() - Chunk.MIN_CHUNK_SIZE)) {
+      Fragment f = isAddrInFragmentFreeSpace(addr);
+      if (f != null) {
+        addr = f.getMemoryAddress() + f.getSize();
+      } else {
+        int curChunkSize = Chunk.getSize(addr);
+        int refCount = Chunk.getRefCount(addr);
+        if (refCount > 0) {
+          result.add(this.ma.chunkFactory.newChunk(addr));
+        }
+        addr += curChunkSize;
+      }
+    }
+  }
+  /**
+   * If addr is in the free space of a fragment then return that fragment; otherwise return null.
+   */
+  private Fragment isAddrInFragmentFreeSpace(long addr) {
+    for (Fragment f: this.fragmentList) {
+      if (addr >= (f.getMemoryAddress() + f.getFreeIndex()) && addr < (f.getMemoryAddress() + f.getSize())) {
+        return f;
+      }
+    }
+    return null;
+  }
+  public long getUsedMemory() {
+    return this.allocatedSize.get();
+  }
+  public long getFreeMemory() {
+    return this.ma.getTotalMemory() - getUsedMemory();
+  }
+  long getFreeFragmentMemory() {
+    long result = 0;
+    for (Fragment f: this.fragmentList) {
+      int freeSpace = f.freeSpace();
+      if (freeSpace >= Chunk.MIN_CHUNK_SIZE) {
+        result += freeSpace;
+      }
+    }
+    return result;
+  }
+  long getFreeTinyMemory() {
+    long tinyFree = 0;
+    for (int i=0; i < this.tinyFreeLists.length(); i++) {
+      SyncChunkStack cl = this.tinyFreeLists.get(i);
+      if (cl != null) {
+        tinyFree += cl.computeTotalSize();
+      }
+    }
+    return tinyFree;
+  }
+  long getFreeHugeMemory() {
+    long hugeFree = 0;
+    for (Chunk c: this.hugeChunkSet) {
+      hugeFree += c.getSize();
+    }
+    return hugeFree;
+  }
+
+  /**
+   * The id of the last fragment we allocated from.
+   */
+  private final AtomicInteger lastFragmentAllocation = new AtomicInteger(0);
+  private final CopyOnWriteArrayList<Fragment> fragmentList;
+  private final SimpleMemoryAllocatorImpl ma;
+
+  public FreeListManager(SimpleMemoryAllocatorImpl ma) {
+    this.ma = ma;
+    UnsafeMemoryChunk[] slabs = ma.getSlabs();
+    Fragment[] tmp = new Fragment[slabs.length];
+    for (int i=0; i < slabs.length; i++) {
+      tmp[i] = new Fragment(slabs[i].getMemoryAddress(), slabs[i].getSize());
+    }
+    this.fragmentList = new CopyOnWriteArrayList<Fragment>(tmp);
+
+    if(ma.validateMemoryWithFill) {
+      fillFragments();
+    }
+  }
+
+  /**
+   * Fills all fragments with a fill used for data integrity validation.
+   */
+  private void fillFragments() {
+    for(Fragment fragment : this.fragmentList) {
+      fragment.fill();
+    }
+  }
+
+  /**
+   * Allocate a chunk of memory of at least the given size.
+   * The basic algorithm is:
+   * 1. Look for a previously allocated and freed chunk close to the size requested.
+   * 2. See if the original chunk is big enough to split. If so do so.
+   * 3. Look for a previously allocated and freed chunk of any size larger than the one requested.
+   *    If we find one split it.
+   * <p>
+   * It might be better not to include step 3 since we expect and freed chunk to be reallocated in the future.
+   * Maybe it would be better for 3 to look for adjacent free blocks that can be merged together.
+   * For now we will just try 1 and 2 and then report out of mem.
+   * @param size minimum bytes the returned chunk must have.
+   * @param chunkType TODO
+   * @return the allocated chunk
+   * @throws IllegalStateException if a chunk can not be allocated.
+   */
+  @SuppressWarnings("synthetic-access")
+  public Chunk allocate(int size, ChunkType chunkType) {
+    Chunk result = null;
+    {
+      assert size > 0;
+      if (chunkType == null) {
+        chunkType = GemFireChunk.TYPE;
+      }
+      result = basicAllocate(size, true, chunkType);
+      result.setDataSize(size);
+    }
+    this.ma.stats.incObjects(1);
+    int resultSize = result.getSize();
+    this.allocatedSize.addAndGet(resultSize);
+    this.ma.stats.incUsedMemory(resultSize);
+    this.ma.stats.incFreeMemory(-resultSize);
+    result.initializeUseCount();
+    this.ma.notifyListeners();
+
+    return result;
+  }
+
+  private Chunk basicAllocate(int size, boolean useSlabs, ChunkType chunkType) {
+    if (useSlabs) {
+      // Every object stored off heap has a header so we need
+      // to adjust the size so that the header gets allocated.
+      // If useSlabs is false then the incoming size has already
+      // been adjusted.
+      size += Chunk.OFF_HEAP_HEADER_SIZE;
+    }
+    if (size <= SimpleMemoryAllocatorImpl.MAX_TINY) {
+      return allocateTiny(size, useSlabs, chunkType);
+    } else {
+      return allocateHuge(size, useSlabs, chunkType);
+    }
+  }
+
+  private Chunk allocateFromFragments(int chunkSize, ChunkType chunkType) {
+    do {
+      final int lastAllocationId = this.lastFragmentAllocation.get();
+      for (int i=lastAllocationId; i < this.fragmentList.size(); i++) {
+        Chunk result = allocateFromFragment(i, chunkSize, chunkType);
+        if (result != null) {
+          return result;
+        }
+      }
+      for (int i=0; i < lastAllocationId; i++) {
+        Chunk result = allocateFromFragment(i, chunkSize, chunkType);
+        if (result != null) {
+          return result;
+        }
+      }
+    } while (compact(chunkSize));
+    // We tried all the fragments and didn't find any free memory.
+    logOffHeapState(chunkSize);
+    final OutOfOffHeapMemoryException failure = new OutOfOffHeapMemoryException("Out of off-heap memory. Could not allocate size of " + chunkSize);
+    try {
+      throw failure;
+    } finally {
+      this.ma.ooohml.outOfOffHeapMemory(failure);
+    }
+  }
+
+  private void logOffHeapState(int chunkSize) {
+    if (InternalDistributedSystem.getAnyInstance() != null) {
+      LogWriter lw = InternalDistributedSystem.getAnyInstance().getLogWriter();
+      lw.info("OutOfOffHeapMemory allocating size of " + chunkSize + ". allocated=" + this.allocatedSize.get() + " compactions=" + this.compactCount.get() + " objects=" + this.ma.stats.getObjects() + " free=" + this.ma.stats.getFreeMemory() + " fragments=" + this.ma.stats.getFragments() + " largestFragment=" + this.ma.stats.getLargestFragment() + " fragmentation=" + this.ma.stats.getFragmentation());
+      logFragmentState(lw);
+      logTinyState(lw);
+      logHugeState(lw);
+    }
+  }
+
+  private void logHugeState(LogWriter lw) {
+    for (Chunk c: this.hugeChunkSet) {
+      lw.info("Free huge of size " + c.getSize());
+    }
+  }
+  private void logTinyState(LogWriter lw) {
+    for (int i=0; i < this.tinyFreeLists.length(); i++) {
+      SyncChunkStack cl = this.tinyFreeLists.get(i);
+      if (cl != null) {
+        cl.logSizes(lw, "Free tiny of size ");
+      }
+    }
+  }
+  private void logFragmentState(LogWriter lw) {
+    for (Fragment f: this.fragmentList) {
+      int freeSpace = f.freeSpace();
+      if (freeSpace > 0) {
+        lw.info("Fragment at " + f.getMemoryAddress() + " of size " + f.getSize() + " has " + freeSpace + " bytes free.");
+      }
+    }
+  }
+
+  private final AtomicInteger compactCount = new AtomicInteger();
+  /**
+   * Compacts memory and returns true if enough memory to allocate chunkSize
+   * is freed. Otherwise returns false;
+   * TODO OFFHEAP: what should be done about contiguous chunks that end up being bigger than 2G?
+   * Currently if we are given slabs bigger than 2G or that just happen to be contiguous and add
+   * up to 2G then the compactor may unify them together into a single Chunk and our 32-bit chunkSize
+   * field will overflow. This code needs to detect this and just create a chunk of 2G and then start
+   * a new one.
+   * Or to prevent it from happening we could just check the incoming slabs and throw away a few bytes
+   * to keep them from being contiguous.
+   */
+  private boolean compact(int chunkSize) {
+    final long startCompactionTime = this.ma.getStats().startCompaction();
+    final int countPreSync = this.compactCount.get();
+    try {
+      synchronized (this) {
+        if (this.compactCount.get() != countPreSync) {
+          // someone else did a compaction while we waited on the sync.
+          // So just return true causing the caller to retry the allocation.
+          return true;
+        }
+        ArrayList<SyncChunkStack> freeChunks = new ArrayList<SyncChunkStack>();
+        collectFreeChunks(freeChunks);
+        final int SORT_ARRAY_BLOCK_SIZE = 128;
+        long[] sorted = new long[SORT_ARRAY_BLOCK_SIZE];
+        int sortedSize = 0;
+        boolean result = false;
+        int largestFragment = 0;
+        for (SyncChunkStack l: freeChunks) {
+          long addr = l.poll();
+          while (addr != 0) {
+            int idx = Arrays.binarySearch(sorted, 0, sortedSize, addr);
+            //System.out.println("DEBUG addr=" + addr + " size=" + Chunk.getSize(addr) + " idx="+idx + " sortedSize=" + sortedSize);
+            if (idx >= 0) {
+              throw new IllegalStateException("duplicate memory address found during compaction!");
+            }
+            idx = -idx;
+            idx--;
+            if (idx == sortedSize) {
+              // addr is > everything in the array
+              if (sortedSize == 0) {
+                // nothing was in the array
+                sorted[0] = addr;
+                sortedSize++;
+              } else {
+                // see if we can conflate into sorted[idx]
+                long lowAddr = sorted[idx-1];
+                int lowSize = Chunk.getSize(lowAddr);
+                if (lowAddr + lowSize == addr) {
+                  // append the addr chunk to lowAddr
+                  Chunk.setSize(lowAddr, lowSize + Chunk.getSize(addr));
+                } else {
+                  if (sortedSize >= sorted.length) {
+                    long[] newSorted = new long[sorted.length+SORT_ARRAY_BLOCK_SIZE];
+                    System.arraycopy(sorted, 0, newSorted, 0, sorted.length);
+                    sorted = newSorted;
+                  }
+                  sortedSize++;
+                  sorted[idx] = addr;
+                }
+              }
+            } else {
+              int addrSize = Chunk.getSize(addr);
+              long highAddr = sorted[idx];
+              if (addr + addrSize == highAddr) {
+                // append highAddr chunk to addr
+                Chunk.setSize(addr, addrSize + Chunk.getSize(highAddr));
+                sorted[idx] = addr;
+              } else {
+                boolean insert = idx==0;
+                if (!insert) {
+                  long lowAddr = sorted[idx-1];
+                  //                  if (lowAddr == 0L) {
+                  //                    long[] tmp = Arrays.copyOf(sorted, sortedSize);
+                  //                    throw new IllegalStateException("addr was zero at idx=" + (idx-1) + " sorted="+ Arrays.toString(tmp));
+                  //                  }
+                  int lowSize = Chunk.getSize(lowAddr);
+                  if (lowAddr + lowSize == addr) {
+                    // append the addr chunk to lowAddr
+                    Chunk.setSize(lowAddr, lowSize + addrSize);
+                  } else {
+                    insert = true;
+                  }
+                }
+                if (insert) {
+                  if (sortedSize >= sorted.length) {
+                    long[] newSorted = new long[sorted.length+SORT_ARRAY_BLOCK_SIZE];
+                    System.arraycopy(sorted, 0, newSorted, 0, idx);
+                    newSorted[idx] = addr;
+                    System.arraycopy(sorted, idx, newSorted, idx+1, sortedSize-idx);
+                    sorted = newSorted;
+                  } else {
+                    System.arraycopy(sorted, idx, sorted, idx+1, sortedSize-idx);
+                    sorted[idx] = addr;
+                  }
+                  sortedSize++;
+                }
+              }
+            }
+            addr = l.poll();
+          }
+        }
+        for (int i=sortedSize-1; i > 0; i--) {
+          long addr = sorted[i];
+          long lowAddr = sorted[i-1];
+          int lowSize = Chunk.getSize(lowAddr);
+          if (lowAddr + lowSize == addr) {
+            // append addr chunk to lowAddr
+            Chunk.setSize(lowAddr, lowSize + Chunk.getSize(addr));
+            sorted[i] = 0L;
+          }
+        }
+        this.lastFragmentAllocation.set(0);
+        ArrayList<Fragment> tmp = new ArrayList<Fragment>();
+        for (int i=sortedSize-1; i >= 0; i--) {
+          long addr = sorted[i];
+          if (addr == 0L) continue;
+          int addrSize = Chunk.getSize(addr);
+          Fragment f = new Fragment(addr, addrSize);
+          if (addrSize >= chunkSize) {
+            result = true;
+          }
+          if (addrSize > largestFragment) {
+            largestFragment = addrSize;
+            // TODO it might be better to sort them biggest first
+            tmp.add(0, f);
+          } else {
+            tmp.add(f);
+          }
+        }
+        this.fragmentList.addAll(tmp);
+
+        // Reinitialize fragments with fill pattern data
+        if(this.ma.validateMemoryWithFill) {
+          fillFragments();
+        }
+
+        // Signal any waiters that a compaction happened.
+        this.compactCount.incrementAndGet();
+
+        this.ma.getStats().setLargestFragment(largestFragment);
+        this.ma.getStats().setFragments(tmp.size());        
+        updateFragmentation();
+
+        return result;
+      } // sync
+    } finally {
+      this.ma.getStats().endCompaction(startCompactionTime);
+    }
+  }
+
+  private void updateFragmentation() {      
+    long freeSize = this.ma.getStats().getFreeMemory();
+
+    // Calculate free space fragmentation only if there is free space available.
+    if(freeSize > 0) {
+      long largestFragment = this.ma.getStats().getLargestFragment();
+      long numerator = freeSize - largestFragment;
+
+      double percentage = (double) numerator / (double) freeSize;
+      percentage *= 100d;
+
+      int wholePercentage = (int) Math.rint(percentage);
+      this.ma.getStats().setFragmentation(wholePercentage);
+    } else {
+      // No free space? Then we have no free space fragmentation.
+      this.ma.getStats().setFragmentation(0);
+    }
+  }
+
+  private void collectFreeChunks(List<SyncChunkStack> l) {
+    collectFreeFragmentChunks(l);
+    collectFreeHugeChunks(l);
+    collectFreeTinyChunks(l);
+  }
+  private void collectFreeFragmentChunks(List<SyncChunkStack> l) {
+    if (this.fragmentList.size() == 0) return;
+    SyncChunkStack result = new SyncChunkStack();
+    for (Fragment f: this.fragmentList) {
+      int offset;
+      int diff;
+      do {
+        offset = f.getFreeIndex();
+        diff = f.getSize() - offset;
+      } while (diff >= Chunk.MIN_CHUNK_SIZE && !f.allocate(offset, offset+diff));
+      if (diff < Chunk.MIN_CHUNK_SIZE) {
+        if (diff > 0) {
+          SimpleMemoryAllocatorImpl.logger.debug("Lost memory of size {}", diff);
+        }
+        // fragment is too small to turn into a chunk
+        // TODO we need to make sure this never happens
+        // by keeping sizes rounded. I think I did this
+        // by introducing MIN_CHUNK_SIZE and by rounding
+        // the size of huge allocations.
+        continue;
+      }
+      long chunkAddr = f.getMemoryAddress()+offset;
+      Chunk.setSize(chunkAddr, diff);
+      result.offer(chunkAddr);
+    }
+    // All the fragments have been turned in to chunks so now clear them
+    // The compaction will create new fragments.
+    this.fragmentList.clear();
+    if (!result.isEmpty()) {
+      l.add(result);
+    }
+  }
+  private void collectFreeTinyChunks(List<SyncChunkStack> l) {
+    for (int i=0; i < this.tinyFreeLists.length(); i++) {
+      SyncChunkStack cl = this.tinyFreeLists.get(i);
+      if (cl != null) {
+        long head = cl.clear();
+        if (head != 0L) {
+          l.add(new SyncChunkStack(head));
+        }
+      }
+    }
+  }
+  private void collectFreeHugeChunks(List<SyncChunkStack> l) {
+    Chunk c = this.hugeChunkSet.pollFirst();
+    SyncChunkStack result = null;
+    while (c != null) {
+      if (result == null) {
+        result = new SyncChunkStack();
+        l.add(result);
+      }
+      result.offer(c.getMemoryAddress());
+      c = this.hugeChunkSet.pollFirst();
+    }
+  }
+
+  private Chunk allocateFromFragment(final int fragIdx, final int chunkSize, ChunkType chunkType) {
+    if (fragIdx >= this.fragmentList.size()) return null;
+    final Fragment fragment;
+    try {
+      fragment = this.fragmentList.get(fragIdx);
+    } catch (IndexOutOfBoundsException ignore) {
+      // A concurrent compaction can cause this.
+      return null;
+    }
+    boolean retryFragment;
+    do {
+      retryFragment = false;
+      int oldOffset = fragment.getFreeIndex();
+      int fragmentSize = fragment.getSize();
+      int fragmentFreeSize = fragmentSize - oldOffset;
+      if (fragmentFreeSize >= chunkSize) {
+        // this fragment has room
+        // Try to allocate up to BATCH_SIZE more chunks from it
+        int allocSize = chunkSize * SimpleMemoryAllocatorImpl.BATCH_SIZE;
+        if (allocSize > fragmentFreeSize) {
+          allocSize = (fragmentFreeSize / chunkSize) * chunkSize;
+        }
+        int newOffset = oldOffset + allocSize;
+        int extraSize = fragmentSize - newOffset;
+        if (extraSize < Chunk.MIN_CHUNK_SIZE) {
+          // include these last few bytes of the fragment in the allocation.
+          // If we don't then they will be lost forever.
+          // The extraSize bytes only apply to the first chunk we allocate (not the batch ones).
+          newOffset += extraSize;
+        } else {
+          extraSize = 0;
+        }
+        if (fragment.allocate(oldOffset, newOffset)) {
+          // We did the allocate!
+          this.lastFragmentAllocation.set(fragIdx);
+          Chunk result = this.ma.chunkFactory.newChunk(fragment.getMemoryAddress()+oldOffset, chunkSize+extraSize, chunkType);
+          allocSize -= chunkSize+extraSize;
+          oldOffset += extraSize;
+          while (allocSize > 0) {
+            oldOffset += chunkSize;
+            // we add the batch ones immediately to the freelist
+            result.readyForFree();
+            free(result.getMemoryAddress(), false);
+            result = this.ma.chunkFactory.newChunk(fragment.getMemoryAddress()+oldOffset, chunkSize, chunkType);
+            allocSize -= chunkSize;
+          }
+
+          if(this.ma.validateMemoryWithFill) {
+            result.validateFill();
+          }
+
+          return result;
+        } else {
+          // TODO OFFHEAP: if batch allocations are disabled should we not call basicAllocate here?
+          // Since we know another thread did a concurrent alloc
+          // that possibly did a batch check the free list again.
+          Chunk result = basicAllocate(chunkSize, false, chunkType);
+          if (result != null) {
+            return result;
+          }
+          retryFragment = true;
+        }
+      }
+    } while (retryFragment);
+    return null; // did not find enough free space in this fragment
+  }
+
+  private int round(int multiple, int value) {
+    return (int) ((((long)value + (multiple-1)) / multiple) * multiple);
+  }
+  private Chunk allocateTiny(int size, boolean useFragments, ChunkType chunkType) {
+    return basicAllocate(getNearestTinyMultiple(size), SimpleMemoryAllocatorImpl.TINY_MULTIPLE, 0, this.tinyFreeLists, useFragments, chunkType);
+  }
+  private Chunk basicAllocate(int idx, int multiple, int offset, AtomicReferenceArray<SyncChunkStack> freeLists, boolean useFragments, ChunkType chunkType) {
+    SyncChunkStack clq = freeLists.get(idx);
+    if (clq != null) {
+      long memAddr = clq.poll();
+      if (memAddr != 0) {
+        Chunk result = this.ma.chunkFactory.newChunk(memAddr, chunkType);
+
+        // Data integrity check.
+        if(this.ma.validateMemoryWithFill) {          
+          result.validateFill();
+        }
+
+        result.readyForAllocation(chunkType);
+        return result;
+      }
+    }
+    if (useFragments) {
+      return allocateFromFragments(((idx+1)*multiple)+offset, chunkType);
+    } else {
+      return null;
+    }
+  }
+  private Chunk allocateHuge(int size, boolean useFragments, ChunkType chunkType) {
+    // sizeHolder is a fake Chunk used to search our sorted hugeChunkSet.
+    Chunk sizeHolder = new FakeChunk(size);
+    NavigableSet<Chunk> ts = this.hugeChunkSet.tailSet(sizeHolder);
+    Chunk result = ts.pollFirst();
+    if (result != null) {
+      if (result.getSize() - (SimpleMemoryAllocatorImpl.HUGE_MULTIPLE - Chunk.OFF_HEAP_HEADER_SIZE) < size) {
+        // close enough to the requested size; just return it.
+
+        // Data integrity check.
+        if(this.ma.validateMemoryWithFill) {          
+          result.validateFill();
+        }
+        if (chunkType.getSrcType() != Chunk.getSrcType(result.getMemoryAddress())) {
+          // The java wrapper class that was cached in the huge chunk list is the wrong type.
+          // So allocate a new one and garbage collect the old one.
+          result = this.ma.chunkFactory.newChunk(result.getMemoryAddress(), chunkType);
+        }
+        result.readyForAllocation(chunkType);
+        return result;
+      } else {
+        this.hugeChunkSet.add(result);
+      }
+    }
+    if (useFragments) {
+      // We round it up to the next multiple of TINY_MULTIPLE to make
+      // sure we always have chunks allocated on an 8 byte boundary.
+      return allocateFromFragments(round(SimpleMemoryAllocatorImpl.TINY_MULTIPLE, size), chunkType);
+    } else {
+      return null;
+    }
+  }
+  
+  /**
+   * Used by the FreeListManager to easily search its
+   * ConcurrentSkipListSet. This is not a real chunk
+   * but only used for searching.
+   */
+  private static class FakeChunk extends Chunk {
+    private final int size;
+    public FakeChunk(int size) {
+      super();
+      this.size = size;
+    }
+    @Override
+    public int getSize() {
+      return this.size;
+    }
+  }
+
+  @SuppressWarnings("synthetic-access")
+  public void free(long addr) {
+    free(addr, true);
+  }
+
+  private void free(long addr, boolean updateStats) {
+    int cSize = Chunk.getSize(addr);
+    if (updateStats) {
+      this.ma.stats.incObjects(-1);
+      this.allocatedSize.addAndGet(-cSize);
+      this.ma.stats.incUsedMemory(-cSize);
+      this.ma.stats.incFreeMemory(cSize);
+      this.ma.notifyListeners();
+    }
+    if (cSize <= SimpleMemoryAllocatorImpl.MAX_TINY) {
+      freeTiny(addr, cSize);
+    } else {
+      freeHuge(addr, cSize);
+    }
+  }
+  private void freeTiny(long addr, int cSize) {
+    basicFree(addr, getNearestTinyMultiple(cSize), this.tinyFreeLists);
+  }
+  private void basicFree(long addr, int idx, AtomicReferenceArray<SyncChunkStack> freeLists) {
+    SyncChunkStack clq = freeLists.get(idx);
+    if (clq != null) {
+      clq.offer(addr);
+    } else {
+      clq = new SyncChunkStack();
+      clq.offer(addr);
+      if (!freeLists.compareAndSet(idx, null, clq)) {
+        clq = freeLists.get(idx);
+        clq.offer(addr);
+      }
+    }
+
+  }
+  private void freeHuge(long addr, int cSize) {
+    this.hugeChunkSet.add(this.ma.chunkFactory.newChunk(addr)); // TODO make this a collection of longs
+  }
+
+  List<MemoryBlock> getOrderedBlocks() {
+    final List<MemoryBlock> value = new ArrayList<MemoryBlock>();
+    addBlocksFromFragments(this.fragmentList, value); // unused fragments
+    addBlocksFromChunks(getLiveChunks(), value); // used chunks
+    addBlocksFromChunks(this.hugeChunkSet, value);    // huge free chunks
+    addMemoryBlocks(getTinyFreeBlocks(), value);           // tiny free chunks
+    Collections.sort(value, 
+        new Comparator<MemoryBlock>() {
+          @Override
+          public int compare(MemoryBlock o1, MemoryBlock o2) {
+            return Long.valueOf(o1.getMemoryAddress()).compareTo(o2.getMemoryAddress());
+          }
+    });
+    return value;
+  }
+  private void addBlocksFromFragments(Collection<Fragment> src, List<MemoryBlock> dest) {
+    for (MemoryBlock block : src) {
+      dest.add(new MemoryBlockNode(this.ma, block));
+    }
+  }
+  
+  private void addBlocksFromChunks(Collection<Chunk> src, List<MemoryBlock> dest) {
+    for (Chunk chunk : src) {
+      dest.add(new MemoryBlockNode(this.ma, chunk));
+    }
+  }
+  
+  private void addMemoryBlocks(Collection<MemoryBlock> src, List<MemoryBlock> dest) {
+    for (MemoryBlock block : src) {
+      dest.add(new MemoryBlockNode(this.ma, block));
+    }
+  }
+  
+  private List<MemoryBlock> getTinyFreeBlocks() {
+    final List<MemoryBlock> value = new ArrayList<MemoryBlock>();
+    final SimpleMemoryAllocatorImpl sma = this.ma;
+    for (int i = 0; i < this.tinyFreeLists.length(); i++) {
+      if (this.tinyFreeLists.get(i) == null) continue;
+      long addr = this.tinyFreeLists.get(i).getTopAddress();
+      while (addr != 0L) {
+        value.add(new MemoryBlockNode(sma, new TinyMemoryBlock(addr, i)));
+        addr = Chunk.getNext(addr);
+      }
+    }
+    return value;
+  }
+  List<MemoryBlock> getAllocatedBlocks() {
+    final List<MemoryBlock> value = new ArrayList<MemoryBlock>();
+    addBlocksFromChunks(getLiveChunks(), value); // used chunks
+    Collections.sort(value, 
+        new Comparator<MemoryBlock>() {
+          @Override
+          public int compare(MemoryBlock o1, MemoryBlock o2) {
+            return Long.valueOf(o1.getMemoryAddress()).compareTo(o2.getMemoryAddress());
+          }
+    });
+    return value;
+  }
+  /**
+   * Used to represent an address from a tiny free list as a MemoryBlock
+   */
+  private static final class TinyMemoryBlock implements MemoryBlock {
+    private final long address;
+    private final int freeListId;
+
+    private TinyMemoryBlock(long address, int freeListId) {
+      this.address = address;
+      this.freeListId = freeListId;
+    }
+
+    @Override
+    public State getState() {
+      return State.DEALLOCATED;
+    }
+
+    @Override
+    public long getMemoryAddress() {
+      return address;
+    }
+
+    @Override
+    public int getBlockSize() {
+      return Chunk.getSize(address);
+    }
+
+    @Override
+    public MemoryBlock getNextBlock() {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public int getSlabId() {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public int getFreeListId() {
+      return freeListId;
+    }
+
+    @Override
+    public int getRefCount() {
+      return 0;
+    }
+
+    @Override
+    public String getDataType() {
+      return "N/A";
+    }
+
+    @Override
+    public boolean isSerialized() {
+      return false;
+    }
+
+    @Override
+    public boolean isCompressed() {
+      return false;
+    }
+
+    @Override
+    public Object getDataValue() {
+      return null;
+    }
+
+    @Override
+    public ChunkType getChunkType() {
+      return null;
+    }
+  }
+}
\ 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/offheap/GemFireChunk.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/GemFireChunk.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/GemFireChunk.java
new file mode 100644
index 0000000..3167613
--- /dev/null
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/GemFireChunk.java
@@ -0,0 +1,55 @@
+/*
+ * 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 chunk that stores a GemFire object.
+ * Currently the object stored in this chunk
+ * is always an entry value of a Region.
+ */
+public class GemFireChunk extends Chunk {
+  public static final ChunkType TYPE = new ChunkType() {
+    @Override
+    public int getSrcType() {
+      return Chunk.SRC_TYPE_GFE;
+    }
+    @Override
+    public Chunk newChunk(long memoryAddress) {
+      return new GemFireChunk(memoryAddress);
+    }
+    @Override
+    public Chunk newChunk(long memoryAddress, int chunkSize) {
+      return new GemFireChunk(memoryAddress, chunkSize);
+    }
+  };
+  public GemFireChunk(long memoryAddress, int chunkSize) {
+    super(memoryAddress, chunkSize, TYPE);
+  }
+
+  public GemFireChunk(long memoryAddress) {
+    super(memoryAddress);
+    // chunkType may be set by caller when it calls readyForAllocation
+  }
+  public GemFireChunk(GemFireChunk chunk) {
+    super(chunk);
+  }
+  @Override
+  public Chunk slice(int position, int limit) {
+    return new GemFireChunkSlice(this, position, limit);
+  }
+}
\ 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/offheap/GemFireChunkFactory.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/GemFireChunkFactory.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/GemFireChunkFactory.java
new file mode 100644
index 0000000..c3f3bcc
--- /dev/null
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/GemFireChunkFactory.java
@@ -0,0 +1,52 @@
+/*
+ * 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 ChunkFactory that produces chunks of type GemFireChunk.
+ */
+public class GemFireChunkFactory implements ChunkFactory {
+  @Override
+  public Chunk newChunk(long address, int chunkSize, ChunkType chunkType) {
+    assert chunkType.equals(GemFireChunk.TYPE);
+    return new GemFireChunk(address,chunkSize);
+  }
+
+  @Override
+  public Chunk newChunk(long address) {
+    return new GemFireChunk(address);
+  }
+
+  @Override
+  public Chunk newChunk(long address, ChunkType chunkType) {
+    assert chunkType.equals(GemFireChunk.TYPE);
+    return new GemFireChunk(address);
+  }
+
+  @Override
+  public ChunkType getChunkTypeForAddress(long address) {
+    assert Chunk.getSrcType(address) == Chunk.SRC_TYPE_GFE;
+    return GemFireChunk.TYPE;
+  }
+
+  @Override
+  public ChunkType getChunkTypeForRawBits(int bits) {
+    assert Chunk.getSrcTypeFromRawBits(bits) == Chunk.SRC_TYPE_GFE;
+    return GemFireChunk.TYPE;
+  }
+}
\ 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/offheap/GemFireChunkSlice.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/GemFireChunkSlice.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/GemFireChunkSlice.java
new file mode 100644
index 0000000..0c27aa3
--- /dev/null
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/GemFireChunkSlice.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.gemstone.gemfire.internal.offheap;
+
+/**
+ * Represents a slice of a GemFireChunk.
+ * A slice is a subsequence of the bytes stored in a GemFireChunk.
+ */
+public class GemFireChunkSlice extends GemFireChunk {
+  private final int offset;
+  private final int capacity;
+  public GemFireChunkSlice(GemFireChunk gemFireChunk, int position, int limit) {
+    super(gemFireChunk);
+    this.offset = gemFireChunk.getBaseDataOffset() + position;
+    this.capacity = limit - position;
+  }
+  @Override
+  public int getDataSize() {
+    return this.capacity;
+  }
+  
+  @Override
+  protected long getBaseDataAddress() {
+    return super.getBaseDataAddress() + this.offset;
+  }
+  @Override
+  protected int getBaseDataOffset() {
+    return this.offset;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4b0925e3/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/LifecycleListener.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/LifecycleListener.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/LifecycleListener.java
new file mode 100644
index 0000000..613b12a
--- /dev/null
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/LifecycleListener.java
@@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.gemstone.gemfire.internal.offheap;
+
+import java.util.Iterator;
+import java.util.List;
+import java.util.concurrent.CopyOnWriteArrayList;
+
+/**
+ * Used by tests to get notifications about the lifecycle of a 
+ * SimpleMemoryAllocatorImpl.
+ * 
+ * @author Kirk Lund
+ */
+public interface LifecycleListener {
+
+  /**
+   * Callback is invoked after creating a new SimpleMemoryAllocatorImpl. 
+   * 
+   * Create occurs during the first initialization of an 
+   * InternalDistributedSystem within the JVM.
+   * 
+   * @param allocator the instance that has just been created
+   */
+  public void afterCreate(SimpleMemoryAllocatorImpl allocator);
+  /**
+   * Callback is invoked after reopening an existing SimpleMemoryAllocatorImpl 
+   * for reuse. 
+   * 
+   * Reuse occurs during any intialization of an 
+   * InternalDistributedSystem after the first one was connected and then
+   * disconnected within the JVM.
+   * 
+   * @param allocator the instance that has just been reopened for reuse
+   */
+  public void afterReuse(SimpleMemoryAllocatorImpl allocator);
+  /**
+   * Callback is invoked before closing the SimpleMemoryAllocatorImpl
+   * 
+   * Close occurs after the InternalDistributedSystem and DistributionManager 
+   * have completely disconnected. 
+   * 
+   * @param allocator the instance that is about to be closed
+   */
+  public void beforeClose(SimpleMemoryAllocatorImpl allocator);
+  
+  static  void invokeBeforeClose(SimpleMemoryAllocatorImpl allocator) {
+    for (Iterator<LifecycleListener> iter = lifecycleListeners.iterator(); iter.hasNext();) {
+      LifecycleListener listener = iter.next();
+      listener.beforeClose(allocator);
+    }
+  }
+  static void invokeAfterReuse(SimpleMemoryAllocatorImpl allocator) {
+    for (Iterator<LifecycleListener> iter = lifecycleListeners.iterator(); iter.hasNext();) {
+      LifecycleListener listener = iter.next();
+      listener.afterReuse(allocator);
+    }
+  }
+  static void invokeAfterCreate(SimpleMemoryAllocatorImpl allocator) {
+    for (Iterator<LifecycleListener> iter = lifecycleListeners.iterator(); iter.hasNext();) {
+      LifecycleListener listener = iter.next();
+      listener.afterCreate(allocator);
+    }
+  }
+  /**
+   * Removes a LifecycleListener. Does nothing if the instance has not been added.
+   * @param listener the instance to remove
+   */
+  public static void removeLifecycleListener(LifecycleListener listener) {
+    lifecycleListeners.remove(listener);
+  }
+  /**
+   * Adds a LifecycleListener.
+   * @param listener the instance to add
+   */
+  public static void addLifecycleListener(LifecycleListener listener) {
+    LifecycleListener.lifecycleListeners.add(listener);
+  }
+
+  /**
+   * Following should be private but java 8 does not support that.
+   */
+  static final List<LifecycleListener> lifecycleListeners = new CopyOnWriteArrayList<LifecycleListener>();
+}
\ 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/offheap/MemoryAllocator.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/MemoryAllocator.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/MemoryAllocator.java
index bf38089..231ff3a 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/MemoryAllocator.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/MemoryAllocator.java
@@ -17,7 +17,6 @@
 package com.gemstone.gemfire.internal.offheap;
 
 import com.gemstone.gemfire.compression.Compressor;
-import com.gemstone.gemfire.internal.offheap.SimpleMemoryAllocatorImpl.ChunkType;
 
 /**
  * Basic contract for a heap that manages off heap memory. Any MemoryChunks allocated from a heap

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4b0925e3/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/MemoryBlock.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/MemoryBlock.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/MemoryBlock.java
index 05cc796..3ad9283 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/MemoryBlock.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/MemoryBlock.java
@@ -16,7 +16,6 @@
  */
 package com.gemstone.gemfire.internal.offheap;
 
-import com.gemstone.gemfire.internal.offheap.SimpleMemoryAllocatorImpl.ChunkType;
 
 /**
  * Basic size and usage information about an off-heap memory block under

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4b0925e3/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/MemoryBlockNode.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/MemoryBlockNode.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/MemoryBlockNode.java
new file mode 100644
index 0000000..3f5f4dc
--- /dev/null
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/MemoryBlockNode.java
@@ -0,0 +1,158 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.gemstone.gemfire.internal.offheap;
+
+import java.io.IOException;
+import java.util.Arrays;
+
+import com.gemstone.gemfire.DataSerializer;
+import com.gemstone.gemfire.cache.CacheClosedException;
+import com.gemstone.gemfire.internal.offheap.MemoryBlock.State;
+
+/**
+ * Basic implementation of MemoryBlock for test validation only.
+ */
+public class MemoryBlockNode implements MemoryBlock {
+  private final SimpleMemoryAllocatorImpl ma;
+  private final MemoryBlock block;
+  MemoryBlockNode(SimpleMemoryAllocatorImpl ma, MemoryBlock block) {
+    this.ma = ma;
+    this.block = block;
+  }
+  @Override
+  public State getState() {
+    return this.block.getState();
+  }
+  @Override
+  public long getMemoryAddress() {
+    return this.block.getMemoryAddress();
+  }
+  @Override
+  public int getBlockSize() {
+    return this.block.getBlockSize();
+  }
+  @Override
+  public MemoryBlock getNextBlock() {
+    return this.ma.getBlockAfter(this);
+  }
+  public int getSlabId() {
+    return this.ma.findSlab(getMemoryAddress());
+  }
+  @Override
+  public int getFreeListId() {
+    return this.block.getFreeListId();
+  }
+  public int getRefCount() {
+    return this.block.getRefCount(); // delegate to fix GEODE-489
+  }
+  public String getDataType() {
+    if (this.block.getDataType() != null) {
+      return this.block.getDataType();
+    }
+    if (!isSerialized()) {
+      // byte array
+      if (isCompressed()) {
+        return "compressed byte[" + ((Chunk)this.block).getDataSize() + "]";
+      } else {
+        return "byte[" + ((Chunk)this.block).getDataSize() + "]";
+      }
+    } else if (isCompressed()) {
+      return "compressed object of size " + ((Chunk)this.block).getDataSize();
+    }
+    //Object obj = EntryEventImpl.deserialize(((Chunk)this.block).getRawBytes());
+    byte[] bytes = ((Chunk)this.block).getRawBytes();
+    return DataType.getDataType(bytes);
+  }
+  public boolean isSerialized() {
+    return this.block.isSerialized();
+  }
+  public boolean isCompressed() {
+    return this.block.isCompressed();
+  }
+  @Override
+  public Object getDataValue() {
+    String dataType = getDataType();
+    if (dataType == null || dataType.equals("N/A")) {
+      return null;
+    } else if (isCompressed()) {
+      return ((Chunk)this.block).getCompressedBytes();
+    } else if (!isSerialized()) {
+      // byte array
+      //return "byte[" + ((Chunk)this.block).getDataSize() + "]";
+      return ((Chunk)this.block).getRawBytes();
+    } else {
+      try {
+        byte[] bytes = ((Chunk)this.block).getRawBytes();
+        return DataSerializer.readObject(DataType.getDataInput(bytes));
+      } catch (IOException e) {
+        e.printStackTrace();
+        return "IOException:" + e.getMessage();
+      } catch (ClassNotFoundException e) {
+        e.printStackTrace();
+        return "ClassNotFoundException:" + e.getMessage();
+      } catch (CacheClosedException e) {
+        e.printStackTrace();
+        return "CacheClosedException:" + e.getMessage();
+      }
+    }
+  }
+  @Override
+  public String toString() {
+    final StringBuilder sb = new StringBuilder(MemoryBlock.class.getSimpleName());
+    sb.append("{");
+    sb.append("MemoryAddress=").append(getMemoryAddress());
+    sb.append(", State=").append(getState());
+    sb.append(", BlockSize=").append(getBlockSize());
+    sb.append(", SlabId=").append(getSlabId());
+    sb.append(", FreeListId=");
+    if (getState() == State.UNUSED || getState() == State.ALLOCATED) {
+      sb.append("NONE");
+    } else if (getFreeListId() == -1) {
+      sb.append("HUGE");
+    } else {
+      sb.append(getFreeListId());
+    }
+    sb.append(", RefCount=").append(getRefCount());
+    ChunkType ct = this.getChunkType();
+    if (ct != null) {
+      sb.append(", " + ct);
+    }
+    sb.append(", isSerialized=").append(isSerialized());
+    sb.append(", isCompressed=").append(isCompressed());
+    sb.append(", DataType=").append(getDataType());
+    {
+      sb.append(", DataValue=");
+      Object dataValue = getDataValue();
+      if (dataValue instanceof byte[]) {
+        byte[] ba = (byte[]) dataValue;
+        if (ba.length < 1024) {
+          sb.append(Arrays.toString(ba));
+        } else {
+          sb.append("<byte array of length " + ba.length + ">");
+        }
+      } else {
+        sb.append(dataValue);
+      }
+    }
+    sb.append("}");
+    return sb.toString();
+  }
+  @Override
+  public ChunkType getChunkType() {
+    return this.block.getChunkType();
+  }
+}
\ 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/offheap/OffHeapCachedDeserializable.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapCachedDeserializable.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapCachedDeserializable.java
index 1bc2cdf..143fb25 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapCachedDeserializable.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapCachedDeserializable.java
@@ -30,7 +30,6 @@ import com.gemstone.gemfire.internal.cache.CachedDeserializableFactory;
 import com.gemstone.gemfire.internal.cache.EntryBits;
 import com.gemstone.gemfire.internal.cache.RegionEntry;
 import com.gemstone.gemfire.internal.lang.StringUtils;
-import com.gemstone.gemfire.internal.offheap.SimpleMemoryAllocatorImpl.Chunk;
 import com.gemstone.gemfire.internal.offheap.annotations.Unretained;
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4b0925e3/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapHelper.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapHelper.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapHelper.java
index b5677cd..4845931 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapHelper.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapHelper.java
@@ -106,9 +106,9 @@ public class OffHeapHelper {
    */
   public static boolean releaseWithNoTracking(@Released Object o) {
     if (o instanceof MemoryChunkWithRefCount) {
-      SimpleMemoryAllocatorImpl.skipRefCountTracking();
+      ReferenceCountHelper.skipRefCountTracking();
       ((MemoryChunkWithRefCount) o).release();
-      SimpleMemoryAllocatorImpl.unskipRefCountTracking();
+      ReferenceCountHelper.unskipRefCountTracking();
       return true;
     } else {
       return false;
@@ -121,9 +121,9 @@ public class OffHeapHelper {
    */
   public static boolean releaseAndTrackOwner(@Released final Object o, final Object owner) {
     if (o instanceof MemoryChunkWithRefCount) {
-      SimpleMemoryAllocatorImpl.setReferenceCountOwner(owner);
+      ReferenceCountHelper.setReferenceCountOwner(owner);
       ((MemoryChunkWithRefCount) o).release();
-      SimpleMemoryAllocatorImpl.setReferenceCountOwner(null);
+      ReferenceCountHelper.setReferenceCountOwner(null);
       return true;
     } else {
       return false;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4b0925e3/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapRegionEntryHelper.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapRegionEntryHelper.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapRegionEntryHelper.java
index 4fcacbf..d1a81f0 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapRegionEntryHelper.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapRegionEntryHelper.java
@@ -24,8 +24,6 @@ import com.gemstone.gemfire.internal.cache.EntryEventImpl;
 import com.gemstone.gemfire.internal.cache.OffHeapRegionEntry;
 import com.gemstone.gemfire.internal.cache.RegionEntryContext;
 import com.gemstone.gemfire.internal.cache.Token;
-import com.gemstone.gemfire.internal.offheap.SimpleMemoryAllocatorImpl.Chunk;
-import com.gemstone.gemfire.internal.offheap.SimpleMemoryAllocatorImpl.DataAsAddress;
 import com.gemstone.gemfire.internal.offheap.annotations.Released;
 import com.gemstone.gemfire.internal.offheap.annotations.Retained;
 import com.gemstone.gemfire.internal.offheap.annotations.Unretained;
@@ -371,9 +369,9 @@ public class OffHeapRegionEntryHelper {
     do {
       oldAddress = re.getAddress();
     } while (!re.setAddress(oldAddress, newAddress));
-    SimpleMemoryAllocatorImpl.setReferenceCountOwner(re);
+    ReferenceCountHelper.setReferenceCountOwner(re);
     releaseAddress(oldAddress);
-    SimpleMemoryAllocatorImpl.setReferenceCountOwner(null);
+    ReferenceCountHelper.setReferenceCountOwner(null);
   }
  
   public static Token getValueAsToken(@Unretained OffHeapRegionEntry re) {
@@ -422,7 +420,7 @@ public class OffHeapRegionEntryHelper {
         long addr2 = re.getAddress();
         retryCount++;
         if (retryCount > 100) {
-          throw new IllegalStateException("retain failed addr=" + addr + " addr2=" + addr + " 100 times" + " history=" + SimpleMemoryAllocatorImpl.getFreeRefCountInfo(addr));
+          throw new IllegalStateException("retain failed addr=" + addr + " addr2=" + addr + " 100 times" + " history=" + ReferenceCountHelper.getFreeRefCountInfo(addr));
         }
         addr = addr2;
         // Since retain returned false our region entry should have a different

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4b0925e3/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/RefCountChangeInfo.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/RefCountChangeInfo.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/RefCountChangeInfo.java
new file mode 100644
index 0000000..56cab97
--- /dev/null
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/RefCountChangeInfo.java
@@ -0,0 +1,112 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.gemstone.gemfire.internal.offheap;
+
+import java.io.ByteArrayOutputStream;
+import java.io.PrintStream;
+
+import com.gemstone.gemfire.internal.shared.StringPrintWriter;
+
+@SuppressWarnings("serial")
+/**
+ * Used by SimpleMemoryAllocatorImpl to debug off-heap memory leaks.
+ */
+public class RefCountChangeInfo extends Throwable {
+  private final String threadName;
+  private final int rc;
+  private final Object owner;
+  private int dupCount;
+  
+  public RefCountChangeInfo(boolean decRefCount, int rc, Object owner) {
+    super(decRefCount ? "FREE" : "USED");
+    this.threadName = Thread.currentThread().getName();
+    this.rc = rc;
+    this.owner = owner;
+  }
+  
+  public Object getOwner() {
+    return this.owner;
+  }
+  
+  public int getDupCount() {
+    return this.dupCount;
+  }
+  public void decDupCount() {
+    this.dupCount--;
+  }
+
+  @Override
+  public String toString() {
+    ByteArrayOutputStream baos = new ByteArrayOutputStream(64*1024);
+    PrintStream ps = new PrintStream(baos);
+    ps.print(this.getMessage());
+    ps.print(" rc=");
+    ps.print(this.rc);
+    if (this.dupCount > 0) {
+      ps.print(" dupCount=");
+      ps.print(this.dupCount);
+    }
+    ps.print(" by ");
+    ps.print(this.threadName);
+    if (this.owner != null) {
+      ps.print(" owner=");
+      ps.print(this.owner.getClass().getName());
+      ps.print("@");
+      ps.print(System.identityHashCode(this.owner));
+    }
+    ps.println(": ");
+    StackTraceElement[] trace = getStackTrace();
+    // skip the initial elements from SimpleMemoryAllocatorImpl
+    int skip=0;
+    for (int i=0; i < trace.length; i++) {
+      if (!trace[i].getClassName().contains("SimpleMemoryAllocatorImpl")) {
+        skip = i;
+        break;
+      }
+    }
+    for (int i=skip; i < trace.length; i++) {
+      ps.println("\tat " + trace[i]);
+    }
+    ps.flush();
+    return baos.toString();
+  }
+  
+  public boolean isDuplicate(RefCountChangeInfo other) {
+    if (!getMessage().equals(other.getMessage())) return false;
+    String trace = getStackTraceString();
+    String traceOther = other.getStackTraceString();
+    if (trace.hashCode() != traceOther.hashCode()) return false;
+    if (trace.equals(traceOther)) {
+      this.dupCount++;
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  private String stackTraceString;
+  private String getStackTraceString() {
+    String result = this.stackTraceString;
+    if (result == null) {
+      StringPrintWriter spr = new StringPrintWriter();
+      printStackTrace(spr);
+      result = spr.getBuilder().toString();
+      this.stackTraceString = result;
+    }
+    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/offheap/ReferenceCountHelper.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/ReferenceCountHelper.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/ReferenceCountHelper.java
new file mode 100644
index 0000000..85a0505
--- /dev/null
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/ReferenceCountHelper.java
@@ -0,0 +1,235 @@
+package com.gemstone.gemfire.internal.offheap;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import com.gemstone.gemfire.internal.cache.RegionEntry;
+
+/**
+ * This class provides static methods to help
+ * debug off-heap reference count problems.
+ * To enable reference count tracking set: -Dgemfire.trackOffHeapRefCounts=true.
+ * To enable free operation tracking set: -Dgemfire.trackOffHeapFreedRefCounts=true.
+ */
+public class ReferenceCountHelper {
+  private ReferenceCountHelper() {
+    // no instances allowed
+  }
+  final static private boolean trackRefCounts = Boolean.getBoolean("gemfire.trackOffHeapRefCounts");
+  final static private boolean trackFreedRefCounts = Boolean.getBoolean("gemfire.trackOffHeapFreedRefCounts");
+  final static private ConcurrentMap<Long, List<RefCountChangeInfo>> stacktraces;
+  final static private ConcurrentMap<Long, List<RefCountChangeInfo>> freedStacktraces;
+  final static private ThreadLocal<Object> refCountOwner;
+  final static private ThreadLocal<AtomicInteger> refCountReenterCount;
+  final static private Object SKIP_REF_COUNT_TRACKING = new Object();
+  final static private List<RefCountChangeInfo> LOCKED = Collections.emptyList();
+
+  static {
+    if (trackRefCounts) {
+      stacktraces = new ConcurrentHashMap<Long, List<RefCountChangeInfo>>();
+      if (trackFreedRefCounts) {
+        freedStacktraces = new ConcurrentHashMap<Long, List<RefCountChangeInfo>>();
+      } else {
+        freedStacktraces = null;
+      }
+      refCountOwner = new ThreadLocal<Object>();
+      refCountReenterCount = new ThreadLocal<AtomicInteger>();
+    } else {
+      stacktraces = null;
+      freedStacktraces = null;
+      refCountOwner = null;
+      refCountReenterCount = null;
+    }
+  }
+  
+  /**
+   * Returns true if reference count tracking is enabled.
+   */
+  public static boolean trackReferenceCounts() {
+    return trackRefCounts;
+  }
+
+  /**
+   * Returns true if free operation tracking is enabled.
+   */
+  public static boolean trackFreedReferenceCounts() {
+    return trackFreedRefCounts;
+  }
+
+  /**
+   * Optional call to tell the tracker the logical "owner"
+   * of the reference count. For example you could set
+   * the particular EntryEventImpl instance that incremented
+   * the reference count and is responsible for decrementing it.
+   * Calling this method is a noop if !trackReferenceCounts.
+   */
+  public static void setReferenceCountOwner(Object owner) {
+    if (trackReferenceCounts()) {
+      if (refCountOwner.get() != null) {
+        AtomicInteger ai = refCountReenterCount.get();
+        if (owner != null) {
+          ai.incrementAndGet();
+        } else {
+          if (ai.decrementAndGet() <= 0) {
+            refCountOwner.set(null);
+            ai.set(0);
+          }
+        }
+      } else {
+        AtomicInteger ai = refCountReenterCount.get();
+        if (ai == null) {
+          ai = new AtomicInteger(0);
+          refCountReenterCount.set(ai);
+        }
+        if (owner != null) {
+          ai.set(1);
+        } else {
+          ai.set(0);
+        }
+        refCountOwner.set(owner);
+      }
+    }
+  }
+
+  /**
+   * Create, set, and return a generic reference count owner object.
+   * Calling this method is a noop and returns null if !trackReferenceCounts.
+   */
+  public static Object createReferenceCountOwner() {
+    Object result = null;
+    if (trackReferenceCounts()) {
+      result = new Object();
+      setReferenceCountOwner(result);
+    }
+    return result;
+  }
+
+  /**
+   * Call this method before incrementing a reference count
+   * if you know that tracking is not needed because you know
+   * that the allocate and free will always be done in the same
+   * code block.
+   * Callers of this method must also call unskipRefCountTracking
+   * after the allocation or free is done.
+   */
+  public static void skipRefCountTracking() {
+    setReferenceCountOwner(SKIP_REF_COUNT_TRACKING);
+  }
+
+  /**
+   * Call this method to undo a call to skipRefCountTracking.
+   */
+  public static void unskipRefCountTracking() {
+    setReferenceCountOwner(null);
+  }
+
+  /**
+   * Returns a list of any reference count tracking information for
+   * the given Chunk address.
+   */
+  public static List<RefCountChangeInfo> getRefCountInfo(long address) {
+    if (!trackReferenceCounts()) return null;
+    List<RefCountChangeInfo> result = stacktraces.get(address);
+    while (result != null && !stacktraces.replace(address, result, LOCKED)) {
+      result = stacktraces.get(address);
+    }
+    return result;
+  }
+
+  /**
+   * Returns a list of any free operation tracking information.
+   * This is used to describe who did the previous free(s) when an extra one
+   * ends up being done and fails.
+   */
+  public static List<RefCountChangeInfo> getFreeRefCountInfo(long address) {
+    if (!trackReferenceCounts() || !trackFreedReferenceCounts()) return null;
+    return freedStacktraces.get(address);
+  }
+
+  /**
+   * Used internally to report that a reference count has changed.
+   */
+  static void refCountChanged(Long address, boolean decRefCount, int rc) {
+    final Object owner = refCountOwner.get();
+    if (owner == SKIP_REF_COUNT_TRACKING) {
+      return;
+    }
+    List<RefCountChangeInfo> list = stacktraces.get(address);
+    if (list == null) {
+      List<RefCountChangeInfo> newList = new ArrayList<RefCountChangeInfo>();
+      List<RefCountChangeInfo> old = stacktraces.putIfAbsent(address, newList);
+      if (old == null) {
+        list = newList;
+      } else {
+        list = old;
+      }
+    }
+    if (decRefCount) {
+      if (owner != null) {
+        synchronized (list) {
+          for (int i=0; i < list.size(); i++) {
+            RefCountChangeInfo info = list.get(i);
+            if (owner instanceof RegionEntry) {
+              // use identity comparison on region entries since sqlf does some wierd stuff in the equals method
+              if (owner == info.getOwner()) {
+                if (info.getDupCount() > 0) {
+                  info.decDupCount();
+                } else {
+                  list.remove(i);
+                }
+                return;
+              }
+            } else if (owner.equals(info.getOwner())) {
+              if (info.getDupCount() > 0) {
+                info.decDupCount();
+              } else {
+                list.remove(i);
+              }
+              return;
+            }
+          }
+        }
+      }
+    }
+    if (list == LOCKED) {
+      SimpleMemoryAllocatorImpl.debugLog("refCount " + (decRefCount ? "deced" : "inced") + " after orphan detected for @" + Long.toHexString(address), true);
+      return;
+    }
+    RefCountChangeInfo info = new RefCountChangeInfo(decRefCount, rc, owner);
+    synchronized (list) {
+      //      if (list.size() == 16) {
+      //        debugLog("dumping @" + Long.toHexString(address) + " history=" + list, false);
+      //        list.clear();
+      //      }
+      for (RefCountChangeInfo e: list) {
+        if (e.isDuplicate(info)) {
+          // No need to add it
+          return;
+        }
+      }
+      list.add(info);
+    }
+  }
+
+  /**
+   * Called internally when free operations are tracked to record
+   * that a free has happened of the given address.
+   */
+  static void freeRefCountInfo(Long address) {
+    if (!trackReferenceCounts()) return;
+    List<RefCountChangeInfo> freedInfo = stacktraces.remove(address);
+    if (freedInfo == LOCKED) {
+      SimpleMemoryAllocatorImpl.debugLog("freed after orphan detected for @" + Long.toHexString(address), true);
+    } else if (trackFreedReferenceCounts()) {
+      if (freedInfo != null) {
+        freedStacktraces.put(address, freedInfo);
+      } else {
+        freedStacktraces.remove(address);
+      }
+    }
+  }
+}


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

Posted by ab...@apache.org.
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