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 2016/04/14 03:50:57 UTC

[1/5] incubator-mnemonic git commit: MNEMONIC-20: Formalize the names of annotations, classes and methods

Repository: incubator-mnemonic
Updated Branches:
  refs/heads/master 254238fb9 -> 1e75e2a52


http://git-wip-us.apache.org/repos/asf/incubator-mnemonic/blob/679bcdc1/core/src/main/java/org/apache/mnemonic/VolatileMemAllocator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/mnemonic/VolatileMemAllocator.java b/core/src/main/java/org/apache/mnemonic/VolatileMemAllocator.java
new file mode 100644
index 0000000..ef13a07
--- /dev/null
+++ b/core/src/main/java/org/apache/mnemonic/VolatileMemAllocator.java
@@ -0,0 +1,278 @@
+/*
+ * 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;
+
+import java.nio.ByteBuffer;
+
+import org.apache.mnemonic.service.allocatorservice.VolatileMemoryAllocatorService;
+import org.flowcomputing.commons.resgc.ResCollector;
+import org.flowcomputing.commons.resgc.ResReclaim;
+
+/**
+ * manage a big native memory pool through libvmem.so that is provied by Intel
+ * nvml library.
+ * 
+ *
+ */
+public class VolatileMemAllocator extends CommonAllocator<VolatileMemAllocator> {
+
+  private boolean m_activegc = true;
+  private long m_gctimeout = 100;
+  private long m_nid = -1;
+  private VolatileMemoryAllocatorService m_vmasvc = null;
+
+  /**
+   * Constructor, it initializes and allocate a memory pool from specified uri
+   * location with specified capacity and an allocator service instance.
+   * usually, the uri points to a mounted memory device or a location of file
+   * system.
+   * 
+   * @param vmasvc
+   *          the volatile memory allocation service instance
+   *
+   * @param capacity
+   *          the capacity of memory pool
+   * 
+   * @param uri
+   *          the location of memory pool will be created
+   * 
+   * @param isnew
+   *          a place holder, always specify it as true
+   */
+  public VolatileMemAllocator(VolatileMemoryAllocatorService vmasvc, long capacity, String uri, boolean isnew) {
+    assert null != vmasvc : "VolatileMemoryAllocatorService object is null";
+    if (capacity <= 0) {
+      throw new IllegalArgumentException("BigDataMemAllocator cannot be initialized with capacity <= 0.");
+    }
+
+    m_vmasvc = vmasvc;
+    m_nid = m_vmasvc.init(capacity, uri, isnew);
+
+    /**
+     * create a resource collector to release specified bytebuffer that backed
+     * by underlying big memory pool.
+     */
+    m_bufcollector = new ResCollector<MemBufferHolder<VolatileMemAllocator>, ByteBuffer>(new ResReclaim<ByteBuffer>() {
+      @Override
+      public void reclaim(ByteBuffer mres) {
+        boolean cb_reclaimed = false;
+        if (null != m_bufferreclaimer) {
+          cb_reclaimed = m_bufferreclaimer.reclaim(mres, Long.valueOf(mres.capacity()));
+        }
+        if (!cb_reclaimed) {
+          m_vmasvc.destroyByteBuffer(m_nid, mres);
+          mres = null;
+        }
+      }
+    });
+
+    /**
+     * create a resource collector to release specified chunk that backed by
+     * underlying big memory pool.
+     */
+    m_chunkcollector = new ResCollector<MemChunkHolder<VolatileMemAllocator>, Long>(new ResReclaim<Long>() {
+      @Override
+      public void reclaim(Long mres) {
+        // System.out.println(String.format("Reclaim: %X", mres));
+        boolean cb_reclaimed = false;
+        if (null != m_chunkreclaimer) {
+          cb_reclaimed = m_chunkreclaimer.reclaim(mres, null);
+        }
+        if (!cb_reclaimed) {
+          m_vmasvc.free(m_nid, mres);
+          mres = null;
+        }
+      }
+    });
+  }
+
+  /**
+   * enable active garbage collection. the GC will be forced to collect garbages
+   * when there is no more space for current allocation request.
+   *
+   * @param timeout
+   *          the timeout is used to yield for GC performing
+   *
+   * @return this allocator
+   */
+  @Override
+  public VolatileMemAllocator enableActiveGC(long timeout) {
+    m_activegc = true;
+    m_gctimeout = timeout;
+    return this;
+  }
+
+  /**
+   * disable active garbage collection.
+   *
+   * @return this allocator
+   */
+  @Override
+  public VolatileMemAllocator disableActiveGC() {
+    m_activegc = false;
+    return this;
+  }
+
+  /**
+   * release the memory pool and close it.
+   *
+   */
+  @Override
+  public void close() {
+    super.close();
+  }
+
+  /**
+   * force to synchronize uncommitted data to backed memory pool (this is a
+   * placeholder).
+   *
+   */
+  @Override
+  public void sync() {
+  }
+
+  /**
+   * re-size a specified chunk on its backed memory pool.
+   * 
+   * @param mholder
+   *          the holder of memory chunk. it can be null.
+   * 
+   * @param size
+   *          specify a new size of memory chunk
+   * 
+   * @return the resized memory chunk handler
+   */
+  @Override
+  public MemChunkHolder<VolatileMemAllocator> resizeChunk(MemChunkHolder<VolatileMemAllocator> mholder, long size) {
+    MemChunkHolder<VolatileMemAllocator> ret = null;
+    boolean ac = null != mholder.getRefId();
+    if (size > 0) {
+      Long addr = m_vmasvc.reallocate(m_nid, mholder.get(), size, true);
+      if (0 == addr && m_activegc) {
+        m_chunkcollector.waitReclaimCoolDown(m_gctimeout);
+        addr = m_vmasvc.reallocate(m_nid, mholder.get(), size, true);
+      }
+      if (0 != addr) {
+        mholder.clear();
+        mholder.destroy();
+        ret = new MemChunkHolder<VolatileMemAllocator>(this, addr, size);
+        if (ac) {
+          m_chunkcollector.register(ret);
+        }
+      }
+    }
+    return ret;
+  }
+
+  /**
+   * resize a specified buffer on its backed memory pool.
+   *
+   * @param mholder
+   *          the holder of memory buffer. it can be null.
+   * 
+   * @param size
+   *          specify a new size of memory chunk
+   * 
+   * @return the resized memory buffer handler
+   *
+   */
+  @Override
+  public MemBufferHolder<VolatileMemAllocator> resizeBuffer(MemBufferHolder<VolatileMemAllocator> mholder, long size) {
+    MemBufferHolder<VolatileMemAllocator> ret = null;
+    boolean ac = null != mholder.getRefId();
+    if (size > 0) {
+      int bufpos = mholder.get().position();
+      int buflimit = mholder.get().limit();
+      ByteBuffer buf = m_vmasvc.resizeByteBuffer(m_nid, mholder.get(), size);
+      if (null == buf && m_activegc) {
+        m_bufcollector.waitReclaimCoolDown(m_gctimeout);
+        buf = m_vmasvc.resizeByteBuffer(m_nid, mholder.get(), size);
+      }
+      if (null != buf) {
+        mholder.clear();
+        mholder.destroy();
+        buf.position(bufpos <= size ? bufpos : 0);
+        buf.limit(buflimit <= size ? buflimit : (int) size);
+        ret = new MemBufferHolder<VolatileMemAllocator>(this, buf);
+        if (ac) {
+          m_bufcollector.register(ret);
+        }
+      }
+    }
+    return ret;
+  }
+
+  /**
+   * create a memory chunk that is managed by its holder.
+   * 
+   * @param size
+   *          specify the size of memory chunk
+   * 
+   * @param autoreclaim
+   *          specify whether or not to reclaim this chunk automatically
+   *
+   * @return a holder contains a memory chunk
+   */
+  @Override
+  public MemChunkHolder<VolatileMemAllocator> createChunk(long size, boolean autoreclaim) {
+    MemChunkHolder<VolatileMemAllocator> ret = null;
+    Long addr = m_vmasvc.allocate(m_nid, size, true);
+    if (0 == addr && m_activegc) {
+      m_chunkcollector.waitReclaimCoolDown(m_gctimeout);
+      addr = m_vmasvc.allocate(m_nid, size, true);
+    }
+    if (0 != addr) {
+      ret = new MemChunkHolder<VolatileMemAllocator>(this, addr, size);
+      ret.setCollector(m_chunkcollector);
+      if (autoreclaim) {
+        m_chunkcollector.register(ret);
+      }
+    }
+    return ret;
+  }
+
+  /**
+   * create a memory buffer that is managed by its holder.
+   * 
+   * @param size
+   *          specify the size of memory buffer
+   * 
+   * @param autoreclaim
+   *          specify whether or not to reclaim this buffer automatically
+   *
+   * @return a holder contains a memory buffer
+   */
+  @Override
+  public MemBufferHolder<VolatileMemAllocator> createBuffer(long size, boolean autoreclaim) {
+    MemBufferHolder<VolatileMemAllocator> ret = null;
+    ByteBuffer bb = m_vmasvc.createByteBuffer(m_nid, size);
+    if (null == bb && m_activegc) {
+      m_bufcollector.waitReclaimCoolDown(m_gctimeout);
+      bb = m_vmasvc.createByteBuffer(m_nid, size);
+    }
+    if (null != bb) {
+      ret = new MemBufferHolder<VolatileMemAllocator>(this, bb);
+      ret.setCollector(m_bufcollector);
+      if (autoreclaim) {
+        m_bufcollector.register(ret);
+      }
+    }
+    return ret;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-mnemonic/blob/679bcdc1/core/src/main/resources/META-INFO/services/javax.annotation.processing.Processor
----------------------------------------------------------------------
diff --git a/core/src/main/resources/META-INFO/services/javax.annotation.processing.Processor b/core/src/main/resources/META-INFO/services/javax.annotation.processing.Processor
index 6981d38..25ba444 100644
--- a/core/src/main/resources/META-INFO/services/javax.annotation.processing.Processor
+++ b/core/src/main/resources/META-INFO/services/javax.annotation.processing.Processor
@@ -1 +1 @@
-org.apache.mnemonic.NonVolatileEntityProcessor
+org.apache.mnemonic.DurableEntityProcessor

http://git-wip-us.apache.org/repos/asf/incubator-mnemonic/blob/679bcdc1/core/src/test/java/org/apache/mnemonic/BigDataMemAllocatorNGTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/mnemonic/BigDataMemAllocatorNGTest.java b/core/src/test/java/org/apache/mnemonic/BigDataMemAllocatorNGTest.java
deleted file mode 100644
index 6453ec0..0000000
--- a/core/src/test/java/org/apache/mnemonic/BigDataMemAllocatorNGTest.java
+++ /dev/null
@@ -1,68 +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 org.apache.mnemonic;
-
-import org.testng.annotations.Test;
-
-import java.util.Random;
-
-/**
- * test the functionality of BigMemAllocator class.
- * 
- */
-public class BigDataMemAllocatorNGTest {
-  /**
-   * test to allocate MemBufferHolder objects and then verify them.
-   */
-  @Test
-  public void testMemByteBuffer() {
-    Random randomGenerator = new Random();
-    Allocator<BigDataMemAllocator> act = new BigDataMemAllocator(Utils.getVolatileMemoryAllocatorService("vmem"),
-        1024 * 1024 * 1024, ".", true);
-    MemBufferHolder<?> mbh;
-    for (int idx = 1; idx <= 5; ++idx) {
-      int size = randomGenerator.nextInt(1024 * 1024) + 1024 * 1024;
-      mbh = act.createBuffer(size);
-      for (int i = 0; i < size; i++) {
-        mbh.get().put((byte) randomGenerator.nextInt(255));
-      }
-      // if (bb.hasArray()) randomGenerator.nextBytes(bb.array());
-      System.out.println(String.format("[Seq.%d] size %d - %d, (%s)", idx, size, mbh.get().capacity(),
-          size == mbh.get().capacity() ? "Correct" : "Failed!!!"));
-      // mbh.destroy();
-    }
-  }
-
-  /**
-   * test to allocate MemChunkHolder objects and then verify them.
-   */
-  @Test
-  public void testMemChunk() {
-    Random randomGenerator = new Random();
-    Allocator<BigDataMemAllocator> act = new BigDataMemAllocator(Utils.getVolatileMemoryAllocatorService("vmem"),
-        1024 * 1024 * 1024, ".", true);
-    MemChunkHolder<?> mch;
-    for (int idx = 1; idx <= 5; ++idx) {
-      int size = randomGenerator.nextInt(1024 * 1024) + 1024 * 1024;
-      mch = act.createChunk(size);
-      System.out.println(String.format("[Seq.%d] addr : %X", idx, size, mch.get()));
-      mch.destroy();
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-mnemonic/blob/679bcdc1/core/src/test/java/org/apache/mnemonic/BigDataPMemAllocatorNGTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/mnemonic/BigDataPMemAllocatorNGTest.java b/core/src/test/java/org/apache/mnemonic/BigDataPMemAllocatorNGTest.java
deleted file mode 100644
index 7de32c5..0000000
--- a/core/src/test/java/org/apache/mnemonic/BigDataPMemAllocatorNGTest.java
+++ /dev/null
@@ -1,132 +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 org.apache.mnemonic;
-
-import java.nio.ByteBuffer;
-import java.util.Random;
-import org.testng.Assert;
-import org.testng.annotations.Test;
-
-/**
- *
- */
-
-public class BigDataPMemAllocatorNGTest {
-  @Test
-  public void testPMemByteBuffer() {
-    Random randomGenerator = new Random();
-    BigDataPMemAllocator act = new BigDataPMemAllocator(Utils.getNonVolatileMemoryAllocatorService("pmalloc"),
-        1024 * 1024 * 1024, "./pmtest.dat", true);
-    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;
-      }
-    });
-    MemBufferHolder<?> mbh;
-    for (int idx = 1; idx <= 500; ++idx) {
-      int size = randomGenerator.nextInt(1024 * 1024) + 1024 * 1024;
-      mbh = act.createBuffer(size);
-      Assert.assertNotNull(mbh);
-      for (int i = 0; i < size; i++) {
-        mbh.get().put((byte) randomGenerator.nextInt(255));
-      }
-      // if (bb.hasArray()) randomGenerator.nextBytes(bb.array());
-      Assert.assertEquals(size, mbh.get().capacity());
-      System.out.println(String.format("[Seq.%d] size %d - %d, (%s)", idx, size, mbh.get().capacity(),
-          size == mbh.get().capacity() ? "Correct" : "Failed!!!"));
-      // mbh.destroy();
-    }
-    act.close();
-  }
-
-  @Test
-  public void testGetBufferAddress() {
-    BigDataPMemAllocator act = new BigDataPMemAllocator(Utils.getNonVolatileMemoryAllocatorService("pmalloc"),
-        1024 * 1024 * 1024, "./pmtest_buffer.dat", true);
-    MemBufferHolder<BigDataPMemAllocator> mbh;
-    mbh = act.createBuffer(20000);
-    long phandler = act.getBufferHandler(mbh);
-    System.out.println(String.format("**** 0x%X", phandler));
-    act.close();
-  }
-
-  @Test
-  public void testGenPMemByteBufferWithKey() {
-    Random randomGenerator = Utils.createRandom();
-    BigDataPMemAllocator act = new BigDataPMemAllocator(Utils.getNonVolatileMemoryAllocatorService("pmalloc"),
-        1024 * 1024 * 1024, "./pmtest_key.dat", true);
-    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;
-      }
-    });
-    MemBufferHolder<BigDataPMemAllocator> mbh;
-    Long phandler;
-    long keycount = act.handlerCapacity();
-    for (int idx = 0; idx < keycount; ++idx) {
-      int size = randomGenerator.nextInt(1024 * 1024) + 1024 * 1024;
-      mbh = act.createBuffer(size);
-      if (6 == idx) {
-        size += 2000;
-        mbh = mbh.resize(size);
-      }
-      Assert.assertNotNull(mbh);
-      mbh.get().putInt(size);
-      Assert.assertEquals(size, mbh.get().capacity());
-      System.out.println(String.format("Generating PKey Value [Seq.%d] size %d - %d, (%s)", idx, size,
-          mbh.get().capacity(), size == mbh.get().capacity() ? "Correct" : "Failed!!!"));
-      phandler = act.getBufferHandler(mbh);
-      System.out.println(String.format("---- 0x%X", phandler));
-      act.setHandler(idx, phandler);
-      mbh.cancelAutoReclaim();
-    }
-    act.close();
-  }
-
-  @Test(dependsOnMethods = { "testGenPMemByteBufferWithKey" })
-  public void testCheckPMemByteBufferWithKey() {
-    BigDataPMemAllocator act = new BigDataPMemAllocator(Utils.getNonVolatileMemoryAllocatorService("pmalloc"),
-        1024 * 1024 * 1024, "./pmtest_key.dat", true);
-    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;
-      }
-    });
-    MemBufferHolder<BigDataPMemAllocator> mbh;
-    for (int idx = 0; idx < act.handlerCapacity(); ++idx) {
-      long phandler = act.getHandler(idx);
-      mbh = act.retrieveBuffer(phandler);
-      Assert.assertNotNull(mbh);
-      int val = mbh.get().getInt();
-      Assert.assertEquals(val, mbh.get().capacity());
-      System.out.println(String.format("Checking PKey Value [Seq.%d] size %d - %d, (%s)", idx, val,
-          mbh.get().capacity(), val == mbh.get().capacity() ? "Correct" : "Failed!!!"));
-    }
-    act.close();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-mnemonic/blob/679bcdc1/core/src/test/java/org/apache/mnemonic/ByteBufferSerializerNGTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/mnemonic/ByteBufferSerializerNGTest.java b/core/src/test/java/org/apache/mnemonic/ByteBufferSerializerNGTest.java
index 1307504..1e652c0 100644
--- a/core/src/test/java/org/apache/mnemonic/ByteBufferSerializerNGTest.java
+++ b/core/src/test/java/org/apache/mnemonic/ByteBufferSerializerNGTest.java
@@ -52,14 +52,14 @@ public class ByteBufferSerializerNGTest {
    */
   @Test
   public void testToFromMemBufferHolder() throws IOException, ClassNotFoundException {
-    BigDataMemAllocator act = new BigDataMemAllocator(Utils.getVolatileMemoryAllocatorService("vmem"),
+    VolatileMemAllocator act = new VolatileMemAllocator(Utils.getVolatileMemoryAllocatorService("vmem"),
         1024 * 1024 * 1024, ".", true);
 
     Random randomGenerator = new Random();
     for (int idx = 0; idx < 100; idx++) {
       Payload pl = new Payload(randomGenerator.nextInt(1024 * 1024),
           String.format("Str is %d", randomGenerator.nextInt(1024 * 1024)), randomGenerator.nextDouble());
-      MemBufferHolder<BigDataMemAllocator> mbh = ByteBufferSerializer.toMemBufferHolder(act, pl);
+      MemBufferHolder<VolatileMemAllocator> mbh = ByteBufferSerializer.toMemBufferHolder(act, pl);
       Payload rpl = ByteBufferSerializer.fromMemBufferHolder(mbh);
       mbh.destroy();
       assertTrue(pl.compareTo(rpl) == 0);

http://git-wip-us.apache.org/repos/asf/incubator-mnemonic/blob/679bcdc1/core/src/test/java/org/apache/mnemonic/DurablePersonNGTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/mnemonic/DurablePersonNGTest.java b/core/src/test/java/org/apache/mnemonic/DurablePersonNGTest.java
new file mode 100644
index 0000000..e66c66f
--- /dev/null
+++ b/core/src/test/java/org/apache/mnemonic/DurablePersonNGTest.java
@@ -0,0 +1,144 @@
+/*
+ * 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;
+
+/**
+ *
+ *
+ */
+
+import java.nio.ByteBuffer;
+import java.util.Random;
+import java.util.UUID;
+
+import org.testng.annotations.Test;
+
+public class DurablePersonNGTest {
+  private long cKEYCAPACITY;
+
+  @Test(expectedExceptions = { OutOfHybridMemory.class })
+  public void testGenPeople() throws OutOfHybridMemory, RetrieveDurableEntityError {
+    Random rand = Utils.createRandom();
+    NonVolatileMemAllocator act = new NonVolatileMemAllocator(Utils.getNonVolatileMemoryAllocatorService("pmalloc"),
+        1024 * 1024 * 8, "./pobj_person.dat", true);
+    cKEYCAPACITY = act.handlerCapacity();
+    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;
+      }
+    });
+    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) {
+      act.setHandler(i, 0L);
+    }
+
+    Person<Integer> mother;
+    Person<Integer> person;
+
+    long keyidx = 0;
+    long val;
+
+    try {
+      while (true) {
+        // if (keyidx >= KEYCAPACITY) break;
+
+        keyidx %= cKEYCAPACITY;
+
+        System.out.printf("************ Generating People on Key %d ***********\n", keyidx);
+
+        val = act.getHandler(keyidx);
+        if (0L != val) {
+          PersonFactory.restore(act, val, true);
+        }
+
+        person = PersonFactory.create(act);
+        person.setAge((short) rand.nextInt(50));
+        person.setName(String.format("Name: [%s]", UUID.randomUUID().toString()), true);
+        person.setName(String.format("Name: [%s]", UUID.randomUUID().toString()), true);
+        person.setName(String.format("Name: [%s]", UUID.randomUUID().toString()), true);
+        person.setName(String.format("Name: [%s]", UUID.randomUUID().toString()), true);
+
+        act.setHandler(keyidx, person.getHandler());
+
+        for (int deep = 0; deep < rand.nextInt(100); ++deep) {
+
+          mother = PersonFactory.create(act);
+          mother.setAge((short) (50 + rand.nextInt(50)));
+          mother.setName(String.format("Name: [%s]", UUID.randomUUID().toString()), true);
+
+          person.setMother(mother, true);
+
+          person = mother;
+
+        }
+        ++keyidx;
+      }
+    } finally {
+      act.close();
+    }
+  }
+
+  @Test(dependsOnMethods = { "testGenPeople" })
+  public void testCheckPeople() throws RetrieveDurableEntityError {
+    NonVolatileMemAllocator act = new NonVolatileMemAllocator(Utils.getNonVolatileMemoryAllocatorService("pmalloc"),
+        1024 * 1024 * 8, "./pobj_person.dat", true);
+    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;
+      }
+    });
+    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;
+      }
+    });
+
+    long val;
+    for (long i = 0; i < cKEYCAPACITY; ++i) {
+      System.out.printf("----------Key %d--------------\n", i);
+      val = act.getHandler(i);
+      if (0L == val) {
+        break;
+      }
+      Person<Integer> person = PersonFactory.restore(act, val, true);
+      while (null != person) {
+        person.testOutput();
+        person = person.getMother();
+      }
+    }
+
+    act.close();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-mnemonic/blob/679bcdc1/core/src/test/java/org/apache/mnemonic/MemBufferHolderCachePoolNGTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/mnemonic/MemBufferHolderCachePoolNGTest.java b/core/src/test/java/org/apache/mnemonic/MemBufferHolderCachePoolNGTest.java
index 723f571..c79ec8a 100644
--- a/core/src/test/java/org/apache/mnemonic/MemBufferHolderCachePoolNGTest.java
+++ b/core/src/test/java/org/apache/mnemonic/MemBufferHolderCachePoolNGTest.java
@@ -39,8 +39,8 @@ public class MemBufferHolderCachePoolNGTest {
     MemBufferHolderCachePool<Integer> mbhcpool = new MemBufferHolderCachePool<Integer>(1024 * 1024 * 10);
     Random randomGenerator = new Random();
     MemClustering.NodeConfig<?> ncs[] = new MemClustering.NodeConfig<?>[] {
-        new MemClustering.NodeConfig<BigDataMemAllocator>(
-            new BigDataMemAllocator(Utils.getVolatileMemoryAllocatorService("vmem"), 1024 * 1024 * 200, ".", true)
+        new MemClustering.NodeConfig<VolatileMemAllocator>(
+            new VolatileMemAllocator(Utils.getVolatileMemoryAllocatorService("vmem"), 1024 * 1024 * 200, ".", true)
                 .disableActiveGC(),
             MemClustering.PerformanceLevel.FAST),
         // new MemClustering.NodeConfig(new BigMemAllocator(1024*1024*20, ".",
@@ -97,8 +97,8 @@ public class MemBufferHolderCachePoolNGTest {
     MemBufferHolderCachePool<Integer> mbhcpool = new MemBufferHolderCachePool<Integer>(1024 * 1024 * 10);
     Random randomGenerator = new Random();
     MemClustering.NodeConfig<?> ncs[] = new MemClustering.NodeConfig<?>[] {
-        new MemClustering.NodeConfig<BigDataMemAllocator>(
-            new BigDataMemAllocator(Utils.getVolatileMemoryAllocatorService("vmem"), 1024 * 1024 * 200, ".", true)
+        new MemClustering.NodeConfig<VolatileMemAllocator>(
+            new VolatileMemAllocator(Utils.getVolatileMemoryAllocatorService("vmem"), 1024 * 1024 * 200, ".", true)
                 .disableActiveGC(),
             MemClustering.PerformanceLevel.FAST),
         // new MemClustering.NodeConfig(new BigMemAllocator(1024*1024*20, ".",

http://git-wip-us.apache.org/repos/asf/incubator-mnemonic/blob/679bcdc1/core/src/test/java/org/apache/mnemonic/MemClusteringNGTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/mnemonic/MemClusteringNGTest.java b/core/src/test/java/org/apache/mnemonic/MemClusteringNGTest.java
index f62bfd8..0c90f30 100644
--- a/core/src/test/java/org/apache/mnemonic/MemClusteringNGTest.java
+++ b/core/src/test/java/org/apache/mnemonic/MemClusteringNGTest.java
@@ -40,15 +40,15 @@ public class MemClusteringNGTest {
     MemClustering.NodeConfig<?> ncs[] = new MemClustering.NodeConfig<?>[] {
         new MemClustering.NodeConfig<SysMemAllocator>(new SysMemAllocator(1024 * 1024 * 20, true).disableActiveGC(),
             MemClustering.PerformanceLevel.FASTEST),
-        new MemClustering.NodeConfig<BigDataMemAllocator>(
-            new BigDataMemAllocator(Utils.getVolatileMemoryAllocatorService("vmem"), 1024 * 1024 * 20, ".", true)
+        new MemClustering.NodeConfig<VolatileMemAllocator>(
+            new VolatileMemAllocator(Utils.getVolatileMemoryAllocatorService("vmem"), 1024 * 1024 * 20, ".", true)
                 .disableActiveGC(),
             MemClustering.PerformanceLevel.FAST),
         // new MemClustering.NodeConfig(new
         // BigMemAllocator(1024*1024*20, ".", true).disableActiveGC(),
         // MemClustering.PerformanceLevel.NORMAL),
-        new MemClustering.NodeConfig<BigDataMemAllocator>(
-            new BigDataMemAllocator(Utils.getVolatileMemoryAllocatorService("vmem"), 1024 * 1024 * 20, ".", true)
+        new MemClustering.NodeConfig<VolatileMemAllocator>(
+            new VolatileMemAllocator(Utils.getVolatileMemoryAllocatorService("vmem"), 1024 * 1024 * 20, ".", true)
                 .disableActiveGC(),
             MemClustering.PerformanceLevel.SLOW), };
     MemClustering mclst = new MemClustering(ncs);
@@ -71,8 +71,8 @@ public class MemClusteringNGTest {
   public void testMemByteBufferWithActiveGC() {
     Random randomGenerator = new Random();
     MemClustering.NodeConfig<?> ncs[] = new MemClustering.NodeConfig<?>[] {
-        new MemClustering.NodeConfig<BigDataMemAllocator>(
-            new BigDataMemAllocator(Utils.getVolatileMemoryAllocatorService("vmem"), 1024 * 1024 * 20, ".", true),
+        new MemClustering.NodeConfig<VolatileMemAllocator>(
+            new VolatileMemAllocator(Utils.getVolatileMemoryAllocatorService("vmem"), 1024 * 1024 * 20, ".", true),
             MemClustering.PerformanceLevel.NORMAL),
         // new MemClustering.NodeConfig(new BigMemAllocator(1024*1024*20, ".",
         // true), MemClustering.PerformanceLevel.SLOW),
@@ -97,15 +97,15 @@ public class MemClusteringNGTest {
   public void testMemByteBufferManualRelease() {
     Random randomGenerator = new Random();
     MemClustering.NodeConfig<?> ncs[] = new MemClustering.NodeConfig<?>[] {
-        new MemClustering.NodeConfig<BigDataMemAllocator>(
-            new BigDataMemAllocator(Utils.getVolatileMemoryAllocatorService("vmem"), 1024 * 1024 * 20, ".", true)
+        new MemClustering.NodeConfig<VolatileMemAllocator>(
+            new VolatileMemAllocator(Utils.getVolatileMemoryAllocatorService("vmem"), 1024 * 1024 * 20, ".", true)
                 .disableActiveGC(),
             MemClustering.PerformanceLevel.FAST),
         // new MemClustering.NodeConfig(new
         // BigMemAllocator(1024*1024*20, ".", true).disableActiveGC(),
         // MemClustering.PerformanceLevel.NORMAL),
-        new MemClustering.NodeConfig<BigDataMemAllocator>(
-            new BigDataMemAllocator(Utils.getVolatileMemoryAllocatorService("vmem"), 1024 * 1024 * 20, ".", true),
+        new MemClustering.NodeConfig<VolatileMemAllocator>(
+            new VolatileMemAllocator(Utils.getVolatileMemoryAllocatorService("vmem"), 1024 * 1024 * 20, ".", true),
             MemClustering.PerformanceLevel.SLOW), };
     MemClustering mclst = new MemClustering(ncs);
     MemBufferHolder<?> mbh;
@@ -135,15 +135,15 @@ public class MemClusteringNGTest {
     MemClustering.NodeConfig<?> ncs[] = new MemClustering.NodeConfig<?>[] {
         new MemClustering.NodeConfig<SysMemAllocator>(new SysMemAllocator(1024 * 1024 * 20, true).disableActiveGC(),
             MemClustering.PerformanceLevel.FASTEST),
-        new MemClustering.NodeConfig<BigDataMemAllocator>(
-            new BigDataMemAllocator(Utils.getVolatileMemoryAllocatorService("vmem"), 1024 * 1024 * 20, ".", true)
+        new MemClustering.NodeConfig<VolatileMemAllocator>(
+            new VolatileMemAllocator(Utils.getVolatileMemoryAllocatorService("vmem"), 1024 * 1024 * 20, ".", true)
                 .disableActiveGC(),
             MemClustering.PerformanceLevel.FAST),
         // new MemClustering.NodeConfig(new
         // BigMemAllocator(1024*1024*20, ".", true).disableActiveGC(),
         // MemClustering.PerformanceLevel.NORMAL),
-        new MemClustering.NodeConfig<BigDataMemAllocator>(
-            new BigDataMemAllocator(Utils.getVolatileMemoryAllocatorService("vmem"), 1024 * 1024 * 20, ".", true),
+        new MemClustering.NodeConfig<VolatileMemAllocator>(
+            new VolatileMemAllocator(Utils.getVolatileMemoryAllocatorService("vmem"), 1024 * 1024 * 20, ".", true),
             MemClustering.PerformanceLevel.SLOW), };
     MemClustering mclst = new MemClustering(ncs) {
     };

http://git-wip-us.apache.org/repos/asf/incubator-mnemonic/blob/679bcdc1/core/src/test/java/org/apache/mnemonic/NonVolatileMemAllocatorNGTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/mnemonic/NonVolatileMemAllocatorNGTest.java b/core/src/test/java/org/apache/mnemonic/NonVolatileMemAllocatorNGTest.java
new file mode 100644
index 0000000..48b6be5
--- /dev/null
+++ b/core/src/test/java/org/apache/mnemonic/NonVolatileMemAllocatorNGTest.java
@@ -0,0 +1,132 @@
+/*
+ * 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;
+
+import java.nio.ByteBuffer;
+import java.util.Random;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+/**
+ *
+ */
+
+public class NonVolatileMemAllocatorNGTest {
+  @Test
+  public void testPMemByteBuffer() {
+    Random randomGenerator = new Random();
+    NonVolatileMemAllocator act = new NonVolatileMemAllocator(Utils.getNonVolatileMemoryAllocatorService("pmalloc"),
+        1024 * 1024 * 1024, "./pmtest.dat", true);
+    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;
+      }
+    });
+    MemBufferHolder<?> mbh;
+    for (int idx = 1; idx <= 500; ++idx) {
+      int size = randomGenerator.nextInt(1024 * 1024) + 1024 * 1024;
+      mbh = act.createBuffer(size);
+      Assert.assertNotNull(mbh);
+      for (int i = 0; i < size; i++) {
+        mbh.get().put((byte) randomGenerator.nextInt(255));
+      }
+      // if (bb.hasArray()) randomGenerator.nextBytes(bb.array());
+      Assert.assertEquals(size, mbh.get().capacity());
+      System.out.println(String.format("[Seq.%d] size %d - %d, (%s)", idx, size, mbh.get().capacity(),
+          size == mbh.get().capacity() ? "Correct" : "Failed!!!"));
+      // mbh.destroy();
+    }
+    act.close();
+  }
+
+  @Test
+  public void testGetBufferAddress() {
+    NonVolatileMemAllocator act = new NonVolatileMemAllocator(Utils.getNonVolatileMemoryAllocatorService("pmalloc"),
+        1024 * 1024 * 1024, "./pmtest_buffer.dat", true);
+    MemBufferHolder<NonVolatileMemAllocator> mbh;
+    mbh = act.createBuffer(20000);
+    long phandler = act.getBufferHandler(mbh);
+    System.out.println(String.format("**** 0x%X", phandler));
+    act.close();
+  }
+
+  @Test
+  public void testGenPMemByteBufferWithKey() {
+    Random randomGenerator = Utils.createRandom();
+    NonVolatileMemAllocator act = new NonVolatileMemAllocator(Utils.getNonVolatileMemoryAllocatorService("pmalloc"),
+        1024 * 1024 * 1024, "./pmtest_key.dat", true);
+    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;
+      }
+    });
+    MemBufferHolder<NonVolatileMemAllocator> mbh;
+    Long phandler;
+    long keycount = act.handlerCapacity();
+    for (int idx = 0; idx < keycount; ++idx) {
+      int size = randomGenerator.nextInt(1024 * 1024) + 1024 * 1024;
+      mbh = act.createBuffer(size);
+      if (6 == idx) {
+        size += 2000;
+        mbh = mbh.resize(size);
+      }
+      Assert.assertNotNull(mbh);
+      mbh.get().putInt(size);
+      Assert.assertEquals(size, mbh.get().capacity());
+      System.out.println(String.format("Generating PKey Value [Seq.%d] size %d - %d, (%s)", idx, size,
+          mbh.get().capacity(), size == mbh.get().capacity() ? "Correct" : "Failed!!!"));
+      phandler = act.getBufferHandler(mbh);
+      System.out.println(String.format("---- 0x%X", phandler));
+      act.setHandler(idx, phandler);
+      mbh.cancelAutoReclaim();
+    }
+    act.close();
+  }
+
+  @Test(dependsOnMethods = { "testGenPMemByteBufferWithKey" })
+  public void testCheckPMemByteBufferWithKey() {
+    NonVolatileMemAllocator act = new NonVolatileMemAllocator(Utils.getNonVolatileMemoryAllocatorService("pmalloc"),
+        1024 * 1024 * 1024, "./pmtest_key.dat", true);
+    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;
+      }
+    });
+    MemBufferHolder<NonVolatileMemAllocator> mbh;
+    for (int idx = 0; idx < act.handlerCapacity(); ++idx) {
+      long phandler = act.getHandler(idx);
+      mbh = act.retrieveBuffer(phandler);
+      Assert.assertNotNull(mbh);
+      int val = mbh.get().getInt();
+      Assert.assertEquals(val, mbh.get().capacity());
+      System.out.println(String.format("Checking PKey Value [Seq.%d] size %d - %d, (%s)", idx, val,
+          mbh.get().capacity(), val == mbh.get().capacity() ? "Correct" : "Failed!!!"));
+    }
+    act.close();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-mnemonic/blob/679bcdc1/core/src/test/java/org/apache/mnemonic/NonVolatilePersonNGTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/mnemonic/NonVolatilePersonNGTest.java b/core/src/test/java/org/apache/mnemonic/NonVolatilePersonNGTest.java
deleted file mode 100644
index 4dc85bd..0000000
--- a/core/src/test/java/org/apache/mnemonic/NonVolatilePersonNGTest.java
+++ /dev/null
@@ -1,144 +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 org.apache.mnemonic;
-
-/**
- *
- *
- */
-
-import java.nio.ByteBuffer;
-import java.util.Random;
-import java.util.UUID;
-
-import org.testng.annotations.Test;
-
-public class NonVolatilePersonNGTest {
-  private long cKEYCAPACITY;
-
-  @Test(expectedExceptions = { OutOfPersistentMemory.class })
-  public void testGenPeople() throws OutOfPersistentMemory, RetrieveNonVolatileEntityError {
-    Random rand = Utils.createRandom();
-    BigDataPMemAllocator act = new BigDataPMemAllocator(Utils.getNonVolatileMemoryAllocatorService("pmalloc"),
-        1024 * 1024 * 8, "./pobj_person.dat", true);
-    cKEYCAPACITY = act.handlerCapacity();
-    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;
-      }
-    });
-    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) {
-      act.setHandler(i, 0L);
-    }
-
-    Person<Integer> mother;
-    Person<Integer> person;
-
-    long keyidx = 0;
-    long val;
-
-    try {
-      while (true) {
-        // if (keyidx >= KEYCAPACITY) break;
-
-        keyidx %= cKEYCAPACITY;
-
-        System.out.printf("************ Generating People on Key %d ***********\n", keyidx);
-
-        val = act.getHandler(keyidx);
-        if (0L != val) {
-          PersonFactory.restore(act, val, true);
-        }
-
-        person = PersonFactory.create(act);
-        person.setAge((short) rand.nextInt(50));
-        person.setName(String.format("Name: [%s]", UUID.randomUUID().toString()), true);
-        person.setName(String.format("Name: [%s]", UUID.randomUUID().toString()), true);
-        person.setName(String.format("Name: [%s]", UUID.randomUUID().toString()), true);
-        person.setName(String.format("Name: [%s]", UUID.randomUUID().toString()), true);
-
-        act.setHandler(keyidx, person.getNonVolatileHandler());
-
-        for (int deep = 0; deep < rand.nextInt(100); ++deep) {
-
-          mother = PersonFactory.create(act);
-          mother.setAge((short) (50 + rand.nextInt(50)));
-          mother.setName(String.format("Name: [%s]", UUID.randomUUID().toString()), true);
-
-          person.setMother(mother, true);
-
-          person = mother;
-
-        }
-        ++keyidx;
-      }
-    } finally {
-      act.close();
-    }
-  }
-
-  @Test(dependsOnMethods = { "testGenPeople" })
-  public void testCheckPeople() throws RetrieveNonVolatileEntityError {
-    BigDataPMemAllocator act = new BigDataPMemAllocator(Utils.getNonVolatileMemoryAllocatorService("pmalloc"),
-        1024 * 1024 * 8, "./pobj_person.dat", true);
-    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;
-      }
-    });
-    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;
-      }
-    });
-
-    long val;
-    for (long i = 0; i < cKEYCAPACITY; ++i) {
-      System.out.printf("----------Key %d--------------\n", i);
-      val = act.getHandler(i);
-      if (0L == val) {
-        break;
-      }
-      Person<Integer> person = PersonFactory.restore(act, val, true);
-      while (null != person) {
-        person.testOutput();
-        person = person.getMother();
-      }
-    }
-
-    act.close();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-mnemonic/blob/679bcdc1/core/src/test/java/org/apache/mnemonic/Person.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/mnemonic/Person.java b/core/src/test/java/org/apache/mnemonic/Person.java
index 9f5afa6..7d5942e 100644
--- a/core/src/test/java/org/apache/mnemonic/Person.java
+++ b/core/src/test/java/org/apache/mnemonic/Person.java
@@ -22,7 +22,7 @@ package org.apache.mnemonic;
  *
  */
 
-@NonVolatileEntity
+@DurableEntity
 public abstract class Person<E> implements Durable, Comparable<Person<E>> {
   E element;
 
@@ -41,7 +41,7 @@ public abstract class Person<E> implements Durable, Comparable<Person<E>> {
 
   }
 
-  public void testOutput() throws RetrieveNonVolatileEntityError {
+  public void testOutput() throws RetrieveDurableEntityError {
     System.out.printf("Person %s, Age: %d ( %s ) \n", getName(), getAge(),
         null == getMother() ? "No Recorded Mother" : "Has Recorded Mother");
   }
@@ -57,28 +57,28 @@ public abstract class Person<E> implements Durable, Comparable<Person<E>> {
     return ret;
   }
 
-  @NonVolatileGetter
+  @DurableGetter
   public abstract Short getAge();
 
-  @NonVolatileSetter
+  @DurableSetter
   public abstract void setAge(Short age);
 
-  @NonVolatileGetter
-  public abstract String getName() throws RetrieveNonVolatileEntityError;
+  @DurableGetter
+  public abstract String getName() throws RetrieveDurableEntityError;
 
-  @NonVolatileSetter
+  @DurableSetter
   public abstract void setName(String name, boolean destroy)
-      throws OutOfPersistentMemory, RetrieveNonVolatileEntityError;
+      throws OutOfHybridMemory, RetrieveDurableEntityError;
 
-  @NonVolatileGetter
-  public abstract Person<E> getMother() throws RetrieveNonVolatileEntityError;
+  @DurableGetter
+  public abstract Person<E> getMother() throws RetrieveDurableEntityError;
 
-  @NonVolatileSetter
-  public abstract void setMother(Person<E> mother, boolean destroy) throws RetrieveNonVolatileEntityError;
+  @DurableSetter
+  public abstract void setMother(Person<E> mother, boolean destroy) throws RetrieveDurableEntityError;
 
-  @NonVolatileGetter
-  public abstract Person<E> getFather() throws RetrieveNonVolatileEntityError;
+  @DurableGetter
+  public abstract Person<E> getFather() throws RetrieveDurableEntityError;
 
-  @NonVolatileSetter
-  public abstract void setFather(Person<E> mother, boolean destroy) throws RetrieveNonVolatileEntityError;
+  @DurableSetter
+  public abstract void setFather(Person<E> mother, boolean destroy) throws RetrieveDurableEntityError;
 }

http://git-wip-us.apache.org/repos/asf/incubator-mnemonic/blob/679bcdc1/core/src/test/java/org/apache/mnemonic/VolatileAllocatorNGTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/mnemonic/VolatileAllocatorNGTest.java b/core/src/test/java/org/apache/mnemonic/VolatileAllocatorNGTest.java
new file mode 100644
index 0000000..a0d7a5b
--- /dev/null
+++ b/core/src/test/java/org/apache/mnemonic/VolatileAllocatorNGTest.java
@@ -0,0 +1,68 @@
+/*
+ * 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;
+
+import org.testng.annotations.Test;
+
+import java.util.Random;
+
+/**
+ * test the functionality of VolatileMemAllocator class.
+ * 
+ */
+public class VolatileAllocatorNGTest {
+  /**
+   * test to allocate MemBufferHolder objects and then verify them.
+   */
+  @Test
+  public void testMemByteBuffer() {
+    Random randomGenerator = new Random();
+    Allocator<VolatileMemAllocator> act = new VolatileMemAllocator(Utils.getVolatileMemoryAllocatorService("vmem"),
+        1024 * 1024 * 1024, ".", true);
+    MemBufferHolder<?> mbh;
+    for (int idx = 1; idx <= 5; ++idx) {
+      int size = randomGenerator.nextInt(1024 * 1024) + 1024 * 1024;
+      mbh = act.createBuffer(size);
+      for (int i = 0; i < size; i++) {
+        mbh.get().put((byte) randomGenerator.nextInt(255));
+      }
+      // if (bb.hasArray()) randomGenerator.nextBytes(bb.array());
+      System.out.println(String.format("[Seq.%d] size %d - %d, (%s)", idx, size, mbh.get().capacity(),
+          size == mbh.get().capacity() ? "Correct" : "Failed!!!"));
+      // mbh.destroy();
+    }
+  }
+
+  /**
+   * test to allocate MemChunkHolder objects and then verify them.
+   */
+  @Test
+  public void testMemChunk() {
+    Random randomGenerator = new Random();
+    Allocator<VolatileMemAllocator> act = new VolatileMemAllocator(Utils.getVolatileMemoryAllocatorService("vmem"),
+        1024 * 1024 * 1024, ".", true);
+    MemChunkHolder<?> mch;
+    for (int idx = 1; idx <= 5; ++idx) {
+      int size = randomGenerator.nextInt(1024 * 1024) + 1024 * 1024;
+      mch = act.createChunk(size);
+      System.out.println(String.format("[Seq.%d] addr : %X", idx, size, mch.get()));
+      mch.destroy();
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-mnemonic/blob/679bcdc1/core/src/test/resources/testng.xml
----------------------------------------------------------------------
diff --git a/core/src/test/resources/testng.xml b/core/src/test/resources/testng.xml
index 2eda957..495811b 100644
--- a/core/src/test/resources/testng.xml
+++ b/core/src/test/resources/testng.xml
@@ -3,18 +3,18 @@
 <suite name="Mnenomic Core test Suite" verbose="1" parallel="tests" thread-count="1">
 	<test name="Memory Clustering test" >
 		<classes>
-			<class name="org.apache.mnemonic.NonVolatileListNGTest" />
+			<class name="org.apache.mnemonic.DurableListNGTest" />
 		</classes>
 	</test>
 </suite>
 
 <!--
 			<class name="org.apache.mnemonic.MemBufferHolderCachePoolNGTest" />
-			<class name="org.apache.mnemonic.BigDataMemAllocatorNGTest" />
+			<class name="org.apache.mnemonic.NonVolatileMemAllocatorNGTest" />
 			<class name="org.apache.mnemonic.ByteBufferSerializerNGTest" />
 			<class name="org.apache.mnemonic.MemClusteringNGTest" />
-			<class name="org.apache.mnemonic.BigDataPMemAllocatorNGTest" />
-			<class name="org.apache.mnemonic.NonVolatilePersonNGTest" />
-			<class name="org.apache.mnemonic.NonVolatileListNGTest" />
+			<class name="org.apache.mnemonic.NonVolatileMemAllocatorNGTest" />
+			<class name="org.apache.mnemonic.DurablePersonNGTest" />
+			<class name="org.apache.mnemonic.DurableListNGTest" />
 
  -->

http://git-wip-us.apache.org/repos/asf/incubator-mnemonic/blob/679bcdc1/examples/src/main/java/org/apache/mnemonic/example/Main.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/mnemonic/example/Main.java b/examples/src/main/java/org/apache/mnemonic/example/Main.java
index 7029971..4ea9dc1 100644
--- a/examples/src/main/java/org/apache/mnemonic/example/Main.java
+++ b/examples/src/main/java/org/apache/mnemonic/example/Main.java
@@ -20,7 +20,7 @@ package org.apache.mnemonic.example;
 import java.nio.ByteBuffer;
 import java.util.Random;
 
-import org.apache.mnemonic.BigDataMemAllocator;
+import org.apache.mnemonic.VolatileMemAllocator;
 import org.apache.mnemonic.CommonAllocator;
 import org.apache.mnemonic.MemBufferHolder;
 import org.apache.mnemonic.MemChunkHolder;
@@ -65,7 +65,7 @@ public class Main {
         // true).disableActiveGC(),
         // MemClustering.PerformanceLevel.NORMAL),
         new MemClustering.NodeConfig(
-            new BigDataMemAllocator(Utils.getVolatileMemoryAllocatorService("vmem"), 1024 * 1024 * 20, ".", true),
+            new VolatileMemAllocator(Utils.getVolatileMemoryAllocatorService("vmem"), 1024 * 1024 * 20, ".", true),
             // Utils.getVolatileMemoryAllocatorService("pmalloc"),
             // 1024 * 1024 * 20, "./example.dat", true),
             MemClustering.PerformanceLevel.SLOW), };


[3/5] incubator-mnemonic git commit: MNEMONIC-20: Formalize the names of annotations, classes and methods

Posted by ga...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-mnemonic/blob/679bcdc1/core/src/main/java/org/apache/mnemonic/AnnotatedDurableEntityClass.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/mnemonic/AnnotatedDurableEntityClass.java b/core/src/main/java/org/apache/mnemonic/AnnotatedDurableEntityClass.java
new file mode 100644
index 0000000..7f25b33
--- /dev/null
+++ b/core/src/main/java/org/apache/mnemonic/AnnotatedDurableEntityClass.java
@@ -0,0 +1,943 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import javax.annotation.processing.Filer;
+import javax.annotation.processing.Messager;
+import javax.lang.model.element.AnnotationMirror;
+import javax.lang.model.element.Element;
+import javax.lang.model.element.ElementKind;
+import javax.lang.model.element.ExecutableElement;
+import javax.lang.model.element.Modifier;
+import javax.lang.model.element.TypeElement;
+import javax.lang.model.element.TypeParameterElement;
+import javax.lang.model.element.VariableElement;
+import javax.lang.model.type.TypeKind;
+import javax.lang.model.type.TypeMirror;
+import javax.lang.model.type.TypeVariable;
+import javax.lang.model.util.Elements;
+import javax.lang.model.util.Types;
+
+import com.squareup.javapoet.AnnotationSpec;
+import com.squareup.javapoet.ArrayTypeName;
+import com.squareup.javapoet.ClassName;
+import com.squareup.javapoet.CodeBlock;
+import com.squareup.javapoet.FieldSpec;
+import com.squareup.javapoet.JavaFile;
+import com.squareup.javapoet.MethodSpec;
+import com.squareup.javapoet.MethodSpec.Builder;
+import com.squareup.javapoet.ParameterSpec;
+import com.squareup.javapoet.ParameterizedTypeName;
+import com.squareup.javapoet.TypeName;
+import com.squareup.javapoet.TypeSpec;
+import com.squareup.javapoet.TypeVariableName;
+
+import sun.misc.Unsafe;
+
+/**
+ * this class managed to generate generic non-volatile concrete object and their
+ * corresponding factory.
+ *
+ */
+@SuppressWarnings("restriction")
+public class AnnotatedDurableEntityClass {
+  protected class MethodInfo {
+    public ExecutableElement elem;
+    public MethodSpec.Builder specbuilder;
+    public TypeName rettype;
+  }
+
+  protected class FieldInfo {
+    public FieldSpec.Builder specbuilder;
+    public TypeName type;
+    public long id;
+    public String name;
+    public String efproxiesname;
+    public String gftypesname;
+    public long fieldoff;
+    public long fieldsize;
+  }
+
+  protected final String cFACTORYNAMESUFFIX = "Factory";
+  protected final String cPMEMNAMEPREFIX = "Durable_";
+  protected final String cFIELDNAMESUFFIX = String.format("_field_%s", Utils.genRandomString());
+  protected final String cALLOCATORFIELDNAME = String.format("alloc_%s", Utils.genRandomString());
+  protected final String cAUTORECLAIMFIELDNAME = String.format("autoreclaim_%s", Utils.genRandomString());
+  protected final String cUNSAFEFIELDNAME = String.format("unsafe_%s", Utils.genRandomString());
+  protected final String cHOLDERFIELDNAME = String.format("holder_%s", Utils.genRandomString());
+  protected final String cALLOCTYPENAME = String.format("ALLOC_PMem3C93D24F59");
+
+  private Types m_typeutils;
+  private Elements m_elemutils;
+  private TypeElement m_elem;
+
+  private String m_factoryname;
+  private String m_entityname;
+
+  private long m_holdersize;
+
+  private String m_packagename;
+
+  private TypeName m_alloctypename = TypeVariableName.get(cALLOCTYPENAME);
+  private TypeName m_factoryproxystypename = TypeName.get(EntityFactoryProxy[].class);
+  private TypeName m_gfieldstypename = TypeName.get(GenericField.GType[].class);
+  private TypeVariableName m_alloctypevarname = TypeVariableName.get(cALLOCTYPENAME,
+      ParameterizedTypeName.get(ClassName.get(CommonDurableAllocator.class), TypeVariableName.get(cALLOCTYPENAME)));
+
+  private Map<String, MethodInfo> m_gettersinfo = new HashMap<String, MethodInfo>();
+  private Map<String, MethodInfo> m_settersinfo = new HashMap<String, MethodInfo>();
+  private Map<String, FieldInfo> m_dynfieldsinfo = new HashMap<String, FieldInfo>();
+  private Map<String, FieldInfo> m_fieldsinfo = new HashMap<String, FieldInfo>();
+
+  private Map<String, MethodInfo> m_durablemtdinfo = new HashMap<String, MethodInfo>();
+  private Map<String, MethodInfo> m_entitymtdinfo = new HashMap<String, MethodInfo>();
+
+  private long computeTypeSize(TypeMirror type) {
+    long ret;
+    switch (type.getKind()) {
+    case BYTE:
+      ret = 1L;
+      break;
+    case BOOLEAN:
+      ret = 1L;
+      break;
+    case CHAR:
+      ret = 2L;
+      break;
+    case DOUBLE:
+      ret = 8L;
+      break;
+    case FLOAT:
+      ret = 4L;
+      break;
+    case SHORT:
+      ret = 2L;
+      break;
+    case INT:
+      ret = 4L;
+      break;
+    case LONG:
+      ret = 8L;
+      break;
+    default:
+      ret = 8L;
+    }
+    return ret;
+  }
+
+  private boolean isUnboxPrimitive(TypeName tn) {
+    TypeName n = tn;
+    try {
+      n = tn.unbox();
+    } catch (UnsupportedOperationException ex) {
+    }
+    return n.isPrimitive();
+  }
+
+  private TypeName unboxTypeName(TypeName tn) {
+    TypeName n = tn;
+    try {
+      n = tn.unbox();
+    } catch (UnsupportedOperationException ex) {
+    }
+    return n;
+  }
+
+  public AnnotatedDurableEntityClass(TypeElement classElement, Types typeUtils, Elements elementUtils,
+      Messager messager) {
+    m_elem = classElement;
+    m_typeutils = typeUtils;
+    m_elemutils = elementUtils;
+
+    m_packagename = m_elemutils.getPackageOf(m_elem).getQualifiedName().toString();
+
+    m_factoryname = String.format("%s%s", m_elem.getSimpleName(), cFACTORYNAMESUFFIX);
+    m_entityname = String.format("%s%s_%s", cPMEMNAMEPREFIX, m_elem.getSimpleName(), Utils.genRandomString());
+
+    m_durablemtdinfo.put("cancelAutoReclaim", new MethodInfo());
+    m_durablemtdinfo.put("registerAutoReclaim", new MethodInfo());
+    m_durablemtdinfo.put("getHandler", new MethodInfo());
+    m_durablemtdinfo.put("autoReclaim", new MethodInfo());
+    m_durablemtdinfo.put("destroy", new MethodInfo());
+    m_durablemtdinfo.put("getNativeFieldInfo", new MethodInfo());
+
+    m_entitymtdinfo.put("initializeDurableEntity", new MethodInfo());
+    m_entitymtdinfo.put("createDurableEntity", new MethodInfo());
+    m_entitymtdinfo.put("restoreDurableEntity", new MethodInfo());
+
+  }
+
+  public void prepareProcessing() throws AnnotationProcessingException {
+    MethodInfo methodinfo = null;
+    FieldInfo fieldinfo;
+    String methodname;
+    long fieldoff = 0;
+    TypeElement intf_durable = m_elemutils.getTypeElement(Durable.class.getCanonicalName());
+    TypeElement intf_entity = m_elemutils.getTypeElement(MemoryDurableEntity.class.getCanonicalName());
+    // System.err.printf("<><><><><> %s ======\n", intf_entity.toString());
+
+    boolean valid = false;
+    for (TypeMirror tm : m_elem.getInterfaces()) {
+      if (tm.toString().equals(Durable.class.getCanonicalName())) {
+        valid = true;
+        break;
+      }
+    }
+    if (!valid) {
+      throw new AnnotationProcessingException(m_elem, "Not implemented Durable Interface by %s.",
+          m_elem.getSimpleName().toString());
+    }
+
+    fieldinfo = new FieldInfo();
+    fieldinfo.name = String.format("m_unsafe_%s", Utils.genRandomString());
+    fieldinfo.type = TypeName.get(m_elemutils.getTypeElement(Unsafe.class.getCanonicalName()).asType());
+    fieldinfo.specbuilder = FieldSpec.builder(fieldinfo.type, fieldinfo.name, Modifier.PRIVATE);
+    m_fieldsinfo.put("unsafe", fieldinfo);
+
+    fieldinfo = new FieldInfo();
+    fieldinfo.name = String.format("m_holder_%s", Utils.genRandomString());
+    fieldinfo.type = ParameterizedTypeName.get(ClassName.get(MemChunkHolder.class), m_alloctypename);
+    fieldinfo.specbuilder = FieldSpec.builder(fieldinfo.type, fieldinfo.name, Modifier.PRIVATE);
+    m_fieldsinfo.put("holder", fieldinfo);
+
+    fieldinfo = new FieldInfo();
+    fieldinfo.name = String.format("m_autoreclaim_%s", Utils.genRandomString());
+    fieldinfo.type = TypeName.get(m_typeutils.getPrimitiveType(TypeKind.BOOLEAN));
+    fieldinfo.specbuilder = FieldSpec.builder(fieldinfo.type, fieldinfo.name, Modifier.PRIVATE, Modifier.VOLATILE);
+    m_fieldsinfo.put("autoreclaim", fieldinfo);
+
+    fieldinfo = new FieldInfo();
+    fieldinfo.name = String.format("m_allocator_%s", Utils.genRandomString());
+    fieldinfo.type = m_alloctypename;
+    fieldinfo.specbuilder = FieldSpec.builder(fieldinfo.type, fieldinfo.name, Modifier.PRIVATE);
+    m_fieldsinfo.put("allocator", fieldinfo);
+
+    fieldinfo = new FieldInfo();
+    fieldinfo.name = String.format("m_factoryproxy_%s", Utils.genRandomString());
+    fieldinfo.type = m_factoryproxystypename;
+    fieldinfo.specbuilder = FieldSpec.builder(fieldinfo.type, fieldinfo.name, Modifier.PRIVATE);
+    m_fieldsinfo.put("factoryproxy", fieldinfo);
+
+    fieldinfo = new FieldInfo();
+    fieldinfo.name = String.format("m_genericfield_%s", Utils.genRandomString());
+    fieldinfo.type = m_gfieldstypename;
+    fieldinfo.specbuilder = FieldSpec.builder(fieldinfo.type, fieldinfo.name, Modifier.PRIVATE);
+    m_fieldsinfo.put("genericfield", fieldinfo);
+
+    for (Element elem : m_elem.getEnclosedElements()) {
+      if (elem.getKind() == ElementKind.METHOD) {
+        methodname = elem.getSimpleName().toString();
+        // System.err.printf("=========== %s ======\n", methodname);
+        DurableGetter pgetter = elem.getAnnotation(DurableGetter.class);
+        if (pgetter != null) {
+          if (!elem.getModifiers().contains(Modifier.ABSTRACT)) {
+            throw new AnnotationProcessingException(elem, "%s annotated with DurableGetter is not abstract.",
+                methodname);
+          }
+          if (null != elem.getAnnotation(DurableSetter.class)) {
+            throw new AnnotationProcessingException(elem, "%s is annotated with DurableSetter as well.",
+                methodname);
+          }
+          if (!methodname.startsWith("get")) {
+            throw new AnnotationProcessingException(elem, "%s does not comply name convention of getter.", methodname);
+          }
+          methodinfo = new MethodInfo();
+          methodinfo.elem = (ExecutableElement) elem;
+          methodinfo.specbuilder = MethodSpec.overriding(methodinfo.elem);
+          methodinfo.rettype = TypeName.get(methodinfo.elem.getReturnType());
+          m_gettersinfo.put(methodname.substring(3), methodinfo);
+          fieldinfo = new FieldInfo();
+          fieldinfo.type = methodinfo.rettype;
+          if (fieldinfo.type.toString().equals(String.class.getCanonicalName())) {
+            fieldinfo.type = ParameterizedTypeName.get(ClassName.get(MemBufferHolder.class), m_alloctypename);
+          }
+          if (fieldinfo.type instanceof TypeVariableName) {
+            fieldinfo.type = ParameterizedTypeName.get(ClassName.get(GenericField.class), m_alloctypename,
+                fieldinfo.type);
+          }
+          fieldinfo.name = String.format("m_%s_%s", methodname.substring(3).toLowerCase(), Utils.genRandomString());
+          fieldinfo.specbuilder = FieldSpec.builder(fieldinfo.type, fieldinfo.name, Modifier.PRIVATE);
+          fieldinfo.fieldsize = computeTypeSize(methodinfo.elem.getReturnType());
+          fieldinfo.fieldoff = fieldoff;
+          fieldoff += fieldinfo.fieldsize;
+          fieldinfo.efproxiesname = pgetter.EntityFactoryProxies();
+          fieldinfo.gftypesname = pgetter.GenericFieldTypes();
+          fieldinfo.id = pgetter.Id();
+          m_dynfieldsinfo.put(methodname.substring(3), fieldinfo);
+
+        }
+        if (null != elem.getAnnotation(DurableSetter.class)) {
+          if (!elem.getModifiers().contains(Modifier.ABSTRACT)) {
+            throw new AnnotationProcessingException(elem, "%s annotated with DurableSetter is not abstract.",
+                methodname);
+          }
+          if (!methodname.startsWith("set")) {
+            throw new AnnotationProcessingException(elem, "%s does not comply name convention of setter.", methodname);
+          }
+          methodinfo = new MethodInfo();
+          methodinfo.elem = (ExecutableElement) elem;
+          methodinfo.specbuilder = MethodSpec.overriding(methodinfo.elem);
+          m_settersinfo.put(methodname.substring(3), methodinfo);
+        }
+      }
+    }
+
+    m_holdersize = fieldoff;
+
+    // MethodInfo minfo = null;
+    for (String name : m_settersinfo.keySet()) {
+      if (!m_gettersinfo.containsKey(name)) {
+        throw new AnnotationProcessingException(null, "%s has no getter.", name);
+      }
+    }
+
+    for (Element elem : intf_durable.getEnclosedElements()) {
+      if (elem.getKind() == ElementKind.METHOD) {
+        methodname = elem.getSimpleName().toString();
+        if (m_durablemtdinfo.containsKey(methodname)) {
+          // System.err.printf("**++++++++++ %s ======\n", methodname);
+          methodinfo = m_durablemtdinfo.get(methodname);
+          methodinfo.elem = (ExecutableElement) elem;
+          methodinfo.specbuilder = MethodSpec.overriding(methodinfo.elem);
+        }
+      }
+    }
+
+    for (Element elem : intf_entity.getEnclosedElements()) {
+      if (elem.getKind() == ElementKind.METHOD) {
+        methodname = elem.getSimpleName().toString();
+        if (m_entitymtdinfo.containsKey(methodname)) {
+          // System.err.printf("**------- %s ======\n", elem.toString());
+          methodinfo = m_entitymtdinfo.get(methodname);
+          methodinfo.elem = (ExecutableElement) elem;
+          methodinfo.specbuilder = overriding(methodinfo.elem, cALLOCTYPENAME);
+
+        }
+      }
+    }
+    genNFieldInfo();
+  }
+
+  protected String transTypeToUnsafeMethod(TypeName tname, boolean isget) throws AnnotationProcessingException {
+    String ret = null;
+    if (isUnboxPrimitive(tname)) {
+      TypeName tn = unboxTypeName(tname);
+      if (tn.equals(TypeName.BOOLEAN)) {
+        ret = isget ? "getByte" : "putByte";
+      }
+      if (tn.equals(TypeName.BYTE)) {
+        ret = isget ? "getByte" : "putByte";
+      }
+      if (tn.equals(TypeName.CHAR)) {
+        ret = isget ? "getChar" : "putChar";
+      }
+      if (tn.equals(TypeName.DOUBLE)) {
+        ret = isget ? "getDouble" : "putDouble";
+      }
+      if (tn.equals(TypeName.FLOAT)) {
+        ret = isget ? "getFloat" : "putFloat";
+      }
+      if (tn.equals(TypeName.INT)) {
+        ret = isget ? "getInt" : "putInt";
+      }
+      if (tn.equals(TypeName.LONG)) {
+        ret = isget ? "getLong" : "putLong";
+      }
+      if (tn.equals(TypeName.SHORT)) {
+        ret = isget ? "getShort" : "putShort";
+      }
+    } else {
+      ret = isget ? "getAddress" : "putAddress";
+    }
+    if (null == ret) {
+      throw new AnnotationProcessingException(null, "%s is not supported by getters or setters.", tname.toString());
+    }
+    return ret;
+  }
+
+  protected String getIntialValueLiteral(TypeName tname) throws AnnotationProcessingException {
+    String ret = null;
+    if (isUnboxPrimitive(tname)) {
+      TypeName tn = unboxTypeName(tname);
+      if (tn.equals(TypeName.BOOLEAN)) {
+        ret = "false";
+      }
+      if (tn.equals(TypeName.BYTE)) {
+        ret = "(byte)0";
+      }
+      if (tn.equals(TypeName.CHAR)) {
+        ret = "(char)0";
+      }
+      if (tn.equals(TypeName.DOUBLE)) {
+        ret = "(double)0.0";
+      }
+      if (tn.equals(TypeName.FLOAT)) {
+        ret = "(float)0.0";
+      }
+      if (tn.equals(TypeName.INT)) {
+        ret = "(int)0";
+      }
+      if (tn.equals(TypeName.LONG)) {
+        ret = "(long)0";
+      }
+      if (tn.equals(TypeName.SHORT)) {
+        ret = "(short)0";
+      }
+    } else {
+      ret = null;
+    }
+    if (null == ret) {
+      throw new AnnotationProcessingException(null, "%s is not supported to determine the inital value.",
+          tname.toString());
+    }
+    return ret;
+  }
+
+  private int getFactoryProxyIndex(TypeName gtname) throws AnnotationProcessingException {
+    int ret = -1;
+    boolean found = false;
+    if (gtname instanceof TypeVariableName) {
+      for (TypeParameterElement tpe : m_elem.getTypeParameters()) {
+        ++ret;
+        if (tpe.toString().equals(gtname.toString())) {
+          found = true;
+          break;
+        }
+      }
+      if (!found) {
+        throw new AnnotationProcessingException(null, "%s type is not found during factory proxy query.",
+            gtname.toString());
+      }
+    } else {
+      throw new AnnotationProcessingException(null, "%s type is not generic type for factory proxy query.",
+          gtname.toString());
+    }
+    return ret;
+  }
+
+  protected void genNFieldInfo() {
+    FieldInfo dynfieldinfo, fieldinfo;
+    List<long[]> finfo = new ArrayList<long[]>();
+    for (String name : m_gettersinfo.keySet()) {
+      dynfieldinfo = m_dynfieldsinfo.get(name);
+      if (dynfieldinfo.id > 0) {
+        finfo.add(new long[]{dynfieldinfo.id, dynfieldinfo.fieldoff, dynfieldinfo.fieldsize});
+      }
+    }
+
+    fieldinfo = new FieldInfo();
+    fieldinfo.name = String.format("m_nfieldinfo_%s", Utils.genRandomString());
+    fieldinfo.type = ArrayTypeName.of(ArrayTypeName.of(TypeName.LONG));
+    String initlstr = Utils.toInitLiteral(finfo);
+    fieldinfo.specbuilder = FieldSpec.builder(fieldinfo.type, fieldinfo.name, Modifier.PRIVATE, Modifier.STATIC)
+        .initializer("$1L", initlstr);
+    m_fieldsinfo.put("nfieldinfo", fieldinfo);
+  }
+
+  protected void buildGettersSpecs(TypeSpec.Builder typespecbuilder) throws AnnotationProcessingException {
+    MethodInfo methodinfo;
+    TypeName ftname;
+    String unsafename = m_fieldsinfo.get("unsafe").name;
+    String holdername = m_fieldsinfo.get("holder").name;
+    String allocname = m_fieldsinfo.get("allocator").name;
+    String autoreclaimname = m_fieldsinfo.get("autoreclaim").name;
+    String factoryproxyname = m_fieldsinfo.get("factoryproxy").name;
+    String genericfieldname = m_fieldsinfo.get("genericfield").name;
+    FieldInfo dynfieldinfo;
+    CodeBlock.Builder code;
+    String codefmt;
+    for (String name : m_gettersinfo.keySet()) {
+      code = CodeBlock.builder();
+      methodinfo = m_gettersinfo.get(name);
+      dynfieldinfo = m_dynfieldsinfo.get(name);
+      ftname = m_dynfieldsinfo.get(name).type;
+      if (isUnboxPrimitive(ftname)) {
+        if (unboxTypeName(ftname).equals(TypeName.BOOLEAN)) {
+          codefmt = "return 1 == $1N.$4L($2N.get() + $3L)";
+        } else {
+          codefmt = "return $1N.$4L($2N.get() + $3L)";
+        }
+        code.addStatement(codefmt, unsafename, holdername, dynfieldinfo.fieldoff,
+            transTypeToUnsafeMethod(ftname, true));
+      } else {
+        if (methodinfo.rettype.toString().equals(String.class.getCanonicalName())) {
+          code.beginControlFlow("if (null == $1N)", dynfieldinfo.name);
+          code.addStatement("long phandler = $1N.getAddress($2N.get() + $3L)", unsafename, holdername,
+              dynfieldinfo.fieldoff);
+          code.beginControlFlow("if (0L != phandler)");
+          code.addStatement("$1N = $2N.retrieveBuffer(phandler, $3N)", dynfieldinfo.name, allocname, autoreclaimname);
+          code.beginControlFlow("if (null == $1N)", dynfieldinfo.name);
+          code.addStatement("throw new RetrieveDurableEntityError(\"Retrieve String Buffer Failure.\")");
+          code.endControlFlow();
+          code.endControlFlow();
+          code.endControlFlow();
+          code.addStatement("return null == $1N ? null : $1N.get().asCharBuffer().toString()", dynfieldinfo.name);
+        } else if (dynfieldinfo.type.toString().startsWith(GenericField.class.getCanonicalName())) {
+          code.beginControlFlow("if (null == $1N)", dynfieldinfo.name);
+          code.addStatement("$1T proxy = null", TypeName.get(EntityFactoryProxy.class));
+          code.addStatement("$1T gftype = null", TypeName.get(GenericField.GType.class));
+          code.addStatement("int gfpidx = $1L", getFactoryProxyIndex(methodinfo.rettype));
+          code.beginControlFlow("if (null != $1N && $1N.length > gfpidx)", factoryproxyname);
+          code.addStatement("proxy = $1L[gfpidx]", factoryproxyname);
+          code.endControlFlow();
+          code.beginControlFlow("if (null != $1N && $1N.length > gfpidx)", genericfieldname);
+          code.addStatement("gftype = $1L[gfpidx]", genericfieldname);
+          code.nextControlFlow("else");
+          code.addStatement("throw new RetrieveDurableEntityError(\"No Generic Field Type Info.\")");
+          code.endControlFlow();
+          code.addStatement("$1N = new $2T(proxy, gftype, $8L, $9L, $3N, $4N, $5N, $6N.get() + $7L)", dynfieldinfo.name,
+              dynfieldinfo.type, allocname, unsafename, autoreclaimname, holdername, dynfieldinfo.fieldoff,
+              dynfieldinfo.efproxiesname, dynfieldinfo.gftypesname);
+          code.endControlFlow();
+          code.addStatement("return $1N.get()", dynfieldinfo.name);
+        } else {
+          code.beginControlFlow("if (null == $1N)", dynfieldinfo.name);
+          code.addStatement("long phandler = $1N.getAddress($2N.get() + $3L)", unsafename, holdername,
+              dynfieldinfo.fieldoff);
+          code.beginControlFlow("if (0L != phandler)");
+          code.addStatement("$1N = $4N.restore($2N, $5L, $6L, phandler, $3N)", dynfieldinfo.name, allocname,
+              autoreclaimname, String.format("%s%s",
+                  m_typeutils.asElement(methodinfo.elem.getReturnType()).getSimpleName(), cFACTORYNAMESUFFIX),
+              dynfieldinfo.efproxiesname, dynfieldinfo.gftypesname);
+          code.endControlFlow();
+          code.endControlFlow();
+          code.addStatement("return $1N", dynfieldinfo.name);
+        }
+      }
+      typespecbuilder.addMethod(methodinfo.specbuilder.addCode(code.build()).build());
+    }
+  }
+
+  protected String gsetterName(String name, boolean isget) {
+    return String.format("%s%s", isget ? "get" : "set", name); // Character.toUpperCase(name.charAt(0))
+                                                               // +
+                                                               // name.substring(1));
+  }
+
+  protected void buildSettersSpecs(TypeSpec.Builder typespecbuilder) throws AnnotationProcessingException {
+    MethodInfo methodinfo;
+    TypeName ftname, valtname;
+    String unsafename = m_fieldsinfo.get("unsafe").name;
+    String holdername = m_fieldsinfo.get("holder").name;
+    String allocname = m_fieldsinfo.get("allocator").name;
+    String autoreclaimname = m_fieldsinfo.get("autoreclaim").name;
+    String factoryproxyname = m_fieldsinfo.get("factoryproxy").name;
+    String genericfieldname = m_fieldsinfo.get("genericfield").name;
+    FieldInfo dynfieldinfo;
+    CodeBlock.Builder code;
+    VariableElement arg0;
+    VariableElement arg1;
+    String codefmt;
+    for (String name : m_settersinfo.keySet()) {
+      code = CodeBlock.builder();
+      methodinfo = m_settersinfo.get(name);
+      dynfieldinfo = m_dynfieldsinfo.get(name);
+      ftname = m_dynfieldsinfo.get(name).type;
+      valtname = m_gettersinfo.get(name).rettype;
+      arg0 = methodinfo.elem.getParameters().get(0);
+      if (!TypeName.get(arg0.asType()).equals(valtname)) {
+        throw new AnnotationProcessingException(null, "%s has inconsistent value type with its getter/setter.", name);
+      }
+      if (isUnboxPrimitive(ftname)) {
+        if (unboxTypeName(ftname).equals(TypeName.BOOLEAN)) {
+          codefmt = "$1N.$4L($2N.get() + $3L, $5L?1:0)";
+        } else {
+          codefmt = "$1N.$4L($2N.get() + $3L, $5L)";
+        }
+        code.addStatement(codefmt, unsafename, holdername, dynfieldinfo.fieldoff,
+            transTypeToUnsafeMethod(ftname, false), arg0);
+      } else {
+        try {
+          arg1 = methodinfo.elem.getParameters().get(1);
+          if (!TypeName.BOOLEAN.equals(TypeName.get(arg1.asType()))) {
+            throw new AnnotationProcessingException(null, "the second parameter of %s's setter is not boolean type.",
+                name);
+          }
+        } catch (IndexOutOfBoundsException ex) {
+          throw new AnnotationProcessingException(null, "%s's setter has no second parameters for non primitive type.",
+              name);
+        }
+        if (valtname.toString().equals(String.class.getCanonicalName())) {
+          code.beginControlFlow("if ($1L && null != $2L())", arg1, gsetterName(name, true));
+          code.addStatement("$1N.destroy()", dynfieldinfo.name);
+          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.putLong($2N.get() + $3L, 0L)", unsafename, holdername, dynfieldinfo.fieldoff);
+          code.nextControlFlow("else");
+          code.addStatement("$1N = $2N.createBuffer($3L.length() * 2, $4N)", dynfieldinfo.name, allocname, arg0,
+              autoreclaimname);
+          code.beginControlFlow("if (null == $1N)", dynfieldinfo.name);
+          code.addStatement("throw new OutOfHybridMemory(\"Create Non-Volatile String Error!\")");
+          code.endControlFlow();
+          code.addStatement("$1N.get().asCharBuffer().put($2L)", dynfieldinfo.name, arg0);
+          code.addStatement("$1N.putLong($2N.get() + $3L, $4N.getBufferHandler($5N))", unsafename, holdername,
+              dynfieldinfo.fieldoff, allocname, dynfieldinfo.name);
+          code.endControlFlow();
+        } else if (dynfieldinfo.type.toString().startsWith(GenericField.class.getCanonicalName())) {
+          code.beginControlFlow("if (null == $1N)", dynfieldinfo.name);
+          code.addStatement("$1T proxy = null", TypeName.get(EntityFactoryProxy.class));
+          code.addStatement("$1T gftype = null", TypeName.get(GenericField.GType.class));
+          code.addStatement("int gfpidx = $1L", getFactoryProxyIndex(valtname));
+          code.beginControlFlow("if (null != $1N && $1N.length > gfpidx)", factoryproxyname);
+          code.addStatement("proxy = $1L[gfpidx]", factoryproxyname);
+          code.endControlFlow();
+          code.beginControlFlow("if (null != $1N && $1N.length > gfpidx)", genericfieldname);
+          code.addStatement("gftype = $1L[gfpidx]", genericfieldname);
+          code.nextControlFlow("else");
+          code.addStatement("throw new RetrieveDurableEntityError(\"No Generic Field Type Info.\")");
+          code.endControlFlow();
+          code.addStatement("$1N = new $2T(proxy, gftype, $8L, $9L, $3N, $4N, $5N, $6N.get() + $7L)", dynfieldinfo.name,
+              dynfieldinfo.type, allocname, unsafename, autoreclaimname, holdername, dynfieldinfo.fieldoff,
+              dynfieldinfo.efproxiesname, dynfieldinfo.gftypesname);
+          code.endControlFlow();
+          code.beginControlFlow("if (null != $1L)", dynfieldinfo.name);
+          code.addStatement("$1N.set($2L, $3L)", dynfieldinfo.name, arg0, arg1);
+          code.nextControlFlow("else");
+          code.addStatement("throw new RetrieveDurableEntityError(\"GenericField is null!\")");
+          code.endControlFlow();
+        } else {
+          code.beginControlFlow("if ($1L && null != $2L())", arg1, gsetterName(name, true));
+          code.addStatement("$1N.destroy()", dynfieldinfo.name);
+          code.addStatement("$1N = null", dynfieldinfo.name);
+          code.addStatement("$1N.putAddress($2N.get() + $3L, 0L)", unsafename, holdername, dynfieldinfo.fieldoff);
+          code.endControlFlow();
+          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);
+        }
+      }
+      typespecbuilder.addMethod(methodinfo.specbuilder.addCode(code.build()).build());
+    }
+  }
+
+  protected void buildDurableMethodSpecs(TypeSpec.Builder typespecbuilder) throws AnnotationProcessingException {
+    MethodInfo methodinfo;
+    CodeBlock.Builder code;
+    FieldInfo dynfieldinfo;
+    String holdername = m_fieldsinfo.get("holder").name;
+    String allocname = m_fieldsinfo.get("allocator").name;
+    String autoreclaimname = m_fieldsinfo.get("autoreclaim").name;
+    for (String name : m_durablemtdinfo.keySet()) {
+      methodinfo = m_durablemtdinfo.get(name);
+      code = CodeBlock.builder();
+      switch (name) {
+      case "cancelAutoReclaim":
+        code.addStatement("$1N.cancelAutoReclaim()", holdername);
+        for (String fname : m_dynfieldsinfo.keySet()) {
+          dynfieldinfo = m_dynfieldsinfo.get(fname);
+          if (!isUnboxPrimitive(dynfieldinfo.type)) {
+            code.beginControlFlow("if (null != $1N)", dynfieldinfo.name);
+            code.addStatement("$1N.cancelAutoReclaim()", dynfieldinfo.name);
+            code.endControlFlow();
+          }
+        }
+        code.addStatement("$1N = false", autoreclaimname);
+        break;
+      case "registerAutoReclaim":
+        code.addStatement("$1N.registerAutoReclaim()", holdername);
+        for (String fname : m_dynfieldsinfo.keySet()) {
+          dynfieldinfo = m_dynfieldsinfo.get(fname);
+          if (!isUnboxPrimitive(dynfieldinfo.type)) {
+            code.beginControlFlow("if (null != $1N)", dynfieldinfo.name);
+            code.addStatement("$1N.registerAutoReclaim()", dynfieldinfo.name);
+            code.endControlFlow();
+          }
+        }
+        code.addStatement("$1N = true", autoreclaimname);
+        break;
+      case "getHandler":
+        code.addStatement("return $1N.getChunkHandler($2N)", allocname, holdername);
+        break;
+      case "autoReclaim":
+        code.addStatement("return $1N", autoreclaimname);
+        break;
+      case "destroy":
+        code.addStatement("$1N.destroy()", holdername);
+        for (String fname : m_dynfieldsinfo.keySet()) {
+          dynfieldinfo = m_dynfieldsinfo.get(fname);
+          if (!isUnboxPrimitive(dynfieldinfo.type)) {
+            code.beginControlFlow("if (null != $1N)", dynfieldinfo.name);
+            code.addStatement("$1N.destroy()", dynfieldinfo.name);
+            code.addStatement("$1N = null", dynfieldinfo.name);
+            code.endControlFlow();
+          }
+        }
+        break;
+      case "getNativeFieldInfo":
+        code.addStatement("return $1N", m_fieldsinfo.get("nfieldinfo").name);
+        break;
+      default:
+        throw new AnnotationProcessingException(null, "Method %s is not supported.", name);
+      }
+      typespecbuilder.addMethod(methodinfo.specbuilder.addCode(code.build()).build());
+    }
+  }
+
+  protected void buildEntityMethodSpecs(TypeSpec.Builder typespecbuilder) throws AnnotationProcessingException {
+    MethodInfo methodinfo;
+    CodeBlock.Builder code;
+    VariableElement arg0, arg1, arg2, arg3, arg4;
+    String unsafename = m_fieldsinfo.get("unsafe").name;
+    String holdername = m_fieldsinfo.get("holder").name;
+    String allocname = m_fieldsinfo.get("allocator").name;
+    String autoreclaimname = m_fieldsinfo.get("autoreclaim").name;
+    String factoryproxyname = m_fieldsinfo.get("factoryproxy").name;
+    String genericfieldname = m_fieldsinfo.get("genericfield").name;
+    for (String name : m_entitymtdinfo.keySet()) {
+      methodinfo = m_entitymtdinfo.get(name);
+      code = CodeBlock.builder();
+      arg0 = methodinfo.elem.getParameters().get(0);
+      arg1 = methodinfo.elem.getParameters().get(1);
+      arg2 = methodinfo.elem.getParameters().get(2);
+      switch (name) {
+      case "initializeDurableEntity":
+        arg3 = methodinfo.elem.getParameters().get(3);
+        code.addStatement("$1N = $2L", allocname, arg0);
+        code.addStatement("$1N = $2L", factoryproxyname, arg1);
+        code.addStatement("$1N = $2L", genericfieldname, arg2);
+        code.addStatement("$1N = $2L", autoreclaimname, arg3);
+        code.beginControlFlow("try");
+        code.addStatement("$1N = $2T.getUnsafe()", unsafename, Utils.class);
+        code.nextControlFlow("catch (Exception e)");
+        code.addStatement("e.printStackTrace()");
+        code.endControlFlow();
+        break;
+      case "createDurableEntity":
+        arg3 = methodinfo.elem.getParameters().get(3);
+        code.addStatement("initializeDurableEntity($1L, $2L, $3L, $4L)", arg0, arg1, arg2, arg3);
+        code.addStatement("$1N = $2N.createChunk($3L, $4N)", holdername, allocname, m_holdersize, autoreclaimname);
+        code.beginControlFlow("if (null == $1N)", holdername);
+        code.addStatement("throw new OutOfHybridMemory(\"Create Non-Volatile Entity Error!\")");
+        code.endControlFlow();
+        // code.beginControlFlow("try");
+        // for (String fname : m_dynfieldsinfo.keySet()) {
+        // dynfieldinfo = m_dynfieldsinfo.get(fname);
+        // if (isUnboxPrimitive(dynfieldinfo.type)) {
+        // code.addStatement("$1N($2L)", gsetterName(fname, false),
+        // getIntialValueLiteral(dynfieldinfo.type));
+        // } else {
+        // code.addStatement("$1N(null, false)", gsetterName(fname, false));
+        // }
+        // }
+        // code.nextControlFlow("catch(RetrieveDurableEntityError ex)");
+        // code.endControlFlow();
+        code.addStatement("initializeAfterCreate()");
+        break;
+      case "restoreDurableEntity":
+        arg3 = methodinfo.elem.getParameters().get(3);
+        arg4 = methodinfo.elem.getParameters().get(4);
+        code.addStatement("initializeDurableEntity($1L, $2L, $3L, $4L)", arg0, arg1, arg2, arg4);
+        code.beginControlFlow("if (0L == $1L)", arg3);
+        code.addStatement("throw new RetrieveDurableEntityError(\"Input handler is null on $1N.\")", name);
+        code.endControlFlow();
+        code.addStatement("$1N = $2N.retrieveChunk($3L, $4N)", holdername, allocname, arg3, autoreclaimname);
+        code.beginControlFlow("if (null == $1N)", holdername);
+        code.addStatement("throw new RetrieveDurableEntityError(\"Retrieve Entity Failure!\")");
+        code.endControlFlow();
+        code.addStatement("initializeAfterRestore()");
+        break;
+      default:
+        throw new AnnotationProcessingException(null, "Method %s is not supported.", name);
+      }
+      typespecbuilder.addMethod(methodinfo.specbuilder.addCode(code.build()).build());
+    }
+  }
+
+  protected void buildFieldSpecs(TypeSpec.Builder typespecbuilder, Map<String, FieldInfo> fieldinfos) {
+    FieldInfo fieldinfo;
+    for (String name : fieldinfos.keySet()) {
+      fieldinfo = fieldinfos.get(name);
+      if (null != fieldinfo.specbuilder) {
+        typespecbuilder.addField(fieldinfo.specbuilder.build());
+      }
+    }
+  }
+
+  protected void buildFactoryMethodSpecs(TypeSpec entityspec, TypeSpec.Builder typespecbuilder)
+      throws AnnotationProcessingException {
+    MethodSpec methodspec;
+    CodeBlock code;
+
+    TypeName entitytn = ParameterizedTypeName.get(ClassName.get(m_packagename, m_entityname),
+        entityspec.typeVariables.toArray(new TypeVariableName[0]));
+
+    ParameterSpec allocparam = ParameterSpec.builder(m_alloctypename, "allocator").build();
+    code = CodeBlock.builder().addStatement("return create($1L, false)", allocparam.name).build();
+    methodspec = MethodSpec.methodBuilder("create").addTypeVariables(entityspec.typeVariables)
+        .addException(OutOfHybridMemory.class).addModifiers(Modifier.PUBLIC, Modifier.STATIC)
+        .returns(TypeName.get(m_elem.asType())).addParameter(allocparam).addCode(code).build();
+    typespecbuilder.addMethod(methodspec);
+
+    ParameterSpec autoreclaimparam = ParameterSpec.builder(TypeName.BOOLEAN, "autoreclaim").build();
+    code = CodeBlock.builder()
+        .addStatement("return create($1L, null, null, $2L)", allocparam.name, autoreclaimparam.name).build();
+    methodspec = MethodSpec.methodBuilder("create").addTypeVariables(entityspec.typeVariables)
+        .addException(OutOfHybridMemory.class).addModifiers(Modifier.PUBLIC, Modifier.STATIC)
+        .returns(TypeName.get(m_elem.asType())).addParameter(allocparam).addParameter(autoreclaimparam).addCode(code)
+        .build();
+    typespecbuilder.addMethod(methodspec);
+
+    ParameterSpec factoryproxysparam = ParameterSpec.builder(m_factoryproxystypename, "factoryproxys").build();
+    ParameterSpec gfieldsparam = ParameterSpec.builder(m_gfieldstypename, "gfields").build();
+    code = CodeBlock.builder().addStatement("$1T entity = new $1T()", entitytn)
+        .addStatement("entity.setupGenericInfo($1N, $2N)", factoryproxysparam.name, gfieldsparam.name)
+        .addStatement("entity.createDurableEntity($1L, $2L, $3L, $4L)", allocparam.name, factoryproxysparam.name,
+            gfieldsparam.name, autoreclaimparam.name)
+        .addStatement("return entity").build();
+    methodspec = MethodSpec.methodBuilder("create").addTypeVariables(entityspec.typeVariables)
+        .addException(OutOfHybridMemory.class).addModifiers(Modifier.PUBLIC, Modifier.STATIC)
+        .returns(TypeName.get(m_elem.asType())).addParameter(allocparam).addParameter(factoryproxysparam)
+        .addParameter(gfieldsparam).addParameter(autoreclaimparam).addCode(code).build();
+    typespecbuilder.addMethod(methodspec);
+
+    ParameterSpec phandlerparam = ParameterSpec.builder(TypeName.LONG, "phandler").build();
+    code = CodeBlock.builder().addStatement("return restore($1L, $2L, false)", allocparam.name, phandlerparam.name)
+        .build();
+    methodspec = MethodSpec.methodBuilder("restore").addTypeVariables(entityspec.typeVariables)
+        .addException(RetrieveDurableEntityError.class).addModifiers(Modifier.PUBLIC, Modifier.STATIC)
+        .returns(TypeName.get(m_elem.asType())).addParameter(allocparam).addParameter(phandlerparam).addCode(code)
+        .build();
+    typespecbuilder.addMethod(methodspec);
+
+    code = CodeBlock.builder().addStatement("return restore($1L, null, null, $2L, $3L)", allocparam.name,
+        phandlerparam.name, autoreclaimparam.name).build();
+    methodspec = MethodSpec.methodBuilder("restore").addTypeVariables(entityspec.typeVariables)
+        .addException(RetrieveDurableEntityError.class).addModifiers(Modifier.PUBLIC, Modifier.STATIC)
+        .returns(TypeName.get(m_elem.asType())).addParameter(allocparam).addParameter(phandlerparam)
+        .addParameter(autoreclaimparam).addCode(code).build();
+    typespecbuilder.addMethod(methodspec);
+
+    code = CodeBlock.builder().addStatement("$1T entity = new $1T()", entitytn)
+        .addStatement("entity.setupGenericInfo($1N, $2N)", factoryproxysparam.name, gfieldsparam.name)
+        .addStatement("entity.restoreDurableEntity($1L, $2L, $3L, $4L, $5L)", allocparam.name,
+            factoryproxysparam.name, gfieldsparam.name, phandlerparam.name, autoreclaimparam.name)
+        .addStatement("return entity").build();
+    methodspec = MethodSpec.methodBuilder("restore").addTypeVariables(entityspec.typeVariables)
+        .addException(RetrieveDurableEntityError.class).addModifiers(Modifier.PUBLIC, Modifier.STATIC)
+        .returns(TypeName.get(m_elem.asType())).addParameter(allocparam).addParameter(factoryproxysparam)
+        .addParameter(gfieldsparam).addParameter(phandlerparam).addParameter(autoreclaimparam).addCode(code).build();
+    typespecbuilder.addMethod(methodspec);
+  }
+
+  public void generateCode(Filer filer) throws IOException, AnnotationProcessingException {
+    AnnotationSpec classannotation = AnnotationSpec.builder(SuppressWarnings.class)
+        .addMember("value", "$S", "restriction").build();
+
+    TypeSpec.Builder entitybuilder = TypeSpec.classBuilder(m_entityname).superclass(TypeName.get(m_elem.asType()))
+        .addModifiers(Modifier.PUBLIC).addAnnotation(classannotation)
+        .addSuperinterface(ParameterizedTypeName.get(ClassName.get(MemoryDurableEntity.class), m_alloctypevarname))
+        .addTypeVariable(m_alloctypevarname);
+
+    for (TypeParameterElement tpe : m_elem.getTypeParameters()) {
+      entitybuilder.addTypeVariable(TypeVariableName.get(tpe));
+    }
+
+    buildFieldSpecs(entitybuilder, m_dynfieldsinfo);
+    buildFieldSpecs(entitybuilder, m_fieldsinfo);
+
+    buildGettersSpecs(entitybuilder);
+    buildSettersSpecs(entitybuilder);
+
+    buildDurableMethodSpecs(entitybuilder);
+    buildEntityMethodSpecs(entitybuilder);
+
+    TypeSpec entityspec = entitybuilder.build();
+
+    JavaFile entityFile = JavaFile.builder(m_packagename, entityspec).build();
+
+    entityFile.writeTo(filer);
+
+    TypeSpec.Builder factorybuilder = TypeSpec.classBuilder(m_factoryname).addModifiers(Modifier.PUBLIC);
+
+    buildFactoryMethodSpecs(entityspec, factorybuilder);
+
+    JavaFile factoryFile = JavaFile.builder(m_packagename, factorybuilder.build()).build();
+
+    factoryFile.writeTo(filer);
+
+  }
+
+  public static Builder overriding(ExecutableElement method, String varname) {
+
+    Set<Modifier> modifiers = method.getModifiers();
+    if (modifiers.contains(Modifier.PRIVATE) || modifiers.contains(Modifier.FINAL)
+        || modifiers.contains(Modifier.STATIC)) {
+      throw new IllegalArgumentException("cannot override method with modifiers: " + modifiers);
+    }
+
+    String methodName = method.getSimpleName().toString();
+    MethodSpec.Builder methodBuilder = MethodSpec.methodBuilder(methodName);
+
+    methodBuilder.addAnnotation(Override.class);
+    for (AnnotationMirror mirror : method.getAnnotationMirrors()) {
+      AnnotationSpec annotationSpec = AnnotationSpec.get(mirror);
+      if (annotationSpec.type.equals(Override.class)) {
+        continue;
+      }
+      methodBuilder.addAnnotation(annotationSpec);
+    }
+
+    modifiers = new LinkedHashSet<>(modifiers);
+    modifiers.remove(Modifier.ABSTRACT);
+    methodBuilder.addModifiers(modifiers);
+
+    for (TypeParameterElement typeParameterElement : method.getTypeParameters()) {
+      TypeVariable var = (TypeVariable) typeParameterElement.asType();
+      methodBuilder.addTypeVariable(TypeVariableName.get(var));
+    }
+
+    methodBuilder.returns(TypeName.get(method.getReturnType()));
+
+    List<? extends VariableElement> parameters = method.getParameters();
+    TypeName type;
+    for (VariableElement parameter : parameters) {
+      if (parameter.asType().getKind() == TypeKind.TYPEVAR && parameter.asType().toString().equals(varname)) {
+        type = TypeVariableName.get(varname);
+      } else {
+        type = TypeName.get(parameter.asType());
+      }
+
+      String name = parameter.getSimpleName().toString();
+      Set<Modifier> parameterModifiers = parameter.getModifiers();
+      ParameterSpec.Builder parameterBuilder = ParameterSpec.builder(type, name)
+          .addModifiers(parameterModifiers.toArray(new Modifier[parameterModifiers.size()]));
+      for (AnnotationMirror mirror : parameter.getAnnotationMirrors()) {
+        parameterBuilder.addAnnotation(AnnotationSpec.get(mirror));
+      }
+      methodBuilder.addParameter(parameterBuilder.build());
+    }
+    methodBuilder.varargs(method.isVarArgs());
+
+    for (TypeMirror thrownType : method.getThrownTypes()) {
+      methodBuilder.addException(TypeName.get(thrownType));
+    }
+
+    return methodBuilder;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-mnemonic/blob/679bcdc1/core/src/main/java/org/apache/mnemonic/AnnotatedNonVolatileEntityClass.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/mnemonic/AnnotatedNonVolatileEntityClass.java b/core/src/main/java/org/apache/mnemonic/AnnotatedNonVolatileEntityClass.java
deleted file mode 100644
index ac736e9..0000000
--- a/core/src/main/java/org/apache/mnemonic/AnnotatedNonVolatileEntityClass.java
+++ /dev/null
@@ -1,943 +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 org.apache.mnemonic;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.LinkedHashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-import javax.annotation.processing.Filer;
-import javax.annotation.processing.Messager;
-import javax.lang.model.element.AnnotationMirror;
-import javax.lang.model.element.Element;
-import javax.lang.model.element.ElementKind;
-import javax.lang.model.element.ExecutableElement;
-import javax.lang.model.element.Modifier;
-import javax.lang.model.element.TypeElement;
-import javax.lang.model.element.TypeParameterElement;
-import javax.lang.model.element.VariableElement;
-import javax.lang.model.type.TypeKind;
-import javax.lang.model.type.TypeMirror;
-import javax.lang.model.type.TypeVariable;
-import javax.lang.model.util.Elements;
-import javax.lang.model.util.Types;
-
-import com.squareup.javapoet.AnnotationSpec;
-import com.squareup.javapoet.ArrayTypeName;
-import com.squareup.javapoet.ClassName;
-import com.squareup.javapoet.CodeBlock;
-import com.squareup.javapoet.FieldSpec;
-import com.squareup.javapoet.JavaFile;
-import com.squareup.javapoet.MethodSpec;
-import com.squareup.javapoet.MethodSpec.Builder;
-import com.squareup.javapoet.ParameterSpec;
-import com.squareup.javapoet.ParameterizedTypeName;
-import com.squareup.javapoet.TypeName;
-import com.squareup.javapoet.TypeSpec;
-import com.squareup.javapoet.TypeVariableName;
-
-import sun.misc.Unsafe;
-
-/**
- * this class managed to generate generic non-volatile concrete object and their
- * corresponding factory.
- *
- */
-@SuppressWarnings("restriction")
-public class AnnotatedNonVolatileEntityClass {
-  protected class MethodInfo {
-    public ExecutableElement elem;
-    public MethodSpec.Builder specbuilder;
-    public TypeName rettype;
-  }
-
-  protected class FieldInfo {
-    public FieldSpec.Builder specbuilder;
-    public TypeName type;
-    public long id;
-    public String name;
-    public String efproxiesname;
-    public String gftypesname;
-    public long fieldoff;
-    public long fieldsize;
-  }
-
-  protected final String cFACTORYNAMESUFFIX = "Factory";
-  protected final String cPMEMNAMEPREFIX = "NonVolatile_";
-  protected final String cFIELDNAMESUFFIX = String.format("_field_%s", Utils.genRandomString());
-  protected final String cALLOCATORFIELDNAME = String.format("alloc_%s", Utils.genRandomString());
-  protected final String cAUTORECLAIMFIELDNAME = String.format("autoreclaim_%s", Utils.genRandomString());
-  protected final String cUNSAFEFIELDNAME = String.format("unsafe_%s", Utils.genRandomString());
-  protected final String cHOLDERFIELDNAME = String.format("holder_%s", Utils.genRandomString());
-  protected final String cALLOCTYPENAME = String.format("ALLOC_PMem3C93D24F59");
-
-  private Types m_typeutils;
-  private Elements m_elemutils;
-  private TypeElement m_elem;
-
-  private String m_factoryname;
-  private String m_entityname;
-
-  private long m_holdersize;
-
-  private String m_packagename;
-
-  private TypeName m_alloctypename = TypeVariableName.get(cALLOCTYPENAME);
-  private TypeName m_factoryproxystypename = TypeName.get(EntityFactoryProxy[].class);
-  private TypeName m_gfieldstypename = TypeName.get(GenericField.GType[].class);
-  private TypeVariableName m_alloctypevarname = TypeVariableName.get(cALLOCTYPENAME,
-      ParameterizedTypeName.get(ClassName.get(CommonPersistAllocator.class), TypeVariableName.get(cALLOCTYPENAME)));
-
-  private Map<String, MethodInfo> m_gettersinfo = new HashMap<String, MethodInfo>();
-  private Map<String, MethodInfo> m_settersinfo = new HashMap<String, MethodInfo>();
-  private Map<String, FieldInfo> m_dynfieldsinfo = new HashMap<String, FieldInfo>();
-  private Map<String, FieldInfo> m_fieldsinfo = new HashMap<String, FieldInfo>();
-
-  private Map<String, MethodInfo> m_durablemtdinfo = new HashMap<String, MethodInfo>();
-  private Map<String, MethodInfo> m_entitymtdinfo = new HashMap<String, MethodInfo>();
-
-  private long computeTypeSize(TypeMirror type) {
-    long ret;
-    switch (type.getKind()) {
-    case BYTE:
-      ret = 1L;
-      break;
-    case BOOLEAN:
-      ret = 1L;
-      break;
-    case CHAR:
-      ret = 2L;
-      break;
-    case DOUBLE:
-      ret = 8L;
-      break;
-    case FLOAT:
-      ret = 4L;
-      break;
-    case SHORT:
-      ret = 2L;
-      break;
-    case INT:
-      ret = 4L;
-      break;
-    case LONG:
-      ret = 8L;
-      break;
-    default:
-      ret = 8L;
-    }
-    return ret;
-  }
-
-  private boolean isUnboxPrimitive(TypeName tn) {
-    TypeName n = tn;
-    try {
-      n = tn.unbox();
-    } catch (UnsupportedOperationException ex) {
-    }
-    return n.isPrimitive();
-  }
-
-  private TypeName unboxTypeName(TypeName tn) {
-    TypeName n = tn;
-    try {
-      n = tn.unbox();
-    } catch (UnsupportedOperationException ex) {
-    }
-    return n;
-  }
-
-  public AnnotatedNonVolatileEntityClass(TypeElement classElement, Types typeUtils, Elements elementUtils,
-      Messager messager) {
-    m_elem = classElement;
-    m_typeutils = typeUtils;
-    m_elemutils = elementUtils;
-
-    m_packagename = m_elemutils.getPackageOf(m_elem).getQualifiedName().toString();
-
-    m_factoryname = String.format("%s%s", m_elem.getSimpleName(), cFACTORYNAMESUFFIX);
-    m_entityname = String.format("%s%s_%s", cPMEMNAMEPREFIX, m_elem.getSimpleName(), Utils.genRandomString());
-
-    m_durablemtdinfo.put("cancelAutoReclaim", new MethodInfo());
-    m_durablemtdinfo.put("registerAutoReclaim", new MethodInfo());
-    m_durablemtdinfo.put("getNonVolatileHandler", new MethodInfo());
-    m_durablemtdinfo.put("autoReclaim", new MethodInfo());
-    m_durablemtdinfo.put("destroy", new MethodInfo());
-    m_durablemtdinfo.put("getNativeFieldInfo", new MethodInfo());
-
-    m_entitymtdinfo.put("initializeNonVolatileEntity", new MethodInfo());
-    m_entitymtdinfo.put("createNonVolatileEntity", new MethodInfo());
-    m_entitymtdinfo.put("restoreNonVolatileEntity", new MethodInfo());
-
-  }
-
-  public void prepareProcessing() throws AnnotationProcessingException {
-    MethodInfo methodinfo = null;
-    FieldInfo fieldinfo;
-    String methodname;
-    long fieldoff = 0;
-    TypeElement intf_durable = m_elemutils.getTypeElement(Durable.class.getCanonicalName());
-    TypeElement intf_entity = m_elemutils.getTypeElement(MemoryNonVolatileEntity.class.getCanonicalName());
-    // System.err.printf("<><><><><> %s ======\n", intf_entity.toString());
-
-    boolean valid = false;
-    for (TypeMirror tm : m_elem.getInterfaces()) {
-      if (tm.toString().equals(Durable.class.getCanonicalName())) {
-        valid = true;
-        break;
-      }
-    }
-    if (!valid) {
-      throw new AnnotationProcessingException(m_elem, "Not implemented Durable Interface by %s.",
-          m_elem.getSimpleName().toString());
-    }
-
-    fieldinfo = new FieldInfo();
-    fieldinfo.name = String.format("m_unsafe_%s", Utils.genRandomString());
-    fieldinfo.type = TypeName.get(m_elemutils.getTypeElement(Unsafe.class.getCanonicalName()).asType());
-    fieldinfo.specbuilder = FieldSpec.builder(fieldinfo.type, fieldinfo.name, Modifier.PRIVATE);
-    m_fieldsinfo.put("unsafe", fieldinfo);
-
-    fieldinfo = new FieldInfo();
-    fieldinfo.name = String.format("m_holder_%s", Utils.genRandomString());
-    fieldinfo.type = ParameterizedTypeName.get(ClassName.get(MemChunkHolder.class), m_alloctypename);
-    fieldinfo.specbuilder = FieldSpec.builder(fieldinfo.type, fieldinfo.name, Modifier.PRIVATE);
-    m_fieldsinfo.put("holder", fieldinfo);
-
-    fieldinfo = new FieldInfo();
-    fieldinfo.name = String.format("m_autoreclaim_%s", Utils.genRandomString());
-    fieldinfo.type = TypeName.get(m_typeutils.getPrimitiveType(TypeKind.BOOLEAN));
-    fieldinfo.specbuilder = FieldSpec.builder(fieldinfo.type, fieldinfo.name, Modifier.PRIVATE, Modifier.VOLATILE);
-    m_fieldsinfo.put("autoreclaim", fieldinfo);
-
-    fieldinfo = new FieldInfo();
-    fieldinfo.name = String.format("m_allocator_%s", Utils.genRandomString());
-    fieldinfo.type = m_alloctypename;
-    fieldinfo.specbuilder = FieldSpec.builder(fieldinfo.type, fieldinfo.name, Modifier.PRIVATE);
-    m_fieldsinfo.put("allocator", fieldinfo);
-
-    fieldinfo = new FieldInfo();
-    fieldinfo.name = String.format("m_factoryproxy_%s", Utils.genRandomString());
-    fieldinfo.type = m_factoryproxystypename;
-    fieldinfo.specbuilder = FieldSpec.builder(fieldinfo.type, fieldinfo.name, Modifier.PRIVATE);
-    m_fieldsinfo.put("factoryproxy", fieldinfo);
-
-    fieldinfo = new FieldInfo();
-    fieldinfo.name = String.format("m_genericfield_%s", Utils.genRandomString());
-    fieldinfo.type = m_gfieldstypename;
-    fieldinfo.specbuilder = FieldSpec.builder(fieldinfo.type, fieldinfo.name, Modifier.PRIVATE);
-    m_fieldsinfo.put("genericfield", fieldinfo);
-
-    for (Element elem : m_elem.getEnclosedElements()) {
-      if (elem.getKind() == ElementKind.METHOD) {
-        methodname = elem.getSimpleName().toString();
-        // System.err.printf("=========== %s ======\n", methodname);
-        NonVolatileGetter pgetter = elem.getAnnotation(NonVolatileGetter.class);
-        if (pgetter != null) {
-          if (!elem.getModifiers().contains(Modifier.ABSTRACT)) {
-            throw new AnnotationProcessingException(elem, "%s annotated with NonVolatileGetter is not abstract.",
-                methodname);
-          }
-          if (null != elem.getAnnotation(NonVolatileSetter.class)) {
-            throw new AnnotationProcessingException(elem, "%s is annotated with NonVolatileSetter as well.",
-                methodname);
-          }
-          if (!methodname.startsWith("get")) {
-            throw new AnnotationProcessingException(elem, "%s does not comply name convention of getter.", methodname);
-          }
-          methodinfo = new MethodInfo();
-          methodinfo.elem = (ExecutableElement) elem;
-          methodinfo.specbuilder = MethodSpec.overriding(methodinfo.elem);
-          methodinfo.rettype = TypeName.get(methodinfo.elem.getReturnType());
-          m_gettersinfo.put(methodname.substring(3), methodinfo);
-          fieldinfo = new FieldInfo();
-          fieldinfo.type = methodinfo.rettype;
-          if (fieldinfo.type.toString().equals(String.class.getCanonicalName())) {
-            fieldinfo.type = ParameterizedTypeName.get(ClassName.get(MemBufferHolder.class), m_alloctypename);
-          }
-          if (fieldinfo.type instanceof TypeVariableName) {
-            fieldinfo.type = ParameterizedTypeName.get(ClassName.get(GenericField.class), m_alloctypename,
-                fieldinfo.type);
-          }
-          fieldinfo.name = String.format("m_%s_%s", methodname.substring(3).toLowerCase(), Utils.genRandomString());
-          fieldinfo.specbuilder = FieldSpec.builder(fieldinfo.type, fieldinfo.name, Modifier.PRIVATE);
-          fieldinfo.fieldsize = computeTypeSize(methodinfo.elem.getReturnType());
-          fieldinfo.fieldoff = fieldoff;
-          fieldoff += fieldinfo.fieldsize;
-          fieldinfo.efproxiesname = pgetter.EntityFactoryProxies();
-          fieldinfo.gftypesname = pgetter.GenericFieldTypes();
-          fieldinfo.id = pgetter.Id();
-          m_dynfieldsinfo.put(methodname.substring(3), fieldinfo);
-
-        }
-        if (null != elem.getAnnotation(NonVolatileSetter.class)) {
-          if (!elem.getModifiers().contains(Modifier.ABSTRACT)) {
-            throw new AnnotationProcessingException(elem, "%s annotated with NonVolatileSetter is not abstract.",
-                methodname);
-          }
-          if (!methodname.startsWith("set")) {
-            throw new AnnotationProcessingException(elem, "%s does not comply name convention of setter.", methodname);
-          }
-          methodinfo = new MethodInfo();
-          methodinfo.elem = (ExecutableElement) elem;
-          methodinfo.specbuilder = MethodSpec.overriding(methodinfo.elem);
-          m_settersinfo.put(methodname.substring(3), methodinfo);
-        }
-      }
-    }
-
-    m_holdersize = fieldoff;
-
-    // MethodInfo minfo = null;
-    for (String name : m_settersinfo.keySet()) {
-      if (!m_gettersinfo.containsKey(name)) {
-        throw new AnnotationProcessingException(null, "%s has no getter.", name);
-      }
-    }
-
-    for (Element elem : intf_durable.getEnclosedElements()) {
-      if (elem.getKind() == ElementKind.METHOD) {
-        methodname = elem.getSimpleName().toString();
-        if (m_durablemtdinfo.containsKey(methodname)) {
-          // System.err.printf("**++++++++++ %s ======\n", methodname);
-          methodinfo = m_durablemtdinfo.get(methodname);
-          methodinfo.elem = (ExecutableElement) elem;
-          methodinfo.specbuilder = MethodSpec.overriding(methodinfo.elem);
-        }
-      }
-    }
-
-    for (Element elem : intf_entity.getEnclosedElements()) {
-      if (elem.getKind() == ElementKind.METHOD) {
-        methodname = elem.getSimpleName().toString();
-        if (m_entitymtdinfo.containsKey(methodname)) {
-          // System.err.printf("**------- %s ======\n", elem.toString());
-          methodinfo = m_entitymtdinfo.get(methodname);
-          methodinfo.elem = (ExecutableElement) elem;
-          methodinfo.specbuilder = overriding(methodinfo.elem, cALLOCTYPENAME);
-
-        }
-      }
-    }
-    genNFieldInfo();
-  }
-
-  protected String transTypeToUnsafeMethod(TypeName tname, boolean isget) throws AnnotationProcessingException {
-    String ret = null;
-    if (isUnboxPrimitive(tname)) {
-      TypeName tn = unboxTypeName(tname);
-      if (tn.equals(TypeName.BOOLEAN)) {
-        ret = isget ? "getByte" : "putByte";
-      }
-      if (tn.equals(TypeName.BYTE)) {
-        ret = isget ? "getByte" : "putByte";
-      }
-      if (tn.equals(TypeName.CHAR)) {
-        ret = isget ? "getChar" : "putChar";
-      }
-      if (tn.equals(TypeName.DOUBLE)) {
-        ret = isget ? "getDouble" : "putDouble";
-      }
-      if (tn.equals(TypeName.FLOAT)) {
-        ret = isget ? "getFloat" : "putFloat";
-      }
-      if (tn.equals(TypeName.INT)) {
-        ret = isget ? "getInt" : "putInt";
-      }
-      if (tn.equals(TypeName.LONG)) {
-        ret = isget ? "getLong" : "putLong";
-      }
-      if (tn.equals(TypeName.SHORT)) {
-        ret = isget ? "getShort" : "putShort";
-      }
-    } else {
-      ret = isget ? "getAddress" : "putAddress";
-    }
-    if (null == ret) {
-      throw new AnnotationProcessingException(null, "%s is not supported by getters or setters.", tname.toString());
-    }
-    return ret;
-  }
-
-  protected String getIntialValueLiteral(TypeName tname) throws AnnotationProcessingException {
-    String ret = null;
-    if (isUnboxPrimitive(tname)) {
-      TypeName tn = unboxTypeName(tname);
-      if (tn.equals(TypeName.BOOLEAN)) {
-        ret = "false";
-      }
-      if (tn.equals(TypeName.BYTE)) {
-        ret = "(byte)0";
-      }
-      if (tn.equals(TypeName.CHAR)) {
-        ret = "(char)0";
-      }
-      if (tn.equals(TypeName.DOUBLE)) {
-        ret = "(double)0.0";
-      }
-      if (tn.equals(TypeName.FLOAT)) {
-        ret = "(float)0.0";
-      }
-      if (tn.equals(TypeName.INT)) {
-        ret = "(int)0";
-      }
-      if (tn.equals(TypeName.LONG)) {
-        ret = "(long)0";
-      }
-      if (tn.equals(TypeName.SHORT)) {
-        ret = "(short)0";
-      }
-    } else {
-      ret = null;
-    }
-    if (null == ret) {
-      throw new AnnotationProcessingException(null, "%s is not supported to determine the inital value.",
-          tname.toString());
-    }
-    return ret;
-  }
-
-  private int getFactoryProxyIndex(TypeName gtname) throws AnnotationProcessingException {
-    int ret = -1;
-    boolean found = false;
-    if (gtname instanceof TypeVariableName) {
-      for (TypeParameterElement tpe : m_elem.getTypeParameters()) {
-        ++ret;
-        if (tpe.toString().equals(gtname.toString())) {
-          found = true;
-          break;
-        }
-      }
-      if (!found) {
-        throw new AnnotationProcessingException(null, "%s type is not found during factory proxy query.",
-            gtname.toString());
-      }
-    } else {
-      throw new AnnotationProcessingException(null, "%s type is not generic type for factory proxy query.",
-          gtname.toString());
-    }
-    return ret;
-  }
-
-  protected void genNFieldInfo() {
-    FieldInfo dynfieldinfo, fieldinfo;
-    List<long[]> finfo = new ArrayList<long[]>();
-    for (String name : m_gettersinfo.keySet()) {
-      dynfieldinfo = m_dynfieldsinfo.get(name);
-      if (dynfieldinfo.id > 0) {
-        finfo.add(new long[]{dynfieldinfo.id, dynfieldinfo.fieldoff, dynfieldinfo.fieldsize});
-      }
-    }
-
-    fieldinfo = new FieldInfo();
-    fieldinfo.name = String.format("m_nfieldinfo_%s", Utils.genRandomString());
-    fieldinfo.type = ArrayTypeName.of(ArrayTypeName.of(TypeName.LONG));
-    String initlstr = Utils.toInitLiteral(finfo);
-    fieldinfo.specbuilder = FieldSpec.builder(fieldinfo.type, fieldinfo.name, Modifier.PRIVATE, Modifier.STATIC)
-        .initializer("$1L", initlstr);
-    m_fieldsinfo.put("nfieldinfo", fieldinfo);
-  }
-
-  protected void buildGettersSpecs(TypeSpec.Builder typespecbuilder) throws AnnotationProcessingException {
-    MethodInfo methodinfo;
-    TypeName ftname;
-    String unsafename = m_fieldsinfo.get("unsafe").name;
-    String holdername = m_fieldsinfo.get("holder").name;
-    String allocname = m_fieldsinfo.get("allocator").name;
-    String autoreclaimname = m_fieldsinfo.get("autoreclaim").name;
-    String factoryproxyname = m_fieldsinfo.get("factoryproxy").name;
-    String genericfieldname = m_fieldsinfo.get("genericfield").name;
-    FieldInfo dynfieldinfo;
-    CodeBlock.Builder code;
-    String codefmt;
-    for (String name : m_gettersinfo.keySet()) {
-      code = CodeBlock.builder();
-      methodinfo = m_gettersinfo.get(name);
-      dynfieldinfo = m_dynfieldsinfo.get(name);
-      ftname = m_dynfieldsinfo.get(name).type;
-      if (isUnboxPrimitive(ftname)) {
-        if (unboxTypeName(ftname).equals(TypeName.BOOLEAN)) {
-          codefmt = "return 1 == $1N.$4L($2N.get() + $3L)";
-        } else {
-          codefmt = "return $1N.$4L($2N.get() + $3L)";
-        }
-        code.addStatement(codefmt, unsafename, holdername, dynfieldinfo.fieldoff,
-            transTypeToUnsafeMethod(ftname, true));
-      } else {
-        if (methodinfo.rettype.toString().equals(String.class.getCanonicalName())) {
-          code.beginControlFlow("if (null == $1N)", dynfieldinfo.name);
-          code.addStatement("long phandler = $1N.getAddress($2N.get() + $3L)", unsafename, holdername,
-              dynfieldinfo.fieldoff);
-          code.beginControlFlow("if (0L != phandler)");
-          code.addStatement("$1N = $2N.retrieveBuffer(phandler, $3N)", dynfieldinfo.name, allocname, autoreclaimname);
-          code.beginControlFlow("if (null == $1N)", dynfieldinfo.name);
-          code.addStatement("throw new RetrieveNonVolatileEntityError(\"Retrieve String Buffer Failure.\")");
-          code.endControlFlow();
-          code.endControlFlow();
-          code.endControlFlow();
-          code.addStatement("return null == $1N ? null : $1N.get().asCharBuffer().toString()", dynfieldinfo.name);
-        } else if (dynfieldinfo.type.toString().startsWith(GenericField.class.getCanonicalName())) {
-          code.beginControlFlow("if (null == $1N)", dynfieldinfo.name);
-          code.addStatement("$1T proxy = null", TypeName.get(EntityFactoryProxy.class));
-          code.addStatement("$1T gftype = null", TypeName.get(GenericField.GType.class));
-          code.addStatement("int gfpidx = $1L", getFactoryProxyIndex(methodinfo.rettype));
-          code.beginControlFlow("if (null != $1N && $1N.length > gfpidx)", factoryproxyname);
-          code.addStatement("proxy = $1L[gfpidx]", factoryproxyname);
-          code.endControlFlow();
-          code.beginControlFlow("if (null != $1N && $1N.length > gfpidx)", genericfieldname);
-          code.addStatement("gftype = $1L[gfpidx]", genericfieldname);
-          code.nextControlFlow("else");
-          code.addStatement("throw new RetrieveNonVolatileEntityError(\"No Generic Field Type Info.\")");
-          code.endControlFlow();
-          code.addStatement("$1N = new $2T(proxy, gftype, $8L, $9L, $3N, $4N, $5N, $6N.get() + $7L)", dynfieldinfo.name,
-              dynfieldinfo.type, allocname, unsafename, autoreclaimname, holdername, dynfieldinfo.fieldoff,
-              dynfieldinfo.efproxiesname, dynfieldinfo.gftypesname);
-          code.endControlFlow();
-          code.addStatement("return $1N.get()", dynfieldinfo.name);
-        } else {
-          code.beginControlFlow("if (null == $1N)", dynfieldinfo.name);
-          code.addStatement("long phandler = $1N.getAddress($2N.get() + $3L)", unsafename, holdername,
-              dynfieldinfo.fieldoff);
-          code.beginControlFlow("if (0L != phandler)");
-          code.addStatement("$1N = $4N.restore($2N, $5L, $6L, phandler, $3N)", dynfieldinfo.name, allocname,
-              autoreclaimname, String.format("%s%s",
-                  m_typeutils.asElement(methodinfo.elem.getReturnType()).getSimpleName(), cFACTORYNAMESUFFIX),
-              dynfieldinfo.efproxiesname, dynfieldinfo.gftypesname);
-          code.endControlFlow();
-          code.endControlFlow();
-          code.addStatement("return $1N", dynfieldinfo.name);
-        }
-      }
-      typespecbuilder.addMethod(methodinfo.specbuilder.addCode(code.build()).build());
-    }
-  }
-
-  protected String gsetterName(String name, boolean isget) {
-    return String.format("%s%s", isget ? "get" : "set", name); // Character.toUpperCase(name.charAt(0))
-                                                               // +
-                                                               // name.substring(1));
-  }
-
-  protected void buildSettersSpecs(TypeSpec.Builder typespecbuilder) throws AnnotationProcessingException {
-    MethodInfo methodinfo;
-    TypeName ftname, valtname;
-    String unsafename = m_fieldsinfo.get("unsafe").name;
-    String holdername = m_fieldsinfo.get("holder").name;
-    String allocname = m_fieldsinfo.get("allocator").name;
-    String autoreclaimname = m_fieldsinfo.get("autoreclaim").name;
-    String factoryproxyname = m_fieldsinfo.get("factoryproxy").name;
-    String genericfieldname = m_fieldsinfo.get("genericfield").name;
-    FieldInfo dynfieldinfo;
-    CodeBlock.Builder code;
-    VariableElement arg0;
-    VariableElement arg1;
-    String codefmt;
-    for (String name : m_settersinfo.keySet()) {
-      code = CodeBlock.builder();
-      methodinfo = m_settersinfo.get(name);
-      dynfieldinfo = m_dynfieldsinfo.get(name);
-      ftname = m_dynfieldsinfo.get(name).type;
-      valtname = m_gettersinfo.get(name).rettype;
-      arg0 = methodinfo.elem.getParameters().get(0);
-      if (!TypeName.get(arg0.asType()).equals(valtname)) {
-        throw new AnnotationProcessingException(null, "%s has inconsistent value type with its getter/setter.", name);
-      }
-      if (isUnboxPrimitive(ftname)) {
-        if (unboxTypeName(ftname).equals(TypeName.BOOLEAN)) {
-          codefmt = "$1N.$4L($2N.get() + $3L, $5L?1:0)";
-        } else {
-          codefmt = "$1N.$4L($2N.get() + $3L, $5L)";
-        }
-        code.addStatement(codefmt, unsafename, holdername, dynfieldinfo.fieldoff,
-            transTypeToUnsafeMethod(ftname, false), arg0);
-      } else {
-        try {
-          arg1 = methodinfo.elem.getParameters().get(1);
-          if (!TypeName.BOOLEAN.equals(TypeName.get(arg1.asType()))) {
-            throw new AnnotationProcessingException(null, "the second parameter of %s's setter is not boolean type.",
-                name);
-          }
-        } catch (IndexOutOfBoundsException ex) {
-          throw new AnnotationProcessingException(null, "%s's setter has no second parameters for non primitive type.",
-              name);
-        }
-        if (valtname.toString().equals(String.class.getCanonicalName())) {
-          code.beginControlFlow("if ($1L && null != $2L())", arg1, gsetterName(name, true));
-          code.addStatement("$1N.destroy()", dynfieldinfo.name);
-          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.putLong($2N.get() + $3L, 0L)", unsafename, holdername, dynfieldinfo.fieldoff);
-          code.nextControlFlow("else");
-          code.addStatement("$1N = $2N.createBuffer($3L.length() * 2, $4N)", dynfieldinfo.name, allocname, arg0,
-              autoreclaimname);
-          code.beginControlFlow("if (null == $1N)", dynfieldinfo.name);
-          code.addStatement("throw new OutOfPersistentMemory(\"Create Non-Volatile String Error!\")");
-          code.endControlFlow();
-          code.addStatement("$1N.get().asCharBuffer().put($2L)", dynfieldinfo.name, arg0);
-          code.addStatement("$1N.putLong($2N.get() + $3L, $4N.getBufferHandler($5N))", unsafename, holdername,
-              dynfieldinfo.fieldoff, allocname, dynfieldinfo.name);
-          code.endControlFlow();
-        } else if (dynfieldinfo.type.toString().startsWith(GenericField.class.getCanonicalName())) {
-          code.beginControlFlow("if (null == $1N)", dynfieldinfo.name);
-          code.addStatement("$1T proxy = null", TypeName.get(EntityFactoryProxy.class));
-          code.addStatement("$1T gftype = null", TypeName.get(GenericField.GType.class));
-          code.addStatement("int gfpidx = $1L", getFactoryProxyIndex(valtname));
-          code.beginControlFlow("if (null != $1N && $1N.length > gfpidx)", factoryproxyname);
-          code.addStatement("proxy = $1L[gfpidx]", factoryproxyname);
-          code.endControlFlow();
-          code.beginControlFlow("if (null != $1N && $1N.length > gfpidx)", genericfieldname);
-          code.addStatement("gftype = $1L[gfpidx]", genericfieldname);
-          code.nextControlFlow("else");
-          code.addStatement("throw new RetrieveNonVolatileEntityError(\"No Generic Field Type Info.\")");
-          code.endControlFlow();
-          code.addStatement("$1N = new $2T(proxy, gftype, $8L, $9L, $3N, $4N, $5N, $6N.get() + $7L)", dynfieldinfo.name,
-              dynfieldinfo.type, allocname, unsafename, autoreclaimname, holdername, dynfieldinfo.fieldoff,
-              dynfieldinfo.efproxiesname, dynfieldinfo.gftypesname);
-          code.endControlFlow();
-          code.beginControlFlow("if (null != $1L)", dynfieldinfo.name);
-          code.addStatement("$1N.set($2L, $3L)", dynfieldinfo.name, arg0, arg1);
-          code.nextControlFlow("else");
-          code.addStatement("throw new RetrieveNonVolatileEntityError(\"GenericField is null!\")");
-          code.endControlFlow();
-        } else {
-          code.beginControlFlow("if ($1L && null != $2L())", arg1, gsetterName(name, true));
-          code.addStatement("$1N.destroy()", dynfieldinfo.name);
-          code.addStatement("$1N = null", dynfieldinfo.name);
-          code.addStatement("$1N.putAddress($2N.get() + $3L, 0L)", unsafename, holdername, dynfieldinfo.fieldoff);
-          code.endControlFlow();
-          code.addStatement("$1N = $2L", dynfieldinfo.name, arg0);
-          code.addStatement("$1N.putLong($2N.get() + $3L, null == $4N ? 0L : $4N.getNonVolatileHandler())", unsafename,
-              holdername, dynfieldinfo.fieldoff, dynfieldinfo.name);
-        }
-      }
-      typespecbuilder.addMethod(methodinfo.specbuilder.addCode(code.build()).build());
-    }
-  }
-
-  protected void buildDurableMethodSpecs(TypeSpec.Builder typespecbuilder) throws AnnotationProcessingException {
-    MethodInfo methodinfo;
-    CodeBlock.Builder code;
-    FieldInfo dynfieldinfo;
-    String holdername = m_fieldsinfo.get("holder").name;
-    String allocname = m_fieldsinfo.get("allocator").name;
-    String autoreclaimname = m_fieldsinfo.get("autoreclaim").name;
-    for (String name : m_durablemtdinfo.keySet()) {
-      methodinfo = m_durablemtdinfo.get(name);
-      code = CodeBlock.builder();
-      switch (name) {
-      case "cancelAutoReclaim":
-        code.addStatement("$1N.cancelAutoReclaim()", holdername);
-        for (String fname : m_dynfieldsinfo.keySet()) {
-          dynfieldinfo = m_dynfieldsinfo.get(fname);
-          if (!isUnboxPrimitive(dynfieldinfo.type)) {
-            code.beginControlFlow("if (null != $1N)", dynfieldinfo.name);
-            code.addStatement("$1N.cancelAutoReclaim()", dynfieldinfo.name);
-            code.endControlFlow();
-          }
-        }
-        code.addStatement("$1N = false", autoreclaimname);
-        break;
-      case "registerAutoReclaim":
-        code.addStatement("$1N.registerAutoReclaim()", holdername);
-        for (String fname : m_dynfieldsinfo.keySet()) {
-          dynfieldinfo = m_dynfieldsinfo.get(fname);
-          if (!isUnboxPrimitive(dynfieldinfo.type)) {
-            code.beginControlFlow("if (null != $1N)", dynfieldinfo.name);
-            code.addStatement("$1N.registerAutoReclaim()", dynfieldinfo.name);
-            code.endControlFlow();
-          }
-        }
-        code.addStatement("$1N = true", autoreclaimname);
-        break;
-      case "getNonVolatileHandler":
-        code.addStatement("return $1N.getChunkHandler($2N)", allocname, holdername);
-        break;
-      case "autoReclaim":
-        code.addStatement("return $1N", autoreclaimname);
-        break;
-      case "destroy":
-        code.addStatement("$1N.destroy()", holdername);
-        for (String fname : m_dynfieldsinfo.keySet()) {
-          dynfieldinfo = m_dynfieldsinfo.get(fname);
-          if (!isUnboxPrimitive(dynfieldinfo.type)) {
-            code.beginControlFlow("if (null != $1N)", dynfieldinfo.name);
-            code.addStatement("$1N.destroy()", dynfieldinfo.name);
-            code.addStatement("$1N = null", dynfieldinfo.name);
-            code.endControlFlow();
-          }
-        }
-        break;
-      case "getNativeFieldInfo":
-        code.addStatement("return $1N", m_fieldsinfo.get("nfieldinfo").name);
-        break;
-      default:
-        throw new AnnotationProcessingException(null, "Method %s is not supported.", name);
-      }
-      typespecbuilder.addMethod(methodinfo.specbuilder.addCode(code.build()).build());
-    }
-  }
-
-  protected void buildEntityMethodSpecs(TypeSpec.Builder typespecbuilder) throws AnnotationProcessingException {
-    MethodInfo methodinfo;
-    CodeBlock.Builder code;
-    VariableElement arg0, arg1, arg2, arg3, arg4;
-    String unsafename = m_fieldsinfo.get("unsafe").name;
-    String holdername = m_fieldsinfo.get("holder").name;
-    String allocname = m_fieldsinfo.get("allocator").name;
-    String autoreclaimname = m_fieldsinfo.get("autoreclaim").name;
-    String factoryproxyname = m_fieldsinfo.get("factoryproxy").name;
-    String genericfieldname = m_fieldsinfo.get("genericfield").name;
-    for (String name : m_entitymtdinfo.keySet()) {
-      methodinfo = m_entitymtdinfo.get(name);
-      code = CodeBlock.builder();
-      arg0 = methodinfo.elem.getParameters().get(0);
-      arg1 = methodinfo.elem.getParameters().get(1);
-      arg2 = methodinfo.elem.getParameters().get(2);
-      switch (name) {
-      case "initializeNonVolatileEntity":
-        arg3 = methodinfo.elem.getParameters().get(3);
-        code.addStatement("$1N = $2L", allocname, arg0);
-        code.addStatement("$1N = $2L", factoryproxyname, arg1);
-        code.addStatement("$1N = $2L", genericfieldname, arg2);
-        code.addStatement("$1N = $2L", autoreclaimname, arg3);
-        code.beginControlFlow("try");
-        code.addStatement("$1N = $2T.getUnsafe()", unsafename, Utils.class);
-        code.nextControlFlow("catch (Exception e)");
-        code.addStatement("e.printStackTrace()");
-        code.endControlFlow();
-        break;
-      case "createNonVolatileEntity":
-        arg3 = methodinfo.elem.getParameters().get(3);
-        code.addStatement("initializeNonVolatileEntity($1L, $2L, $3L, $4L)", arg0, arg1, arg2, arg3);
-        code.addStatement("$1N = $2N.createChunk($3L, $4N)", holdername, allocname, m_holdersize, autoreclaimname);
-        code.beginControlFlow("if (null == $1N)", holdername);
-        code.addStatement("throw new OutOfPersistentMemory(\"Create Non-Volatile Entity Error!\")");
-        code.endControlFlow();
-        // code.beginControlFlow("try");
-        // for (String fname : m_dynfieldsinfo.keySet()) {
-        // dynfieldinfo = m_dynfieldsinfo.get(fname);
-        // if (isUnboxPrimitive(dynfieldinfo.type)) {
-        // code.addStatement("$1N($2L)", gsetterName(fname, false),
-        // getIntialValueLiteral(dynfieldinfo.type));
-        // } else {
-        // code.addStatement("$1N(null, false)", gsetterName(fname, false));
-        // }
-        // }
-        // code.nextControlFlow("catch(RetrieveNonVolatileEntityError ex)");
-        // code.endControlFlow();
-        code.addStatement("initializeAfterCreate()");
-        break;
-      case "restoreNonVolatileEntity":
-        arg3 = methodinfo.elem.getParameters().get(3);
-        arg4 = methodinfo.elem.getParameters().get(4);
-        code.addStatement("initializeNonVolatileEntity($1L, $2L, $3L, $4L)", arg0, arg1, arg2, arg4);
-        code.beginControlFlow("if (0L == $1L)", arg3);
-        code.addStatement("throw new RetrieveNonVolatileEntityError(\"Input handler is null on $1N.\")", name);
-        code.endControlFlow();
-        code.addStatement("$1N = $2N.retrieveChunk($3L, $4N)", holdername, allocname, arg3, autoreclaimname);
-        code.beginControlFlow("if (null == $1N)", holdername);
-        code.addStatement("throw new RetrieveNonVolatileEntityError(\"Retrieve Entity Failure!\")");
-        code.endControlFlow();
-        code.addStatement("initializeAfterRestore()");
-        break;
-      default:
-        throw new AnnotationProcessingException(null, "Method %s is not supported.", name);
-      }
-      typespecbuilder.addMethod(methodinfo.specbuilder.addCode(code.build()).build());
-    }
-  }
-
-  protected void buildFieldSpecs(TypeSpec.Builder typespecbuilder, Map<String, FieldInfo> fieldinfos) {
-    FieldInfo fieldinfo;
-    for (String name : fieldinfos.keySet()) {
-      fieldinfo = fieldinfos.get(name);
-      if (null != fieldinfo.specbuilder) {
-        typespecbuilder.addField(fieldinfo.specbuilder.build());
-      }
-    }
-  }
-
-  protected void buildFactoryMethodSpecs(TypeSpec entityspec, TypeSpec.Builder typespecbuilder)
-      throws AnnotationProcessingException {
-    MethodSpec methodspec;
-    CodeBlock code;
-
-    TypeName entitytn = ParameterizedTypeName.get(ClassName.get(m_packagename, m_entityname),
-        entityspec.typeVariables.toArray(new TypeVariableName[0]));
-
-    ParameterSpec allocparam = ParameterSpec.builder(m_alloctypename, "allocator").build();
-    code = CodeBlock.builder().addStatement("return create($1L, false)", allocparam.name).build();
-    methodspec = MethodSpec.methodBuilder("create").addTypeVariables(entityspec.typeVariables)
-        .addException(OutOfPersistentMemory.class).addModifiers(Modifier.PUBLIC, Modifier.STATIC)
-        .returns(TypeName.get(m_elem.asType())).addParameter(allocparam).addCode(code).build();
-    typespecbuilder.addMethod(methodspec);
-
-    ParameterSpec autoreclaimparam = ParameterSpec.builder(TypeName.BOOLEAN, "autoreclaim").build();
-    code = CodeBlock.builder()
-        .addStatement("return create($1L, null, null, $2L)", allocparam.name, autoreclaimparam.name).build();
-    methodspec = MethodSpec.methodBuilder("create").addTypeVariables(entityspec.typeVariables)
-        .addException(OutOfPersistentMemory.class).addModifiers(Modifier.PUBLIC, Modifier.STATIC)
-        .returns(TypeName.get(m_elem.asType())).addParameter(allocparam).addParameter(autoreclaimparam).addCode(code)
-        .build();
-    typespecbuilder.addMethod(methodspec);
-
-    ParameterSpec factoryproxysparam = ParameterSpec.builder(m_factoryproxystypename, "factoryproxys").build();
-    ParameterSpec gfieldsparam = ParameterSpec.builder(m_gfieldstypename, "gfields").build();
-    code = CodeBlock.builder().addStatement("$1T entity = new $1T()", entitytn)
-        .addStatement("entity.setupGenericInfo($1N, $2N)", factoryproxysparam.name, gfieldsparam.name)
-        .addStatement("entity.createNonVolatileEntity($1L, $2L, $3L, $4L)", allocparam.name, factoryproxysparam.name,
-            gfieldsparam.name, autoreclaimparam.name)
-        .addStatement("return entity").build();
-    methodspec = MethodSpec.methodBuilder("create").addTypeVariables(entityspec.typeVariables)
-        .addException(OutOfPersistentMemory.class).addModifiers(Modifier.PUBLIC, Modifier.STATIC)
-        .returns(TypeName.get(m_elem.asType())).addParameter(allocparam).addParameter(factoryproxysparam)
-        .addParameter(gfieldsparam).addParameter(autoreclaimparam).addCode(code).build();
-    typespecbuilder.addMethod(methodspec);
-
-    ParameterSpec phandlerparam = ParameterSpec.builder(TypeName.LONG, "phandler").build();
-    code = CodeBlock.builder().addStatement("return restore($1L, $2L, false)", allocparam.name, phandlerparam.name)
-        .build();
-    methodspec = MethodSpec.methodBuilder("restore").addTypeVariables(entityspec.typeVariables)
-        .addException(RetrieveNonVolatileEntityError.class).addModifiers(Modifier.PUBLIC, Modifier.STATIC)
-        .returns(TypeName.get(m_elem.asType())).addParameter(allocparam).addParameter(phandlerparam).addCode(code)
-        .build();
-    typespecbuilder.addMethod(methodspec);
-
-    code = CodeBlock.builder().addStatement("return restore($1L, null, null, $2L, $3L)", allocparam.name,
-        phandlerparam.name, autoreclaimparam.name).build();
-    methodspec = MethodSpec.methodBuilder("restore").addTypeVariables(entityspec.typeVariables)
-        .addException(RetrieveNonVolatileEntityError.class).addModifiers(Modifier.PUBLIC, Modifier.STATIC)
-        .returns(TypeName.get(m_elem.asType())).addParameter(allocparam).addParameter(phandlerparam)
-        .addParameter(autoreclaimparam).addCode(code).build();
-    typespecbuilder.addMethod(methodspec);
-
-    code = CodeBlock.builder().addStatement("$1T entity = new $1T()", entitytn)
-        .addStatement("entity.setupGenericInfo($1N, $2N)", factoryproxysparam.name, gfieldsparam.name)
-        .addStatement("entity.restoreNonVolatileEntity($1L, $2L, $3L, $4L, $5L)", allocparam.name,
-            factoryproxysparam.name, gfieldsparam.name, phandlerparam.name, autoreclaimparam.name)
-        .addStatement("return entity").build();
-    methodspec = MethodSpec.methodBuilder("restore").addTypeVariables(entityspec.typeVariables)
-        .addException(RetrieveNonVolatileEntityError.class).addModifiers(Modifier.PUBLIC, Modifier.STATIC)
-        .returns(TypeName.get(m_elem.asType())).addParameter(allocparam).addParameter(factoryproxysparam)
-        .addParameter(gfieldsparam).addParameter(phandlerparam).addParameter(autoreclaimparam).addCode(code).build();
-    typespecbuilder.addMethod(methodspec);
-  }
-
-  public void generateCode(Filer filer) throws IOException, AnnotationProcessingException {
-    AnnotationSpec classannotation = AnnotationSpec.builder(SuppressWarnings.class)
-        .addMember("value", "$S", "restriction").build();
-
-    TypeSpec.Builder entitybuilder = TypeSpec.classBuilder(m_entityname).superclass(TypeName.get(m_elem.asType()))
-        .addModifiers(Modifier.PUBLIC).addAnnotation(classannotation)
-        .addSuperinterface(ParameterizedTypeName.get(ClassName.get(MemoryNonVolatileEntity.class), m_alloctypevarname))
-        .addTypeVariable(m_alloctypevarname);
-
-    for (TypeParameterElement tpe : m_elem.getTypeParameters()) {
-      entitybuilder.addTypeVariable(TypeVariableName.get(tpe));
-    }
-
-    buildFieldSpecs(entitybuilder, m_dynfieldsinfo);
-    buildFieldSpecs(entitybuilder, m_fieldsinfo);
-
-    buildGettersSpecs(entitybuilder);
-    buildSettersSpecs(entitybuilder);
-
-    buildDurableMethodSpecs(entitybuilder);
-    buildEntityMethodSpecs(entitybuilder);
-
-    TypeSpec entityspec = entitybuilder.build();
-
-    JavaFile entityFile = JavaFile.builder(m_packagename, entityspec).build();
-
-    entityFile.writeTo(filer);
-
-    TypeSpec.Builder factorybuilder = TypeSpec.classBuilder(m_factoryname).addModifiers(Modifier.PUBLIC);
-
-    buildFactoryMethodSpecs(entityspec, factorybuilder);
-
-    JavaFile factoryFile = JavaFile.builder(m_packagename, factorybuilder.build()).build();
-
-    factoryFile.writeTo(filer);
-
-  }
-
-  public static Builder overriding(ExecutableElement method, String varname) {
-
-    Set<Modifier> modifiers = method.getModifiers();
-    if (modifiers.contains(Modifier.PRIVATE) || modifiers.contains(Modifier.FINAL)
-        || modifiers.contains(Modifier.STATIC)) {
-      throw new IllegalArgumentException("cannot override method with modifiers: " + modifiers);
-    }
-
-    String methodName = method.getSimpleName().toString();
-    MethodSpec.Builder methodBuilder = MethodSpec.methodBuilder(methodName);
-
-    methodBuilder.addAnnotation(Override.class);
-    for (AnnotationMirror mirror : method.getAnnotationMirrors()) {
-      AnnotationSpec annotationSpec = AnnotationSpec.get(mirror);
-      if (annotationSpec.type.equals(Override.class)) {
-        continue;
-      }
-      methodBuilder.addAnnotation(annotationSpec);
-    }
-
-    modifiers = new LinkedHashSet<>(modifiers);
-    modifiers.remove(Modifier.ABSTRACT);
-    methodBuilder.addModifiers(modifiers);
-
-    for (TypeParameterElement typeParameterElement : method.getTypeParameters()) {
-      TypeVariable var = (TypeVariable) typeParameterElement.asType();
-      methodBuilder.addTypeVariable(TypeVariableName.get(var));
-    }
-
-    methodBuilder.returns(TypeName.get(method.getReturnType()));
-
-    List<? extends VariableElement> parameters = method.getParameters();
-    TypeName type;
-    for (VariableElement parameter : parameters) {
-      if (parameter.asType().getKind() == TypeKind.TYPEVAR && parameter.asType().toString().equals(varname)) {
-        type = TypeVariableName.get(varname);
-      } else {
-        type = TypeName.get(parameter.asType());
-      }
-
-      String name = parameter.getSimpleName().toString();
-      Set<Modifier> parameterModifiers = parameter.getModifiers();
-      ParameterSpec.Builder parameterBuilder = ParameterSpec.builder(type, name)
-          .addModifiers(parameterModifiers.toArray(new Modifier[parameterModifiers.size()]));
-      for (AnnotationMirror mirror : parameter.getAnnotationMirrors()) {
-        parameterBuilder.addAnnotation(AnnotationSpec.get(mirror));
-      }
-      methodBuilder.addParameter(parameterBuilder.build());
-    }
-    methodBuilder.varargs(method.isVarArgs());
-
-    for (TypeMirror thrownType : method.getThrownTypes()) {
-      methodBuilder.addException(TypeName.get(thrownType));
-    }
-
-    return methodBuilder;
-  }
-
-}



[4/5] incubator-mnemonic git commit: MNEMONIC-20: Formalize the names of annotations, classes and methods

Posted by ga...@apache.org.
MNEMONIC-20: Formalize the names of annotations, classes and methods


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

Branch: refs/heads/master
Commit: 679bcdc151213ca4e1bf57cd6cebddd2ae36841c
Parents: 254238f
Author: Wang, Gang(Gary) <ga...@intel.com>
Authored: Wed Apr 13 18:28:35 2016 -0700
Committer: Wang, Gang(Gary) <ga...@intel.com>
Committed: Wed Apr 13 18:28:35 2016 -0700

----------------------------------------------------------------------
 README.md                                       |  53 +-
 collections/pom.xml                             |   4 +-
 .../mnemonic/collections/DurableNodeValue.java  | 172 ++++
 .../collections/NonVolatileNodeValue.java       | 172 ----
 .../collections/DurableNodeValueNGTest.java     | 274 ++++++
 .../collections/DurablePersonNGTest.java        | 148 +++
 .../collections/NonVolatileNodeValueNGTest.java | 274 ------
 .../collections/NonVolatilePersonNGTest.java    | 148 ---
 .../org/apache/mnemonic/collections/Person.java |  42 +-
 collections/src/test/resources/testng.xml       |   8 +-
 core/pom.xml                                    |   2 +-
 .../mnemonic/AnnotatedDurableEntityClass.java   | 943 +++++++++++++++++++
 .../AnnotatedNonVolatileEntityClass.java        | 943 -------------------
 .../apache/mnemonic/BigDataMemAllocator.java    | 278 ------
 .../apache/mnemonic/BigDataPMemAllocator.java   | 470 ---------
 .../apache/mnemonic/CommonDurableAllocator.java | 140 +++
 .../apache/mnemonic/CommonPersistAllocator.java | 140 ---
 .../main/java/org/apache/mnemonic/Durable.java  |   4 +-
 .../java/org/apache/mnemonic/DurableEntity.java |  34 +
 .../apache/mnemonic/DurableEntityProcessor.java | 147 +++
 .../java/org/apache/mnemonic/DurableGetter.java |  38 +
 .../java/org/apache/mnemonic/DurableSetter.java |  34 +
 .../org/apache/mnemonic/EntityFactoryProxy.java |   2 +-
 .../java/org/apache/mnemonic/GenericField.java  |  16 +-
 .../apache/mnemonic/MemoryDurableEntity.java    |  36 +
 .../mnemonic/MemoryNonVolatileEntity.java       |  36 -
 .../apache/mnemonic/NVMAddressTranslator.java   |  61 ++
 .../org/apache/mnemonic/NonVolatileEntity.java  |  34 -
 .../mnemonic/NonVolatileEntityProcessor.java    | 147 ---
 .../org/apache/mnemonic/NonVolatileGetter.java  |  38 -
 .../mnemonic/NonVolatileMemAllocator.java       | 474 ++++++++++
 .../org/apache/mnemonic/NonVolatileSetter.java  |  34 -
 .../org/apache/mnemonic/OutOfHybridMemory.java  |  32 +
 .../apache/mnemonic/OutOfPersistentMemory.java  |  32 -
 .../apache/mnemonic/PMAddressTranslator.java    |  61 --
 .../mnemonic/RetrieveDurableEntityError.java    |  35 +
 .../RetrieveNonVolatileEntityError.java         |  35 -
 .../apache/mnemonic/VolatileMemAllocator.java   | 278 ++++++
 .../javax.annotation.processing.Processor       |   2 +-
 .../mnemonic/BigDataMemAllocatorNGTest.java     |  68 --
 .../mnemonic/BigDataPMemAllocatorNGTest.java    | 132 ---
 .../mnemonic/ByteBufferSerializerNGTest.java    |   4 +-
 .../apache/mnemonic/DurablePersonNGTest.java    | 144 +++
 .../MemBufferHolderCachePoolNGTest.java         |   8 +-
 .../apache/mnemonic/MemClusteringNGTest.java    |  28 +-
 .../mnemonic/NonVolatileMemAllocatorNGTest.java | 132 +++
 .../mnemonic/NonVolatilePersonNGTest.java       | 144 ---
 .../test/java/org/apache/mnemonic/Person.java   |  32 +-
 .../mnemonic/VolatileAllocatorNGTest.java       |  68 ++
 core/src/test/resources/testng.xml              |  10 +-
 .../java/org/apache/mnemonic/example/Main.java  |   4 +-
 51 files changed, 3300 insertions(+), 3295 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-mnemonic/blob/679bcdc1/README.md
----------------------------------------------------------------------
diff --git a/README.md b/README.md
index 6b8d5c2..3feacf0 100644
--- a/README.md
+++ b/README.md
@@ -16,6 +16,7 @@ This library comes up with a new programming model we call it non-volatile objec
 * Reduce GC Overheads as the following chart shown (collected from Apache Spark experiments)
 * [Coming major feature]: Distributed Object Graphs (DOG)
 * [Coming major feature]: Columnar-aware object graphs & collections (Apache Arrow based optimization)
+* [Coming major feature]: Native Massive Object Graph (NMOG) Computing
 
 ![Mnemonic_GC_stats](http://nonvolatilecomputing.github.io/Mnemonic/images/mnemonic_GC_stats.png)
 
@@ -33,14 +34,14 @@ This library comes up with a new programming model we call it non-volatile objec
 
 ```java
 /**
- * a non-volatile class should be abstract, implement Durable interface and marked with @NonVolatileEntity annotation
+ * a durable class should be abstract, implement Durable interface and marked with @DurableEntity annotation
  */
-@NonVolatileEntity
+@DurableEntity
 public abstract class Person<E> implements Durable, Comparable<Person<E>> {
         E element; // Generic Type
 
         /**
-         * callback for this non-volatile object creation
+         * callback for this durable object creation
          */
         @Override
         public void initializeAfterCreate() { 
@@ -48,7 +49,7 @@ public abstract class Person<E> implements Durable, Comparable<Person<E>> {
         }
         
         /**
-         * callback for this non-valatile object recovery
+         * callback for this durable object recovery
          */
         @Override
         public void initializeAfterRestore() { 
@@ -64,7 +65,7 @@ public abstract class Person<E> implements Durable, Comparable<Person<E>> {
         }
 
         @Test
-        public void testOutput() throws RetrieveNonVolatileEntityError {
+        public void testOutput() throws RetrieveDurableEntityError {
                 System.out.printf("Person %s, Age: %d ( %s ) \n", getName(), getAge(),
                                 null == getMother()? "No Recorded Mother" : "Has Recorded Mother");
         }
@@ -77,27 +78,27 @@ public abstract class Person<E> implements Durable, Comparable<Person<E>> {
         }
 
         /**
-         * Getters and Setters for non-volatile fields marked with @NonVolatileGetter and @NonVolatileSetter
+         * Getters and Setters for non-volatile fields marked with @DurableGetter and @DurableSetter
          */
-        @NonVolatileGetter
+        @DurableGetter
         abstract public Short getAge();
-        @NonVolatileSetter
+        @DurableSetter
         abstract public void setAge(Short age);
 
-        @NonVolatileGetter
-        abstract public String getName() throws RetrieveNonVolatileEntityError;
-        @NonVolatileSetter
-        abstract public void setName(String name, boolean destroy) throws OutOfPersistentMemory, RetrieveNonVolatileEntityError;
+        @DurableGetter
+        abstract public String getName() throws RetrieveDurableEntityError;
+        @DurableSetter
+        abstract public void setName(String name, boolean destroy) throws OutOfPersistentMemory, RetrieveDurableEntityError;
 
-        @NonVolatileGetter
-        abstract public Person<E> getMother() throws RetrieveNonVolatileEntityError;
-        @NonVolatileSetter
-        abstract public void setMother(Person<E> mother, boolean destroy) throws RetrieveNonVolatileEntityError;
+        @DurableGetter
+        abstract public Person<E> getMother() throws RetrieveDurableEntityError;
+        @DurableSetter
+        abstract public void setMother(Person<E> mother, boolean destroy) throws RetrieveDurableEntityError;
 
-        @NonVolatileGetter
-        abstract public Person<E> getFather() throws RetrieveNonVolatileEntityError;
-        @NonVolatileSetter
-        abstract public void setFather(Person<E> mother, boolean destroy) throws RetrieveNonVolatileEntityError;
+        @DurableGetter
+        abstract public Person<E> getFather() throws RetrieveDurableEntityError;
+        @DurableSetter
+        abstract public void setFather(Person<E> mother, boolean destroy) throws RetrieveDurableEntityError;
 }
 
 ```
@@ -107,7 +108,7 @@ public abstract class Person<E> implements Durable, Comparable<Person<E>> {
 ##### Setup an allocator for non-volatile object graphs.
 ```java
         // create an allocator instance
-        BigDataPMemAllocator act = new BigDataPMemAllocator(1024 * 1024 * 8, "./pobj_person.dat", true);
+        NonVolatileMemAllocator act = new NonVolatileMemAllocator(1024 * 1024 * 8, "./pobj_person.dat", true);
         
         // fetch handler store capacity from this non-volatile storage managed by this allocator
         KEYCAPACITY = act.handlerCapacity();
@@ -126,7 +127,7 @@ public abstract class Person<E> implements Durable, Comparable<Person<E>> {
         person.setName(String.format("Name: [%s]", UUID.randomUUID().toString()), true);
 
         // keep this person on non-volatile handler store
-        act.setHandler(keyidx, person.getNonVolatileHandler());
+        act.setHandler(keyidx, person.getHandler());
 
         for (int deep = 0; deep < rand.nextInt(100); ++deep) {
         
@@ -219,15 +220,15 @@ To run an example:
 To run several test cases:
 ```bash
   
-  $ mvn -Dtest=NonVolatilePersonNGTest test -pl core -DskipTests=false # a testcase for module "core" that requires 'pmalloc' allocator service to pass
+  $ mvn -Dtest=DurablePersonNGTest test -pl core -DskipTests=false # a testcase for module "core" that requires 'pmalloc' allocator service to pass
   
-  $ mvn -Dtest=BigDataMemAllocatorNGTest test -pl core -DskipTests=false # the second testcase for module "core" that requires 'vmem' allocator service to pass
+  $ mvn -Dtest=NonVolatileMemAllocatorNGTest test -pl core -DskipTests=false # the second testcase for module "core" that requires 'vmem' allocator service to pass
   
   $ mvn -Dtest=MemClusteringNGTest test -pl core -DskipTests=false # the third testcase for module "core" that requires 'vmem allocator service to pass
   
-  $ mvn -Dtest=NonVolatileNodeValueNGTest  test -pl collections -DskipTests=false # a testcase for module "collection" that requires 'pmalloc' allocator service to pass
+  $ mvn -Dtest=DurableNodeValueNGTest  test -pl collections -DskipTests=false # a testcase for module "collection" that requires 'pmalloc' allocator service to pass
   
-  $ mvn -Dtest=NonVolatilePersonNGTest  test -pl collections -DskipTests=false # another testcase for module "collection" that requires 'pmalloc' allocator service to pass
+  $ mvn -Dtest=DurablePersonNGTest  test -pl collections -DskipTests=false # another testcase for module "collection" that requires 'pmalloc' allocator service to pass
 ```
 
 ### Where is the document ?

http://git-wip-us.apache.org/repos/asf/incubator-mnemonic/blob/679bcdc1/collections/pom.xml
----------------------------------------------------------------------
diff --git a/collections/pom.xml b/collections/pom.xml
index cc5bd0b..0210b78 100644
--- a/collections/pom.xml
+++ b/collections/pom.xml
@@ -80,7 +80,7 @@
             <configuration>
               <compilerArguments>-XDenableSunApiLintControl</compilerArguments>
               <processors>
-                <processor>${project.parent.groupId}.NonVolatileEntityProcessor</processor>
+                <processor>${project.parent.groupId}.DurableEntityProcessor</processor>
               </processors>
             </configuration>
           </execution>
@@ -91,7 +91,7 @@
             <configuration>
               <compilerArguments>-XDenableSunApiLintControl</compilerArguments>
               <processors>
-                <processor>${project.parent.groupId}.NonVolatileEntityProcessor</processor>
+                <processor>${project.parent.groupId}.DurableEntityProcessor</processor>
               </processors>
             </configuration>
           </execution>

http://git-wip-us.apache.org/repos/asf/incubator-mnemonic/blob/679bcdc1/collections/src/main/java/org/apache/mnemonic/collections/DurableNodeValue.java
----------------------------------------------------------------------
diff --git a/collections/src/main/java/org/apache/mnemonic/collections/DurableNodeValue.java b/collections/src/main/java/org/apache/mnemonic/collections/DurableNodeValue.java
new file mode 100644
index 0000000..91084a2
--- /dev/null
+++ b/collections/src/main/java/org/apache/mnemonic/collections/DurableNodeValue.java
@@ -0,0 +1,172 @@
+/*
+ * 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.util.Iterator;
+import java.util.NoSuchElementException;
+
+import org.apache.mnemonic.Durable;
+import org.apache.mnemonic.EntityFactoryProxy;
+import org.apache.mnemonic.GenericField;
+import org.apache.mnemonic.DurableEntity;
+import org.apache.mnemonic.DurableGetter;
+import org.apache.mnemonic.DurableSetter;
+
+/**
+ * this class defines a non-volatile node for a generic value to form a
+ * unidirectional link
+ *
+ */
+@DurableEntity
+public abstract class DurableNodeValue<E> implements Durable, Iterable<E> {
+  protected transient EntityFactoryProxy[] m_node_efproxies;
+  protected transient GenericField.GType[] 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, GenericField.GType[] gftypes) {
+    m_node_efproxies = efproxies;
+    m_node_gftypes = gftypes;
+  }
+
+  /**
+   * get the item value of this node
+   *
+   * @return the item value of this node
+   */
+  @DurableGetter(Id = 1L, EntityFactoryProxies = "m_node_efproxies", GenericFieldTypes = "m_node_gftypes")
+  public abstract E getItem();
+
+  /**
+   * set a value to this node item
+   * 
+   * @param value
+   *          the value to be set
+   *
+   * @param destroy
+   *          true if want to destroy exist one
+   *
+   */
+  @DurableSetter
+  public abstract void setItem(E value, boolean destroy);
+
+  /**
+   * get next node
+   *
+   * @return the next node
+   *
+   */
+  @DurableGetter(Id = 2L, EntityFactoryProxies = "m_node_efproxies", GenericFieldTypes = "m_node_gftypes")
+  public abstract DurableNodeValue<E> getNext();
+
+  /**
+   * set next node
+   *
+   * @param next
+   *          specify the next node
+   *
+   * @param destroy
+   *          true if want to destroy the exist node
+   */
+  @DurableSetter
+  public abstract void setNext(DurableNodeValue<E> next, boolean destroy);
+
+  /**
+   * get an iterator instance of this list
+   *
+   * @return an iterator of this list
+   */
+  @Override
+  public Iterator<E> iterator() {
+    return new Intr(this);
+  }
+
+  /**
+   * this class defines a iterator for this non-volatile list
+   *
+   */
+  private class Intr implements Iterator<E> {
+
+    protected DurableNodeValue<E> next = null;
+
+    /**
+     * Constructor
+     *
+     * @param head
+     *          the start point for this iterator
+     *
+     */
+    Intr(DurableNodeValue<E> head) {
+      next = head;
+    }
+
+    /**
+     * determine the existing of next
+     *
+     * @return true if there is a next node
+     *
+     */
+    @Override
+    public boolean hasNext() {
+      return null != next;
+    }
+
+    /**
+     * get next node
+     *
+     * @return the next node
+     */
+    @Override
+    public E next() {
+      if (null == next) {
+        new NoSuchElementException();
+      }
+      E ret = next.getItem();
+      next = next.getNext();
+      return ret;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-mnemonic/blob/679bcdc1/collections/src/main/java/org/apache/mnemonic/collections/NonVolatileNodeValue.java
----------------------------------------------------------------------
diff --git a/collections/src/main/java/org/apache/mnemonic/collections/NonVolatileNodeValue.java b/collections/src/main/java/org/apache/mnemonic/collections/NonVolatileNodeValue.java
deleted file mode 100644
index c4d3664..0000000
--- a/collections/src/main/java/org/apache/mnemonic/collections/NonVolatileNodeValue.java
+++ /dev/null
@@ -1,172 +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 org.apache.mnemonic.collections;
-
-import java.util.Iterator;
-import java.util.NoSuchElementException;
-
-import org.apache.mnemonic.Durable;
-import org.apache.mnemonic.EntityFactoryProxy;
-import org.apache.mnemonic.GenericField;
-import org.apache.mnemonic.NonVolatileEntity;
-import org.apache.mnemonic.NonVolatileGetter;
-import org.apache.mnemonic.NonVolatileSetter;
-
-/**
- * this class defines a non-volatile node for a generic value to form a
- * unidirectional link
- *
- */
-@NonVolatileEntity
-public abstract class NonVolatileNodeValue<E> implements Durable, Iterable<E> {
-  protected transient EntityFactoryProxy[] m_node_efproxies;
-  protected transient GenericField.GType[] 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, GenericField.GType[] gftypes) {
-    m_node_efproxies = efproxies;
-    m_node_gftypes = gftypes;
-  }
-
-  /**
-   * get the item value of this node
-   *
-   * @return the item value of this node
-   */
-  @NonVolatileGetter(Id = 1L, EntityFactoryProxies = "m_node_efproxies", GenericFieldTypes = "m_node_gftypes")
-  public abstract E getItem();
-
-  /**
-   * set a value to this node item
-   * 
-   * @param value
-   *          the value to be set
-   *
-   * @param destroy
-   *          true if want to destroy exist one
-   *
-   */
-  @NonVolatileSetter
-  public abstract void setItem(E value, boolean destroy);
-
-  /**
-   * get next node
-   *
-   * @return the next node
-   *
-   */
-  @NonVolatileGetter(Id = 2L, EntityFactoryProxies = "m_node_efproxies", GenericFieldTypes = "m_node_gftypes")
-  public abstract NonVolatileNodeValue<E> getNext();
-
-  /**
-   * set next node
-   *
-   * @param next
-   *          specify the next node
-   *
-   * @param destroy
-   *          true if want to destroy the exist node
-   */
-  @NonVolatileSetter
-  public abstract void setNext(NonVolatileNodeValue<E> next, boolean destroy);
-
-  /**
-   * get an iterator instance of this list
-   *
-   * @return an iterator of this list
-   */
-  @Override
-  public Iterator<E> iterator() {
-    return new Intr(this);
-  }
-
-  /**
-   * this class defines a iterator for this non-volatile list
-   *
-   */
-  private class Intr implements Iterator<E> {
-
-    protected NonVolatileNodeValue<E> next = null;
-
-    /**
-     * Constructor
-     *
-     * @param head
-     *          the start point for this iterator
-     *
-     */
-    Intr(NonVolatileNodeValue<E> head) {
-      next = head;
-    }
-
-    /**
-     * determine the existing of next
-     *
-     * @return true if there is a next node
-     *
-     */
-    @Override
-    public boolean hasNext() {
-      return null != next;
-    }
-
-    /**
-     * get next node
-     *
-     * @return the next node
-     */
-    @Override
-    public E next() {
-      if (null == next) {
-        new NoSuchElementException();
-      }
-      E ret = next.getItem();
-      next = next.getNext();
-      return ret;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-mnemonic/blob/679bcdc1/collections/src/test/java/org/apache/mnemonic/collections/DurableNodeValueNGTest.java
----------------------------------------------------------------------
diff --git a/collections/src/test/java/org/apache/mnemonic/collections/DurableNodeValueNGTest.java b/collections/src/test/java/org/apache/mnemonic/collections/DurableNodeValueNGTest.java
new file mode 100644
index 0000000..0cc0095
--- /dev/null
+++ b/collections/src/test/java/org/apache/mnemonic/collections/DurableNodeValueNGTest.java
@@ -0,0 +1,274 @@
+/*
+ * 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 java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Random;
+
+import org.apache.mnemonic.NonVolatileMemAllocator;
+import org.apache.mnemonic.CommonDurableAllocator;
+import org.apache.mnemonic.Durable;
+import org.apache.mnemonic.EntityFactoryProxy;
+import org.apache.mnemonic.GenericField;
+import org.apache.mnemonic.Reclaim;
+import org.apache.mnemonic.Utils;
+import org.testng.AssertJUnit;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+/**
+ *
+ *
+ */
+
+public class DurableNodeValueNGTest {
+  private long cKEYCAPACITY;
+  private Random m_rand;
+  private NonVolatileMemAllocator m_act;
+
+  @BeforeClass
+  public void setUp() {
+    m_rand = Utils.createRandom();
+    m_act = new NonVolatileMemAllocator(Utils.getNonVolatileMemoryAllocatorService("pmalloc"), 1024 * 1024 * 1024,
+        "./pobj_NodeValue.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 = false)
+  public void testSingleNodeValueWithInteger() {
+    int val = m_rand.nextInt();
+    GenericField.GType gtypes[] = {GenericField.GType.INTEGER};
+    DurableNodeValue<Integer> plln = DurableNodeValueFactory.create(m_act, null, gtypes, false);
+    plln.setItem(val, false);
+    Long handler = plln.getHandler();
+    System.err.println("-------------Start to Restore Integer -----------");
+    DurableNodeValue<Integer> plln2 = DurableNodeValueFactory.restore(m_act, null, gtypes, handler, false);
+    AssertJUnit.assertEquals(val, (int) plln2.getItem());
+  }
+
+  @Test(enabled = false)
+  public void testNodeValueWithString() {
+    String val = Utils.genRandomString();
+    GenericField.GType gtypes[] = {GenericField.GType.STRING};
+    DurableNodeValue<String> plln = DurableNodeValueFactory.create(m_act, null, gtypes, false);
+    plln.setItem(val, false);
+    Long handler = plln.getHandler();
+    System.err.println("-------------Start to Restore String-----------");
+    DurableNodeValue<String> plln2 = DurableNodeValueFactory.restore(m_act, null, gtypes, handler, false);
+    AssertJUnit.assertEquals(val, plln2.getItem());
+  }
+
+  @Test(enabled = false)
+  public void testNodeValueWithPerson() {
+
+    Person<Long> person = PersonFactory.create(m_act);
+    person.setAge((short) 31);
+
+    GenericField.GType gtypes[] = {GenericField.GType.DURABLE};
+    EntityFactoryProxy efproxies[] = {new EntityFactoryProxy() {
+      @Override
+      public <A extends CommonDurableAllocator<A>> Durable restore(A allocator, EntityFactoryProxy[] factoryproxys,
+          GenericField.GType[] gfields, long phandler, boolean autoreclaim) {
+        return PersonFactory.restore(allocator, factoryproxys, gfields, phandler, autoreclaim);
+      }
+    } };
+
+    DurableNodeValue<Person<Long>> plln = DurableNodeValueFactory.create(m_act, efproxies, gtypes, false);
+    plln.setItem(person, false);
+    Long handler = plln.getHandler();
+
+    DurableNodeValue<Person<Long>> plln2 = DurableNodeValueFactory.restore(m_act, efproxies, gtypes, handler,
+        false);
+    AssertJUnit.assertEquals(31, (int) plln2.getItem().getAge());
+
+  }
+
+  @Test(enabled = false)
+  public void testLinkedNodeValueWithPerson() {
+
+    int elem_count = 10;
+    List<Long> referlist = new ArrayList();
+
+    GenericField.GType listgftypes[] = {GenericField.GType.DURABLE};
+    EntityFactoryProxy listefproxies[] = {new EntityFactoryProxy() {
+      @Override
+      public <A extends CommonDurableAllocator<A>> Durable restore(A allocator, EntityFactoryProxy[] factoryproxys,
+          GenericField.GType[] gfields, long phandler, boolean autoreclaim) {
+        return PersonFactory.restore(allocator, factoryproxys, gfields, phandler, autoreclaim);
+      }
+    } };
+
+    DurableNodeValue<Person<Long>> firstnv = DurableNodeValueFactory.create(m_act, listefproxies, listgftypes,
+        false);
+
+    DurableNodeValue<Person<Long>> nextnv = firstnv;
+
+    Person<Long> person;
+    long val;
+    DurableNodeValue<Person<Long>> newnv;
+    for (int i = 0; i < elem_count; ++i) {
+      person = PersonFactory.create(m_act);
+      person.setAge((short) m_rand.nextInt(50));
+      person.setName(String.format("Name: [%s]", Utils.genRandomString()), true);
+      nextnv.setItem(person, false);
+      newnv = DurableNodeValueFactory.create(m_act, listefproxies, listgftypes, false);
+      nextnv.setNext(newnv, false);
+      nextnv = newnv;
+    }
+
+    Person<Long> eval;
+    DurableNodeValue<Person<Long>> iternv = firstnv;
+    while (null != iternv) {
+      System.out.printf(" Stage 1 --->\n");
+      eval = iternv.getItem();
+      if (null != eval) {
+        eval.testOutput();
+      }
+      iternv = iternv.getNext();
+    }
+
+    long handler = firstnv.getHandler();
+
+    DurableNodeValue<Person<Long>> firstnv2 = DurableNodeValueFactory.restore(m_act, listefproxies, listgftypes,
+        handler, false);
+
+    for (Person<Long> eval2 : firstnv2) {
+      System.out.printf(" Stage 2 ---> \n");
+      if (null != eval2) {
+        eval2.testOutput();
+      }
+    }
+
+    // Assert.assert, expected);(plist, plist2);
+
+  }
+
+  @Test(enabled = true)
+  public void testLinkedNodeValueWithLinkedNodeValue() {
+
+    int elem_count = 10;
+    long slotKeyId = 10;
+
+    GenericField.GType[] elem_gftypes = {GenericField.GType.DOUBLE};
+    EntityFactoryProxy[] elem_efproxies = null;
+
+    GenericField.GType linkedgftypes[] = {GenericField.GType.DURABLE, GenericField.GType.DOUBLE};
+    EntityFactoryProxy linkedefproxies[] = {new EntityFactoryProxy() {
+      @Override
+      public <A extends CommonDurableAllocator<A>> Durable restore(A allocator, EntityFactoryProxy[] factoryproxys,
+          GenericField.GType[] gfields, long phandler, boolean autoreclaim) {
+        EntityFactoryProxy[] val_efproxies = null;
+        GenericField.GType[] val_gftypes = null;
+        if (null != factoryproxys && factoryproxys.length >= 2) {
+          val_efproxies = Arrays.copyOfRange(factoryproxys, 1, factoryproxys.length);
+        }
+        if (null != gfields && gfields.length >= 2) {
+          val_gftypes = Arrays.copyOfRange(gfields, 1, gfields.length);
+        }
+        return DurableNodeValueFactory.restore(allocator, val_efproxies, val_gftypes, phandler, autoreclaim);
+      }
+    } };
+
+    DurableNodeValue<DurableNodeValue<Double>> nextnv = null, pre_nextnv = null;
+    DurableNodeValue<Double> elem = null, pre_elem = null, first_elem = null;
+
+    Long linkhandler = 0L;
+
+    System.out.printf(" Stage 1 -testLinkedNodeValueWithLinkedNodeValue--> \n");
+
+    pre_nextnv = null;
+    Double val;
+    for (int i = 0; i < elem_count; ++i) {
+      first_elem = null;
+      pre_elem = null;
+      for (int v = 0; v < 3; ++v) {
+        elem = DurableNodeValueFactory.create(m_act, elem_efproxies, elem_gftypes, false);
+        val = m_rand.nextDouble();
+        elem.setItem(val, false);
+        if (null == pre_elem) {
+          first_elem = elem;
+        } else {
+          pre_elem.setNext(elem, false);
+        }
+        pre_elem = elem;
+        System.out.printf("%f ", val);
+      }
+
+      nextnv = DurableNodeValueFactory.create(m_act, linkedefproxies, linkedgftypes, false);
+      nextnv.setItem(first_elem, false);
+      if (null == pre_nextnv) {
+        linkhandler = nextnv.getHandler();
+      } else {
+        pre_nextnv.setNext(nextnv, false);
+      }
+      pre_nextnv = nextnv;
+      System.out.printf(" generated an item... \n");
+    }
+    m_act.setHandler(slotKeyId, linkhandler);
+
+    long handler = m_act.getHandler(slotKeyId);
+
+    DurableNodeValue<DurableNodeValue<Double>> linkedvals = DurableNodeValueFactory.restore(m_act,
+        linkedefproxies, linkedgftypes, handler, false);
+    Iterator<DurableNodeValue<Double>> iter = linkedvals.iterator();
+    Iterator<Double> elemiter = null;
+
+    System.out.printf(" Stage 2 -testLinkedNodeValueWithLinkedNodeValue--> \n");
+    while (iter.hasNext()) {
+      elemiter = iter.next().iterator();
+      while (elemiter.hasNext()) {
+        System.out.printf("%f ", elemiter.next());
+      }
+      System.out.printf(" Fetched an item... \n");
+    }
+
+    // Assert.assert, expected);(plist, plist2);
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-mnemonic/blob/679bcdc1/collections/src/test/java/org/apache/mnemonic/collections/DurablePersonNGTest.java
----------------------------------------------------------------------
diff --git a/collections/src/test/java/org/apache/mnemonic/collections/DurablePersonNGTest.java b/collections/src/test/java/org/apache/mnemonic/collections/DurablePersonNGTest.java
new file mode 100644
index 0000000..7694b6d
--- /dev/null
+++ b/collections/src/test/java/org/apache/mnemonic/collections/DurablePersonNGTest.java
@@ -0,0 +1,148 @@
+/*
+ * 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.NonVolatileMemAllocator;
+import org.apache.mnemonic.OutOfHybridMemory;
+import org.apache.mnemonic.Reclaim;
+import org.apache.mnemonic.RetrieveDurableEntityError;
+import org.apache.mnemonic.Utils;
+import org.testng.annotations.Test;
+import java.nio.ByteBuffer;
+import java.util.Random;
+import java.util.UUID;
+
+/**
+ *
+ *
+ */
+
+public class DurablePersonNGTest {
+  private long cKEYCAPACITY;
+
+  @Test(expectedExceptions = { OutOfHybridMemory.class })
+  public void testGenPeople() throws OutOfHybridMemory, RetrieveDurableEntityError {
+    Random rand = Utils.createRandom();
+    NonVolatileMemAllocator act = new NonVolatileMemAllocator(Utils.getNonVolatileMemoryAllocatorService("pmalloc"),
+        1024 * 1024 * 8, "./pobj_person.dat", true);
+    cKEYCAPACITY = act.handlerCapacity();
+    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;
+      }
+    });
+    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) {
+      act.setHandler(i, 0L);
+    }
+
+    Person<Integer> mother;
+    Person<Integer> person;
+
+    long keyidx = 0;
+    long val;
+
+    try {
+      while (true) {
+        // if (keyidx >= KEYCAPACITY) break;
+
+        keyidx %= cKEYCAPACITY;
+
+        System.out.printf("************ Generating People on Key %d ***********\n", keyidx);
+
+        val = act.getHandler(keyidx);
+        if (0L != val) {
+          PersonFactory.restore(act, val, true);
+        }
+
+        person = PersonFactory.create(act);
+        person.setAge((short) rand.nextInt(50));
+        person.setName(String.format("Name: [%s]", UUID.randomUUID().toString()), true);
+        person.setName(String.format("Name: [%s]", UUID.randomUUID().toString()), true);
+        person.setName(String.format("Name: [%s]", UUID.randomUUID().toString()), true);
+        person.setName(String.format("Name: [%s]", UUID.randomUUID().toString()), true);
+
+        act.setHandler(keyidx, person.getHandler());
+
+        for (int deep = 0; deep < rand.nextInt(100); ++deep) {
+
+          mother = PersonFactory.create(act);
+          mother.setAge((short) (50 + rand.nextInt(50)));
+          mother.setName(String.format("Name: [%s]", UUID.randomUUID().toString()), true);
+
+          person.setMother(mother, true);
+
+          person = mother;
+
+        }
+        ++keyidx;
+      }
+    } finally {
+      act.close();
+    }
+  }
+
+  @Test(dependsOnMethods = { "testGenPeople" })
+  public void testCheckPeople() throws RetrieveDurableEntityError {
+    NonVolatileMemAllocator act = new NonVolatileMemAllocator(Utils.getNonVolatileMemoryAllocatorService("pmalloc"),
+        1024 * 1024 * 8, "./pobj_person.dat", true);
+    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;
+      }
+    });
+    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;
+      }
+    });
+
+    long val;
+    for (long i = 0; i < cKEYCAPACITY; ++i) {
+      System.out.printf("----------Key %d--------------\n", i);
+      val = act.getHandler(i);
+      if (0L == val) {
+        break;
+      }
+      Person<Integer> person = PersonFactory.restore(act, val, true);
+      while (null != person) {
+        person.testOutput();
+        person = person.getMother();
+      }
+    }
+
+    act.close();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-mnemonic/blob/679bcdc1/collections/src/test/java/org/apache/mnemonic/collections/NonVolatileNodeValueNGTest.java
----------------------------------------------------------------------
diff --git a/collections/src/test/java/org/apache/mnemonic/collections/NonVolatileNodeValueNGTest.java b/collections/src/test/java/org/apache/mnemonic/collections/NonVolatileNodeValueNGTest.java
deleted file mode 100644
index ae122f7..0000000
--- a/collections/src/test/java/org/apache/mnemonic/collections/NonVolatileNodeValueNGTest.java
+++ /dev/null
@@ -1,274 +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 org.apache.mnemonic.collections;
-
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Random;
-
-import org.apache.mnemonic.BigDataPMemAllocator;
-import org.apache.mnemonic.CommonPersistAllocator;
-import org.apache.mnemonic.Durable;
-import org.apache.mnemonic.EntityFactoryProxy;
-import org.apache.mnemonic.GenericField;
-import org.apache.mnemonic.Reclaim;
-import org.apache.mnemonic.Utils;
-import org.testng.AssertJUnit;
-import org.testng.annotations.AfterClass;
-import org.testng.annotations.BeforeClass;
-import org.testng.annotations.Test;
-
-/**
- *
- *
- */
-
-public class NonVolatileNodeValueNGTest {
-  private long cKEYCAPACITY;
-  private Random m_rand;
-  private BigDataPMemAllocator m_act;
-
-  @BeforeClass
-  public void setUp() {
-    m_rand = Utils.createRandom();
-    m_act = new BigDataPMemAllocator(Utils.getNonVolatileMemoryAllocatorService("pmalloc"), 1024 * 1024 * 1024,
-        "./pobj_NodeValue.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 = false)
-  public void testSingleNodeValueWithInteger() {
-    int val = m_rand.nextInt();
-    GenericField.GType gtypes[] = {GenericField.GType.INTEGER};
-    NonVolatileNodeValue<Integer> plln = NonVolatileNodeValueFactory.create(m_act, null, gtypes, false);
-    plln.setItem(val, false);
-    Long handler = plln.getNonVolatileHandler();
-    System.err.println("-------------Start to Restore Integer -----------");
-    NonVolatileNodeValue<Integer> plln2 = NonVolatileNodeValueFactory.restore(m_act, null, gtypes, handler, false);
-    AssertJUnit.assertEquals(val, (int) plln2.getItem());
-  }
-
-  @Test(enabled = false)
-  public void testNodeValueWithString() {
-    String val = Utils.genRandomString();
-    GenericField.GType gtypes[] = {GenericField.GType.STRING};
-    NonVolatileNodeValue<String> plln = NonVolatileNodeValueFactory.create(m_act, null, gtypes, false);
-    plln.setItem(val, false);
-    Long handler = plln.getNonVolatileHandler();
-    System.err.println("-------------Start to Restore String-----------");
-    NonVolatileNodeValue<String> plln2 = NonVolatileNodeValueFactory.restore(m_act, null, gtypes, handler, false);
-    AssertJUnit.assertEquals(val, plln2.getItem());
-  }
-
-  @Test(enabled = false)
-  public void testNodeValueWithPerson() {
-
-    Person<Long> person = PersonFactory.create(m_act);
-    person.setAge((short) 31);
-
-    GenericField.GType gtypes[] = {GenericField.GType.DURABLE};
-    EntityFactoryProxy efproxies[] = {new EntityFactoryProxy() {
-      @Override
-      public <A extends CommonPersistAllocator<A>> Durable restore(A allocator, EntityFactoryProxy[] factoryproxys,
-          GenericField.GType[] gfields, long phandler, boolean autoreclaim) {
-        return PersonFactory.restore(allocator, factoryproxys, gfields, phandler, autoreclaim);
-      }
-    } };
-
-    NonVolatileNodeValue<Person<Long>> plln = NonVolatileNodeValueFactory.create(m_act, efproxies, gtypes, false);
-    plln.setItem(person, false);
-    Long handler = plln.getNonVolatileHandler();
-
-    NonVolatileNodeValue<Person<Long>> plln2 = NonVolatileNodeValueFactory.restore(m_act, efproxies, gtypes, handler,
-        false);
-    AssertJUnit.assertEquals(31, (int) plln2.getItem().getAge());
-
-  }
-
-  @Test(enabled = false)
-  public void testLinkedNodeValueWithPerson() {
-
-    int elem_count = 10;
-    List<Long> referlist = new ArrayList();
-
-    GenericField.GType listgftypes[] = {GenericField.GType.DURABLE};
-    EntityFactoryProxy listefproxies[] = {new EntityFactoryProxy() {
-      @Override
-      public <A extends CommonPersistAllocator<A>> Durable restore(A allocator, EntityFactoryProxy[] factoryproxys,
-          GenericField.GType[] gfields, long phandler, boolean autoreclaim) {
-        return PersonFactory.restore(allocator, factoryproxys, gfields, phandler, autoreclaim);
-      }
-    } };
-
-    NonVolatileNodeValue<Person<Long>> firstnv = NonVolatileNodeValueFactory.create(m_act, listefproxies, listgftypes,
-        false);
-
-    NonVolatileNodeValue<Person<Long>> nextnv = firstnv;
-
-    Person<Long> person;
-    long val;
-    NonVolatileNodeValue<Person<Long>> newnv;
-    for (int i = 0; i < elem_count; ++i) {
-      person = PersonFactory.create(m_act);
-      person.setAge((short) m_rand.nextInt(50));
-      person.setName(String.format("Name: [%s]", Utils.genRandomString()), true);
-      nextnv.setItem(person, false);
-      newnv = NonVolatileNodeValueFactory.create(m_act, listefproxies, listgftypes, false);
-      nextnv.setNext(newnv, false);
-      nextnv = newnv;
-    }
-
-    Person<Long> eval;
-    NonVolatileNodeValue<Person<Long>> iternv = firstnv;
-    while (null != iternv) {
-      System.out.printf(" Stage 1 --->\n");
-      eval = iternv.getItem();
-      if (null != eval) {
-        eval.testOutput();
-      }
-      iternv = iternv.getNext();
-    }
-
-    long handler = firstnv.getNonVolatileHandler();
-
-    NonVolatileNodeValue<Person<Long>> firstnv2 = NonVolatileNodeValueFactory.restore(m_act, listefproxies, listgftypes,
-        handler, false);
-
-    for (Person<Long> eval2 : firstnv2) {
-      System.out.printf(" Stage 2 ---> \n");
-      if (null != eval2) {
-        eval2.testOutput();
-      }
-    }
-
-    // Assert.assert, expected);(plist, plist2);
-
-  }
-
-  @Test(enabled = true)
-  public void testLinkedNodeValueWithLinkedNodeValue() {
-
-    int elem_count = 10;
-    long slotKeyId = 10;
-
-    GenericField.GType[] elem_gftypes = {GenericField.GType.DOUBLE};
-    EntityFactoryProxy[] elem_efproxies = null;
-
-    GenericField.GType linkedgftypes[] = {GenericField.GType.DURABLE, GenericField.GType.DOUBLE};
-    EntityFactoryProxy linkedefproxies[] = {new EntityFactoryProxy() {
-      @Override
-      public <A extends CommonPersistAllocator<A>> Durable restore(A allocator, EntityFactoryProxy[] factoryproxys,
-          GenericField.GType[] gfields, long phandler, boolean autoreclaim) {
-        EntityFactoryProxy[] val_efproxies = null;
-        GenericField.GType[] val_gftypes = null;
-        if (null != factoryproxys && factoryproxys.length >= 2) {
-          val_efproxies = Arrays.copyOfRange(factoryproxys, 1, factoryproxys.length);
-        }
-        if (null != gfields && gfields.length >= 2) {
-          val_gftypes = Arrays.copyOfRange(gfields, 1, gfields.length);
-        }
-        return NonVolatileNodeValueFactory.restore(allocator, val_efproxies, val_gftypes, phandler, autoreclaim);
-      }
-    } };
-
-    NonVolatileNodeValue<NonVolatileNodeValue<Double>> nextnv = null, pre_nextnv = null;
-    NonVolatileNodeValue<Double> elem = null, pre_elem = null, first_elem = null;
-
-    Long linkhandler = 0L;
-
-    System.out.printf(" Stage 1 -testLinkedNodeValueWithLinkedNodeValue--> \n");
-
-    pre_nextnv = null;
-    Double val;
-    for (int i = 0; i < elem_count; ++i) {
-      first_elem = null;
-      pre_elem = null;
-      for (int v = 0; v < 3; ++v) {
-        elem = NonVolatileNodeValueFactory.create(m_act, elem_efproxies, elem_gftypes, false);
-        val = m_rand.nextDouble();
-        elem.setItem(val, false);
-        if (null == pre_elem) {
-          first_elem = elem;
-        } else {
-          pre_elem.setNext(elem, false);
-        }
-        pre_elem = elem;
-        System.out.printf("%f ", val);
-      }
-
-      nextnv = NonVolatileNodeValueFactory.create(m_act, linkedefproxies, linkedgftypes, false);
-      nextnv.setItem(first_elem, false);
-      if (null == pre_nextnv) {
-        linkhandler = nextnv.getNonVolatileHandler();
-      } else {
-        pre_nextnv.setNext(nextnv, false);
-      }
-      pre_nextnv = nextnv;
-      System.out.printf(" generated an item... \n");
-    }
-    m_act.setHandler(slotKeyId, linkhandler);
-
-    long handler = m_act.getHandler(slotKeyId);
-
-    NonVolatileNodeValue<NonVolatileNodeValue<Double>> linkedvals = NonVolatileNodeValueFactory.restore(m_act,
-        linkedefproxies, linkedgftypes, handler, false);
-    Iterator<NonVolatileNodeValue<Double>> iter = linkedvals.iterator();
-    Iterator<Double> elemiter = null;
-
-    System.out.printf(" Stage 2 -testLinkedNodeValueWithLinkedNodeValue--> \n");
-    while (iter.hasNext()) {
-      elemiter = iter.next().iterator();
-      while (elemiter.hasNext()) {
-        System.out.printf("%f ", elemiter.next());
-      }
-      System.out.printf(" Fetched an item... \n");
-    }
-
-    // Assert.assert, expected);(plist, plist2);
-
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-mnemonic/blob/679bcdc1/collections/src/test/java/org/apache/mnemonic/collections/NonVolatilePersonNGTest.java
----------------------------------------------------------------------
diff --git a/collections/src/test/java/org/apache/mnemonic/collections/NonVolatilePersonNGTest.java b/collections/src/test/java/org/apache/mnemonic/collections/NonVolatilePersonNGTest.java
deleted file mode 100644
index 42cfc16..0000000
--- a/collections/src/test/java/org/apache/mnemonic/collections/NonVolatilePersonNGTest.java
+++ /dev/null
@@ -1,148 +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 org.apache.mnemonic.collections;
-
-import org.apache.mnemonic.BigDataPMemAllocator;
-import org.apache.mnemonic.OutOfPersistentMemory;
-import org.apache.mnemonic.Reclaim;
-import org.apache.mnemonic.RetrieveNonVolatileEntityError;
-import org.apache.mnemonic.Utils;
-import org.testng.annotations.Test;
-import java.nio.ByteBuffer;
-import java.util.Random;
-import java.util.UUID;
-
-/**
- *
- *
- */
-
-public class NonVolatilePersonNGTest {
-  private long cKEYCAPACITY;
-
-  @Test(expectedExceptions = { OutOfPersistentMemory.class })
-  public void testGenPeople() throws OutOfPersistentMemory, RetrieveNonVolatileEntityError {
-    Random rand = Utils.createRandom();
-    BigDataPMemAllocator act = new BigDataPMemAllocator(Utils.getNonVolatileMemoryAllocatorService("pmalloc"),
-        1024 * 1024 * 8, "./pobj_person.dat", true);
-    cKEYCAPACITY = act.handlerCapacity();
-    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;
-      }
-    });
-    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) {
-      act.setHandler(i, 0L);
-    }
-
-    Person<Integer> mother;
-    Person<Integer> person;
-
-    long keyidx = 0;
-    long val;
-
-    try {
-      while (true) {
-        // if (keyidx >= KEYCAPACITY) break;
-
-        keyidx %= cKEYCAPACITY;
-
-        System.out.printf("************ Generating People on Key %d ***********\n", keyidx);
-
-        val = act.getHandler(keyidx);
-        if (0L != val) {
-          PersonFactory.restore(act, val, true);
-        }
-
-        person = PersonFactory.create(act);
-        person.setAge((short) rand.nextInt(50));
-        person.setName(String.format("Name: [%s]", UUID.randomUUID().toString()), true);
-        person.setName(String.format("Name: [%s]", UUID.randomUUID().toString()), true);
-        person.setName(String.format("Name: [%s]", UUID.randomUUID().toString()), true);
-        person.setName(String.format("Name: [%s]", UUID.randomUUID().toString()), true);
-
-        act.setHandler(keyidx, person.getNonVolatileHandler());
-
-        for (int deep = 0; deep < rand.nextInt(100); ++deep) {
-
-          mother = PersonFactory.create(act);
-          mother.setAge((short) (50 + rand.nextInt(50)));
-          mother.setName(String.format("Name: [%s]", UUID.randomUUID().toString()), true);
-
-          person.setMother(mother, true);
-
-          person = mother;
-
-        }
-        ++keyidx;
-      }
-    } finally {
-      act.close();
-    }
-  }
-
-  @Test(dependsOnMethods = { "testGenPeople" })
-  public void testCheckPeople() throws RetrieveNonVolatileEntityError {
-    BigDataPMemAllocator act = new BigDataPMemAllocator(Utils.getNonVolatileMemoryAllocatorService("pmalloc"),
-        1024 * 1024 * 8, "./pobj_person.dat", true);
-    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;
-      }
-    });
-    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;
-      }
-    });
-
-    long val;
-    for (long i = 0; i < cKEYCAPACITY; ++i) {
-      System.out.printf("----------Key %d--------------\n", i);
-      val = act.getHandler(i);
-      if (0L == val) {
-        break;
-      }
-      Person<Integer> person = PersonFactory.restore(act, val, true);
-      while (null != person) {
-        person.testOutput();
-        person = person.getMother();
-      }
-    }
-
-    act.close();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-mnemonic/blob/679bcdc1/collections/src/test/java/org/apache/mnemonic/collections/Person.java
----------------------------------------------------------------------
diff --git a/collections/src/test/java/org/apache/mnemonic/collections/Person.java b/collections/src/test/java/org/apache/mnemonic/collections/Person.java
index 259bee9..5210847 100644
--- a/collections/src/test/java/org/apache/mnemonic/collections/Person.java
+++ b/collections/src/test/java/org/apache/mnemonic/collections/Person.java
@@ -20,11 +20,11 @@ package org.apache.mnemonic.collections;
 import org.apache.mnemonic.Durable;
 import org.apache.mnemonic.EntityFactoryProxy;
 import org.apache.mnemonic.GenericField;
-import org.apache.mnemonic.NonVolatileEntity;
-import org.apache.mnemonic.NonVolatileGetter;
-import org.apache.mnemonic.NonVolatileSetter;
-import org.apache.mnemonic.OutOfPersistentMemory;
-import org.apache.mnemonic.RetrieveNonVolatileEntityError;
+import org.apache.mnemonic.DurableEntity;
+import org.apache.mnemonic.DurableGetter;
+import org.apache.mnemonic.DurableSetter;
+import org.apache.mnemonic.OutOfHybridMemory;
+import org.apache.mnemonic.RetrieveDurableEntityError;
 import org.testng.annotations.Test;
 
 /**
@@ -32,7 +32,7 @@ import org.testng.annotations.Test;
  *
  */
 
-@NonVolatileEntity
+@DurableEntity
 public abstract class Person<E> implements Durable, Comparable<Person<E>> {
   E element;
 
@@ -52,7 +52,7 @@ public abstract class Person<E> implements Durable, Comparable<Person<E>> {
   }
 
   @Test
-  public void testOutput() throws RetrieveNonVolatileEntityError {
+  public void testOutput() throws RetrieveDurableEntityError {
     System.out.printf("Person %s, Age: %d ( %s ) \n", getName(), getAge(),
         null == getMother() ? "No Recorded Mother" : "Has Recorded Mother");
   }
@@ -68,28 +68,28 @@ public abstract class Person<E> implements Durable, Comparable<Person<E>> {
     return ret;
   }
 
-  @NonVolatileGetter(Id = 1L)
+  @DurableGetter(Id = 1L)
   public abstract Short getAge();
 
-  @NonVolatileSetter
+  @DurableSetter
   public abstract void setAge(Short age);
 
-  @NonVolatileGetter(Id = 2L)
-  public abstract String getName() throws RetrieveNonVolatileEntityError;
+  @DurableGetter(Id = 2L)
+  public abstract String getName() throws RetrieveDurableEntityError;
 
-  @NonVolatileSetter
+  @DurableSetter
   public abstract void setName(String name, boolean destroy)
-      throws OutOfPersistentMemory, RetrieveNonVolatileEntityError;
+      throws OutOfHybridMemory, RetrieveDurableEntityError;
 
-  @NonVolatileGetter(Id = 3L)
-  public abstract Person<E> getMother() throws RetrieveNonVolatileEntityError;
+  @DurableGetter(Id = 3L)
+  public abstract Person<E> getMother() throws RetrieveDurableEntityError;
 
-  @NonVolatileSetter
-  public abstract void setMother(Person<E> mother, boolean destroy) throws RetrieveNonVolatileEntityError;
+  @DurableSetter
+  public abstract void setMother(Person<E> mother, boolean destroy) throws RetrieveDurableEntityError;
 
-  @NonVolatileGetter(Id = 4L)
-  public abstract Person<E> getFather() throws RetrieveNonVolatileEntityError;
+  @DurableGetter(Id = 4L)
+  public abstract Person<E> getFather() throws RetrieveDurableEntityError;
 
-  @NonVolatileSetter
-  public abstract void setFather(Person<E> mother, boolean destroy) throws RetrieveNonVolatileEntityError;
+  @DurableSetter
+  public abstract void setFather(Person<E> mother, boolean destroy) throws RetrieveDurableEntityError;
 }

http://git-wip-us.apache.org/repos/asf/incubator-mnemonic/blob/679bcdc1/collections/src/test/resources/testng.xml
----------------------------------------------------------------------
diff --git a/collections/src/test/resources/testng.xml b/collections/src/test/resources/testng.xml
index eeacf58..eb82560 100644
--- a/collections/src/test/resources/testng.xml
+++ b/collections/src/test/resources/testng.xml
@@ -3,7 +3,7 @@
 <suite name="Suite" verbose="1" parallel="tests" thread-count="1">
   <test name="Test">
     <classes>
-      <class name="org.apache.mnemonic.collections.NonVolatileNodeValueNGTest"/> 
+      <class name="org.apache.mnemonic.collections.DurableNodeValueNGTest"/> 
     </classes>
   </test> <!-- Test -->
 </suite> <!-- Suite -->
@@ -11,7 +11,7 @@
 
 <!--
       <class name="org.apache.mnemonic.collections.Person"/>
-      <class name="org.apache.mnemonic.collections.NonVolatilePersonNGTest"/>
-      <class name="org.apache.mnemonic.collections.NonVolatileListNGTest"/>
-      <class name="org.apache.mnemonic.collections.NonVolatileNodeValueNGTest"/> 
+      <class name="org.apache.mnemonic.collections.DurablePersonNGTest"/>
+      <class name="org.apache.mnemonic.collections.DurableListNGTest"/>
+      <class name="org.apache.mnemonic.collections.DurableNodeValueNGTest"/> 
  -->

http://git-wip-us.apache.org/repos/asf/incubator-mnemonic/blob/679bcdc1/core/pom.xml
----------------------------------------------------------------------
diff --git a/core/pom.xml b/core/pom.xml
index 91751f7..00b2855 100644
--- a/core/pom.xml
+++ b/core/pom.xml
@@ -62,7 +62,7 @@
             <configuration>
               <compilerArguments>-XDenableSunApiLintControl</compilerArguments>
               <processors>
-                <processor>${project.groupId}.NonVolatileEntityProcessor</processor>
+                <processor>${project.groupId}.DurableEntityProcessor</processor>
               </processors>
             </configuration>
           </execution>


[2/5] incubator-mnemonic git commit: MNEMONIC-20: Formalize the names of annotations, classes and methods

Posted by ga...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-mnemonic/blob/679bcdc1/core/src/main/java/org/apache/mnemonic/BigDataMemAllocator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/mnemonic/BigDataMemAllocator.java b/core/src/main/java/org/apache/mnemonic/BigDataMemAllocator.java
deleted file mode 100644
index 2f0a49f..0000000
--- a/core/src/main/java/org/apache/mnemonic/BigDataMemAllocator.java
+++ /dev/null
@@ -1,278 +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 org.apache.mnemonic;
-
-import java.nio.ByteBuffer;
-
-import org.apache.mnemonic.service.allocatorservice.VolatileMemoryAllocatorService;
-import org.flowcomputing.commons.resgc.ResCollector;
-import org.flowcomputing.commons.resgc.ResReclaim;
-
-/**
- * manage a big native memory pool through libvmem.so that is provied by Intel
- * nvml library.
- * 
- *
- */
-public class BigDataMemAllocator extends CommonAllocator<BigDataMemAllocator> {
-
-  private boolean m_activegc = true;
-  private long m_gctimeout = 100;
-  private long m_nid = -1;
-  private VolatileMemoryAllocatorService m_vmasvc = null;
-
-  /**
-   * Constructor, it initializes and allocate a memory pool from specified uri
-   * location with specified capacity and an allocator service instance.
-   * usually, the uri points to a mounted memory device or a location of file
-   * system.
-   * 
-   * @param vmasvc
-   *          the volatile memory allocation service instance
-   *
-   * @param capacity
-   *          the capacity of memory pool
-   * 
-   * @param uri
-   *          the location of memory pool will be created
-   * 
-   * @param isnew
-   *          a place holder, always specify it as true
-   */
-  public BigDataMemAllocator(VolatileMemoryAllocatorService vmasvc, long capacity, String uri, boolean isnew) {
-    assert null != vmasvc : "VolatileMemoryAllocatorService object is null";
-    if (capacity <= 0) {
-      throw new IllegalArgumentException("BigDataMemAllocator cannot be initialized with capacity <= 0.");
-    }
-
-    m_vmasvc = vmasvc;
-    m_nid = m_vmasvc.init(capacity, uri, isnew);
-
-    /**
-     * create a resource collector to release specified bytebuffer that backed
-     * by underlying big memory pool.
-     */
-    m_bufcollector = new ResCollector<MemBufferHolder<BigDataMemAllocator>, ByteBuffer>(new ResReclaim<ByteBuffer>() {
-      @Override
-      public void reclaim(ByteBuffer mres) {
-        boolean cb_reclaimed = false;
-        if (null != m_bufferreclaimer) {
-          cb_reclaimed = m_bufferreclaimer.reclaim(mres, Long.valueOf(mres.capacity()));
-        }
-        if (!cb_reclaimed) {
-          m_vmasvc.destroyByteBuffer(m_nid, mres);
-          mres = null;
-        }
-      }
-    });
-
-    /**
-     * create a resource collector to release specified chunk that backed by
-     * underlying big memory pool.
-     */
-    m_chunkcollector = new ResCollector<MemChunkHolder<BigDataMemAllocator>, Long>(new ResReclaim<Long>() {
-      @Override
-      public void reclaim(Long mres) {
-        // System.out.println(String.format("Reclaim: %X", mres));
-        boolean cb_reclaimed = false;
-        if (null != m_chunkreclaimer) {
-          cb_reclaimed = m_chunkreclaimer.reclaim(mres, null);
-        }
-        if (!cb_reclaimed) {
-          m_vmasvc.free(m_nid, mres);
-          mres = null;
-        }
-      }
-    });
-  }
-
-  /**
-   * enable active garbage collection. the GC will be forced to collect garbages
-   * when there is no more space for current allocation request.
-   *
-   * @param timeout
-   *          the timeout is used to yield for GC performing
-   *
-   * @return this allocator
-   */
-  @Override
-  public BigDataMemAllocator enableActiveGC(long timeout) {
-    m_activegc = true;
-    m_gctimeout = timeout;
-    return this;
-  }
-
-  /**
-   * disable active garbage collection.
-   *
-   * @return this allocator
-   */
-  @Override
-  public BigDataMemAllocator disableActiveGC() {
-    m_activegc = false;
-    return this;
-  }
-
-  /**
-   * release the memory pool and close it.
-   *
-   */
-  @Override
-  public void close() {
-    super.close();
-  }
-
-  /**
-   * force to synchronize uncommitted data to backed memory pool (this is a
-   * placeholder).
-   *
-   */
-  @Override
-  public void sync() {
-  }
-
-  /**
-   * re-size a specified chunk on its backed memory pool.
-   * 
-   * @param mholder
-   *          the holder of memory chunk. it can be null.
-   * 
-   * @param size
-   *          specify a new size of memory chunk
-   * 
-   * @return the resized memory chunk handler
-   */
-  @Override
-  public MemChunkHolder<BigDataMemAllocator> resizeChunk(MemChunkHolder<BigDataMemAllocator> mholder, long size) {
-    MemChunkHolder<BigDataMemAllocator> ret = null;
-    boolean ac = null != mholder.getRefId();
-    if (size > 0) {
-      Long addr = m_vmasvc.reallocate(m_nid, mholder.get(), size, true);
-      if (0 == addr && m_activegc) {
-        m_chunkcollector.waitReclaimCoolDown(m_gctimeout);
-        addr = m_vmasvc.reallocate(m_nid, mholder.get(), size, true);
-      }
-      if (0 != addr) {
-        mholder.clear();
-        mholder.destroy();
-        ret = new MemChunkHolder<BigDataMemAllocator>(this, addr, size);
-        if (ac) {
-          m_chunkcollector.register(ret);
-        }
-      }
-    }
-    return ret;
-  }
-
-  /**
-   * resize a specified buffer on its backed memory pool.
-   *
-   * @param mholder
-   *          the holder of memory buffer. it can be null.
-   * 
-   * @param size
-   *          specify a new size of memory chunk
-   * 
-   * @return the resized memory buffer handler
-   *
-   */
-  @Override
-  public MemBufferHolder<BigDataMemAllocator> resizeBuffer(MemBufferHolder<BigDataMemAllocator> mholder, long size) {
-    MemBufferHolder<BigDataMemAllocator> ret = null;
-    boolean ac = null != mholder.getRefId();
-    if (size > 0) {
-      int bufpos = mholder.get().position();
-      int buflimit = mholder.get().limit();
-      ByteBuffer buf = m_vmasvc.resizeByteBuffer(m_nid, mholder.get(), size);
-      if (null == buf && m_activegc) {
-        m_bufcollector.waitReclaimCoolDown(m_gctimeout);
-        buf = m_vmasvc.resizeByteBuffer(m_nid, mholder.get(), size);
-      }
-      if (null != buf) {
-        mholder.clear();
-        mholder.destroy();
-        buf.position(bufpos <= size ? bufpos : 0);
-        buf.limit(buflimit <= size ? buflimit : (int) size);
-        ret = new MemBufferHolder<BigDataMemAllocator>(this, buf);
-        if (ac) {
-          m_bufcollector.register(ret);
-        }
-      }
-    }
-    return ret;
-  }
-
-  /**
-   * create a memory chunk that is managed by its holder.
-   * 
-   * @param size
-   *          specify the size of memory chunk
-   * 
-   * @param autoreclaim
-   *          specify whether or not to reclaim this chunk automatically
-   *
-   * @return a holder contains a memory chunk
-   */
-  @Override
-  public MemChunkHolder<BigDataMemAllocator> createChunk(long size, boolean autoreclaim) {
-    MemChunkHolder<BigDataMemAllocator> ret = null;
-    Long addr = m_vmasvc.allocate(m_nid, size, true);
-    if (0 == addr && m_activegc) {
-      m_chunkcollector.waitReclaimCoolDown(m_gctimeout);
-      addr = m_vmasvc.allocate(m_nid, size, true);
-    }
-    if (0 != addr) {
-      ret = new MemChunkHolder<BigDataMemAllocator>(this, addr, size);
-      ret.setCollector(m_chunkcollector);
-      if (autoreclaim) {
-        m_chunkcollector.register(ret);
-      }
-    }
-    return ret;
-  }
-
-  /**
-   * create a memory buffer that is managed by its holder.
-   * 
-   * @param size
-   *          specify the size of memory buffer
-   * 
-   * @param autoreclaim
-   *          specify whether or not to reclaim this buffer automatically
-   *
-   * @return a holder contains a memory buffer
-   */
-  @Override
-  public MemBufferHolder<BigDataMemAllocator> createBuffer(long size, boolean autoreclaim) {
-    MemBufferHolder<BigDataMemAllocator> ret = null;
-    ByteBuffer bb = m_vmasvc.createByteBuffer(m_nid, size);
-    if (null == bb && m_activegc) {
-      m_bufcollector.waitReclaimCoolDown(m_gctimeout);
-      bb = m_vmasvc.createByteBuffer(m_nid, size);
-    }
-    if (null != bb) {
-      ret = new MemBufferHolder<BigDataMemAllocator>(this, bb);
-      ret.setCollector(m_bufcollector);
-      if (autoreclaim) {
-        m_bufcollector.register(ret);
-      }
-    }
-    return ret;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-mnemonic/blob/679bcdc1/core/src/main/java/org/apache/mnemonic/BigDataPMemAllocator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/mnemonic/BigDataPMemAllocator.java b/core/src/main/java/org/apache/mnemonic/BigDataPMemAllocator.java
deleted file mode 100644
index e412d0c..0000000
--- a/core/src/main/java/org/apache/mnemonic/BigDataPMemAllocator.java
+++ /dev/null
@@ -1,470 +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 org.apache.mnemonic;
-
-import java.nio.ByteBuffer;
-
-import org.apache.mnemonic.service.allocatorservice.NonVolatileMemoryAllocatorService;
-import org.flowcomputing.commons.resgc.ResCollector;
-import org.flowcomputing.commons.resgc.ResReclaim;
-
-/**
- * manage a big native persistent memory pool through libpmalloc.so provided by
- * pmalloc project.
- * 
- *
- */
-public class BigDataPMemAllocator extends CommonPersistAllocator<BigDataPMemAllocator> implements PMAddressTranslator {
-
-  private boolean m_activegc = true;
-  private long m_gctimeout = 100;
-  private long m_nid = -1;
-  private long b_addr = 0;
-  private NonVolatileMemoryAllocatorService m_nvmasvc = null;
-
-  /**
-   * Constructor, it initializes and allocate a memory pool from specified uri
-   * location with specified capacity and an allocator service instance.
-   * usually, the uri points to a mounted non-volatile memory device or a
-   * location of file system.
-   * 
-   * @param nvmasvc
-   *          the non-volatile memory allocation service instance
-   *
-   * @param capacity
-   *          the capacity of memory pool
-   * 
-   * @param uri
-   *          the location of memory pool will be created
-   * 
-   * @param isnew
-   *          a place holder, always specify it as true
-   */
-  public BigDataPMemAllocator(NonVolatileMemoryAllocatorService nvmasvc, long capacity, String uri, boolean isnew) {
-    assert null != nvmasvc : "NonVolatileMemoryAllocatorService object is null";
-    if (capacity <= 0) {
-      throw new IllegalArgumentException("BigDataPMemAllocator cannot be initialized with capacity <= 0.");
-    }
-
-    m_nvmasvc = nvmasvc;
-
-    m_nid = m_nvmasvc.init(capacity, uri, isnew);
-    b_addr = m_nvmasvc.getBaseAddress(m_nid);
-
-    /**
-     * create a resource collector to release specified chunk that backed by
-     * underlying big memory pool.
-     */
-    m_chunkcollector = new ResCollector<MemChunkHolder<BigDataPMemAllocator>, Long>(new ResReclaim<Long>() {
-      @Override
-      public void reclaim(Long mres) {
-        // System.out.println(String.format("Reclaim: %X", mres));
-        boolean cb_reclaimed = false;
-        if (null != m_chunkreclaimer) {
-          cb_reclaimed = m_chunkreclaimer.reclaim(mres, null);
-        }
-        if (!cb_reclaimed) {
-          m_nvmasvc.free(m_nid, mres);
-          mres = null;
-        }
-      }
-    });
-
-    /**
-     * create a resource collector to release specified bytebuffer that backed
-     * by underlying big memory pool.
-     */
-    m_bufcollector = new ResCollector<MemBufferHolder<BigDataPMemAllocator>, ByteBuffer>(new ResReclaim<ByteBuffer>() {
-      @Override
-      public void reclaim(ByteBuffer mres) {
-        boolean cb_reclaimed = false;
-        if (null != m_bufferreclaimer) {
-          cb_reclaimed = m_bufferreclaimer.reclaim(mres, Long.valueOf(mres.capacity()));
-        }
-        if (!cb_reclaimed) {
-          m_nvmasvc.destroyByteBuffer(m_nid, mres);
-          mres = null;
-        }
-      }
-    });
-  }
-
-  /**
-   * enable active garbage collection. the GC will be forced to collect garbages
-   * when there is no more space for current allocation request.
-   * 
-   * @param timeout
-   *          the timeout is used to yield for GC performing
-   */
-  @Override
-  public BigDataPMemAllocator enableActiveGC(long timeout) {
-    m_activegc = true;
-    m_gctimeout = timeout;
-    return this;
-  }
-
-  /**
-   * disable active garbage collection.
-   * 
-   */
-  @Override
-  public BigDataPMemAllocator disableActiveGC() {
-    m_activegc = false;
-    return this;
-  }
-
-  /**
-   * Release the memory pool and close it.
-   *
-   */
-  @Override
-  public void close() {
-    super.close();
-    m_nvmasvc.close(m_nid);
-  }
-
-  /**
-   * force to synchronize uncommitted data to backed memory pool (this is a
-   * placeholder).
-   *
-   */
-  @Override
-  public void sync() {
-  }
-
-  /**
-   * re-size a specified chunk on its backed memory pool.
-   * 
-   * @param mholder
-   *          the holder of memory chunk. it can be null.
-   * 
-   * @param size
-   *          specify a new size of memory chunk
-   * 
-   * @return the resized memory chunk handler
-   */
-  @Override
-  public MemChunkHolder<BigDataPMemAllocator> resizeChunk(MemChunkHolder<BigDataPMemAllocator> mholder, long size) {
-    MemChunkHolder<BigDataPMemAllocator> ret = null;
-    boolean ac = null != mholder.getRefId();
-    if (size > 0) {
-      Long addr = m_nvmasvc.reallocate(m_nid, mholder.get(), size, true);
-      if (0 == addr && m_activegc) {
-        m_chunkcollector.waitReclaimCoolDown(m_gctimeout);
-        addr = m_nvmasvc.reallocate(m_nid, mholder.get(), size, true);
-      }
-      if (0 != addr) {
-        mholder.clear();
-        mholder.destroy();
-        ret = new MemChunkHolder<BigDataPMemAllocator>(this, addr, size);
-        if (ac) {
-          m_chunkcollector.register(ret);
-        }
-      }
-    }
-    return ret;
-  }
-
-  /**
-   * resize a specified buffer on its backed memory pool.
-   *
-   * @param mholder
-   *          the holder of memory buffer. it can be null.
-   * 
-   * @param size
-   *          specify a new size of memory chunk
-   * 
-   * @return the resized memory buffer handler
-   *
-   */
-  @Override
-  public MemBufferHolder<BigDataPMemAllocator> resizeBuffer(MemBufferHolder<BigDataPMemAllocator> mholder, long size) {
-    MemBufferHolder<BigDataPMemAllocator> ret = null;
-    boolean ac = null != mholder.getRefId();
-    if (size > 0) {
-      int bufpos = mholder.get().position();
-      int buflimit = mholder.get().limit();
-      ByteBuffer buf = m_nvmasvc.resizeByteBuffer(m_nid, mholder.get(), size);
-      if (null == buf && m_activegc) {
-        m_bufcollector.waitReclaimCoolDown(m_gctimeout);
-        buf = m_nvmasvc.resizeByteBuffer(m_nid, mholder.get(), size);
-      }
-      if (null != buf) {
-        mholder.clear();
-        mholder.destroy();
-        buf.position(bufpos <= size ? bufpos : 0);
-        buf.limit(buflimit <= size ? buflimit : (int) size);
-        ret = new MemBufferHolder<BigDataPMemAllocator>(this, buf);
-        if (ac) {
-          m_bufcollector.register(ret);
-        }
-      }
-    }
-    return ret;
-  }
-
-  /**
-   * create a memory chunk that is managed by its holder.
-   * 
-   * @param size
-   *          specify the size of memory chunk
-   * 
-   * @param autoreclaim
-   *          specify whether or not to reclaim this chunk automatically
-   *
-   * @return a holder contains a memory chunk
-   */
-  @Override
-  public MemChunkHolder<BigDataPMemAllocator> createChunk(long size, boolean autoreclaim) {
-    MemChunkHolder<BigDataPMemAllocator> ret = null;
-    Long addr = m_nvmasvc.allocate(m_nid, size, true);
-    if ((null == addr || 0 == addr) && m_activegc) {
-      m_chunkcollector.waitReclaimCoolDown(m_gctimeout);
-      addr = m_nvmasvc.allocate(m_nid, size, true);
-    }
-    if (null != addr && 0 != addr) {
-      ret = new MemChunkHolder<BigDataPMemAllocator>(this, addr, size);
-      ret.setCollector(m_chunkcollector);
-      if (autoreclaim) {
-        m_chunkcollector.register(ret);
-      }
-    }
-    return ret;
-  }
-
-  /**
-   * create a memory buffer that is managed by its holder.
-   * 
-   * @param size
-   *          specify the size of memory buffer
-   * 
-   * @param autoreclaim
-   *          specify whether or not to reclaim this buffer automatically
-   *
-   * @return a holder contains a memory buffer
-   */
-  @Override
-  public MemBufferHolder<BigDataPMemAllocator> createBuffer(long size, boolean autoreclaim) {
-    MemBufferHolder<BigDataPMemAllocator> ret = null;
-    ByteBuffer bb = m_nvmasvc.createByteBuffer(m_nid, size);
-    if (null == bb && m_activegc) {
-      m_bufcollector.waitReclaimCoolDown(m_gctimeout);
-      bb = m_nvmasvc.createByteBuffer(m_nid, size);
-    }
-    if (null != bb) {
-      ret = new MemBufferHolder<BigDataPMemAllocator>(this, bb);
-      ret.setCollector(m_bufcollector);
-      if (autoreclaim) {
-        m_bufcollector.register(ret);
-      }
-    }
-    return ret;
-  }
-
-  /**
-   * retrieve a memory buffer from its backed memory allocator.
-   * 
-   * @param phandler
-   *          specify the handler of memory buffer to retrieve
-   *
-   * @param autoreclaim
-   *          specify whether this retrieved memory buffer can be reclaimed
-   *          automatically or not
-   * 
-   * @return a holder contains the retrieved memory buffer
-   */
-  @Override
-  public MemBufferHolder<BigDataPMemAllocator> retrieveBuffer(long phandler, boolean autoreclaim) {
-    MemBufferHolder<BigDataPMemAllocator> ret = null;
-    ByteBuffer bb = m_nvmasvc.retrieveByteBuffer(m_nid, getEffectiveAddress(phandler));
-    if (null != bb) {
-      ret = new MemBufferHolder<BigDataPMemAllocator>(this, bb);
-      if (autoreclaim) {
-        m_bufcollector.register(ret);
-      }
-    }
-    return ret;
-  }
-
-  /**
-   * retrieve a memory chunk from its backed memory allocator.
-   * 
-   * @param phandler
-   *          specify the handler of memory chunk to retrieve
-   *
-   * @param autoreclaim
-   *          specify whether this retrieved memory chunk can be reclaimed
-   *          automatically or not
-   * 
-   * @return a holder contains the retrieved memory chunk
-   */
-  @Override
-  public MemChunkHolder<BigDataPMemAllocator> retrieveChunk(long phandler, boolean autoreclaim) {
-    MemChunkHolder<BigDataPMemAllocator> ret = null;
-    long eaddr = getEffectiveAddress(phandler);
-    long sz = m_nvmasvc.retrieveSize(m_nid, eaddr);
-    if (sz > 0L) {
-      ret = new MemChunkHolder<BigDataPMemAllocator>(this, eaddr, sz);
-      if (autoreclaim) {
-        m_chunkcollector.register(ret);
-      }
-    }
-    return ret;
-  }
-
-  /**
-   * get the handler from a memory buffer holder.
-   * 
-   * @param mbuf
-   *          specify the memory buffer holder
-   *
-   * @return a handler that could be used to retrieve its memory buffer
-   */
-  @Override
-  public long getBufferHandler(MemBufferHolder<BigDataPMemAllocator> mbuf) {
-    return getPortableAddress(m_nvmasvc.getByteBufferHandler(m_nid, mbuf.get()));
-  }
-
-  /**
-   * get the handler from a memory chunk holder.
-   * 
-   * @param mchunk
-   *          specify the memory chunk holder
-   *
-   * @return a handler that could be used to retrieve its memory chunk
-   */
-  @Override
-  public long getChunkHandler(MemChunkHolder<BigDataPMemAllocator> mchunk) {
-    return getPortableAddress(mchunk.get());
-  }
-
-  /**
-   * determine whether this allocator supports to store non-volatile handler or
-   * not. (it is a placeholder)
-   *
-   * @return true if there is
-   */
-  @Override
-  public boolean hasNonVolatileHandlerStore() {
-    return true;
-  }
-
-  /**
-   * start a application level transaction on this allocator. (it is a place
-   * holder)
-   *
-   */
-  @Override
-  public void beginTransaction() {
-    throw new UnsupportedOperationException("Transaction Unsupported.");
-  }
-
-  /**
-   * end a application level transaction on this allocator. (it is a place
-   * holder)
-   *
-   */
-  @Override
-  public void endTransaction() {
-    throw new UnsupportedOperationException("Transaction Unsupported.");
-  }
-
-  /**
-   * set a handler on key.
-   * 
-   * @param key
-   *          the key to set its value
-   * 
-   * @param handler
-   *          the handler
-   */
-  public void setHandler(long key, long handler) {
-    m_nvmasvc.setHandler(m_nid, key, handler);
-  }
-
-  /**
-   * get a handler value.
-   * 
-   * @param key
-   *          the key to set its value
-   * 
-   * @return the value of handler
-   */
-  public long getHandler(long key) {
-    return m_nvmasvc.getHandler(m_nid, key);
-  }
-
-  /**
-   * return the capacity of non-volatile handler store.
-   * 
-   * @return the capacity of handler store
-   * 
-   */
-  public long handlerCapacity() {
-    return m_nvmasvc.handlerCapacity(m_nid);
-  }
-
-  /**
-   * calculate the portable address
-   *
-   * @param addr
-   *          the address to be calculated
-   *
-   * @return the portable address
-   */
-  @Override
-  public long getPortableAddress(long addr) {
-    return addr - b_addr;
-  }
-
-  /**
-   * calculate the effective address
-   *
-   * @param addr
-   *          the address to be calculated
-   *
-   * @return the effective address
-   */
-  @Override
-  public long getEffectiveAddress(long addr) {
-    return addr + b_addr;
-  }
-
-  /**
-   * get the base address
-   *
-   * @return the base address
-   */
-  @Override
-  public long getBaseAddress() {
-    return b_addr;
-  }
-
-  /**
-   * set the base address for calculation
-   *
-   * @param addr
-   *          the base address
-   *
-   */
-  @Override
-  public long setBaseAddress(long addr) {
-    return b_addr = addr;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-mnemonic/blob/679bcdc1/core/src/main/java/org/apache/mnemonic/CommonDurableAllocator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/mnemonic/CommonDurableAllocator.java b/core/src/main/java/org/apache/mnemonic/CommonDurableAllocator.java
new file mode 100644
index 0000000..af27f22
--- /dev/null
+++ b/core/src/main/java/org/apache/mnemonic/CommonDurableAllocator.java
@@ -0,0 +1,140 @@
+/*
+ * 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;
+
+/**
+ * an abstract common class for persistent memory allocator to provide common
+ * functionalities.
+ *
+ */
+public abstract class CommonDurableAllocator<A extends CommonAllocator<A>> extends CommonAllocator<A> {
+
+  /**
+   * determine whether the allocator supports transaction feature or not
+   *
+   * @return true if supported
+   */
+  public boolean supportTransaction() {
+    return false;
+  }
+
+  /**
+   * determine whether the allocator does atomic operations on memory pool
+   *
+   * @return true if it is
+   *
+   */
+  public boolean isAtomicOperation() {
+    return false;
+  }
+
+  /**
+   * determine whether this allocator supports to store durable handler or
+   * not
+   *
+   * @return true if there is
+   */
+  public boolean hasDurableHandlerStore() {
+    return false;
+  }
+
+  /**
+   * retrieve a memory buffer from its backed memory allocator.
+   * 
+   * @param phandler
+   *          specify the handler of memory buffer to retrieve
+   *
+   * @return a holder contains the retrieved memory buffer
+   */
+  public MemBufferHolder<A> retrieveBuffer(long phandler) {
+    return retrieveBuffer(phandler, true);
+  }
+
+  /**
+   * retrieve a memory chunk from its backed memory allocator.
+   * 
+   * @param phandler
+   *          specify the handler of memory chunk to retrieve
+   *
+   * @return a holder contains the retrieved memory chunk
+   */
+  public MemChunkHolder<A> retrieveChunk(long phandler) {
+    return retrieveChunk(phandler, true);
+  }
+
+  /**
+   * retrieve a memory buffer from its backed memory allocator.
+   * 
+   * @param phandler
+   *          specify the handler of memory buffer to retrieve
+   *
+   * @param autoreclaim
+   *          specify whether this retrieved memory buffer can be reclaimed
+   *          automatically or not
+   * 
+   * @return a holder contains the retrieved memory buffer
+   */
+  public abstract MemBufferHolder<A> retrieveBuffer(long phandler, boolean autoreclaim);
+
+  /**
+   * retrieve a memory chunk from its backed memory allocator.
+   * 
+   * @param phandler
+   *          specify the handler of memory chunk to retrieve
+   *
+   * @param autoreclaim
+   *          specify whether this retrieved memory chunk can be reclaimed
+   *          automatically or not
+   * 
+   * @return a holder contains the retrieved memory chunk
+   */
+  public abstract MemChunkHolder<A> retrieveChunk(long phandler, boolean autoreclaim);
+
+  /**
+   * get the handler from a memory buffer holder.
+   * 
+   * @param mbuf
+   *          specify the memory buffer holder
+   *
+   * @return a handler that could be used to retrieve its memory buffer
+   */
+  public abstract long getBufferHandler(MemBufferHolder<A> mbuf);
+
+  /**
+   * get the handler from a memory chunk holder.
+   * 
+   * @param mchunk
+   *          specify the memory chunk holder
+   *
+   * @return a handler that could be used to retrieve its memory chunk
+   */
+  public abstract long getChunkHandler(MemChunkHolder<A> mchunk);
+
+  /**
+   * start a application level transaction on this allocator.
+   *
+   */
+  public abstract void beginTransaction();
+
+  /**
+   * end a application level transaction on this allocator.
+   *
+   */
+  public abstract void endTransaction();
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-mnemonic/blob/679bcdc1/core/src/main/java/org/apache/mnemonic/CommonPersistAllocator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/mnemonic/CommonPersistAllocator.java b/core/src/main/java/org/apache/mnemonic/CommonPersistAllocator.java
deleted file mode 100644
index 0f6aae3..0000000
--- a/core/src/main/java/org/apache/mnemonic/CommonPersistAllocator.java
+++ /dev/null
@@ -1,140 +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 org.apache.mnemonic;
-
-/**
- * an abstract common class for persistent memory allocator to provide common
- * functionalities.
- *
- */
-public abstract class CommonPersistAllocator<A extends CommonAllocator<A>> extends CommonAllocator<A> {
-
-  /**
-   * determine whether the allocator supports transaction feature or not
-   *
-   * @return true if supported
-   */
-  public boolean supportTransaction() {
-    return false;
-  }
-
-  /**
-   * determine whether the allocator does atomic operations on memory pool
-   *
-   * @return true if it is
-   *
-   */
-  public boolean isAtomicOperation() {
-    return false;
-  }
-
-  /**
-   * determine whether this allocator supports to store non-volatile handler or
-   * not
-   *
-   * @return true if there is
-   */
-  public boolean hasNonVolatileHandlerStore() {
-    return false;
-  }
-
-  /**
-   * retrieve a memory buffer from its backed memory allocator.
-   * 
-   * @param phandler
-   *          specify the handler of memory buffer to retrieve
-   *
-   * @return a holder contains the retrieved memory buffer
-   */
-  public MemBufferHolder<A> retrieveBuffer(long phandler) {
-    return retrieveBuffer(phandler, true);
-  }
-
-  /**
-   * retrieve a memory chunk from its backed memory allocator.
-   * 
-   * @param phandler
-   *          specify the handler of memory chunk to retrieve
-   *
-   * @return a holder contains the retrieved memory chunk
-   */
-  public MemChunkHolder<A> retrieveChunk(long phandler) {
-    return retrieveChunk(phandler, true);
-  }
-
-  /**
-   * retrieve a memory buffer from its backed memory allocator.
-   * 
-   * @param phandler
-   *          specify the handler of memory buffer to retrieve
-   *
-   * @param autoreclaim
-   *          specify whether this retrieved memory buffer can be reclaimed
-   *          automatically or not
-   * 
-   * @return a holder contains the retrieved memory buffer
-   */
-  public abstract MemBufferHolder<A> retrieveBuffer(long phandler, boolean autoreclaim);
-
-  /**
-   * retrieve a memory chunk from its backed memory allocator.
-   * 
-   * @param phandler
-   *          specify the handler of memory chunk to retrieve
-   *
-   * @param autoreclaim
-   *          specify whether this retrieved memory chunk can be reclaimed
-   *          automatically or not
-   * 
-   * @return a holder contains the retrieved memory chunk
-   */
-  public abstract MemChunkHolder<A> retrieveChunk(long phandler, boolean autoreclaim);
-
-  /**
-   * get the handler from a memory buffer holder.
-   * 
-   * @param mbuf
-   *          specify the memory buffer holder
-   *
-   * @return a handler that could be used to retrieve its memory buffer
-   */
-  public abstract long getBufferHandler(MemBufferHolder<A> mbuf);
-
-  /**
-   * get the handler from a memory chunk holder.
-   * 
-   * @param mchunk
-   *          specify the memory chunk holder
-   *
-   * @return a handler that could be used to retrieve its memory chunk
-   */
-  public abstract long getChunkHandler(MemChunkHolder<A> mchunk);
-
-  /**
-   * start a application level transaction on this allocator.
-   *
-   */
-  public abstract void beginTransaction();
-
-  /**
-   * end a application level transaction on this allocator.
-   *
-   */
-  public abstract void endTransaction();
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-mnemonic/blob/679bcdc1/core/src/main/java/org/apache/mnemonic/Durable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/mnemonic/Durable.java b/core/src/main/java/org/apache/mnemonic/Durable.java
index 5666fa5..56f9ac1 100644
--- a/core/src/main/java/org/apache/mnemonic/Durable.java
+++ b/core/src/main/java/org/apache/mnemonic/Durable.java
@@ -71,7 +71,7 @@ public interface Durable {
    *
    * @return the handler of this object
    */
-  long getNonVolatileHandler();
+  long getHandler();
 
   /**
    * return the setting for auto-reclaim
@@ -84,7 +84,7 @@ public interface Durable {
    * manually destroy this object and release its memory resource
    *
    */
-  void destroy() throws RetrieveNonVolatileEntityError;
+  void destroy() throws RetrieveDurableEntityError;
 
   /**
    * return the native field map info for native processing.

http://git-wip-us.apache.org/repos/asf/incubator-mnemonic/blob/679bcdc1/core/src/main/java/org/apache/mnemonic/DurableEntity.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/mnemonic/DurableEntity.java b/core/src/main/java/org/apache/mnemonic/DurableEntity.java
new file mode 100644
index 0000000..66e3137
--- /dev/null
+++ b/core/src/main/java/org/apache/mnemonic/DurableEntity.java
@@ -0,0 +1,34 @@
+/*
+ * 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;
+
+import java.lang.annotation.Retention;
+import java.lang.annotation.Target;
+import java.lang.annotation.ElementType;
+import java.lang.annotation.RetentionPolicy;
+
+/**
+ * this class defines a annotation for non-volatile entity
+ *
+ */
+
+@Target(ElementType.TYPE)
+@Retention(RetentionPolicy.CLASS)
+public @interface DurableEntity {
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-mnemonic/blob/679bcdc1/core/src/main/java/org/apache/mnemonic/DurableEntityProcessor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/mnemonic/DurableEntityProcessor.java b/core/src/main/java/org/apache/mnemonic/DurableEntityProcessor.java
new file mode 100644
index 0000000..7610001
--- /dev/null
+++ b/core/src/main/java/org/apache/mnemonic/DurableEntityProcessor.java
@@ -0,0 +1,147 @@
+/*
+ * 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;
+
+/**
+ * a non-volatile annotation processor
+ *
+ */
+
+import java.io.IOException;
+import java.util.LinkedHashSet;
+import java.util.Set;
+
+import javax.annotation.processing.AbstractProcessor;
+import javax.annotation.processing.Filer;
+import javax.annotation.processing.Messager;
+import javax.annotation.processing.ProcessingEnvironment;
+import javax.annotation.processing.RoundEnvironment;
+import javax.lang.model.SourceVersion;
+import javax.lang.model.element.Element;
+import javax.lang.model.element.ElementKind;
+import javax.lang.model.element.TypeElement;
+import javax.lang.model.util.Elements;
+import javax.lang.model.util.Types;
+
+import javax.tools.Diagnostic;
+
+public class DurableEntityProcessor extends AbstractProcessor {
+  private Types typeUtils;
+  private Elements elementUtils;
+  private Filer filer;
+  private Messager messager;
+  // private Map<String, FactoryGroupedClasses> factoryClasses = new
+  // LinkedHashMap<String, FactoryGroupedClasses>();
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public synchronized void init(ProcessingEnvironment processingEnv) {
+    super.init(processingEnv);
+    typeUtils = processingEnv.getTypeUtils();
+    elementUtils = processingEnv.getElementUtils();
+    filer = processingEnv.getFiler();
+    messager = processingEnv.getMessager();
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public Set<String> getSupportedAnnotationTypes() {
+    Set<String> annotataions = new LinkedHashSet<String>();
+    annotataions.add(DurableEntity.class.getCanonicalName());
+    return annotataions;
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public SourceVersion getSupportedSourceVersion() {
+    return SourceVersion.latestSupported();
+  }
+
+  /**
+   * triggered if an error issued during processing
+   *
+   * @param e
+   *          the element in question
+   *
+   * @param msg
+   *          the message issued
+   */
+  public void error(Element e, String msg) {
+    messager.printMessage(Diagnostic.Kind.ERROR, msg, e);
+  }
+
+  /**
+   * triggered if a note issued during processing
+   *
+   * @param e
+   *          the element in question
+   *
+   * @param msg
+   *          the message issued
+   */
+  public void note(Element e, String msg) {
+    messager.printMessage(Diagnostic.Kind.NOTE, msg, e);
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public boolean process(Set<? extends TypeElement> annotations, RoundEnvironment roundEnv) {
+
+    try {
+
+      for (Element annotatedElement : roundEnv.getElementsAnnotatedWith(DurableEntity.class)) {
+
+        String outputstr = String.format("++++++++++%s+++++++++++", annotatedElement.getSimpleName());
+        note(annotatedElement, outputstr);
+        // System.err.println(outputstr);
+
+        if (annotatedElement.getKind() != ElementKind.CLASS) {
+          throw new AnnotationProcessingException(annotatedElement, "Only classes can be annotated with @%s",
+              DurableEntity.class.getSimpleName());
+        }
+
+        // We can cast it, because we know that it of ElementKind.CLASS
+        TypeElement typeelem = (TypeElement) annotatedElement;
+
+        AnnotatedDurableEntityClass annotatedClass = new AnnotatedDurableEntityClass(typeelem, typeUtils,
+            elementUtils, messager);
+
+        annotatedClass.prepareProcessing();
+
+        annotatedClass.generateCode(filer);
+
+      }
+
+    } catch (AnnotationProcessingException e) {
+      error(e.getElement(), e.getMessage());
+    } catch (IOException e) {
+      error(null, e.getMessage());
+    }
+
+    return true;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-mnemonic/blob/679bcdc1/core/src/main/java/org/apache/mnemonic/DurableGetter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/mnemonic/DurableGetter.java b/core/src/main/java/org/apache/mnemonic/DurableGetter.java
new file mode 100644
index 0000000..5a11af1
--- /dev/null
+++ b/core/src/main/java/org/apache/mnemonic/DurableGetter.java
@@ -0,0 +1,38 @@
+/*
+ * 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;
+
+/**
+ * this class defines an annotation for getter methods of non-volatile entity
+ *
+ */
+
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+@Target(ElementType.METHOD)
+@Retention(RetentionPolicy.CLASS)
+public @interface DurableGetter {
+  String EntityFactoryProxies() default "null";
+
+  String GenericFieldTypes() default "null";
+
+  long Id() default -1L;
+}

http://git-wip-us.apache.org/repos/asf/incubator-mnemonic/blob/679bcdc1/core/src/main/java/org/apache/mnemonic/DurableSetter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/mnemonic/DurableSetter.java b/core/src/main/java/org/apache/mnemonic/DurableSetter.java
new file mode 100644
index 0000000..01021b9
--- /dev/null
+++ b/core/src/main/java/org/apache/mnemonic/DurableSetter.java
@@ -0,0 +1,34 @@
+/*
+ * 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;
+
+/**
+ * this class defines an annotation for setter methods of non-volatile entity
+ *
+ */
+
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+@Target(ElementType.METHOD)
+@Retention(RetentionPolicy.CLASS)
+public @interface DurableSetter {
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-mnemonic/blob/679bcdc1/core/src/main/java/org/apache/mnemonic/EntityFactoryProxy.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/mnemonic/EntityFactoryProxy.java b/core/src/main/java/org/apache/mnemonic/EntityFactoryProxy.java
index 621f580..0a36b02 100644
--- a/core/src/main/java/org/apache/mnemonic/EntityFactoryProxy.java
+++ b/core/src/main/java/org/apache/mnemonic/EntityFactoryProxy.java
@@ -48,6 +48,6 @@ public interface EntityFactoryProxy {
    * @return the restored non-volatile object from this factory proxy
    *
    */
-  <A extends CommonPersistAllocator<A>> Durable restore(A allocator, EntityFactoryProxy[] factoryproxys,
+  <A extends CommonDurableAllocator<A>> Durable restore(A allocator, EntityFactoryProxy[] factoryproxys,
       GenericField.GType[] gfields, long phandler, boolean autoreclaim);
 }

http://git-wip-us.apache.org/repos/asf/incubator-mnemonic/blob/679bcdc1/core/src/main/java/org/apache/mnemonic/GenericField.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/mnemonic/GenericField.java b/core/src/main/java/org/apache/mnemonic/GenericField.java
index a1d3c9e..b9d1df6 100644
--- a/core/src/main/java/org/apache/mnemonic/GenericField.java
+++ b/core/src/main/java/org/apache/mnemonic/GenericField.java
@@ -25,7 +25,7 @@ package org.apache.mnemonic;
 import sun.misc.Unsafe;
 
 @SuppressWarnings("restriction")
-public class GenericField<A extends CommonPersistAllocator<A>, E> implements Durable {
+public class GenericField<A extends CommonDurableAllocator<A>, E> implements Durable {
 
   /**
    * defines the types of generic field
@@ -134,7 +134,7 @@ public class GenericField<A extends CommonPersistAllocator<A>, E> implements Dur
       } else {
         m_strfield = m_allocator.createBuffer(str.length() * 2, m_autoreclaim);
         if (null == m_strfield) {
-          throw new OutOfPersistentMemory("Create Persistent String Error!");
+          throw new OutOfHybridMemory("Create Durable String Error!");
         }
         m_strfield.get().asCharBuffer().put(str);
         m_unsafe.putAddress(m_fpos, m_allocator.getBufferHandler(m_strfield));
@@ -147,7 +147,7 @@ public class GenericField<A extends CommonPersistAllocator<A>, E> implements Dur
         m_unsafe.putAddress(m_fpos, 0L);
       }
       m_field = (Durable) e;
-      m_unsafe.putAddress(m_fpos, null == m_field ? 0L : m_field.getNonVolatileHandler());
+      m_unsafe.putAddress(m_fpos, null == m_field ? 0L : m_field.getHandler());
       break;
     }
 
@@ -192,7 +192,7 @@ public class GenericField<A extends CommonPersistAllocator<A>, E> implements Dur
         if (0L != phandler) {
           m_strfield = m_allocator.retrieveBuffer(phandler, m_autoreclaim);
           if (null == m_strfield) {
-            throw new RetrieveNonVolatileEntityError("Retrieve String Buffer Failure.");
+            throw new RetrieveDurableEntityError("Retrieve String Buffer Failure.");
           }
         }
       }
@@ -203,7 +203,7 @@ public class GenericField<A extends CommonPersistAllocator<A>, E> implements Dur
         long phandler = m_unsafe.getAddress(m_fpos);
         if (0L != phandler) {
           if (null == m_defproxy) {
-            throw new RetrieveNonVolatileEntityError("Proxy not specified for Non-Volatile Generic entity.");
+            throw new RetrieveDurableEntityError("Proxy not specified for Non-Volatile Generic entity.");
           }
           m_field = m_defproxy.restore(m_allocator, m_efproxies, m_gftypes, phandler, m_autoreclaim);
         }
@@ -262,8 +262,8 @@ public class GenericField<A extends CommonPersistAllocator<A>, E> implements Dur
    * {@inheritDoc}
    */
   @Override
-  public long getNonVolatileHandler() {
-    throw new UnsupportedOperationException("GenericField.getNonVolatileHandler()");
+  public long getHandler() {
+    throw new UnsupportedOperationException("GenericField.getHandler()");
   }
 
   /**
@@ -278,7 +278,7 @@ public class GenericField<A extends CommonPersistAllocator<A>, E> implements Dur
    * {@inheritDoc}
    */
   @Override
-  public void destroy() throws RetrieveNonVolatileEntityError {
+  public void destroy() throws RetrieveDurableEntityError {
     if (null != m_field) {
       m_field.destroy();
     }

http://git-wip-us.apache.org/repos/asf/incubator-mnemonic/blob/679bcdc1/core/src/main/java/org/apache/mnemonic/MemoryDurableEntity.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/mnemonic/MemoryDurableEntity.java b/core/src/main/java/org/apache/mnemonic/MemoryDurableEntity.java
new file mode 100644
index 0000000..a2c56bf
--- /dev/null
+++ b/core/src/main/java/org/apache/mnemonic/MemoryDurableEntity.java
@@ -0,0 +1,36 @@
+/*
+ * 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;
+
+/**
+ *
+ *
+ */
+
+public interface MemoryDurableEntity<ALLOC_PMem3C93D24F59 extends CommonDurableAllocator<ALLOC_PMem3C93D24F59>> {
+
+  void initializeDurableEntity(ALLOC_PMem3C93D24F59 allocator, EntityFactoryProxy[] efproxys,
+      GenericField.GType[] gfields, boolean autoreclaim);
+
+  void createDurableEntity(ALLOC_PMem3C93D24F59 allocator, EntityFactoryProxy[] efproxys,
+      GenericField.GType[] gfields, boolean autoreclaim) throws OutOfHybridMemory;
+
+  void restoreDurableEntity(ALLOC_PMem3C93D24F59 allocator, EntityFactoryProxy[] efproxys,
+      GenericField.GType[] gfields, long phandler, boolean autoreclaim) throws RetrieveDurableEntityError;
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-mnemonic/blob/679bcdc1/core/src/main/java/org/apache/mnemonic/MemoryNonVolatileEntity.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/mnemonic/MemoryNonVolatileEntity.java b/core/src/main/java/org/apache/mnemonic/MemoryNonVolatileEntity.java
deleted file mode 100644
index 4ae615a..0000000
--- a/core/src/main/java/org/apache/mnemonic/MemoryNonVolatileEntity.java
+++ /dev/null
@@ -1,36 +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 org.apache.mnemonic;
-
-/**
- *
- *
- */
-
-public interface MemoryNonVolatileEntity<ALLOC_PMem3C93D24F59 extends CommonPersistAllocator<ALLOC_PMem3C93D24F59>> {
-
-  void initializeNonVolatileEntity(ALLOC_PMem3C93D24F59 allocator, EntityFactoryProxy[] efproxys,
-      GenericField.GType[] gfields, boolean autoreclaim);
-
-  void createNonVolatileEntity(ALLOC_PMem3C93D24F59 allocator, EntityFactoryProxy[] efproxys,
-      GenericField.GType[] gfields, boolean autoreclaim) throws OutOfPersistentMemory;
-
-  void restoreNonVolatileEntity(ALLOC_PMem3C93D24F59 allocator, EntityFactoryProxy[] efproxys,
-      GenericField.GType[] gfields, long phandler, boolean autoreclaim) throws RetrieveNonVolatileEntityError;
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-mnemonic/blob/679bcdc1/core/src/main/java/org/apache/mnemonic/NVMAddressTranslator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/mnemonic/NVMAddressTranslator.java b/core/src/main/java/org/apache/mnemonic/NVMAddressTranslator.java
new file mode 100644
index 0000000..0c4ff32
--- /dev/null
+++ b/core/src/main/java/org/apache/mnemonic/NVMAddressTranslator.java
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.mnemonic;
+
+/**
+ * translate persistent memory address for allocator
+ *
+ */
+public interface NVMAddressTranslator {
+
+  /**
+   * calculate the portable address
+   *
+   * @param addr
+   *          the address to be calculated
+   *
+   * @return the portable address
+   */
+  long getPortableAddress(long addr);
+
+  /**
+   * calculate the effective address
+   *
+   * @param addr
+   *          the address to be calculated
+   *
+   * @return the effective address
+   */
+  long getEffectiveAddress(long addr);
+
+  /**
+   * get the base address
+   *
+   * @return the base address
+   */
+  long getBaseAddress();
+
+  /**
+   * set the base address for calculation
+   *
+   * @param addr
+   *          the base address
+   *
+   */
+  long setBaseAddress(long addr);
+}

http://git-wip-us.apache.org/repos/asf/incubator-mnemonic/blob/679bcdc1/core/src/main/java/org/apache/mnemonic/NonVolatileEntity.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/mnemonic/NonVolatileEntity.java b/core/src/main/java/org/apache/mnemonic/NonVolatileEntity.java
deleted file mode 100644
index 8ba6569..0000000
--- a/core/src/main/java/org/apache/mnemonic/NonVolatileEntity.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.mnemonic;
-
-import java.lang.annotation.Retention;
-import java.lang.annotation.Target;
-import java.lang.annotation.ElementType;
-import java.lang.annotation.RetentionPolicy;
-
-/**
- * this class defines a annotation for non-volatile entity
- *
- */
-
-@Target(ElementType.TYPE)
-@Retention(RetentionPolicy.CLASS)
-public @interface NonVolatileEntity {
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-mnemonic/blob/679bcdc1/core/src/main/java/org/apache/mnemonic/NonVolatileEntityProcessor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/mnemonic/NonVolatileEntityProcessor.java b/core/src/main/java/org/apache/mnemonic/NonVolatileEntityProcessor.java
deleted file mode 100644
index c10b7b2..0000000
--- a/core/src/main/java/org/apache/mnemonic/NonVolatileEntityProcessor.java
+++ /dev/null
@@ -1,147 +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 org.apache.mnemonic;
-
-/**
- * a non-volatile annotation processor
- *
- */
-
-import java.io.IOException;
-import java.util.LinkedHashSet;
-import java.util.Set;
-
-import javax.annotation.processing.AbstractProcessor;
-import javax.annotation.processing.Filer;
-import javax.annotation.processing.Messager;
-import javax.annotation.processing.ProcessingEnvironment;
-import javax.annotation.processing.RoundEnvironment;
-import javax.lang.model.SourceVersion;
-import javax.lang.model.element.Element;
-import javax.lang.model.element.ElementKind;
-import javax.lang.model.element.TypeElement;
-import javax.lang.model.util.Elements;
-import javax.lang.model.util.Types;
-
-import javax.tools.Diagnostic;
-
-public class NonVolatileEntityProcessor extends AbstractProcessor {
-  private Types typeUtils;
-  private Elements elementUtils;
-  private Filer filer;
-  private Messager messager;
-  // private Map<String, FactoryGroupedClasses> factoryClasses = new
-  // LinkedHashMap<String, FactoryGroupedClasses>();
-
-  /**
-   * {@inheritDoc}
-   */
-  @Override
-  public synchronized void init(ProcessingEnvironment processingEnv) {
-    super.init(processingEnv);
-    typeUtils = processingEnv.getTypeUtils();
-    elementUtils = processingEnv.getElementUtils();
-    filer = processingEnv.getFiler();
-    messager = processingEnv.getMessager();
-  }
-
-  /**
-   * {@inheritDoc}
-   */
-  @Override
-  public Set<String> getSupportedAnnotationTypes() {
-    Set<String> annotataions = new LinkedHashSet<String>();
-    annotataions.add(NonVolatileEntity.class.getCanonicalName());
-    return annotataions;
-  }
-
-  /**
-   * {@inheritDoc}
-   */
-  @Override
-  public SourceVersion getSupportedSourceVersion() {
-    return SourceVersion.latestSupported();
-  }
-
-  /**
-   * triggered if an error issued during processing
-   *
-   * @param e
-   *          the element in question
-   *
-   * @param msg
-   *          the message issued
-   */
-  public void error(Element e, String msg) {
-    messager.printMessage(Diagnostic.Kind.ERROR, msg, e);
-  }
-
-  /**
-   * triggered if a note issued during processing
-   *
-   * @param e
-   *          the element in question
-   *
-   * @param msg
-   *          the message issued
-   */
-  public void note(Element e, String msg) {
-    messager.printMessage(Diagnostic.Kind.NOTE, msg, e);
-  }
-
-  /**
-   * {@inheritDoc}
-   */
-  @Override
-  public boolean process(Set<? extends TypeElement> annotations, RoundEnvironment roundEnv) {
-
-    try {
-
-      for (Element annotatedElement : roundEnv.getElementsAnnotatedWith(NonVolatileEntity.class)) {
-
-        String outputstr = String.format("++++++++++%s+++++++++++", annotatedElement.getSimpleName());
-        note(annotatedElement, outputstr);
-        // System.err.println(outputstr);
-
-        if (annotatedElement.getKind() != ElementKind.CLASS) {
-          throw new AnnotationProcessingException(annotatedElement, "Only classes can be annotated with @%s",
-              NonVolatileEntity.class.getSimpleName());
-        }
-
-        // We can cast it, because we know that it of ElementKind.CLASS
-        TypeElement typeelem = (TypeElement) annotatedElement;
-
-        AnnotatedNonVolatileEntityClass annotatedClass = new AnnotatedNonVolatileEntityClass(typeelem, typeUtils,
-            elementUtils, messager);
-
-        annotatedClass.prepareProcessing();
-
-        annotatedClass.generateCode(filer);
-
-      }
-
-    } catch (AnnotationProcessingException e) {
-      error(e.getElement(), e.getMessage());
-    } catch (IOException e) {
-      error(null, e.getMessage());
-    }
-
-    return true;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-mnemonic/blob/679bcdc1/core/src/main/java/org/apache/mnemonic/NonVolatileGetter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/mnemonic/NonVolatileGetter.java b/core/src/main/java/org/apache/mnemonic/NonVolatileGetter.java
deleted file mode 100644
index 39f49a4..0000000
--- a/core/src/main/java/org/apache/mnemonic/NonVolatileGetter.java
+++ /dev/null
@@ -1,38 +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 org.apache.mnemonic;
-
-/**
- * this class defines an annotation for getter methods of non-volatile entity
- *
- */
-
-import java.lang.annotation.ElementType;
-import java.lang.annotation.Retention;
-import java.lang.annotation.RetentionPolicy;
-import java.lang.annotation.Target;
-
-@Target(ElementType.METHOD)
-@Retention(RetentionPolicy.CLASS)
-public @interface NonVolatileGetter {
-  String EntityFactoryProxies() default "null";
-
-  String GenericFieldTypes() default "null";
-
-  long Id() default -1L;
-}

http://git-wip-us.apache.org/repos/asf/incubator-mnemonic/blob/679bcdc1/core/src/main/java/org/apache/mnemonic/NonVolatileMemAllocator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/mnemonic/NonVolatileMemAllocator.java b/core/src/main/java/org/apache/mnemonic/NonVolatileMemAllocator.java
new file mode 100644
index 0000000..6c07c31
--- /dev/null
+++ b/core/src/main/java/org/apache/mnemonic/NonVolatileMemAllocator.java
@@ -0,0 +1,474 @@
+/*
+ * 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;
+
+import java.nio.ByteBuffer;
+
+import org.apache.mnemonic.service.allocatorservice.NonVolatileMemoryAllocatorService;
+import org.flowcomputing.commons.resgc.ResCollector;
+import org.flowcomputing.commons.resgc.ResReclaim;
+
+/**
+ * manage a big native persistent memory pool through libpmalloc.so provided by
+ * pmalloc project.
+ * 
+ *
+ */
+public class NonVolatileMemAllocator extends CommonDurableAllocator<NonVolatileMemAllocator>
+    implements NVMAddressTranslator {
+
+  private boolean m_activegc = true;
+  private long m_gctimeout = 100;
+  private long m_nid = -1;
+  private long b_addr = 0;
+  private NonVolatileMemoryAllocatorService m_nvmasvc = null;
+
+  /**
+   * Constructor, it initializes and allocate a memory pool from specified uri
+   * location with specified capacity and an allocator service instance.
+   * usually, the uri points to a mounted non-volatile memory device or a
+   * location of file system.
+   * 
+   * @param nvmasvc
+   *          the non-volatile memory allocation service instance
+   *
+   * @param capacity
+   *          the capacity of memory pool
+   * 
+   * @param uri
+   *          the location of memory pool will be created
+   * 
+   * @param isnew
+   *          a place holder, always specify it as true
+   */
+  public NonVolatileMemAllocator(NonVolatileMemoryAllocatorService nvmasvc, long capacity, String uri, boolean isnew) {
+    assert null != nvmasvc : "NonVolatileMemoryAllocatorService object is null";
+    if (capacity <= 0) {
+      throw new IllegalArgumentException("BigDataPMemAllocator cannot be initialized with capacity <= 0.");
+    }
+
+    m_nvmasvc = nvmasvc;
+
+    m_nid = m_nvmasvc.init(capacity, uri, isnew);
+    b_addr = m_nvmasvc.getBaseAddress(m_nid);
+
+    /**
+     * create a resource collector to release specified chunk that backed by
+     * underlying big memory pool.
+     */
+    m_chunkcollector = new ResCollector<MemChunkHolder<NonVolatileMemAllocator>, Long>(new ResReclaim<Long>() {
+      @Override
+      public void reclaim(Long mres) {
+        // System.out.println(String.format("Reclaim: %X", mres));
+        boolean cb_reclaimed = false;
+        if (null != m_chunkreclaimer) {
+          cb_reclaimed = m_chunkreclaimer.reclaim(mres, null);
+        }
+        if (!cb_reclaimed) {
+          m_nvmasvc.free(m_nid, mres);
+          mres = null;
+        }
+      }
+    });
+
+    /**
+     * create a resource collector to release specified bytebuffer that backed
+     * by underlying big memory pool.
+     */
+    m_bufcollector = new ResCollector<MemBufferHolder<NonVolatileMemAllocator>, ByteBuffer>(
+        new ResReclaim<ByteBuffer>() {
+          @Override
+          public void reclaim(ByteBuffer mres) {
+            boolean cb_reclaimed = false;
+            if (null != m_bufferreclaimer) {
+              cb_reclaimed = m_bufferreclaimer.reclaim(mres, Long.valueOf(mres.capacity()));
+            }
+            if (!cb_reclaimed) {
+              m_nvmasvc.destroyByteBuffer(m_nid, mres);
+              mres = null;
+            }
+          }
+        });
+  }
+
+  /**
+   * enable active garbage collection. the GC will be forced to collect garbages
+   * when there is no more space for current allocation request.
+   * 
+   * @param timeout
+   *          the timeout is used to yield for GC performing
+   */
+  @Override
+  public NonVolatileMemAllocator enableActiveGC(long timeout) {
+    m_activegc = true;
+    m_gctimeout = timeout;
+    return this;
+  }
+
+  /**
+   * disable active garbage collection.
+   * 
+   */
+  @Override
+  public NonVolatileMemAllocator disableActiveGC() {
+    m_activegc = false;
+    return this;
+  }
+
+  /**
+   * Release the memory pool and close it.
+   *
+   */
+  @Override
+  public void close() {
+    super.close();
+    m_nvmasvc.close(m_nid);
+  }
+
+  /**
+   * force to synchronize uncommitted data to backed memory pool (this is a
+   * placeholder).
+   *
+   */
+  @Override
+  public void sync() {
+  }
+
+  /**
+   * re-size a specified chunk on its backed memory pool.
+   * 
+   * @param mholder
+   *          the holder of memory chunk. it can be null.
+   * 
+   * @param size
+   *          specify a new size of memory chunk
+   * 
+   * @return the resized memory chunk handler
+   */
+  @Override
+  public MemChunkHolder<NonVolatileMemAllocator> resizeChunk(MemChunkHolder<NonVolatileMemAllocator> mholder,
+      long size) {
+    MemChunkHolder<NonVolatileMemAllocator> ret = null;
+    boolean ac = null != mholder.getRefId();
+    if (size > 0) {
+      Long addr = m_nvmasvc.reallocate(m_nid, mholder.get(), size, true);
+      if (0 == addr && m_activegc) {
+        m_chunkcollector.waitReclaimCoolDown(m_gctimeout);
+        addr = m_nvmasvc.reallocate(m_nid, mholder.get(), size, true);
+      }
+      if (0 != addr) {
+        mholder.clear();
+        mholder.destroy();
+        ret = new MemChunkHolder<NonVolatileMemAllocator>(this, addr, size);
+        if (ac) {
+          m_chunkcollector.register(ret);
+        }
+      }
+    }
+    return ret;
+  }
+
+  /**
+   * resize a specified buffer on its backed memory pool.
+   *
+   * @param mholder
+   *          the holder of memory buffer. it can be null.
+   * 
+   * @param size
+   *          specify a new size of memory chunk
+   * 
+   * @return the resized memory buffer handler
+   *
+   */
+  @Override
+  public MemBufferHolder<NonVolatileMemAllocator> resizeBuffer(MemBufferHolder<NonVolatileMemAllocator> mholder,
+      long size) {
+    MemBufferHolder<NonVolatileMemAllocator> ret = null;
+    boolean ac = null != mholder.getRefId();
+    if (size > 0) {
+      int bufpos = mholder.get().position();
+      int buflimit = mholder.get().limit();
+      ByteBuffer buf = m_nvmasvc.resizeByteBuffer(m_nid, mholder.get(), size);
+      if (null == buf && m_activegc) {
+        m_bufcollector.waitReclaimCoolDown(m_gctimeout);
+        buf = m_nvmasvc.resizeByteBuffer(m_nid, mholder.get(), size);
+      }
+      if (null != buf) {
+        mholder.clear();
+        mholder.destroy();
+        buf.position(bufpos <= size ? bufpos : 0);
+        buf.limit(buflimit <= size ? buflimit : (int) size);
+        ret = new MemBufferHolder<NonVolatileMemAllocator>(this, buf);
+        if (ac) {
+          m_bufcollector.register(ret);
+        }
+      }
+    }
+    return ret;
+  }
+
+  /**
+   * create a memory chunk that is managed by its holder.
+   * 
+   * @param size
+   *          specify the size of memory chunk
+   * 
+   * @param autoreclaim
+   *          specify whether or not to reclaim this chunk automatically
+   *
+   * @return a holder contains a memory chunk
+   */
+  @Override
+  public MemChunkHolder<NonVolatileMemAllocator> createChunk(long size, boolean autoreclaim) {
+    MemChunkHolder<NonVolatileMemAllocator> ret = null;
+    Long addr = m_nvmasvc.allocate(m_nid, size, true);
+    if ((null == addr || 0 == addr) && m_activegc) {
+      m_chunkcollector.waitReclaimCoolDown(m_gctimeout);
+      addr = m_nvmasvc.allocate(m_nid, size, true);
+    }
+    if (null != addr && 0 != addr) {
+      ret = new MemChunkHolder<NonVolatileMemAllocator>(this, addr, size);
+      ret.setCollector(m_chunkcollector);
+      if (autoreclaim) {
+        m_chunkcollector.register(ret);
+      }
+    }
+    return ret;
+  }
+
+  /**
+   * create a memory buffer that is managed by its holder.
+   * 
+   * @param size
+   *          specify the size of memory buffer
+   * 
+   * @param autoreclaim
+   *          specify whether or not to reclaim this buffer automatically
+   *
+   * @return a holder contains a memory buffer
+   */
+  @Override
+  public MemBufferHolder<NonVolatileMemAllocator> createBuffer(long size, boolean autoreclaim) {
+    MemBufferHolder<NonVolatileMemAllocator> ret = null;
+    ByteBuffer bb = m_nvmasvc.createByteBuffer(m_nid, size);
+    if (null == bb && m_activegc) {
+      m_bufcollector.waitReclaimCoolDown(m_gctimeout);
+      bb = m_nvmasvc.createByteBuffer(m_nid, size);
+    }
+    if (null != bb) {
+      ret = new MemBufferHolder<NonVolatileMemAllocator>(this, bb);
+      ret.setCollector(m_bufcollector);
+      if (autoreclaim) {
+        m_bufcollector.register(ret);
+      }
+    }
+    return ret;
+  }
+
+  /**
+   * retrieve a memory buffer from its backed memory allocator.
+   * 
+   * @param phandler
+   *          specify the handler of memory buffer to retrieve
+   *
+   * @param autoreclaim
+   *          specify whether this retrieved memory buffer can be reclaimed
+   *          automatically or not
+   * 
+   * @return a holder contains the retrieved memory buffer
+   */
+  @Override
+  public MemBufferHolder<NonVolatileMemAllocator> retrieveBuffer(long phandler, boolean autoreclaim) {
+    MemBufferHolder<NonVolatileMemAllocator> ret = null;
+    ByteBuffer bb = m_nvmasvc.retrieveByteBuffer(m_nid, getEffectiveAddress(phandler));
+    if (null != bb) {
+      ret = new MemBufferHolder<NonVolatileMemAllocator>(this, bb);
+      if (autoreclaim) {
+        m_bufcollector.register(ret);
+      }
+    }
+    return ret;
+  }
+
+  /**
+   * retrieve a memory chunk from its backed memory allocator.
+   * 
+   * @param phandler
+   *          specify the handler of memory chunk to retrieve
+   *
+   * @param autoreclaim
+   *          specify whether this retrieved memory chunk can be reclaimed
+   *          automatically or not
+   * 
+   * @return a holder contains the retrieved memory chunk
+   */
+  @Override
+  public MemChunkHolder<NonVolatileMemAllocator> retrieveChunk(long phandler, boolean autoreclaim) {
+    MemChunkHolder<NonVolatileMemAllocator> ret = null;
+    long eaddr = getEffectiveAddress(phandler);
+    long sz = m_nvmasvc.retrieveSize(m_nid, eaddr);
+    if (sz > 0L) {
+      ret = new MemChunkHolder<NonVolatileMemAllocator>(this, eaddr, sz);
+      if (autoreclaim) {
+        m_chunkcollector.register(ret);
+      }
+    }
+    return ret;
+  }
+
+  /**
+   * get the handler from a memory buffer holder.
+   * 
+   * @param mbuf
+   *          specify the memory buffer holder
+   *
+   * @return a handler that could be used to retrieve its memory buffer
+   */
+  @Override
+  public long getBufferHandler(MemBufferHolder<NonVolatileMemAllocator> mbuf) {
+    return getPortableAddress(m_nvmasvc.getByteBufferHandler(m_nid, mbuf.get()));
+  }
+
+  /**
+   * get the handler from a memory chunk holder.
+   * 
+   * @param mchunk
+   *          specify the memory chunk holder
+   *
+   * @return a handler that could be used to retrieve its memory chunk
+   */
+  @Override
+  public long getChunkHandler(MemChunkHolder<NonVolatileMemAllocator> mchunk) {
+    return getPortableAddress(mchunk.get());
+  }
+
+  /**
+   * determine whether this allocator supports to store non-volatile handler or
+   * not. (it is a placeholder)
+   *
+   * @return true if there is
+   */
+  @Override
+  public boolean hasDurableHandlerStore() {
+    return true;
+  }
+
+  /**
+   * start a application level transaction on this allocator. (it is a place
+   * holder)
+   *
+   */
+  @Override
+  public void beginTransaction() {
+    throw new UnsupportedOperationException("Transaction Unsupported.");
+  }
+
+  /**
+   * end a application level transaction on this allocator. (it is a place
+   * holder)
+   *
+   */
+  @Override
+  public void endTransaction() {
+    throw new UnsupportedOperationException("Transaction Unsupported.");
+  }
+
+  /**
+   * set a handler on key.
+   * 
+   * @param key
+   *          the key to set its value
+   * 
+   * @param handler
+   *          the handler
+   */
+  public void setHandler(long key, long handler) {
+    m_nvmasvc.setHandler(m_nid, key, handler);
+  }
+
+  /**
+   * get a handler value.
+   * 
+   * @param key
+   *          the key to set its value
+   * 
+   * @return the value of handler
+   */
+  public long getHandler(long key) {
+    return m_nvmasvc.getHandler(m_nid, key);
+  }
+
+  /**
+   * return the capacity of non-volatile handler store.
+   * 
+   * @return the capacity of handler store
+   * 
+   */
+  public long handlerCapacity() {
+    return m_nvmasvc.handlerCapacity(m_nid);
+  }
+
+  /**
+   * calculate the portable address
+   *
+   * @param addr
+   *          the address to be calculated
+   *
+   * @return the portable address
+   */
+  @Override
+  public long getPortableAddress(long addr) {
+    return addr - b_addr;
+  }
+
+  /**
+   * calculate the effective address
+   *
+   * @param addr
+   *          the address to be calculated
+   *
+   * @return the effective address
+   */
+  @Override
+  public long getEffectiveAddress(long addr) {
+    return addr + b_addr;
+  }
+
+  /**
+   * get the base address
+   *
+   * @return the base address
+   */
+  @Override
+  public long getBaseAddress() {
+    return b_addr;
+  }
+
+  /**
+   * set the base address for calculation
+   *
+   * @param addr
+   *          the base address
+   *
+   */
+  @Override
+  public long setBaseAddress(long addr) {
+    return b_addr = addr;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-mnemonic/blob/679bcdc1/core/src/main/java/org/apache/mnemonic/NonVolatileSetter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/mnemonic/NonVolatileSetter.java b/core/src/main/java/org/apache/mnemonic/NonVolatileSetter.java
deleted file mode 100644
index 31cf4d1..0000000
--- a/core/src/main/java/org/apache/mnemonic/NonVolatileSetter.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.mnemonic;
-
-/**
- * this class defines an annotation for setter methods of non-volatile entity
- *
- */
-
-import java.lang.annotation.ElementType;
-import java.lang.annotation.Retention;
-import java.lang.annotation.RetentionPolicy;
-import java.lang.annotation.Target;
-
-@Target(ElementType.METHOD)
-@Retention(RetentionPolicy.CLASS)
-public @interface NonVolatileSetter {
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-mnemonic/blob/679bcdc1/core/src/main/java/org/apache/mnemonic/OutOfHybridMemory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/mnemonic/OutOfHybridMemory.java b/core/src/main/java/org/apache/mnemonic/OutOfHybridMemory.java
new file mode 100644
index 0000000..d9c99ca
--- /dev/null
+++ b/core/src/main/java/org/apache/mnemonic/OutOfHybridMemory.java
@@ -0,0 +1,32 @@
+/*
+ * 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;
+
+/**
+ * this is an exception that should be throw once out of persistent memory
+ *
+ */
+
+public class OutOfHybridMemory extends RuntimeException {
+
+  private static final long serialVersionUID = -6315943783592441148L;
+
+  public OutOfHybridMemory(String s) {
+    super(s);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-mnemonic/blob/679bcdc1/core/src/main/java/org/apache/mnemonic/OutOfPersistentMemory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/mnemonic/OutOfPersistentMemory.java b/core/src/main/java/org/apache/mnemonic/OutOfPersistentMemory.java
deleted file mode 100644
index 390790a..0000000
--- a/core/src/main/java/org/apache/mnemonic/OutOfPersistentMemory.java
+++ /dev/null
@@ -1,32 +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 org.apache.mnemonic;
-
-/**
- * this is an exception that should be throw once out of persistent memory
- *
- */
-
-public class OutOfPersistentMemory extends RuntimeException {
-
-  private static final long serialVersionUID = -6315943783592441148L;
-
-  public OutOfPersistentMemory(String s) {
-    super(s);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-mnemonic/blob/679bcdc1/core/src/main/java/org/apache/mnemonic/PMAddressTranslator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/mnemonic/PMAddressTranslator.java b/core/src/main/java/org/apache/mnemonic/PMAddressTranslator.java
deleted file mode 100644
index c5c9105..0000000
--- a/core/src/main/java/org/apache/mnemonic/PMAddressTranslator.java
+++ /dev/null
@@ -1,61 +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 org.apache.mnemonic;
-
-/**
- * translate persistent memory address for allocator
- *
- */
-public interface PMAddressTranslator {
-
-  /**
-   * calculate the portable address
-   *
-   * @param addr
-   *          the address to be calculated
-   *
-   * @return the portable address
-   */
-  long getPortableAddress(long addr);
-
-  /**
-   * calculate the effective address
-   *
-   * @param addr
-   *          the address to be calculated
-   *
-   * @return the effective address
-   */
-  long getEffectiveAddress(long addr);
-
-  /**
-   * get the base address
-   *
-   * @return the base address
-   */
-  long getBaseAddress();
-
-  /**
-   * set the base address for calculation
-   *
-   * @param addr
-   *          the base address
-   *
-   */
-  long setBaseAddress(long addr);
-}

http://git-wip-us.apache.org/repos/asf/incubator-mnemonic/blob/679bcdc1/core/src/main/java/org/apache/mnemonic/RetrieveDurableEntityError.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/mnemonic/RetrieveDurableEntityError.java b/core/src/main/java/org/apache/mnemonic/RetrieveDurableEntityError.java
new file mode 100644
index 0000000..4ff5a3e
--- /dev/null
+++ b/core/src/main/java/org/apache/mnemonic/RetrieveDurableEntityError.java
@@ -0,0 +1,35 @@
+/*
+ * 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;
+
+/**
+ *
+ *
+ */
+
+public class RetrieveDurableEntityError extends RuntimeException {
+
+  /**
+   * 
+   */
+  private static final long serialVersionUID = 3985459401986971131L;
+
+  public RetrieveDurableEntityError(String s) {
+    super(s);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-mnemonic/blob/679bcdc1/core/src/main/java/org/apache/mnemonic/RetrieveNonVolatileEntityError.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/mnemonic/RetrieveNonVolatileEntityError.java b/core/src/main/java/org/apache/mnemonic/RetrieveNonVolatileEntityError.java
deleted file mode 100644
index b38af5e..0000000
--- a/core/src/main/java/org/apache/mnemonic/RetrieveNonVolatileEntityError.java
+++ /dev/null
@@ -1,35 +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 org.apache.mnemonic;
-
-/**
- *
- *
- */
-
-public class RetrieveNonVolatileEntityError extends RuntimeException {
-
-  /**
-   * 
-   */
-  private static final long serialVersionUID = 3985459401986971131L;
-
-  public RetrieveNonVolatileEntityError(String s) {
-    super(s);
-  }
-}



[5/5] incubator-mnemonic git commit: fix the name of VolatileAllocatorNGTest, changed to VolatileMemAllocatorNGTest

Posted by ga...@apache.org.
fix the name of VolatileAllocatorNGTest, changed to VolatileMemAllocatorNGTest


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

Branch: refs/heads/master
Commit: 1e75e2a52cfde1ccfa6e0b062da20373be204ad2
Parents: 679bcdc
Author: Wang, Gang(Gary) <ga...@intel.com>
Authored: Wed Apr 13 18:36:34 2016 -0700
Committer: Wang, Gang(Gary) <ga...@intel.com>
Committed: Wed Apr 13 18:43:32 2016 -0700

----------------------------------------------------------------------
 README.md                                       |  4 +-
 .../mnemonic/VolatileAllocatorNGTest.java       | 68 --------------------
 .../mnemonic/VolatileMemAllocatorNGTest.java    | 68 ++++++++++++++++++++
 core/src/test/resources/testng.xml              |  2 +-
 4 files changed, 72 insertions(+), 70 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-mnemonic/blob/1e75e2a5/README.md
----------------------------------------------------------------------
diff --git a/README.md b/README.md
index 3feacf0..578b5f5 100644
--- a/README.md
+++ b/README.md
@@ -222,7 +222,9 @@ To run several test cases:
   
   $ mvn -Dtest=DurablePersonNGTest test -pl core -DskipTests=false # a testcase for module "core" that requires 'pmalloc' allocator service to pass
   
-  $ mvn -Dtest=NonVolatileMemAllocatorNGTest test -pl core -DskipTests=false # the second testcase for module "core" that requires 'vmem' allocator service to pass
+  $ mvn -Dtest=NonVolatileMemAllocatorNGTest test -pl core -DskipTests=false # the second testcase for module "core" that requires 'pmalloc' allocator service to pass
+  
+  $ mvn -Dtest=VolatileMemAllocatorNGTest test -pl core -DskipTests=false # the second testcase for module "core" that requires 'vmem' allocator service to pass
   
   $ mvn -Dtest=MemClusteringNGTest test -pl core -DskipTests=false # the third testcase for module "core" that requires 'vmem allocator service to pass
   

http://git-wip-us.apache.org/repos/asf/incubator-mnemonic/blob/1e75e2a5/core/src/test/java/org/apache/mnemonic/VolatileAllocatorNGTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/mnemonic/VolatileAllocatorNGTest.java b/core/src/test/java/org/apache/mnemonic/VolatileAllocatorNGTest.java
deleted file mode 100644
index a0d7a5b..0000000
--- a/core/src/test/java/org/apache/mnemonic/VolatileAllocatorNGTest.java
+++ /dev/null
@@ -1,68 +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 org.apache.mnemonic;
-
-import org.testng.annotations.Test;
-
-import java.util.Random;
-
-/**
- * test the functionality of VolatileMemAllocator class.
- * 
- */
-public class VolatileAllocatorNGTest {
-  /**
-   * test to allocate MemBufferHolder objects and then verify them.
-   */
-  @Test
-  public void testMemByteBuffer() {
-    Random randomGenerator = new Random();
-    Allocator<VolatileMemAllocator> act = new VolatileMemAllocator(Utils.getVolatileMemoryAllocatorService("vmem"),
-        1024 * 1024 * 1024, ".", true);
-    MemBufferHolder<?> mbh;
-    for (int idx = 1; idx <= 5; ++idx) {
-      int size = randomGenerator.nextInt(1024 * 1024) + 1024 * 1024;
-      mbh = act.createBuffer(size);
-      for (int i = 0; i < size; i++) {
-        mbh.get().put((byte) randomGenerator.nextInt(255));
-      }
-      // if (bb.hasArray()) randomGenerator.nextBytes(bb.array());
-      System.out.println(String.format("[Seq.%d] size %d - %d, (%s)", idx, size, mbh.get().capacity(),
-          size == mbh.get().capacity() ? "Correct" : "Failed!!!"));
-      // mbh.destroy();
-    }
-  }
-
-  /**
-   * test to allocate MemChunkHolder objects and then verify them.
-   */
-  @Test
-  public void testMemChunk() {
-    Random randomGenerator = new Random();
-    Allocator<VolatileMemAllocator> act = new VolatileMemAllocator(Utils.getVolatileMemoryAllocatorService("vmem"),
-        1024 * 1024 * 1024, ".", true);
-    MemChunkHolder<?> mch;
-    for (int idx = 1; idx <= 5; ++idx) {
-      int size = randomGenerator.nextInt(1024 * 1024) + 1024 * 1024;
-      mch = act.createChunk(size);
-      System.out.println(String.format("[Seq.%d] addr : %X", idx, size, mch.get()));
-      mch.destroy();
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-mnemonic/blob/1e75e2a5/core/src/test/java/org/apache/mnemonic/VolatileMemAllocatorNGTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/mnemonic/VolatileMemAllocatorNGTest.java b/core/src/test/java/org/apache/mnemonic/VolatileMemAllocatorNGTest.java
new file mode 100644
index 0000000..6c96be7
--- /dev/null
+++ b/core/src/test/java/org/apache/mnemonic/VolatileMemAllocatorNGTest.java
@@ -0,0 +1,68 @@
+/*
+ * 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;
+
+import org.testng.annotations.Test;
+
+import java.util.Random;
+
+/**
+ * test the functionality of VolatileMemAllocator class.
+ * 
+ */
+public class VolatileMemAllocatorNGTest {
+  /**
+   * test to allocate MemBufferHolder objects and then verify them.
+   */
+  @Test
+  public void testMemByteBuffer() {
+    Random randomGenerator = new Random();
+    Allocator<VolatileMemAllocator> act = new VolatileMemAllocator(Utils.getVolatileMemoryAllocatorService("vmem"),
+        1024 * 1024 * 1024, ".", true);
+    MemBufferHolder<?> mbh;
+    for (int idx = 1; idx <= 5; ++idx) {
+      int size = randomGenerator.nextInt(1024 * 1024) + 1024 * 1024;
+      mbh = act.createBuffer(size);
+      for (int i = 0; i < size; i++) {
+        mbh.get().put((byte) randomGenerator.nextInt(255));
+      }
+      // if (bb.hasArray()) randomGenerator.nextBytes(bb.array());
+      System.out.println(String.format("[Seq.%d] size %d - %d, (%s)", idx, size, mbh.get().capacity(),
+          size == mbh.get().capacity() ? "Correct" : "Failed!!!"));
+      // mbh.destroy();
+    }
+  }
+
+  /**
+   * test to allocate MemChunkHolder objects and then verify them.
+   */
+  @Test
+  public void testMemChunk() {
+    Random randomGenerator = new Random();
+    Allocator<VolatileMemAllocator> act = new VolatileMemAllocator(Utils.getVolatileMemoryAllocatorService("vmem"),
+        1024 * 1024 * 1024, ".", true);
+    MemChunkHolder<?> mch;
+    for (int idx = 1; idx <= 5; ++idx) {
+      int size = randomGenerator.nextInt(1024 * 1024) + 1024 * 1024;
+      mch = act.createChunk(size);
+      System.out.println(String.format("[Seq.%d] addr : %X", idx, size, mch.get()));
+      mch.destroy();
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-mnemonic/blob/1e75e2a5/core/src/test/resources/testng.xml
----------------------------------------------------------------------
diff --git a/core/src/test/resources/testng.xml b/core/src/test/resources/testng.xml
index 495811b..595f68d 100644
--- a/core/src/test/resources/testng.xml
+++ b/core/src/test/resources/testng.xml
@@ -13,7 +13,7 @@
 			<class name="org.apache.mnemonic.NonVolatileMemAllocatorNGTest" />
 			<class name="org.apache.mnemonic.ByteBufferSerializerNGTest" />
 			<class name="org.apache.mnemonic.MemClusteringNGTest" />
-			<class name="org.apache.mnemonic.NonVolatileMemAllocatorNGTest" />
+			<class name="org.apache.mnemonic.VolatileMemAllocatorNGTest" />
 			<class name="org.apache.mnemonic.DurablePersonNGTest" />
 			<class name="org.apache.mnemonic.DurableListNGTest" />