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/05/06 00:41:01 UTC

[03/13] incubator-mnemonic git commit: MNEMONIC-40 - Clean up assembled source package for release MNEMONIC-41 - Create a script for preparing a release of a single package with signing and hashing MNEMONIC-42 - Normalize the names of project submodules

http://git-wip-us.apache.org/repos/asf/incubator-mnemonic/blob/30836536/mnemonic-core/src/main/java/org/apache/mnemonic/NonVolatileMemAllocator.java
----------------------------------------------------------------------
diff --git a/mnemonic-core/src/main/java/org/apache/mnemonic/NonVolatileMemAllocator.java b/mnemonic-core/src/main/java/org/apache/mnemonic/NonVolatileMemAllocator.java
new file mode 100644
index 0000000..6c07c31
--- /dev/null
+++ b/mnemonic-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/30836536/mnemonic-core/src/main/java/org/apache/mnemonic/OutOfHybridMemory.java
----------------------------------------------------------------------
diff --git a/mnemonic-core/src/main/java/org/apache/mnemonic/OutOfHybridMemory.java b/mnemonic-core/src/main/java/org/apache/mnemonic/OutOfHybridMemory.java
new file mode 100644
index 0000000..d9c99ca
--- /dev/null
+++ b/mnemonic-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/30836536/mnemonic-core/src/main/java/org/apache/mnemonic/Reclaim.java
----------------------------------------------------------------------
diff --git a/mnemonic-core/src/main/java/org/apache/mnemonic/Reclaim.java b/mnemonic-core/src/main/java/org/apache/mnemonic/Reclaim.java
new file mode 100644
index 0000000..84b85ac
--- /dev/null
+++ b/mnemonic-core/src/main/java/org/apache/mnemonic/Reclaim.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.mnemonic;
+
+/**
+ * an interface to reclaim its memory resource.
+ * 
+ */
+public interface Reclaim<MRES> {
+
+  /**
+   * reclaim specified resources.
+   * 
+   * @param mres
+   *          a resource to be reclaimed
+   * 
+   * @param size
+   *          the size of resource, it will be set as null if unknown
+   * 
+   * @return <tt>true</tt> if this resource has been reclaimed by this callback.
+   *         <tt>false</tt> if this resource needs to be claimed by default
+   *         reclaimer
+   */
+  boolean reclaim(MRES mres, Long size);
+}

