You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mnemonic.apache.org by ga...@apache.org on 2017/03/07 01:18:52 UTC

incubator-mnemonic git commit: MNEMONIC-208: Adding PUT/GET/DELETE for durable maps.

Repository: incubator-mnemonic
Updated Branches:
  refs/heads/master 81a4878f4 -> be32da856


MNEMONIC-208: Adding PUT/GET/DELETE for durable maps.


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

Branch: refs/heads/master
Commit: be32da856026a765012b2b9eced47d588c657732
Parents: 81a4878
Author: Johnu George <jo...@cisco.com>
Authored: Mon Mar 6 10:38:27 2017 -0800
Committer: Johnu George <jo...@cisco.com>
Committed: Mon Mar 6 17:01:06 2017 -0800

----------------------------------------------------------------------
 .../mnemonic/collections/DurableHashMap.java    | 133 ++++++
 .../collections/DurableHashMapFactory.java      |  70 +++
 .../collections/DurableHashMapImpl.java         | 447 +++++++++++++++++++
 .../apache/mnemonic/collections/MapEntry.java   | 119 +++++
 .../collections/DurableHashMapNGTest.java       | 366 +++++++++++++++
 .../org/apache/mnemonic/collections/Person.java |  14 +-
 .../mnemonic/AnnotatedDurableEntityClass.java   |  22 +-
 7 files changed, 1158 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-mnemonic/blob/be32da85/mnemonic-collections/src/main/java/org/apache/mnemonic/collections/DurableHashMap.java
----------------------------------------------------------------------
diff --git a/mnemonic-collections/src/main/java/org/apache/mnemonic/collections/DurableHashMap.java b/mnemonic-collections/src/main/java/org/apache/mnemonic/collections/DurableHashMap.java
new file mode 100644
index 0000000..2931cc2
--- /dev/null
+++ b/mnemonic-collections/src/main/java/org/apache/mnemonic/collections/DurableHashMap.java
@@ -0,0 +1,133 @@
+/*
+ * 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 org.apache.mnemonic.collections;
+
+import org.apache.mnemonic.Durable;
+import org.apache.mnemonic.EntityFactoryProxy;
+import org.apache.mnemonic.DurableType;
+
+public abstract class DurableHashMap<K, V> implements Durable {
+  protected transient EntityFactoryProxy[] m_node_efproxies;
+  protected transient DurableType[] m_node_gftypes;
+  protected long threshold;
+  protected long totalCapacity;
+  protected long mapSize = 0;
+  
+  /**
+   * creation callback for initialization
+   *
+   */
+  @Override
+  public void initializeAfterCreate() {
+  //   System.out.println("Initializing After Created");
+  }
+
+  /**
+   * restore callback for initialization
+   *
+   */
+  @Override
+  public void initializeAfterRestore() {
+  //   System.out.println("Initializing After Restored");
+  }
+
+  /**
+   * this function will be invoked by its factory to setup generic related info
+   * to avoid expensive operations from reflection
+   *
+   * @param efproxies
+   *          specify a array of factory to proxy the restoring of its generic
+   *          field objects
+   *
+   * @param gftypes
+   *          specify a array of types corresponding to efproxies
+   */
+  @Override
+  public void setupGenericInfo(EntityFactoryProxy[] efproxies, DurableType[] gftypes) {
+    m_node_efproxies = efproxies;
+    m_node_gftypes = gftypes;
+  }
+ 
+  /**
+   * Add a new key-value pair to map
+   * 
+   * @param key
+   *          the key to be set
+   *
+   * @param value
+   *          the value to be set
+   *
+   * @return previous value with key else return null
+   */
+  public abstract V put(K key, V value);
+
+  /**
+   * Return a value to which key is mapped
+   * 
+   * @param key
+   *          the key whose value is to be retrieved
+   *
+   * @return previous value with key else return null
+   */
+  public abstract V get(K key);
+
+  /**
+   * Remove a mapping for a specified key
+   * 
+   * @param key
+   *          the key whose value is to be removed
+   *
+   * @return previous value with key else return null
+   */
+  public abstract V remove(K key);
+
+  /**
+   * Apply hash function to given hash code
+   * 
+   * @param hashcode
+   *          the hashcode of the object
+   *
+   * @return result of the hash function 
+   */
+  public int hash(int hashcode) {
+    hashcode ^= (hashcode >>> 20) ^ (hashcode >>> 12);
+    return hashcode ^ (hashcode >>> 7) ^ (hashcode >>> 4);
+  }
+
+  /**
+   * Map the hash to a bucket
+   * 
+   * @param hash
+   *          the hashcode of the object
+   *
+   * @return index of the bucket
+   */
+  public long getBucketIndex(int hash) {
+    return hash & (totalCapacity - 1);
+  }
+  /**
+   * Get the number of elements in the map
+   * 
+   * @return size of the map
+   */
+  public long getSize() {
+    return mapSize;
+  }
+
+}
+    

http://git-wip-us.apache.org/repos/asf/incubator-mnemonic/blob/be32da85/mnemonic-collections/src/main/java/org/apache/mnemonic/collections/DurableHashMapFactory.java
----------------------------------------------------------------------
diff --git a/mnemonic-collections/src/main/java/org/apache/mnemonic/collections/DurableHashMapFactory.java b/mnemonic-collections/src/main/java/org/apache/mnemonic/collections/DurableHashMapFactory.java
new file mode 100644
index 0000000..5f04da7
--- /dev/null
+++ b/mnemonic-collections/src/main/java/org/apache/mnemonic/collections/DurableHashMapFactory.java
@@ -0,0 +1,70 @@
+/*
+ * 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 org.apache.mnemonic.collections;
+
+import org.apache.mnemonic.DurableType;
+import org.apache.mnemonic.EntityFactoryProxy;
+import org.apache.mnemonic.OutOfHybridMemory;
+import org.apache.mnemonic.RestorableAllocator;
+import org.apache.mnemonic.RestoreDurableEntityError;
+
+public class DurableHashMapFactory {
+  public static <A extends RestorableAllocator<A>, K, V> DurableHashMap<K, V>
+              create(A allocator) throws OutOfHybridMemory {
+    return create(allocator, 0L, false);
+  }
+
+  public static <A extends RestorableAllocator<A>, K, V> DurableHashMap<K, V> 
+              create(A allocator, long initialCapacity) throws OutOfHybridMemory {
+    return create(allocator, initialCapacity, false);
+  }
+
+  public static <A extends RestorableAllocator<A>, K, V> DurableHashMap<K, V> 
+              create(A allocator, long initialCapacity, boolean autoreclaim) throws OutOfHybridMemory {
+    return create(allocator, null, null, initialCapacity, autoreclaim);
+  }
+
+  public static <A extends RestorableAllocator<A>, K, V> DurableHashMap<K, V> 
+              create(A allocator, EntityFactoryProxy[] factoryproxys, DurableType[] gfields, 
+                   long initialCapacity, boolean autoreclaim) throws OutOfHybridMemory {
+    DurableHashMapImpl<A, K, V> entity = new DurableHashMapImpl<A, K, V>();
+    entity.setCapacityHint(initialCapacity);
+    entity.setupGenericInfo(factoryproxys, gfields);
+    entity.createDurableEntity(allocator, factoryproxys, gfields, autoreclaim);
+    return entity;
+  }
+
+  public static <A extends RestorableAllocator<A>, K, V> DurableHashMap<K, V> 
+              restore(A allocator, long phandler) throws RestoreDurableEntityError {
+    return restore(allocator, phandler, false);
+  }
+
+  public static <A extends RestorableAllocator<A>, K, V> DurableHashMap<K, V> 
+              restore(A allocator, long phandler, boolean autoreclaim) throws RestoreDurableEntityError {
+    return restore(allocator, null, null, phandler, autoreclaim);
+  }
+
+  public static <A extends RestorableAllocator<A>, K, V> DurableHashMap<K, V> 
+              restore(A allocator, EntityFactoryProxy[] factoryproxys, DurableType[] gfields,
+                   long phandler, boolean autoreclaim) throws RestoreDurableEntityError {
+    DurableHashMapImpl<A, K, V> entity = new DurableHashMapImpl<A, K, V>();
+    entity.setupGenericInfo(factoryproxys, gfields);
+    entity.restoreDurableEntity(allocator, factoryproxys, gfields, phandler, autoreclaim);
+    return entity;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-mnemonic/blob/be32da85/mnemonic-collections/src/main/java/org/apache/mnemonic/collections/DurableHashMapImpl.java
----------------------------------------------------------------------
diff --git a/mnemonic-collections/src/main/java/org/apache/mnemonic/collections/DurableHashMapImpl.java b/mnemonic-collections/src/main/java/org/apache/mnemonic/collections/DurableHashMapImpl.java
new file mode 100644
index 0000000..2706ff2
--- /dev/null
+++ b/mnemonic-collections/src/main/java/org/apache/mnemonic/collections/DurableHashMapImpl.java
@@ -0,0 +1,447 @@
+/*
+ * 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 org.apache.mnemonic.collections;
+
+import org.apache.mnemonic.EntityFactoryProxy;
+import org.apache.mnemonic.DurableType;
+import org.apache.mnemonic.MemChunkHolder;
+import org.apache.mnemonic.MemoryDurableEntity;
+import org.apache.mnemonic.OutOfHybridMemory;
+import org.apache.mnemonic.RestorableAllocator;
+import org.apache.mnemonic.RestoreDurableEntityError;
+import org.apache.mnemonic.RetrieveDurableEntityError;
+import org.apache.mnemonic.Utils;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.commons.lang3.ArrayUtils;
+import sun.misc.Unsafe;
+
+@SuppressWarnings("restriction")
+public class DurableHashMapImpl<A extends RestorableAllocator<A>, K, V>
+        extends DurableHashMap<K, V> implements MemoryDurableEntity<A> {
+
+  private static final long DEFAULT_MAP_SIZE = 16;
+  private static final float DEFAULT_MAP_LOAD_FACTOR = 0.75f;
+  private static final long MAX_OBJECT_SIZE = 8;
+  private static long[][] fieldInfo;
+  private Unsafe unsafe;
+  private EntityFactoryProxy[] factoryProxy;
+  private EntityFactoryProxy[] listefproxies;
+  private DurableType[] genericField;
+  private DurableType[] listgftypes;
+  private volatile boolean autoResize = true;
+  private volatile boolean autoReclaim;
+  private MemChunkHolder<A> holder;
+  private MemChunkHolder<A> chunkAddr;
+  private A allocator;
+  /**
+   * Set initial capacity for a hashmap. It can grow in size.
+   * 
+   * @param capacity
+   *          Initial capacity to be set
+   */
+  public void setCapacityHint(long capacity) {
+    if (0 == capacity) {
+      totalCapacity = DEFAULT_MAP_SIZE;
+    } else {
+      totalCapacity = 1;
+      while (totalCapacity < capacity) {
+        totalCapacity <<= 1;
+      }
+    }
+    threshold = (long) (totalCapacity * DEFAULT_MAP_LOAD_FACTOR);
+  }
+
+  /**
+   * Add a new key-value pair to map
+   * 
+   * @param key
+   *          the key to be set
+   *
+   * @param value
+   *          the value to be set
+   *
+   * @return previous value with key else return null
+   */
+  @Override
+  public V put(K key, V value) {
+    int hash = hash(key.hashCode());
+    long bucketIndex = getBucketIndex(hash);
+    long bucketAddr = holder.get() + MAX_OBJECT_SIZE * bucketIndex;
+    V retVal = addEntry(key, value, bucketAddr);
+    if (autoResize && (mapSize >= threshold)) {
+      resize(2 * totalCapacity);
+    }
+    return retVal;
+  }
+
+  /**
+   * Add a new key-value pair to map at a given bucket address
+   * 
+   * @param key
+   *          the key to be set
+   *
+   * @param value
+   *          the value to be set
+   *
+   * @param bucketAddr
+   *          the addr of the bucket where key is hashed
+   *
+   * @return previous value with key else return null
+   */
+  public V addEntry(K key, V value, long bucketAddr) {
+    V retValue = null;
+    long handler = unsafe.getAddress(bucketAddr);
+    if (0L == handler) {
+      DurableSinglyLinkedList<MapEntry<K, V>> head = DurableSinglyLinkedListFactory.create(allocator, 
+          listefproxies, listgftypes, false);
+      MapEntry<K, V> entry = MapEntryFactory.create(allocator, factoryProxy, genericField, false);
+      entry.setKey(key, false);
+      entry.setValue(value, false);
+      head.setItem(entry, false);
+      unsafe.putLong(bucketAddr, head.getHandler());
+      mapSize++;
+    } else {
+      DurableSinglyLinkedList<MapEntry<K, V>> head = DurableSinglyLinkedListFactory.restore(allocator,
+          listefproxies, listgftypes, handler, false);
+      DurableSinglyLinkedList<MapEntry<K, V>> prev = head;
+      boolean found = false;
+      while (null != head) {
+        MapEntry<K, V> mapEntry = head.getItem();
+        K entryKey = mapEntry.getKey();
+        if (entryKey == key || entryKey.equals(key)) {
+          retValue = mapEntry.getValue();
+          mapEntry.setValue(value, false);
+          found = true;
+          break;
+        }
+        prev = head;
+        head = head.getNext();
+      }
+      if (true != found) {
+        DurableSinglyLinkedList<MapEntry<K, V>> newNode = DurableSinglyLinkedListFactory.create(allocator, 
+            listefproxies, listgftypes, false);
+        MapEntry<K, V> entry = MapEntryFactory.create(allocator, factoryProxy, genericField, false);
+        entry.setKey(key, false);
+        entry.setValue(value, false);
+        newNode.setItem(entry, false);
+        prev.setNext(newNode, false);
+        mapSize++;
+      }
+    }
+    return retValue;
+  }
+
+  /**
+   * Return a value to which key is mapped
+   * 
+   * @param key
+   *          the key whose value is to be retrieved
+   *
+   * @return previous value with key else return null
+   */
+  @Override
+  public V get(K key) {
+    int hash = hash(key.hashCode());
+    long bucketIndex = getBucketIndex(hash);
+    long bucketAddr = holder.get() + MAX_OBJECT_SIZE * bucketIndex;
+    return getEntry(key, bucketAddr);
+  }
+
+  /**
+   * Return a value to which key is mapped given a bucket address
+   * 
+   * @param key
+   *          the key whose value is to be retrieved
+   *
+   * @param bucketAddr
+   *          the addr of the bucket where key is hashed
+   *
+   * @return previous value with key else return null
+   */
+  public V getEntry(K key, long bucketAddr) {
+    V retValue = null;
+    long handler = unsafe.getAddress(bucketAddr);
+    if (0L != handler) {
+      DurableSinglyLinkedList<MapEntry<K, V>> head = DurableSinglyLinkedListFactory.restore(allocator,
+          listefproxies, listgftypes, handler, false);
+      while (null != head) {
+        MapEntry<K, V> mapEntry = head.getItem();
+        K entryKey = mapEntry.getKey();
+        if (entryKey == key || entryKey.equals(key)) {
+          retValue = mapEntry.getValue();
+          break;
+        }
+        head = head.getNext();
+      }
+    }
+    return retValue;
+  }
+
+  /**
+   * Remove a mapping for a specified key
+   * 
+   * @param key
+   *          the key whose value is to be removed
+   *
+   * @return previous value with key else return null
+   */
+  @Override
+  public V remove(K key) {
+    int hash = hash(key.hashCode());
+    long bucketIndex = getBucketIndex(hash);
+    long bucketAddr = holder.get() + MAX_OBJECT_SIZE * bucketIndex;
+    return removeEntry(key, bucketAddr);
+  }
+
+  /**
+   * Remove a mapping for a specified key at given bucket address
+   * 
+   * @param key
+   *          the key whose value is to be removed
+   *
+   * @param bucketAddr
+   *          the addr of the bucket where key is hashed
+   *
+   * @return previous value with key else return null
+   */
+  public V removeEntry(K key, long bucketAddr) {
+    V retValue = null;
+    long handler = unsafe.getAddress(bucketAddr);
+    if (0L != handler) {
+      DurableSinglyLinkedList<MapEntry<K, V>> head = DurableSinglyLinkedListFactory.restore(allocator,
+          listefproxies, listgftypes, handler, false);
+      DurableSinglyLinkedList<MapEntry<K, V>> prev = null;
+      boolean found = false;
+      while (null != head) {
+        MapEntry<K, V> mapEntry = head.getItem();
+        K entryKey = mapEntry.getKey();
+        if (entryKey == key || entryKey.equals(key)) {
+          retValue = mapEntry.getValue();
+          found = true;
+          break;
+        }
+        prev = head;
+        head = head.getNext();
+      }
+      if (true == found) {
+        if (null == prev) { 
+          if (null == head.getNext()) {
+            unsafe.putAddress(bucketAddr, 0L);
+            head.destroy();
+          } else {
+            unsafe.putAddress(bucketAddr, head.getNext().getHandler());        
+            head.setNext(null, false);
+            head.destroy(); // #TODO: better way to delete one node
+          }
+        } else {
+          prev.setNext(head.getNext(), false);
+          head.setNext(null, false);
+          head.destroy(); // #TODO: better way to delete one node
+        }
+        mapSize--;
+      }       
+    }
+    return retValue;
+  }
+
+  /**
+   * Rehashes the entire map into a new map of given capacity
+   * 
+   * @param newCapacity
+   *          the capacity of new map
+   */
+  public void resize(long newCapacity) {
+    MemChunkHolder<A> prevHolder = holder; 
+    long bucketAddr = prevHolder.get();
+    long maxbucketAddr = bucketAddr + MAX_OBJECT_SIZE * totalCapacity;
+    totalCapacity = newCapacity;
+    threshold = (long) (totalCapacity * DEFAULT_MAP_LOAD_FACTOR);
+    holder = allocator.createChunk(MAX_OBJECT_SIZE * totalCapacity, autoReclaim);
+    unsafe.putLong(chunkAddr.get(), allocator.getChunkHandler(holder));
+    while (bucketAddr < maxbucketAddr) {
+      long handler = unsafe.getAddress(bucketAddr);
+      if (0L != handler) {
+        DurableSinglyLinkedList<MapEntry<K, V>> head = DurableSinglyLinkedListFactory.restore(allocator,
+            listefproxies, listgftypes, handler, false);
+        DurableSinglyLinkedList<MapEntry<K, V>> curr = head;
+        while (null != curr) {
+          curr = curr.getNext();
+          transfer(head);
+          head = curr;
+        }
+      }
+      bucketAddr += MAX_OBJECT_SIZE;
+    }
+    prevHolder.destroy();
+  }
+
+  /**
+   * Transfers a map item from old map to the new map
+   * 
+   * @param elem
+   *          the item in the old map
+   */
+  public void transfer(DurableSinglyLinkedList<MapEntry<K, V>> elem) {
+    int hash = hash(elem.getItem().getKey().hashCode());
+    long bucketIndex = getBucketIndex(hash);
+    long bucketAddr = holder.get() + MAX_OBJECT_SIZE * bucketIndex;
+    long handler = unsafe.getAddress(bucketAddr);
+    if (0L != handler) {
+      DurableSinglyLinkedList<MapEntry<K, V>> head = DurableSinglyLinkedListFactory.restore(allocator,
+          listefproxies, listgftypes, handler, false);
+      elem.setNext(head, false);
+    } else {
+      elem.setNext(null, false);
+    }
+    unsafe.putLong(bucketAddr, elem.getHandler());
+  }
+
+  /**
+   * Recomputes the size of the map during restore without persistence
+   * 
+   */
+  public long recomputeMapSize() {
+    long size = 0;
+    long bucketAddr = holder.get();
+    long maxbucketAddr = bucketAddr + MAX_OBJECT_SIZE * totalCapacity;
+    while (bucketAddr < maxbucketAddr) {
+      long handler = unsafe.getAddress(bucketAddr);
+      if (0L != handler) {
+        DurableSinglyLinkedList<MapEntry<K, V>> head = DurableSinglyLinkedListFactory.restore(allocator,
+            listefproxies, listgftypes, handler, false);
+        while (null != head) {
+          size++;
+          head = head.getNext();
+        }
+      }
+      bucketAddr += MAX_OBJECT_SIZE;
+    }
+    return size;
+  }
+
+  @Override
+  public boolean autoReclaim() {
+    return autoReclaim;
+  }
+
+  @Override
+  public long[][] getNativeFieldInfo() {
+    return fieldInfo;
+  }
+
+  @Override
+  public void destroy() throws RetrieveDurableEntityError {
+    long bucketAddr = holder.get();
+    long maxbucketAddr = bucketAddr + MAX_OBJECT_SIZE * totalCapacity;
+    while (bucketAddr < maxbucketAddr) {
+      long handler = unsafe.getAddress(bucketAddr);
+      if (0L != handler) {
+        DurableSinglyLinkedList<MapEntry<K, V>> head = DurableSinglyLinkedListFactory.restore(allocator,
+            listefproxies, listgftypes, handler, false);
+        head.destroy();
+        }
+      bucketAddr += MAX_OBJECT_SIZE;
+    }
+    holder.destroy();
+    chunkAddr.destroy();
+  }
+
+  @Override
+  public void cancelAutoReclaim() {
+    holder.cancelAutoReclaim();
+    autoReclaim = false;
+  }
+
+  @Override
+  public void registerAutoReclaim() {
+    holder.registerAutoReclaim();
+    autoReclaim = true;
+  }
+
+  @Override
+  public long getHandler() {
+    return allocator.getChunkHandler(chunkAddr);
+  }
+
+  @Override
+  public void restoreDurableEntity(A allocator, EntityFactoryProxy[] factoryProxy, 
+             DurableType[] gField, long phandler, boolean autoreclaim) throws RestoreDurableEntityError {
+    initializeDurableEntity(allocator, factoryProxy, gField, autoreclaim);
+    if (0L == phandler) {
+      throw new RestoreDurableEntityError("Input handler is null on restoreDurableEntity.");
+    }
+    chunkAddr = allocator.retrieveChunk(phandler, autoreclaim);
+    long chunkHandler = unsafe.getLong(chunkAddr.get());
+    holder = allocator.retrieveChunk(chunkHandler, autoReclaim);
+    if (null == holder || null == chunkAddr) {
+      throw new RestoreDurableEntityError("Retrieve Entity Failure!");
+    }
+    setCapacityHint(holder.getSize() / MAX_OBJECT_SIZE);
+    mapSize = recomputeMapSize();
+    initializeAfterRestore();
+  }
+
+
+  @Override
+  public void initializeDurableEntity(A allocator, EntityFactoryProxy[] factoryProxy, 
+              DurableType[] gField, boolean autoReclaim) {
+    this.allocator = allocator;
+    this.factoryProxy = factoryProxy;
+    this.genericField = gField;
+    this.autoReclaim = autoReclaim;
+    DurableType gftypes[] = {DurableType.DURABLE};
+    this.listgftypes = ArrayUtils.addAll(gftypes, genericField);
+    EntityFactoryProxy efproxies[] = {new EntityFactoryProxy() {
+      @Override
+      public <A extends RestorableAllocator<A>> MapEntry<K, V> restore(
+          A allocator, EntityFactoryProxy[] factoryproxys,
+          DurableType[] gfields, long phandler, boolean autoreclaim) {
+          Pair<DurableType[], EntityFactoryProxy[]> dpt = Utils.shiftDurableParams(gfields, factoryproxys, 1);
+        return MapEntryFactory.restore(allocator, dpt.getRight(), dpt.getLeft(), phandler, autoreclaim);
+          }
+      @Override
+      public <A extends RestorableAllocator<A>> MapEntry<K, V> create(
+          A allocator, EntityFactoryProxy[] factoryproxys,
+          DurableType[] gfields, boolean autoreclaim) {
+          Pair<DurableType[], EntityFactoryProxy[]> dpt = Utils.shiftDurableParams(gfields, factoryproxys, 1);
+        return MapEntryFactory.create(allocator, dpt.getRight(), dpt.getLeft(), autoreclaim);
+          }
+    }
+    };
+    this.listefproxies = ArrayUtils.addAll(efproxies, factoryProxy);
+    try {
+      this.unsafe = Utils.getUnsafe();
+    } catch (Exception e) {
+      e.printStackTrace();
+    }
+  }
+
+  @Override
+  public void createDurableEntity(A allocator, EntityFactoryProxy[] factoryProxy, 
+              DurableType[] gField, boolean autoreclaim) throws OutOfHybridMemory {
+    initializeDurableEntity(allocator, factoryProxy, gField, autoreclaim);
+    this.holder = allocator.createChunk(MAX_OBJECT_SIZE * totalCapacity, autoreclaim);
+    this.chunkAddr = allocator.createChunk(MAX_OBJECT_SIZE, autoreclaim);
+    unsafe.putLong(chunkAddr.get(), allocator.getChunkHandler(holder));
+    if (null == this.holder || null == this.chunkAddr) {
+      throw new OutOfHybridMemory("Create Durable Entity Error!");
+    }
+    initializeAfterCreate();
+  }
+}
+ 