http://git-wip-us.apache.org/repos/asf/incubator-mnemonic/blob/30836536/mnemonic-core/src/main/java/org/apache/mnemonic/RetrieveDurableEntityError.java
----------------------------------------------------------------------
diff --git a/mnemonic-core/src/main/java/org/apache/mnemonic/RetrieveDurableEntityError.java b/mnemonic-core/src/main/java/org/apache/mnemonic/RetrieveDurableEntityError.java
new file mode 100644
index 0000000..4ff5a3e
--- /dev/null
+++ b/mnemonic-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/30836536/mnemonic-core/src/main/java/org/apache/mnemonic/SysMemAllocator.java
----------------------------------------------------------------------
diff --git a/mnemonic-core/src/main/java/org/apache/mnemonic/SysMemAllocator.java b/mnemonic-core/src/main/java/org/apache/mnemonic/SysMemAllocator.java
new file mode 100644
index 0000000..1d68937
--- /dev/null
+++ b/mnemonic-core/src/main/java/org/apache/mnemonic/SysMemAllocator.java
@@ -0,0 +1,303 @@
+/*
+ * 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.reflect.Field;
+import java.nio.ByteBuffer;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.flowcomputing.commons.resgc.ResCollector;
+import org.flowcomputing.commons.resgc.ResReclaim;
+
+import sun.misc.Unsafe;
+import sun.misc.Cleaner;
+
+/**
+ * manage a system memory pool as a internal volatile allocator
+ * 
+ */
+@SuppressWarnings("restriction")
+public class SysMemAllocator extends CommonAllocator<SysMemAllocator> {
+
+  private boolean m_activegc = true;
+  private long m_gctimeout = 100;
+  private Unsafe m_unsafe = null;
+  private AtomicLong currentMemory = new AtomicLong(0L);
+  private long maxStoreCapacity = 0L;
+  private Map<Long, Long> m_chunksize = new ConcurrentHashMap<Long, Long>();
+
+  /**
+   * Constructor, it initialize and allocate a memory pool from Java off-heap
+   * with specified capacity.
+   * 
+   * @param capacity
+   *          specify the capacity of a system memory pool
+   * 
+   * @param isnew
+   *          a place holder, always specify it as true
+   * 
+   * @throws Exception
+   *           fail to retrieve Unsafe object
+   * 
+   */
+  public SysMemAllocator(long capacity, boolean isnew) throws Exception {
+    m_unsafe = Utils.getUnsafe();
+    maxStoreCapacity = capacity;
+    /**
+     * create a resource collector to release specified bytebuffer that backed
+     * by Java off-heap.
+     */
+    m_bufcollector = new ResCollector<MemBufferHolder<SysMemAllocator>, ByteBuffer>(new ResReclaim<ByteBuffer>() {
+      @Override
+      public synchronized void reclaim(ByteBuffer mres) {
+        Long sz = Long.valueOf(mres.capacity());
+        boolean cb_reclaimed = false;
+        if (null != m_bufferreclaimer) {
+          cb_reclaimed = m_bufferreclaimer.reclaim(mres, sz);
+        }
+        if (!cb_reclaimed) {
+          try {
+            Field cleanerField;
+            cleanerField = mres.getClass().getDeclaredField("cleaner");
+            cleanerField.setAccessible(true);
+            Cleaner cleaner = (Cleaner) cleanerField.get(mres);
+            cleaner.clean();
+          } catch (NoSuchFieldException | SecurityException | IllegalArgumentException | IllegalAccessException e) {
+            e.printStackTrace();
+          }
+          mres = null;
+        }
+        currentMemory.addAndGet(-sz);
+      }
+    });
+
+    /**
+     * create a resource collector to release specified chunk that backed by
+     * Java off-heap.
+     */
+    m_chunkcollector = new ResCollector<MemChunkHolder<SysMemAllocator>, Long>(new ResReclaim<Long>() {
+      @Override
+      public synchronized void reclaim(Long mres) {
+        // System.out.println(String.format("Reclaim: %X ...", mres));
+        Long sz = m_chunksize.remove(mres);
+        boolean cb_reclaimed = false;
+        if (null != m_chunkreclaimer) {
+          cb_reclaimed = m_chunkreclaimer.reclaim(mres, sz);
+        }
+        if (!cb_reclaimed) {
+          m_unsafe.freeMemory(mres);
+          mres = null;
+        }
+        if (null != sz) {
+          currentMemory.addAndGet(-sz);
+        }
+      }
+    });
+  }
+
+  /**
+   * 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 SysMemAllocator enableActiveGC(long timeout) {
+    m_activegc = true;
+    m_gctimeout = timeout;
+    return this;
+  }
+
+  /**
+   * disable active garbage collection.
+   *
+   * @return this allocator
+   */
+  @Override
+  public SysMemAllocator 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<SysMemAllocator> resizeChunk(MemChunkHolder<SysMemAllocator> mholder, long size) {
+    MemChunkHolder<SysMemAllocator> ret = null;
+    boolean ac = null != mholder.getRefId();
+    if (size > 0) {
+      if (currentMemory.get() + size > maxStoreCapacity) {
+        if (m_activegc) {
+          m_chunkcollector.waitReclaimCoolDown(m_gctimeout);
+        }
+      }
+      if (currentMemory.get() + size <= maxStoreCapacity) {
+        Long addr = m_unsafe.reallocateMemory(mholder.get(), size);
+        if (0 != addr) {
+          mholder.clear();
+          mholder.destroy();
+          ret = new MemChunkHolder<SysMemAllocator>(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<SysMemAllocator> resizeBuffer(MemBufferHolder<SysMemAllocator> mholder, long size) {
+    MemBufferHolder<SysMemAllocator> ret = null;
+    boolean ac = null != mholder.getRefId();
+    if (size > 0) {
+      int bufpos = mholder.get().position();
+      int buflimit = mholder.get().limit();
+      if (currentMemory.get() + size > maxStoreCapacity) {
+        if (m_activegc) {
+          m_bufcollector.waitReclaimCoolDown(m_gctimeout);
+        }
+      }
+      if (currentMemory.get() + size <= maxStoreCapacity) {
+        ByteBuffer buf = Utils.resizeByteBuffer(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<SysMemAllocator>(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<SysMemAllocator> createChunk(long size, boolean autoreclaim) {
+    MemChunkHolder<SysMemAllocator> ret = null;
+    Long addr = null;
+    if (currentMemory.get() + size > maxStoreCapacity) {
+      if (m_activegc) {
+        m_chunkcollector.waitReclaimCoolDown(m_gctimeout);
+      }
+    }
+    if (currentMemory.get() + size <= maxStoreCapacity) {
+      addr = m_unsafe.allocateMemory(size);
+    }
+    if (null != addr && 0 != addr) {
+      ret = new MemChunkHolder<SysMemAllocator>(this, addr, size);
+      ret.setCollector(m_chunkcollector);
+      if (autoreclaim) {
+        m_chunkcollector.register(ret);
+      }
+      m_chunksize.put(addr, size);
+      currentMemory.getAndAdd(size);
+    }
+    return ret;
+  }
+
+  /**
+   * create a memory buffer that is managed by its holder.
+   * 
+   * @param size
+   *          specify the size of memory buffer
+   * 
+   * @return a holder contains a memory buffer
+   */
+  @Override
+  public MemBufferHolder<SysMemAllocator> createBuffer(long size, boolean autoreclaim) {
+    MemBufferHolder<SysMemAllocator> ret = null;
+    ByteBuffer bb = null;
+    if (currentMemory.get() + size > maxStoreCapacity) {
+      if (m_activegc) {
+        m_bufcollector.waitReclaimCoolDown(m_gctimeout);
+      }
+    }
+    if (currentMemory.get() + size <= maxStoreCapacity) {
+      bb = ByteBuffer.allocateDirect((int) size);
+    }
+    if (null != bb) {
+      ret = new MemBufferHolder<SysMemAllocator>(this, bb);
+      ret.setCollector(m_bufcollector);
+      if (autoreclaim) {
+        m_bufcollector.register(ret);
+      }
+      currentMemory.getAndAdd(size);
+    }
+    return ret;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-mnemonic/blob/30836536/mnemonic-core/src/main/java/org/apache/mnemonic/Utils.java
----------------------------------------------------------------------
diff --git a/mnemonic-core/src/main/java/org/apache/mnemonic/Utils.java b/mnemonic-core/src/main/java/org/apache/mnemonic/Utils.java
new file mode 100644
index 0000000..49794db
--- /dev/null
+++ b/mnemonic-core/src/main/java/org/apache/mnemonic/Utils.java
@@ -0,0 +1,399 @@
+/*
+ * 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.reflect.Field;
+import java.nio.ByteBuffer;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Date;
+import java.util.Random;
+import java.util.UUID;
+
+import sun.misc.Unsafe;
+import java.util.Iterator;
+import java.util.List;
+import java.util.ServiceLoader;
+import org.apache.mnemonic.service.allocatorservice.VolatileMemoryAllocatorService;
+import org.apache.mnemonic.service.allocatorservice.NonVolatileMemoryAllocatorService;
+
+/**
+ * <p>
+ * Utilities for project.
+ * </p>
+ * 
+ */
+@SuppressWarnings("restriction")
+public class Utils {
+  public static final String ANSI_RESET = "\u001B[0m";
+  public static final String ANSI_BLACK = "\u001B[30m";
+  public static final String ANSI_RED = "\u001B[31m";
+  public static final String ANSI_GREEN = "\u001B[32m";
+  public static final String ANSI_YELLOW = "\u001B[33m";
+  public static final String ANSI_BLUE = "\u001B[34m";
+  public static final String ANSI_PURPLE = "\u001B[35m";
+  public static final String ANSI_CYAN = "\u001B[36m";
+  public static final String ANSI_WHITE = "\u001B[37m";
+
+  private static Unsafe m_unsafe = null;
+
+  private static ServiceLoader<VolatileMemoryAllocatorService> m_vmasvcloader = null;
+  private static ServiceLoader<NonVolatileMemoryAllocatorService> m_nvmasvcloader = null;
+
+  /**
+   * retrieve a volatile memory allocator service
+   * 
+   * @param id
+   *          specify a name of allocator to retrieve
+   *
+   * @return the volatile memory allocator service instance
+   */
+  public static VolatileMemoryAllocatorService getVolatileMemoryAllocatorService(String id) {
+    return getVolatileMemoryAllocatorService(id, true);
+  }
+
+  /**
+   * retrieve a volatile memory allocator service
+   * 
+   * @param id
+   *          specify a name of allocator to retrieve
+   *
+   * @param allownvmsvc
+   *          specify whether allow to treat non-volatile memory allocator as
+   *          volatile one during searching
+   *
+   * @return the volatile memory allocator service instance
+   */
+  public static VolatileMemoryAllocatorService getVolatileMemoryAllocatorService(String id, boolean allownvmsvc) {
+    VolatileMemoryAllocatorService ret = null;
+    if (null == m_vmasvcloader) {
+      m_vmasvcloader = ServiceLoader.load(VolatileMemoryAllocatorService.class);
+    }
+    Iterator<VolatileMemoryAllocatorService> svcit = m_vmasvcloader.iterator();
+    VolatileMemoryAllocatorService svc = null;
+    while (null == ret && svcit.hasNext()) {
+      svc = svcit.next();
+      if (svc.getServiceId().equals(id)) {
+        ret = svc;
+      }
+    }
+    if (null == ret && allownvmsvc) {
+      ret = getNonVolatileMemoryAllocatorService(id);
+    }
+    assert null != ret : "VolatileMemoryAllocatorService \'" + id + "\' not found!";
+    return ret;
+  }
+
+  /**
+   * retrieve a non-volatile memory allocator service
+   * 
+   * @param id
+   *          specify a name of allocator to retrieve
+   *
+   * @return the non-volatile memory allocator service instance
+   */
+  public static NonVolatileMemoryAllocatorService getNonVolatileMemoryAllocatorService(String id) {
+    NonVolatileMemoryAllocatorService ret = null;
+    if (null == m_nvmasvcloader) {
+      m_nvmasvcloader = ServiceLoader.load(NonVolatileMemoryAllocatorService.class);
+    }
+    Iterator<NonVolatileMemoryAllocatorService> svcit = m_nvmasvcloader.iterator();
+    NonVolatileMemoryAllocatorService svc = null;
+    while (null == ret && svcit.hasNext()) {
+      svc = svcit.next();
+      if (svc.getServiceId().equals(id)) {
+        ret = svc;
+      }
+    }
+    assert null != ret : "NonVolatileMemoryAllocatorService \'" + id + "\' not found!";
+    return ret;
+  }
+
+  /**
+   * Generates a unique name that contains current timestamp.
+   * 
+   * @param format
+   *          the template that is used to generate unique name.
+   *
+   * @return unique path name.
+   */
+  public static String genUniquePathname(String format) {
+    String ret = null;
+    if (null != format && !format.isEmpty()) {
+      ret = String.format(format, (new SimpleDateFormat("ddMMyy-hhmmss.SSS").format(new Date())));
+    }
+    return ret;
+  }
+
+  /**
+   * retrieve the usage of memory.
+   * 
+   * @return the size of memory has been occupied
+   */
+  public static long getMemoryUse(long timeout) {
+    putOutTheGarbage(timeout);
+    long totalMemory = Runtime.getRuntime().totalMemory();
+    putOutTheGarbage(timeout);
+    long freeMemory = Runtime.getRuntime().freeMemory();
+    return (totalMemory - freeMemory);
+  }
+
+  /**
+   * run garbage collections.
+   */
+  private static void putOutTheGarbage(long timeout) {
+    collectGarbage(timeout);
+    collectGarbage(timeout);
+  }
+
+  /**
+   * run a garbage collection.
+   */
+  public static void collectGarbage(long timeout) {
+    try {
+      System.gc();
+      Thread.sleep(timeout);
+      System.runFinalization();
+      Thread.sleep(timeout);
+    } catch (InterruptedException ex) {
+      ex.printStackTrace();
+    }
+  }
+
+  /**
+   * Retrieve an Unsafe object.
+   *
+   * @throws Exception
+   *           Error when get Unsafe object from runtime
+   *
+   * @return an unsafe object
+   */
+  public static Unsafe getUnsafe() throws Exception {
+    if (null == m_unsafe) {
+      Field field = sun.misc.Unsafe.class.getDeclaredField("theUnsafe");
+      field.setAccessible(true);
+      m_unsafe = (sun.misc.Unsafe) field.get(null);
+    }
+    return m_unsafe;
+  }
+
+  /**
+   * resize a bytebuffer with a new instance
+   *
+   * @param buf
+   *          specify a buf to resize
+   *
+   * @param size
+   *          specify the size for resizing
+   *
+   * @return the resized bytebuffer instance
+   */
+  public static ByteBuffer resizeByteBuffer(ByteBuffer buf, long size) {
+    ByteBuffer ret = ByteBuffer.allocateDirect((int) size);
+    if (ret != null) {
+      if (null != buf) {
+        ret.put(buf);
+        ret.flip();
+      }
+    }
+    return ret;
+  }
+
+  /**
+   * create a new instance of Random using default fixed seed
+   *
+   * @return the instance of Random
+   */
+  public static Random createRandom() {
+    return createRandom(0L);
+  }
+
+  /**
+   * create a new instance of Random
+   *
+   * @param rgenseed
+   *          specify a random seed
+   *
+   * @return the instance of Random
+   */
+  public static Random createRandom(long rgenseed) {
+    Random ret = new Random();
+    if (0L == rgenseed) {
+      rgenseed = System.currentTimeMillis();
+      System.out.println("Random number generator seed is " + rgenseed);
+    } else {
+      System.out.println("Fixed Random number generator seed is " + rgenseed);
+    }
+    ret.setSeed(rgenseed);
+    return ret;
+  }
+
+  /**
+   * generate a random string with fixed length
+   *
+   * @return the random string
+   */
+  public static String genRandomString() {
+    return genRandomString(6);
+  }
+
+  /**
+   * generate a random string
+   *
+   * @param len
+   *          specify the length of this random string
+   *
+   * @return the random string
+   */
+  public static String genRandomString(int len) {
+    return UUID.randomUUID().toString().replaceAll("-", "").toUpperCase().substring(0, len);
+  }
+
+  /**
+   * assert the equality of two generic objects using compareTo() operator
+   *
+   * @param <T>
+   *          the type of comparable objects
+   *
+   * @param actual
+   *          specify a object to be compared
+   *
+   * @param expected
+   *          specify a object to be expected
+   *
+   * @return true if equal according to compareTo()
+   */
+  public static <T extends Comparable<T>> boolean assertComparison(T actual, T expected) {
+    boolean ret = false;
+    if ((expected == null) && (actual == null)) {
+      ret = true;
+    } else if (expected != null) {
+      ret = expected.compareTo(actual) == 0;
+    }
+    return ret;
+  }
+
+  /**
+   * convert a long array to a initializer literal string.
+   *
+   * @param larr
+   *          specify a long array to be converted
+   *
+   * @return a literal string represent the initializer
+   */
+  public static String toInitLiteral(long[] larr) {
+    return Arrays.toString(larr).replaceAll("\\[", "{").replaceAll("\\]", "}");
+  }
+
+  /**
+   * convert a list of long array to a initializer literal string.
+   *
+   * @param llarr
+   *          specify a list of long array to be converted
+   *
+   * @return a literal string represent the initializer
+   */
+  public static String toInitLiteral(List<long[]> llarr) {
+    List<String> slist = new ArrayList<String>();
+    for (long[] larr : llarr) {
+      slist.add(toInitLiteral(larr));
+    }
+    return "{" + String.join(",", slist) + "}";
+  }
+
+  /**
+   * retrieve a set of native field info from a list of object field info
+   * according to the field id info. it forms a value info stack for native code
+   * to use as one standardized parameter
+   *
+   * @param objstack
+   *          a stack of object info retrieved from
+   *          Durable.getNativeFieldInfo(), order matters
+   *
+   * @param fidinfostack
+   *          a stack of field id in the form of (next_fid, next_level_fid)
+   *          order follows objstack the last next_level_fid specifies the
+   *          value's fid. the last item of next_fid could be null if there is
+   *          no next node if it is null that means the last item is a object
+   *          instead of node
+   *
+   * @return the stack of native field info
+   *
+   */
+  public static List<long[]> getNativeParamForm(List<long[][]> objstack, long[][] fidinfostack) {
+    List<long[]> ret = new ArrayList<long[]>();
+    if (null == objstack || null == fidinfostack || fidinfostack.length != objstack.size()) {
+      throw new IllegalArgumentException("Not the same depth");
+    }
+    for (int idx = 0; idx < fidinfostack.length; ++idx) {
+      ret.add(genNativeStackItem(objstack.get(idx), fidinfostack[idx], idx == fidinfostack.length - 1));
+    }
+    return ret;
+  }
+
+  /**
+   * generate an item of native stack.
+   *
+   * @param oinfo
+   *          a object field info
+   *
+   * @param fidinfo
+   *          a pair of field id info
+   *
+   * @param allowfidnull
+   *          allow the first field id is null
+   *
+   * @return the native item
+   */
+  public static long[] genNativeStackItem(long[][] oinfo, long[] fidinfo, boolean allowfidnull) {
+    long[] ret = new long[4];
+    long fid;
+    boolean found;
+    if (fidinfo.length != 2) {
+      throw new IllegalArgumentException("the length of field id array is not exactly 2");
+    }
+    for (int idx = 0; idx < fidinfo.length; ++idx) {
+      ret[idx * 2] = 0;
+      ret[idx * 2 + 1] = 0;
+      fid = fidinfo[idx];
+      if (fid <= 0) {
+        if (allowfidnull && 0 == idx) {
+          continue;
+        } else {
+          throw new IllegalArgumentException("the field id is not greater than 0");
+        }
+      }
+      found = false;
+      for (long[] finfo : oinfo) {
+        if (finfo.length != 3) {
+          throw new IllegalArgumentException("the length of field array is not exactly 3");
+        }
+        if (fid == finfo[0]) {
+          ret[idx * 2] = finfo[1];
+          ret[idx * 2 + 1] = finfo[2];
+          found = true;
+        }
+      }
+      if (!found) {
+        throw new IllegalArgumentException("field id not found");
+      }
+    }
+    return ret;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-mnemonic/blob/30836536/mnemonic-core/src/main/java/org/apache/mnemonic/VolatileMemAllocator.java
----------------------------------------------------------------------
diff --git a/mnemonic-core/src/main/java/org/apache/mnemonic/VolatileMemAllocator.java b/mnemonic-core/src/main/java/org/apache/mnemonic/VolatileMemAllocator.java
new file mode 100644
index 0000000..ef13a07
--- /dev/null
+++ b/mnemonic-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/30836536/mnemonic-core/src/main/java/org/apache/mnemonic/service/allocatorservice/NonVolatileMemoryAllocatorService.java
----------------------------------------------------------------------
diff --git a/mnemonic-core/src/main/java/org/apache/mnemonic/service/allocatorservice/NonVolatileMemoryAllocatorService.java b/mnemonic-core/src/main/java/org/apache/mnemonic/service/allocatorservice/NonVolatileMemoryAllocatorService.java
new file mode 100644
index 0000000..1eda1d9
--- /dev/null
+++ b/mnemonic-core/src/main/java/org/apache/mnemonic/service/allocatorservice/NonVolatileMemoryAllocatorService.java
@@ -0,0 +1,113 @@
+/*
+ * 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.service.allocatorservice;
+
+import java.nio.ByteBuffer;
+
+public interface NonVolatileMemoryAllocatorService extends VolatileMemoryAllocatorService {
+
+  /**
+   * retrieve a bytebuffer from its handler
+   *
+   * @param id
+   *          the identifier of backed memory pool
+   * 
+   * @param handler
+   *          the handler of a nonvolatile bytebuffer
+   *
+   * @return the nonvolatile bytebuffer
+   *
+   */
+  ByteBuffer retrieveByteBuffer(long id, long handler);
+
+  /**
+   * retrieve the size of a nonvolatile memory object
+   *
+   * @param id
+   *          the identifier of backed memory pool
+   * 
+   * @param handler
+   *          the handler of a nonvolatile object
+   *
+   * @return the size of nonvolatile object
+   *
+   */
+  long retrieveSize(long id, long handler);
+
+  /**
+   * get the handler of a nonvolatile bytebuffer
+   *
+   * @param id
+   *          the identifier of backed memory pool
+   * 
+   * @param buf
+   *          the nonvolatile bytebuffer
+   *
+   * @return the handler of this specified nonvolatile bytebuffer
+   *
+   */
+  long getByteBufferHandler(long id, ByteBuffer buf);
+
+  /**
+   * set a handler to a key.
+   * 
+   * @param id
+   *          the identifier of backed memory pool
+   * 
+   * @param key
+   *          the key to set this handler
+   * 
+   * @param handler
+   *          the handler
+   */
+  void setHandler(long id, long key, long handler);
+
+  /**
+   * get a handler from specified key.
+   * 
+   * @param id
+   *          the identifier of backed memory pool
+   * 
+   * @param key
+   *          the key to get its handler
+   * 
+   * @return the handler of the specified key
+   */
+  long getHandler(long id, long key);
+
+  /**
+   * return the number of available keys to use.
+   * 
+   * @param id
+   *          the identifier of backed memory pool
+   * 
+   * @return the number of keys
+   */
+  long handlerCapacity(long id);
+
+  /**
+   * return the base address of this persistent memory pool.
+   * 
+   * @param id
+   *          the identifier of backed memory pool
+   * 
+   * @return the base address of this pmem pool
+   */
+  long getBaseAddress(long id);
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-mnemonic/blob/30836536/mnemonic-core/src/main/java/org/apache/mnemonic/service/allocatorservice/VolatileMemoryAllocatorService.java
----------------------------------------------------------------------
diff --git a/mnemonic-core/src/main/java/org/apache/mnemonic/service/allocatorservice/VolatileMemoryAllocatorService.java b/mnemonic-core/src/main/java/org/apache/mnemonic/service/allocatorservice/VolatileMemoryAllocatorService.java
new file mode 100644
index 0000000..cef38e1
--- /dev/null
+++ b/mnemonic-core/src/main/java/org/apache/mnemonic/service/allocatorservice/VolatileMemoryAllocatorService.java
@@ -0,0 +1,143 @@
+/*
+ * 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.service.allocatorservice;
+
+import java.nio.ByteBuffer;
+
+public interface VolatileMemoryAllocatorService {
+
+  /**
+   * Provide the service identifier for this allocator
+   *
+   * @return the service identifer of this allocator
+   */
+  String getServiceId();
+
+  /**
+   * Initialize a memory pool through native interface backed by native library.
+   * 
+   * @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
+   *
+   * @return the identifier of created memory pool
+   */
+  long init(long capacity, String uri, boolean isnew);
+
+  /**
+   * close the memory pool through native interface.
+   * 
+   */
+  void close(long id);
+
+  /**
+   * force to synchronize uncommitted data to backed memory pool through native
+   * interface.
+   */
+  void sync(long id);
+
+  /**
+   * allocate specified size of memory block from backed memory pool.
+   * 
+   * @param id
+   *          the identifier of backed memory pool
+   * 
+   * @param size
+   *          specify size of memory block to be allocated
+   * 
+   * @return the address of allocated memory block from native memory pool
+   */
+  long allocate(long id, long size, boolean initzero);
+
+  /**
+   * reallocate a specified size of memory block from backed memory pool.
+   * 
+   * @param id
+   *          the identifier of backed memory pool
+   * 
+   * @param address
+   *          the address of previous allocated memory block. it can be null.
+   * 
+   * @param size
+   *          specify new size of memory block to be reallocated
+   * 
+   * @return the address of reallocated memory block from native memory pool
+   */
+  long reallocate(long id, long address, long size, boolean initzero);
+
+  /**
+   * free a memory block by specify its address into backed memory pool.
+   * 
+   * @param id
+   *          the identifier of backed memory pool
+   * 
+   * @param address
+   *          the address of allocated memory block.
+   */
+  void free(long id, long address);
+
+  /**
+   * create a ByteBuffer object which backed buffer is coming from backed native
+   * memory pool.
+   * 
+   * @param id
+   *          the identifier of backed memory pool
+   * 
+   * @param size
+   *          the size of backed buffer that is managed by created ByteBuffer
+   *          object.
+   * 
+   * @return a created ByteBuffer object with a backed native memory block
+   */
+  ByteBuffer createByteBuffer(long id, long size);
+
+  /**
+   * resize a ByteBuffer object which backed buffer is coming from backed native
+   * memory pool. NOTE: the ByteBuffer object will be renewed and lost metadata
+   * e.g. position, mark and etc.
+   * 
+   * @param id
+   *          the identifier of backed memory pool
+   * 
+   * @param bytebuf
+   *          the specified ByteBuffer object to be destroyed
+   * 
+   * @param size
+   *          the new size of backed buffer that is managed by created
+   *          ByteBuffer object.
+   * 
+   * @return a created ByteBuffer object with a backed native memory block
+   */
+  ByteBuffer resizeByteBuffer(long id, ByteBuffer bytebuf, long size);
+
+  /**
+   * destroy a native memory block backed ByteBuffer object.
+   * 
+   * @param id
+   *          the identifier of backed memory pool
+   * 
+   * @param bytebuf
+   *          the specified ByteBuffer object to be destroyed
+   */
+  void destroyByteBuffer(long id, ByteBuffer bytebuf);
+}

http://git-wip-us.apache.org/repos/asf/incubator-mnemonic/blob/30836536/mnemonic-core/src/main/resources/META-INFO/services/javax.annotation.processing.Processor
----------------------------------------------------------------------
diff --git a/mnemonic-core/src/main/resources/META-INFO/services/javax.annotation.processing.Processor b/mnemonic-core/src/main/resources/META-INFO/services/javax.annotation.processing.Processor
new file mode 100644
index 0000000..25ba444
--- /dev/null
+++ b/mnemonic-core/src/main/resources/META-INFO/services/javax.annotation.processing.Processor
@@ -0,0 +1 @@
+org.apache.mnemonic.DurableEntityProcessor

http://git-wip-us.apache.org/repos/asf/incubator-mnemonic/blob/30836536/mnemonic-core/src/test/java/org/apache/mnemonic/ByteBufferSerializerNGTest.java
----------------------------------------------------------------------
diff --git a/mnemonic-core/src/test/java/org/apache/mnemonic/ByteBufferSerializerNGTest.java b/mnemonic-core/src/test/java/org/apache/mnemonic/ByteBufferSerializerNGTest.java
new file mode 100644
index 0000000..1e652c0
--- /dev/null
+++ b/mnemonic-core/src/test/java/org/apache/mnemonic/ByteBufferSerializerNGTest.java
@@ -0,0 +1,69 @@
+/*
+ * 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 static org.testng.Assert.assertTrue;
+import org.testng.annotations.Test;
+
+import java.nio.ByteBuffer;
+import java.util.Random;
+import java.io.IOException;
+
+/**
+ * test the functionalities of ByteBufferSerializer class
+ * 
+ */
+public class ByteBufferSerializerNGTest {
+
+  /**
+   * test to convert any serializable object from/to ByteBuffer object that is
+   * backed by Java heap.
+   */
+  @Test
+  public void testToFromByteBuffer() throws IOException, ClassNotFoundException {
+    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());
+      ByteBuffer bb = ByteBufferSerializer.toByteBuffer(pl);
+      Payload rpl = ByteBufferSerializer.toObject(bb);
+      assertTrue(pl.compareTo(rpl) == 0);
+    }
+  }
+
+  /**
+   * test to convert any serializable object from/to MemBufferHolder object that
+   * is backed by native memory pool.
+   */
+  @Test
+  public void testToFromMemBufferHolder() throws IOException, ClassNotFoundException {
+    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<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/30836536/mnemonic-core/src/test/java/org/apache/mnemonic/DurablePersonNGTest.java
----------------------------------------------------------------------
diff --git a/mnemonic-core/src/test/java/org/apache/mnemonic/DurablePersonNGTest.java b/mnemonic-core/src/test/java/org/apache/mnemonic/DurablePersonNGTest.java
new file mode 100644
index 0000000..e66c66f
--- /dev/null
+++ b/mnemonic-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/30836536/mnemonic-core/src/test/java/org/apache/mnemonic/MemBufferHolderCachePoolNGTest.java
----------------------------------------------------------------------
diff --git a/mnemonic-core/src/test/java/org/apache/mnemonic/MemBufferHolderCachePoolNGTest.java b/mnemonic-core/src/test/java/org/apache/mnemonic/MemBufferHolderCachePoolNGTest.java
new file mode 100644
index 0000000..c79ec8a
--- /dev/null
+++ b/mnemonic-core/src/test/java/org/apache/mnemonic/MemBufferHolderCachePoolNGTest.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;
+
+import static org.testng.Assert.assertNotNull;
+import static org.testng.Assert.assertEquals;
+
+import org.testng.annotations.Test;
+import java.util.Random;
+
+/**
+ * test the functionalities of MemBufferHolderCachePool class
+ * 
+ */
+public class MemBufferHolderCachePoolNGTest {
+
+  /**
+   * test to aggressively allow any MemBufferHolder objects in pool to be able
+   * to drop at will that prevents this cache pool from overflowing.
+   */
+  @Test
+  public void testMemBufferHolderCachePoolWithoutOverflow() {
+
+    MemBufferHolderCachePool<Integer> mbhcpool = new MemBufferHolderCachePool<Integer>(1024 * 1024 * 10);
+    Random randomGenerator = new Random();
+    MemClustering.NodeConfig<?> ncs[] = new MemClustering.NodeConfig<?>[] {
+        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, ".",
+        // true).disableActiveGC(), MemClustering.PerformanceLevel.NORMAL),
+        // new MemClustering.NodeConfig(new BigMemAllocator(1024*1024*20, ".",
+        // true).disableActiveGC(), MemClustering.PerformanceLevel.SLOW),
+    };
+    MemClustering mclst = new MemClustering(ncs);
+    MemBufferHolder<?> mbh;
+
+    DropEvent<Integer, MemBufferHolder<?>> dropevt = new DropEvent<Integer, MemBufferHolder<?>>() {
+      @Override
+      public void drop(CachePool<Integer, MemBufferHolder<?>> pool, Integer k, MemBufferHolder<?> v) {
+        System.out.println(String.format("dropping idx: %d", k));
+        v.destroy();
+      }
+    };
+
+    EvictFilter<Integer, MemBufferHolder<?>> dfilter = new EvictFilter<Integer, MemBufferHolder<?>>() {
+      @Override
+      public boolean validate(CachePool<Integer, MemBufferHolder<?>> pool, Integer k, MemBufferHolder<?> v) {
+        System.out.println(String.format("validating idx: %d", k));
+        return true;
+      }
+    };
+
+    for (int idx = 1; idx <= 100; ++idx) {
+      int size = randomGenerator.nextInt(1024 * 1024) + 1024 * 1024;
+      mbh = mclst.createBuffer(size);
+      System.out
+          .println(String.format("\nallocating idx: %d - size: %d, FreeCap: %d", idx, size, mbhcpool.freeCapacity()));
+      assertNotNull(mbh);
+      assertNotNull(mbh.get());
+      for (int i = 0; i < size; i++) {
+        mbh.get().put((byte) randomGenerator.nextInt(255));
+      }
+
+      mbh.get().flip();
+
+      assertEquals(size, mbh.get().capacity());
+
+      mbhcpool.put(idx, mbh, dropevt, dfilter);
+
+    }
+  }
+
+  /**
+   * test to overflow a cache pool of MemBufferHolder objects that is caused by
+   * preventing any objected in pool from dropping.
+   */
+  @Test(expectedExceptions = ContainerOverflowException.class)
+  public void testMemBufferHolderCachePoolWithOverflow() {
+
+    MemBufferHolderCachePool<Integer> mbhcpool = new MemBufferHolderCachePool<Integer>(1024 * 1024 * 10);
+    Random randomGenerator = new Random();
+    MemClustering.NodeConfig<?> ncs[] = new MemClustering.NodeConfig<?>[] {
+        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, ".",
+        // true).disableActiveGC(), MemClustering.PerformanceLevel.NORMAL),
+        // new MemClustering.NodeConfig(new BigMemAllocator(1024*1024*20, ".",
+        // true).disableActiveGC(), MemClustering.PerformanceLevel.SLOW),
+    };
+    MemClustering mclst = new MemClustering(ncs);
+    MemBufferHolder<?> mbh;
+
+    DropEvent<Integer, MemBufferHolder<?>> dropevt = new DropEvent<Integer, MemBufferHolder<?>>() {
+      @Override
+      public void drop(CachePool<Integer, MemBufferHolder<?>> pool, Integer k, MemBufferHolder<?> v) {
+        System.out.println(String.format("dropping idx: %d", k));
+        v.destroy();
+      }
+    };
+
+    EvictFilter<Integer, MemBufferHolder<?>> dfilter = new EvictFilter<Integer, MemBufferHolder<?>>() {
+      @Override
+      public boolean validate(CachePool<Integer, MemBufferHolder<?>> pool, Integer k, MemBufferHolder<?> v) {
+        System.out.println(String.format("validating idx: %d", k));
+        return false;
+      }
+    };
+
+    for (int idx = 1; idx <= 100; ++idx) {
+      int size = randomGenerator.nextInt(1024 * 1024) + 1024 * 1024;
+      mbh = mclst.createBuffer(size);
+      System.out
+          .println(String.format("\nallocating idx: %d - size: %d, FreeCap: %d", idx, size, mbhcpool.freeCapacity()));
+      assertNotNull(mbh);
+      assertNotNull(mbh.get());
+      for (int i = 0; i < size; i++) {
+        mbh.get().put((byte) randomGenerator.nextInt(255));
+      }
+
+      mbh.get().flip();
+
+      assertEquals(size, mbh.get().capacity());
+
+      mbhcpool.put(idx, mbh, dropevt, dfilter);
+
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-mnemonic/blob/30836536/mnemonic-core/src/test/java/org/apache/mnemonic/MemClusteringNGTest.java
----------------------------------------------------------------------
diff --git a/mnemonic-core/src/test/java/org/apache/mnemonic/MemClusteringNGTest.java b/mnemonic-core/src/test/java/org/apache/mnemonic/MemClusteringNGTest.java
new file mode 100644
index 0000000..0c90f30
--- /dev/null
+++ b/mnemonic-core/src/test/java/org/apache/mnemonic/MemClusteringNGTest.java
@@ -0,0 +1,192 @@
+/*
+ * 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 static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertTrue;
+
+import org.testng.annotations.Test;
+import java.util.Random;
+
+/**
+ * test the functionalities of MemClustering class
+ */
+public class MemClusteringNGTest {
+
+  /**
+   * test to fill up memory pool without reclaim unused memory blocks that will
+   * cause additional operations of allocation will be failed.
+   * 
+   * @throws Exception
+   */
+  @Test(expectedExceptions = NullPointerException.class)
+  public void testMemByteBufferWithoutActiveGC() throws Exception {
+    Random randomGenerator = new Random();
+    MemClustering.NodeConfig<?> ncs[] = new MemClustering.NodeConfig<?>[] {
+        new MemClustering.NodeConfig<SysMemAllocator>(new SysMemAllocator(1024 * 1024 * 20, true).disableActiveGC(),
+            MemClustering.PerformanceLevel.FASTEST),
+        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<VolatileMemAllocator>(
+            new VolatileMemAllocator(Utils.getVolatileMemoryAllocatorService("vmem"), 1024 * 1024 * 20, ".", true)
+                .disableActiveGC(),
+            MemClustering.PerformanceLevel.SLOW), };
+    MemClustering mclst = new MemClustering(ncs);
+    MemBufferHolder<?> mbh;
+    for (int idx = 1; idx <= 200; ++idx) {
+      int size = randomGenerator.nextInt(1024 * 1024) + 1024 * 1024;
+      mbh = mclst.createBuffer(size);
+      for (int i = 0; i < size; i++) {
+        mbh.get().put((byte) randomGenerator.nextInt(255));
+      }
+      assertEquals(size, mbh.get().capacity());
+    }
+  }
+
+  /**
+   * test to try to fill up memory pool and reclaim unused memory block as
+   * required.
+   */
+  @Test
+  public void testMemByteBufferWithActiveGC() {
+    Random randomGenerator = new Random();
+    MemClustering.NodeConfig<?> ncs[] = new MemClustering.NodeConfig<?>[] {
+        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),
+    };
+    MemClustering mclst = new MemClustering(ncs);
+    MemBufferHolder<?> mbh;
+    for (int idx = 1; idx <= 200; ++idx) {
+      int size = randomGenerator.nextInt(1024 * 1024) + 1024 * 1024;
+      mbh = mclst.createBuffer(size);
+      for (int i = 0; i < size; i++) {
+        mbh.get().put((byte) randomGenerator.nextInt(255));
+      }
+      assertEquals(size, mbh.get().capacity());
+    }
+  }
+
+  /**
+   * test to manually release memory resource once that is intended to be
+   * unused.
+   */
+  @Test
+  public void testMemByteBufferManualRelease() {
+    Random randomGenerator = new Random();
+    MemClustering.NodeConfig<?> ncs[] = new MemClustering.NodeConfig<?>[] {
+        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<VolatileMemAllocator>(
+            new VolatileMemAllocator(Utils.getVolatileMemoryAllocatorService("vmem"), 1024 * 1024 * 20, ".", true),
+            MemClustering.PerformanceLevel.SLOW), };
+    MemClustering mclst = new MemClustering(ncs);
+    MemBufferHolder<?> mbh;
+    for (int idx = 1; idx <= 200; ++idx) {
+      int size = randomGenerator.nextInt(1024 * 1024) + 1024 * 1024;
+      mbh = mclst.createBuffer(size);
+      for (int i = 0; i < size; i++) {
+        mbh.get().put((byte) randomGenerator.nextInt(255));
+      }
+      assertEquals(size, mbh.get().capacity());
+      // System.out.println("testMemByteBufferManualRelease");
+      mbh.destroy();
+    }
+  }
+
+  public boolean actriggered = false, pctriggered = false;
+
+  /**
+   * test change events that will be triggered by memory pool downgrading or
+   * memory pool switching caused by fill up.
+   * 
+   * @throws Exception
+   */
+  @Test
+  public void testMemByteBufferWithChange() throws Exception {
+    Random randomGenerator = new Random();
+    MemClustering.NodeConfig<?> ncs[] = new MemClustering.NodeConfig<?>[] {
+        new MemClustering.NodeConfig<SysMemAllocator>(new SysMemAllocator(1024 * 1024 * 20, true).disableActiveGC(),
+            MemClustering.PerformanceLevel.FASTEST),
+        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<VolatileMemAllocator>(
+            new VolatileMemAllocator(Utils.getVolatileMemoryAllocatorService("vmem"), 1024 * 1024 * 20, ".", true),
+            MemClustering.PerformanceLevel.SLOW), };
+    MemClustering mclst = new MemClustering(ncs) {
+    };
+
+    actriggered = false;
+    pctriggered = false;
+    mclst.setAllocatorChange(new MemClustering.AllocatorChange() {
+      @Override
+      public void changed(MemClustering.PerformanceLevel lvl, CommonAllocator<?> prevallocator,
+          CommonAllocator<?> tgtallocator) {
+        System.out.println(String.format("Allocator Changed: %s, %X -> %X", lvl.name(),
+            System.identityHashCode(prevallocator), System.identityHashCode(tgtallocator)));
+        actriggered = true;
+      }
+    });
+    mclst.setPerformanceLevelChange(new MemClustering.PerformanceLevelChange() {
+      @Override
+      public void changed(MemClustering.PerformanceLevel prevlvl, MemClustering.PerformanceLevel lvl) {
+        System.out.println(
+            String.format("Perf.Level Changed: %s -> %s", null == prevlvl ? "NULL" : prevlvl.name(), lvl.name()));
+        pctriggered = true;
+      }
+    });
+
+    MemBufferHolder<?> mbh;
+    for (int idx = 1; idx <= 100; ++idx) {
+      int size = randomGenerator.nextInt(1024 * 1024) + 1024 * 1024;
+      mbh = mclst.createBuffer(size);
+      for (int i = 0; i < size; i++) {
+        mbh.get().put((byte) randomGenerator.nextInt(255));
+      }
+      // mbh.destroy();
+    }
+    assertTrue(actriggered && pctriggered);
+
+  }
+
+  /*
+   * @Test public void testMemChunk() { Random randomGenerator = new Random();
+   * Allocator act = new BigMemAllocator(1024*1024*1024, "/home/wg/bm", true);
+   * MemChunkHolder mch; for (int idx = 1; idx <= 50000; ++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(); } }
+   */
+}