http://git-wip-us.apache.org/repos/asf/incubator-mnemonic/blob/be32da85/mnemonic-collections/src/main/java/org/apache/mnemonic/collections/MapEntry.java
----------------------------------------------------------------------
diff --git a/mnemonic-collections/src/main/java/org/apache/mnemonic/collections/MapEntry.java b/mnemonic-collections/src/main/java/org/apache/mnemonic/collections/MapEntry.java
new file mode 100644
index 0000000..4b82cd4
--- /dev/null
+++ b/mnemonic-collections/src/main/java/org/apache/mnemonic/collections/MapEntry.java
@@ -0,0 +1,119 @@
+/*
+ * 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 org.apache.mnemonic.collections;
+
+
+import org.apache.mnemonic.Durable;
+import org.apache.mnemonic.EntityFactoryProxy;
+import org.apache.mnemonic.DurableType;
+import org.apache.mnemonic.DurableEntity;
+import org.apache.mnemonic.DurableGetter;
+import org.apache.mnemonic.DurableSetter;
+import org.apache.mnemonic.RetrieveDurableEntityError;
+
+/**
+ * this class defines a non-volatile node for a generic value to form a
+ * unidirectional link
+ *
+ */
+@DurableEntity
+public abstract class MapEntry<K, V> implements Durable {
+  protected transient EntityFactoryProxy[] m_node_efproxies;
+  protected transient DurableType[] m_node_gftypes;
+
+  /**
+   * creation callback for initialization
+   *
+   */
+  @Override
+  public void initializeAfterCreate() {
+    // System.out.println("Initializing After Created");
+  }
+
+  /**
+   * restore callback for initialization
+   *
+   */
+  @Override
+  public void initializeAfterRestore() {
+    // System.out.println("Initializing After Restored");
+  }
+
+  /**
+   * this function will be invoked by its factory to setup generic related info
+   * to avoid expensive operations from reflection
+   *
+   * @param efproxies
+   *          specify a array of factory to proxy the restoring of its generic
+   *          field objects
+   *
+   * @param gftypes
+   *          specify a array of types corresponding to efproxies
+   */
+  @Override
+  public void setupGenericInfo(EntityFactoryProxy[] efproxies, DurableType[] gftypes) {
+    m_node_efproxies = efproxies;
+    m_node_gftypes = gftypes;
+  }
+
+  /**
+   * get the key field of this node
+   *
+   * @return the key field of this node
+   */
+  @DurableGetter(Id = 1L, EntityFactoryProxies = "m_node_efproxies", GenericFieldTypes = "m_node_gftypes")
+  public abstract K getKey();
+
+  /**
+   * set key field in the node
+   * 
+   * @param key
+   *          the key to be set
+   *
+   * @param destroy
+   *          true if want to destroy exist one
+   *
+   */
+  @DurableSetter
+  public abstract void setKey(K key, boolean destroy);
+
+  /**
+   * get value field of this node
+   *
+   * @return the value field of this node
+   *
+   */
+  @DurableGetter(Id = 2L, EntityFactoryProxies = "m_node_efproxies", GenericFieldTypes = "m_node_gftypes")
+  public abstract V getValue();
+
+  /**
+   * set value field in the node
+   *
+   * @param value
+   *          the value to be set
+   *
+   * @param destroy
+   *          true if want to destroy the exist node
+   */
+  @DurableSetter
+  public abstract void setValue(V value, boolean destroy);
+
+  public void testOutput() throws RetrieveDurableEntityError {
+     System.out.println(" Key: " + getKey() + " Value: " + getValue());
+   } 
+}

http://git-wip-us.apache.org/repos/asf/incubator-mnemonic/blob/be32da85/mnemonic-collections/src/test/java/org/apache/mnemonic/collections/DurableHashMapNGTest.java
----------------------------------------------------------------------
diff --git a/mnemonic-collections/src/test/java/org/apache/mnemonic/collections/DurableHashMapNGTest.java b/mnemonic-collections/src/test/java/org/apache/mnemonic/collections/DurableHashMapNGTest.java
new file mode 100644
index 0000000..8b1d588
--- /dev/null
+++ b/mnemonic-collections/src/test/java/org/apache/mnemonic/collections/DurableHashMapNGTest.java
@@ -0,0 +1,366 @@
+/*
+ * 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 org.apache.mnemonic.collections;
+
+import java.nio.ByteBuffer;
+
+import org.apache.mnemonic.Utils;
+import org.apache.mnemonic.RestorableAllocator;
+import org.apache.mnemonic.NonVolatileMemAllocator;
+import org.apache.mnemonic.EntityFactoryProxy;
+import org.apache.mnemonic.DurableType;
+import org.apache.mnemonic.Reclaim;
+import org.apache.mnemonic.Durable;
+import org.apache.commons.lang3.tuple.Pair;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+import org.testng.AssertJUnit;
+/**
+ *
+ *
+ */
+
+public class DurableHashMapNGTest {
+  private long cKEYCAPACITY;
+  private NonVolatileMemAllocator m_act;
+  private long mInitialCapacity = 1;
+
+  @BeforeClass
+  public void setUp() {
+    m_act = new NonVolatileMemAllocator(Utils.getNonVolatileMemoryAllocatorService("pmalloc"), 1024 * 1024 * 1024,
+        "./pobj_hashmaps.dat", true);
+    cKEYCAPACITY = m_act.handlerCapacity();
+    m_act.setBufferReclaimer(new Reclaim<ByteBuffer>() {
+      @Override
+      public boolean reclaim(ByteBuffer mres, Long sz) {
+        System.out.println(String.format("Reclaim Memory Buffer: %X  Size: %s", System.identityHashCode(mres),
+            null == sz ? "NULL" : sz.toString()));
+        return false;
+      }
+    });
+    m_act.setChunkReclaimer(new Reclaim<Long>() {
+      @Override
+      public boolean reclaim(Long mres, Long sz) {
+        System.out.println(String.format("Reclaim Memory Chunk: %X  Size: %s", System.identityHashCode(mres),
+            null == sz ? "NULL" : sz.toString()));
+        return false;
+      }
+    });
+
+    for (long i = 0; i < cKEYCAPACITY; ++i) {
+      m_act.setHandler(i, 0L);
+    }
+  }
+
+  @AfterClass
+  public void tearDown() {
+    m_act.close();
+  }
+
+  @Test(enabled = true)
+  public void testGetPutRemovePrimitives() {
+    DurableType gtypes[] = {DurableType.STRING, DurableType.INTEGER};
+    DurableHashMap<String, Integer> map = DurableHashMapFactory.create(m_act, null, gtypes, mInitialCapacity, false);
+    
+    Long handler = map.getHandler();
+    Integer val = map.put("hello", 1);
+    AssertJUnit.assertNull(val);
+    val = map.put("hello", 1);
+    AssertJUnit.assertEquals(1, val.intValue());
+    val = map.put("world", 2);
+    AssertJUnit.assertNull(val);
+    val = map.put("hello", 2);
+    AssertJUnit.assertEquals(1, val.intValue());
+    val = map.put("hello", 3);
+    AssertJUnit.assertEquals(2, val.intValue());
+
+    val = map.get("hello");
+    AssertJUnit.assertEquals(3, val.intValue());
+    val = map.get("world"); 
+    AssertJUnit.assertEquals(2, val.intValue());
+    val = map.get("test");
+    AssertJUnit.assertNull(val);
+    val = map.put("testing", 5);
+    AssertJUnit.assertNull(val);
+    val = map.get("testing");
+    AssertJUnit.assertEquals(5, val.intValue());
+    val = map.remove("testing");
+    AssertJUnit.assertEquals(5, val.intValue());
+    val = map.get("testing");
+    AssertJUnit.assertNull(val);
+    val = map.remove("testing");
+    AssertJUnit.assertNull(val);
+    val = map.remove("world");
+    AssertJUnit.assertEquals(2, val.intValue());
+
+
+    DurableHashMap<String, Integer> restoredMap = DurableHashMapFactory.restore(m_act, null, gtypes, handler, 
+        false);
+    val = restoredMap.get("hello");
+    AssertJUnit.assertEquals(3, val.intValue());
+    val = restoredMap.get("world"); 
+    AssertJUnit.assertNull(val);
+    val = restoredMap.get("test");
+    AssertJUnit.assertNull(val);
+    val = map.get("testing");
+    AssertJUnit.assertNull(val);
+    val = map.put("test", 4);
+    AssertJUnit.assertNull(val);
+    val = restoredMap.get("test");
+    AssertJUnit.assertEquals(4, val.intValue());
+  }
+
+  @Test(enabled = true)
+  public void testGetPutKeyDurable() {
+    
+    DurableType gtypes[] = {DurableType.DURABLE, DurableType.STRING};
+    EntityFactoryProxy efproxies[] = {new EntityFactoryProxy() {
+      @Override
+      public <A extends RestorableAllocator<A>> Person<Long> restore(
+          A allocator, EntityFactoryProxy[] factoryproxys,
+          DurableType[] gfields, long phandler, boolean autoreclaim) {
+        return PersonFactory.restore(allocator, factoryproxys, gfields, phandler, autoreclaim);
+      }
+      @Override
+      public <A extends RestorableAllocator<A>> Person<Long> create(
+          A allocator, EntityFactoryProxy[] factoryproxys,
+          DurableType[] gfields, boolean autoreclaim) {
+        return PersonFactory.create(allocator, factoryproxys, gfields, autoreclaim);
+      }
+    } };
+    
+    Person<Long> person =  (Person<Long>) efproxies[0].create(m_act, null, null, false);
+    person.setAge((short) 31);
+    person.setName("Bob", true);
+
+    Person<Long> anotherPerson =  (Person<Long>) efproxies[0].create(m_act, null, null, false);
+    anotherPerson.setAge((short) 30);
+    anotherPerson.setName("Alice", true);
+
+    DurableHashMap<Person<Long>, String> map = DurableHashMapFactory.create(m_act, 
+                      efproxies, gtypes, mInitialCapacity, false);
+    String str = map.put(person, "hello");
+    AssertJUnit.assertNull(str); 
+    str = map.get(person);
+    AssertJUnit.assertEquals(str, "hello");
+    str = map.put(person, "world");
+    AssertJUnit.assertEquals(str, "hello");
+    str = map.get(person);
+    AssertJUnit.assertEquals(str, "world");
+
+    str = map.put(anotherPerson, "testing");
+    AssertJUnit.assertNull(str); 
+    str = map.get(anotherPerson);
+    AssertJUnit.assertEquals(str, "testing");
+    str = map.get(person);
+    AssertJUnit.assertEquals(str, "world");
+    
+    Person<Long> third =  (Person<Long>) efproxies[0].create(m_act, null, null, false);
+    third.setAge((short) 31);
+    third.setName("Bob", true);
+
+    str = map.get(third);
+    AssertJUnit.assertEquals(str, "world");
+  }
+
+  @Test(enabled = true)
+  public void testGetPutValueDurable() {
+    
+    DurableType gtypes[] = {DurableType.STRING, DurableType.DURABLE};
+    EntityFactoryProxy efproxies[] = {null, new EntityFactoryProxy() {
+      @Override
+      public <A extends RestorableAllocator<A>> Person<Long> restore(
+          A allocator, EntityFactoryProxy[] factoryproxys,
+          DurableType[] gfields, long phandler, boolean autoreclaim) {
+        return PersonFactory.restore(allocator, factoryproxys, gfields, phandler, autoreclaim);
+      }
+      @Override
+      public <A extends RestorableAllocator<A>> Person<Long> create(
+          A allocator, EntityFactoryProxy[] factoryproxys,
+          DurableType[] gfields, boolean autoreclaim) {
+        return PersonFactory.create(allocator, factoryproxys, gfields, autoreclaim);
+      }
+    } };
+    
+    Person<Long> person =  (Person<Long>) efproxies[1].create(m_act, null, null, false);
+    person.setAge((short) 31);
+    DurableHashMap<String, Person<Long>> map = DurableHashMapFactory.create(m_act, 
+                            efproxies, gtypes, mInitialCapacity, false);
+    map.put("hello", person);
+    
+    Person<Long> per = map.get("hello");
+    AssertJUnit.assertEquals(31, (int)per.getAge()); 
+  }
+
+  @Test(enabled = true)
+  public void testGetPutKeyValueDurable() {
+    
+    DurableType gtypes[] = {DurableType.DURABLE, DurableType.DURABLE};
+    EntityFactoryProxy efproxies[] = {new EntityFactoryProxy() {
+      @Override
+      public <A extends RestorableAllocator<A>> Person<Long> restore(
+          A allocator, EntityFactoryProxy[] factoryproxys,
+          DurableType[] gfields, long phandler, boolean autoreclaim) {
+        return PersonFactory.restore(allocator, factoryproxys, gfields, phandler, autoreclaim);
+      }
+      @Override
+      public <A extends RestorableAllocator<A>> Person<Long> create(
+          A allocator, EntityFactoryProxy[] factoryproxys,
+          DurableType[] gfields, boolean autoreclaim) {
+        return PersonFactory.create(allocator, factoryproxys, gfields, autoreclaim);
+      }
+    }, new EntityFactoryProxy() {
+      @Override
+      public <A extends RestorableAllocator<A>> Person<Long> restore(
+          A allocator, EntityFactoryProxy[] factoryproxys,
+          DurableType[] gfields, long phandler, boolean autoreclaim) {
+        return PersonFactory.restore(allocator, factoryproxys, gfields, phandler, autoreclaim);
+      }
+      @Override
+      public <A extends RestorableAllocator<A>> Person<Long> create(
+          A allocator, EntityFactoryProxy[] factoryproxys,
+          DurableType[] gfields, boolean autoreclaim) {
+        return PersonFactory.create(allocator, factoryproxys, gfields, autoreclaim);
+      }
+    } };
+   
+    Person<Long> person =  (Person<Long>) efproxies[0].create(m_act, null, null, false);
+    person.setAge((short) 31);
+    person.setName("Bob", true);
+
+    Person<Long> anotherPerson =  (Person<Long>) efproxies[1].create(m_act, null, null, false);
+    anotherPerson.setAge((short) 30);
+    anotherPerson.setName("Alice", true);
+
+    DurableHashMap<Person<Long>, Person<Long>> map = DurableHashMapFactory.create(m_act, 
+                            efproxies, gtypes, mInitialCapacity, false);
+    map.put(person, anotherPerson);
+    
+    Person<Long> per = map.get(person);
+    AssertJUnit.assertEquals(30, (int)per.getAge()); 
+    per = map.get(anotherPerson);
+    AssertJUnit.assertNull(per);
+  }
+
+  @Test(enabled = true)
+  public void testGetPutMapOfMapDurable() {
+    DurableType gtypes[] = {DurableType.STRING, DurableType.DURABLE};
+    EntityFactoryProxy efproxies[] = {null, new EntityFactoryProxy() {
+      @Override
+      public <A extends RestorableAllocator<A>> Person<Long> restore(
+          A allocator, EntityFactoryProxy[] factoryproxys,
+          DurableType[] gfields, long phandler, boolean autoreclaim) {
+        return PersonFactory.restore(allocator, factoryproxys, gfields, phandler, autoreclaim);
+      }
+      @Override
+      public <A extends RestorableAllocator<A>> Person<Long> create(
+          A allocator, EntityFactoryProxy[] factoryproxys,
+          DurableType[] gfields, boolean autoreclaim) {
+        return PersonFactory.create(allocator, factoryproxys, gfields, autoreclaim);
+      }
+    } };
+    DurableType mapgtypes[] = {DurableType.STRING, DurableType.DURABLE, DurableType.STRING, DurableType.DURABLE};
+    EntityFactoryProxy mapefproxies[] = {null, new EntityFactoryProxy() {
+      @Override
+      public <A extends RestorableAllocator<A>> Durable restore(
+          A allocator, EntityFactoryProxy[] factoryproxys,
+          DurableType[] gfields, long phandler, boolean autoreclaim) {
+        Pair<DurableType[], EntityFactoryProxy[]> dpt = Utils.shiftDurableParams(gfields, factoryproxys, 2);
+        return DurableHashMapFactory.restore(allocator, dpt.getRight(), dpt.getLeft(), phandler, autoreclaim);
+      }
+      @Override
+      public <A extends RestorableAllocator<A>> Durable create(
+          A allocator, EntityFactoryProxy[] factoryproxys,
+          DurableType[] gfields, boolean autoreclaim) {
+        Pair<DurableType[], EntityFactoryProxy[]> dpt = Utils.shiftDurableParams(gfields, factoryproxys, 2);
+        return DurableHashMapFactory.create(allocator, dpt.getRight(), dpt.getLeft(), mInitialCapacity, autoreclaim);
+      }
+    }, null, new EntityFactoryProxy() {
+      @Override
+      public <A extends RestorableAllocator<A>> Person<Long> restore(
+          A allocator, EntityFactoryProxy[] factoryproxys,
+          DurableType[] gfields, long phandler, boolean autoreclaim) {
+        return PersonFactory.restore(allocator, factoryproxys, gfields, phandler, autoreclaim);
+      }
+      @Override
+      public <A extends RestorableAllocator<A>> Person<Long> create(
+          A allocator, EntityFactoryProxy[] factoryproxys,
+          DurableType[] gfields, boolean autoreclaim) {
+        return PersonFactory.create(allocator, factoryproxys, gfields, autoreclaim);
+      }
+    } };
+    
+    Person<Long> person =  PersonFactory.create(m_act, null, null, false);
+    person.setAge((short) 31);
+    person.setName("Bob", true);
+
+
+    Person<Long> anotherPerson =  (Person<Long>) efproxies[1].create(m_act, null, null, false);
+    anotherPerson.setAge((short) 30);
+    anotherPerson.setName("Alice", true);
+    
+    DurableHashMap<String, Person<Long>> map = DurableHashMapFactory.create(m_act, 
+                            efproxies, gtypes, mInitialCapacity, false);
+    map.put("world", person);
+    Person<Long> per = map.get("world");
+    AssertJUnit.assertEquals(31, (int)per.getAge());
+
+    DurableHashMap<String, DurableHashMap<String, Person<Long>>> bigMap = DurableHashMapFactory.create(m_act,
+                            mapefproxies, mapgtypes, mInitialCapacity, false);
+    bigMap.put("hello", map);
+    per = bigMap.get("hello").get("world");
+    AssertJUnit.assertEquals(31, (int)per.getAge());
+    bigMap.get("hello").put("testing", anotherPerson);
+    per = bigMap.get("hello").get("testing");
+    AssertJUnit.assertEquals("Alice", per.getName());
+  }
+    
+  @Test(enabled = true)
+  public void testAutoResizeMaps() {
+    DurableType gtypes[] = {DurableType.STRING, DurableType.INTEGER};
+    DurableHashMap<String, Integer> map = DurableHashMapFactory.create(m_act, null, gtypes, mInitialCapacity, false);
+    Long handler = map.getHandler();
+    Integer val = 0;
+    for (int i = 0; i < 200; i++) {
+      val = map.put("str" + i, i);
+      AssertJUnit.assertNull(val);
+    }
+    AssertJUnit.assertEquals(map.getSize(), 200);
+    for (int i = 0; i < 200; i++) {
+      AssertJUnit.assertEquals(map.get("str" + i).intValue(), i);
+    } 
+    DurableHashMap<String, Integer> restoredMap = DurableHashMapFactory.restore(m_act, null, gtypes, handler, 
+        false);
+    AssertJUnit.assertEquals(restoredMap.getSize(), 200);
+    for (int i = 0; i < 200; i++) {
+      AssertJUnit.assertEquals(restoredMap.get("str" + i).intValue(), i);
+    } 
+    for (int i = 0; i < 100; i++) {
+      AssertJUnit.assertEquals(restoredMap.remove("str" + i).intValue(), i);
+    }
+    AssertJUnit.assertEquals(restoredMap.getSize(), 100);
+    for (int i = 0; i < 200; i++) {
+      if (i < 100) {
+        AssertJUnit.assertNull(restoredMap.get("str" + i));
+      } else {
+      AssertJUnit.assertEquals(restoredMap.get("str" + i).intValue(), i);
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-mnemonic/blob/be32da85/mnemonic-collections/src/test/java/org/apache/mnemonic/collections/Person.java
----------------------------------------------------------------------
diff --git a/mnemonic-collections/src/test/java/org/apache/mnemonic/collections/Person.java b/mnemonic-collections/src/test/java/org/apache/mnemonic/collections/Person.java
index dc68354..56b1ea4 100644
--- a/mnemonic-collections/src/test/java/org/apache/mnemonic/collections/Person.java
+++ b/mnemonic-collections/src/test/java/org/apache/mnemonic/collections/Person.java
@@ -17,6 +17,8 @@
 
 package org.apache.mnemonic.collections;
 
+import java.util.Objects;
+
 import org.apache.mnemonic.Durable;
 import org.apache.mnemonic.EntityFactoryProxy;
 import org.apache.mnemonic.DurableEntity;
@@ -38,12 +40,12 @@ public abstract class Person<E> implements Durable, Comparable<Person<E>> {
 
   @Override
   public void initializeAfterCreate() {
-    System.out.println("Initializing After Created");
+    //System.out.println("Initializing After Created");
   }
 
   @Override
   public void initializeAfterRestore() {
-    System.out.println("Initializing After Restored");
+    //System.out.println("Initializing After Restored");
   }
 
   @Override
@@ -68,6 +70,14 @@ public abstract class Person<E> implements Durable, Comparable<Person<E>> {
     return ret;
   }
 
+  public int hashCode() {
+    return Objects.hash(getAge(), getName());
+  }
+
+  @Override
+  public boolean equals(Object anotherPerson) {
+    return (0 == this.compareTo((Person<E>)anotherPerson)) ? true : false; 
+  }
   @DurableGetter(Id = 1L)
   public abstract Short getAge();
 

http://git-wip-us.apache.org/repos/asf/incubator-mnemonic/blob/be32da85/mnemonic-core/src/main/java/org/apache/mnemonic/AnnotatedDurableEntityClass.java
----------------------------------------------------------------------
diff --git a/mnemonic-core/src/main/java/org/apache/mnemonic/AnnotatedDurableEntityClass.java b/mnemonic-core/src/main/java/org/apache/mnemonic/AnnotatedDurableEntityClass.java
index c4a7403..84fd9ab 100644
--- a/mnemonic-core/src/main/java/org/apache/mnemonic/AnnotatedDurableEntityClass.java
+++ b/mnemonic-core/src/main/java/org/apache/mnemonic/AnnotatedDurableEntityClass.java
@@ -606,18 +606,18 @@ public class AnnotatedDurableEntityClass {
           code.addStatement("$1N = null", dynfieldinfo.name);
           code.addStatement("$1N.putAddress($2N.get() + $3L, 0L)", unsafename, holdername, dynfieldinfo.fieldoff);
           code.endControlFlow();
-          code.beginControlFlow("if ($1L.getAllocator() != $2N)", arg0, allocname);
+          code.beginControlFlow("if ((null != $1L) && ($1L.getAllocator() != $2N))", arg0, allocname);
           code.addStatement("throw new IllegalAllocatorError(\"This chunk is allocated by another allocator!\")");
           code.endControlFlow();
-          code.beginControlFlow("if (null != $1L)", arg0);
           code.addStatement("$1N = $2L", dynfieldinfo.name, arg0);
+          code.addStatement("$1N.putLong($2N.get() + $3L, null == $4N ? 0L : $4N.getHandler())",
+              unsafename, holdername, dynfieldinfo.fieldoff, dynfieldinfo.name);
+          code.beginControlFlow("if (null != $1L)", dynfieldinfo.name);
           code.beginControlFlow("if ($1N)", autoreclaimname);
           code.addStatement("$1N.registerAutoReclaim();", dynfieldinfo.name);
           code.nextControlFlow("else");
           code.addStatement("$1N.cancelAutoReclaim();", dynfieldinfo.name);
           code.endControlFlow();
-          code.addStatement("$1N.putLong($2N.get() + $3L, null == $4N ? 0L : $4N.getHandler())",
-              unsafename, holdername, dynfieldinfo.fieldoff, dynfieldinfo.name);
           code.endControlFlow();
         } else if (valtname.toString().startsWith(DurableBuffer.class.getCanonicalName())) {
           code.beginControlFlow("if ($1L && null != $2L())", arg1, gsetterName(name, true));
@@ -625,18 +625,18 @@ public class AnnotatedDurableEntityClass {
           code.addStatement("$1N = null", dynfieldinfo.name);
           code.addStatement("$1N.putAddress($2N.get() + $3L, 0L)", unsafename, holdername, dynfieldinfo.fieldoff);
           code.endControlFlow();
-          code.beginControlFlow("if ($1L.getAllocator() != $2N)", arg0, allocname);
+          code.beginControlFlow("if ((null != $1L) && ($1L.getAllocator() != $2N))", arg0, allocname);
           code.addStatement("throw new IllegalAllocatorError(\"This buffer is allocated by another allocator!\")");
           code.endControlFlow();
-          code.beginControlFlow("if (null != $1L)", arg0);
           code.addStatement("$1N = $2L", dynfieldinfo.name, arg0);
+          code.addStatement("$1N.putLong($2N.get() + $3L, null == $4N ? 0L : $4N.getHandler())",
+              unsafename, holdername, dynfieldinfo.fieldoff, dynfieldinfo.name);
+          code.beginControlFlow("if (null != $1L)", dynfieldinfo.name);
           code.beginControlFlow("if ($1N)", autoreclaimname);
           code.addStatement("$1N.registerAutoReclaim();", dynfieldinfo.name);
           code.nextControlFlow("else");
           code.addStatement("$1N.cancelAutoReclaim();", dynfieldinfo.name);
           code.endControlFlow();
-          code.addStatement("$1N.putLong($2N.get() + $3L, null == $4N ? 0L : $4N.getHandler())",
-              unsafename, holdername, dynfieldinfo.fieldoff, dynfieldinfo.name);
           code.endControlFlow();
         } else if (valtname.toString().equals(String.class.getCanonicalName())) {
           code.beginControlFlow("if ($1L && null != $2L())", arg1, gsetterName(name, true));
@@ -684,15 +684,15 @@ public class AnnotatedDurableEntityClass {
           code.addStatement("$1N = null", dynfieldinfo.name);
           code.addStatement("$1N.putAddress($2N.get() + $3L, 0L)", unsafename, holdername, dynfieldinfo.fieldoff);
           code.endControlFlow();
-          code.beginControlFlow("if (null != $1L)", arg0);
           code.addStatement("$1N = $2L", dynfieldinfo.name, arg0);
+          code.addStatement("$1N.putLong($2N.get() + $3L, null == $4N ? 0L : $4N.getHandler())", unsafename,
+              holdername, dynfieldinfo.fieldoff, dynfieldinfo.name);
+          code.beginControlFlow("if (null != $1L)", dynfieldinfo.name);
           code.beginControlFlow("if ($1N)", autoreclaimname);
           code.addStatement("$1N.registerAutoReclaim();", dynfieldinfo.name);
           code.nextControlFlow("else");
           code.addStatement("$1N.cancelAutoReclaim();", dynfieldinfo.name);
           code.endControlFlow();
-          code.addStatement("$1N.putLong($2N.get() + $3L, null == $4N ? 0L : $4N.getHandler())", unsafename,
-              holdername, dynfieldinfo.fieldoff, dynfieldinfo.name);
           code.endControlFlow();
         }
       }