Added unsafe backing implementation of MemoryBuffer

git-svn-id: https://svn.apache.org/repos/asf/directmemory/trunk@1402295 13f79535-47bb-0310-9956-ffa450edef68
diff --git a/directmemory-cache/src/main/java/org/apache/directmemory/memory/AllocationPolicy.java b/directmemory-cache/src/main/java/org/apache/directmemory/memory/AllocationPolicy.java
index 3b98043..a48257d 100644
--- a/directmemory-cache/src/main/java/org/apache/directmemory/memory/AllocationPolicy.java
+++ b/directmemory-cache/src/main/java/org/apache/directmemory/memory/AllocationPolicy.java
@@ -21,7 +21,7 @@
 
 import java.util.List;
 
-import org.apache.directmemory.memory.allocator.ByteBufferAllocator;
+import org.apache.directmemory.memory.allocator.Allocator;
 
 /**
  * Interface describing the buffer allocation policy.
@@ -40,16 +40,16 @@
      *
      * @param buffers
      */
-    void init( List<ByteBufferAllocator> allocators );
+    void init( List<Allocator> allocators );
 
     /**
-     * Returns the {@link ByteBufferAllocator} to use to allocate.
+     * Returns the {@link Allocator} to use to allocate.
      *
-     * @param previousAllocator : the previously used {@link ByteBufferAllocator}, or null if it's the first allocation
+     * @param previousAllocator : the previously used {@link Allocator}, or null if it's the first allocation
      * @param allocationNumber : the number of time the allocation has already failed.
-     * @return the {@link ByteBufferAllocator} to use, or null if allocation has failed.
+     * @return the {@link Allocator} to use, or null if allocation has failed.
      */
-    ByteBufferAllocator getActiveAllocator( ByteBufferAllocator previousAllocator, int allocationNumber );
+    Allocator getActiveAllocator( Allocator previousAllocator, int allocationNumber );
 
     /**
      * Reset internal state
diff --git a/directmemory-cache/src/main/java/org/apache/directmemory/memory/MemoryManagerServiceImpl.java b/directmemory-cache/src/main/java/org/apache/directmemory/memory/MemoryManagerServiceImpl.java
index e0c938d..8e6ccd4 100644
--- a/directmemory-cache/src/main/java/org/apache/directmemory/memory/MemoryManagerServiceImpl.java
+++ b/directmemory-cache/src/main/java/org/apache/directmemory/memory/MemoryManagerServiceImpl.java
@@ -26,7 +26,7 @@
 import java.util.List;
 
 import org.apache.directmemory.measures.Ram;
-import org.apache.directmemory.memory.allocator.ByteBufferAllocator;
+import org.apache.directmemory.memory.allocator.Allocator;
 import org.apache.directmemory.memory.allocator.MergingByteBufferAllocatorImpl;
 import org.apache.directmemory.memory.buffer.MemoryBuffer;
 import org.slf4j.Logger;
@@ -38,7 +38,7 @@
 
     protected static Logger logger = LoggerFactory.getLogger( MemoryManager.class );
 
-    List<ByteBufferAllocator> allocators;
+    List<Allocator> allocators;
 
     protected final AllocationPolicy allocationPolicy;
 
@@ -62,11 +62,11 @@
     public void init( int numberOfBuffers, int size )
     {
 
-        allocators = new ArrayList<ByteBufferAllocator>( numberOfBuffers );
+        allocators = new ArrayList<Allocator>( numberOfBuffers );
 
         for ( int i = 0; i < numberOfBuffers; i++ )
         {
-            final ByteBufferAllocator allocator = instanciateByteBufferAllocator( i, size );
+            final Allocator allocator = instanciateByteBufferAllocator( i, size );
             allocators.add( allocator );
         }
 
@@ -76,7 +76,7 @@
     }
 
 
-    protected ByteBufferAllocator instanciateByteBufferAllocator( final int allocatorNumber, final int size )
+    protected Allocator instanciateByteBufferAllocator( final int allocatorNumber, final int size )
     {
         final MergingByteBufferAllocatorImpl allocator = new MergingByteBufferAllocatorImpl( allocatorNumber, size );
 
@@ -87,12 +87,12 @@
         return allocator;
     }
 
-    protected ByteBufferAllocator getAllocator( int allocatorIndex )
+    protected Allocator getAllocator( int allocatorIndex )
     {
         return allocators.get( allocatorIndex );
     }
 
-    protected ByteBufferAllocator getCurrentAllocator()
+    protected Allocator getCurrentAllocator()
     {
         return allocationPolicy.getActiveAllocator( null, 0 );
     }
@@ -101,7 +101,7 @@
     public Pointer<V> store( byte[] payload, long expiresIn )
     {
         Pointer<V> p = null;
-        ByteBufferAllocator allocator = null;
+        Allocator allocator = null;
         int allocationNumber = 0;
         do
         {
@@ -179,7 +179,7 @@
     public long capacity()
     {
         long totalCapacity = 0;
-        for ( ByteBufferAllocator allocator : allocators )
+        for ( Allocator allocator : allocators )
         {
             totalCapacity += allocator.getCapacity();
         }
@@ -187,7 +187,7 @@
     }
 
   
-    protected List<ByteBufferAllocator> getAllocators()
+    protected List<Allocator> getAllocators()
     {
         return allocators;
     }
@@ -199,7 +199,7 @@
     {
 
         Pointer<V> p = null;
-        ByteBufferAllocator allocator = null;
+        Allocator allocator = null;
         int allocationNumber = 0;
         do
         {
@@ -243,7 +243,7 @@
             pointer.setFree( true );
         }
         pointers.clear();
-        for ( ByteBufferAllocator allocator : allocators )
+        for ( Allocator allocator : allocators )
         {
             allocator.clear();
         }
diff --git a/directmemory-cache/src/main/java/org/apache/directmemory/memory/RoundRobinAllocationPolicy.java b/directmemory-cache/src/main/java/org/apache/directmemory/memory/RoundRobinAllocationPolicy.java
index 80cf397..76c580e 100644
--- a/directmemory-cache/src/main/java/org/apache/directmemory/memory/RoundRobinAllocationPolicy.java
+++ b/directmemory-cache/src/main/java/org/apache/directmemory/memory/RoundRobinAllocationPolicy.java
@@ -22,7 +22,7 @@
 import java.util.List;
 import java.util.concurrent.atomic.AtomicInteger;
 
-import org.apache.directmemory.memory.allocator.ByteBufferAllocator;
+import org.apache.directmemory.memory.allocator.Allocator;
 
 /**
  * Round Robin allocation policy. An internal counter is incremented (modulo the size of the buffer), so each calls to
@@ -39,7 +39,7 @@
     private static final int BUFFERS_INDEX_INITIAL_VALUE = -1;
 
     // All the buffers to allocate
-    private List<ByteBufferAllocator> allocators;
+    private List<Allocator> allocators;
 
     // Cyclic counter
     private AtomicInteger buffersIndexCounter = new AtomicInteger( BUFFERS_INDEX_INITIAL_VALUE );
@@ -56,13 +56,13 @@
     }
 
     @Override
-    public void init( final List<ByteBufferAllocator> allocators )
+    public void init( final List<Allocator> allocators )
     {
         this.allocators = allocators;
     }
 
     @Override
-    public ByteBufferAllocator getActiveAllocator( final ByteBufferAllocator previousAllocator, final int allocationNumber )
+    public Allocator getActiveAllocator( final Allocator previousAllocator, final int allocationNumber )
     {
         // If current allocation is more than the limit, return a null buffer.
         if ( allocationNumber > maxAllocations )
@@ -73,7 +73,7 @@
         // Thread safely increment and get the next buffer's index
         int i = incrementAndGetBufferIndex();
 
-        final ByteBufferAllocator allocator = allocators.get( i );
+        final Allocator allocator = allocators.get( i );
 
         return allocator;
     }
diff --git a/directmemory-cache/src/main/java/org/apache/directmemory/memory/allocator/AbstractByteBufferAllocator.java b/directmemory-cache/src/main/java/org/apache/directmemory/memory/allocator/AbstractByteBufferAllocator.java
index 468db89..c9dc0dd 100644
--- a/directmemory-cache/src/main/java/org/apache/directmemory/memory/allocator/AbstractByteBufferAllocator.java
+++ b/directmemory-cache/src/main/java/org/apache/directmemory/memory/allocator/AbstractByteBufferAllocator.java
@@ -19,7 +19,6 @@
  * under the License.
  */
 
-import org.apache.directmemory.memory.buffer.MemoryBuffer;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -28,7 +27,7 @@
 
 
 public abstract class AbstractByteBufferAllocator
-    implements ByteBufferAllocator
+    implements Allocator
 {
 
     protected final Logger logger = LoggerFactory.getLogger( this.getClass() );
diff --git a/directmemory-cache/src/main/java/org/apache/directmemory/memory/allocator/ByteBufferAllocator.java b/directmemory-cache/src/main/java/org/apache/directmemory/memory/allocator/Allocator.java
similarity index 90%
rename from directmemory-cache/src/main/java/org/apache/directmemory/memory/allocator/ByteBufferAllocator.java
rename to directmemory-cache/src/main/java/org/apache/directmemory/memory/allocator/Allocator.java
index 8556dca..c46ac54 100644
--- a/directmemory-cache/src/main/java/org/apache/directmemory/memory/allocator/ByteBufferAllocator.java
+++ b/directmemory-cache/src/main/java/org/apache/directmemory/memory/allocator/Allocator.java
@@ -24,11 +24,11 @@
 import java.io.Closeable;
 
 /**
- * Interface defining interaction with {@link ByteBuffer}
+ * Interface defining interaction with {@link MemoryBuffer}
  * 
  * @since 0.6
  */
-public interface ByteBufferAllocator
+public interface Allocator
     extends Closeable
 {
     
@@ -47,7 +47,7 @@
     MemoryBuffer allocate( final int size );
     
     /**
-     * Clear all allocated {@link MemoryBuffer}, resulting in a empty and ready to deserve {@link ByteBufferAllocator}
+     * Clear all allocated {@link MemoryBuffer}, resulting in a empty and ready to deserve {@link Allocator}
      */
     void clear();
     
@@ -57,7 +57,7 @@
     int getCapacity();
     
     /**
-     * @return the internal identifier of the {@link ByteBufferAllocator}
+     * @return the internal identifier of the {@link Allocator}
      */
     int getNumber();
     
diff --git a/directmemory-cache/src/main/java/org/apache/directmemory/memory/allocator/FixedSizeByteBufferAllocatorImpl.java b/directmemory-cache/src/main/java/org/apache/directmemory/memory/allocator/FixedSizeByteBufferAllocatorImpl.java
index 3c29b6e..dfc1dd4 100644
--- a/directmemory-cache/src/main/java/org/apache/directmemory/memory/allocator/FixedSizeByteBufferAllocatorImpl.java
+++ b/directmemory-cache/src/main/java/org/apache/directmemory/memory/allocator/FixedSizeByteBufferAllocatorImpl.java
@@ -34,7 +34,7 @@
 import static com.google.common.base.Preconditions.checkState;
 
 /**
- * {@link ByteBufferAllocator} implementation that instantiate {@link ByteBuffer}s of fixed size, called slices.
+ * {@link Allocator} implementation that instantiate {@link ByteBuffer}s of fixed size, called slices.
  *
  * @since 0.6
  */
diff --git a/directmemory-cache/src/main/java/org/apache/directmemory/memory/allocator/FixedSizeUnsafeAllocatorImpl.java b/directmemory-cache/src/main/java/org/apache/directmemory/memory/allocator/FixedSizeUnsafeAllocatorImpl.java
new file mode 100644
index 0000000..b1da057
--- /dev/null
+++ b/directmemory-cache/src/main/java/org/apache/directmemory/memory/allocator/FixedSizeUnsafeAllocatorImpl.java
@@ -0,0 +1,253 @@
+package org.apache.directmemory.memory.allocator;
+
+import java.io.IOException;
+import java.nio.ByteOrder;
+import java.util.Iterator;
+import java.util.Set;
+import java.util.concurrent.ConcurrentSkipListSet;
+
+import org.apache.directmemory.memory.buffer.AbstractMemoryBuffer;
+import org.apache.directmemory.memory.buffer.MemoryBuffer;
+
+public class FixedSizeUnsafeAllocatorImpl
+    implements Allocator
+{
+
+    private final sun.misc.Unsafe unsafe = UnsafeUtils.getUnsafe();
+
+    private final Set<UnsafeMemoryBuffer> memoryBuffers = new ConcurrentSkipListSet<UnsafeMemoryBuffer>();
+
+    private final int number;
+
+    public FixedSizeUnsafeAllocatorImpl( int number )
+    {
+        this.number = number;
+
+        if ( unsafe == null )
+        {
+            throw new IllegalStateException( "This JVM has no sun.misc.Unsafe support, "
+                + "please choose another MemoryManager implementation" );
+        }
+    }
+
+    @Override
+    public void close()
+        throws IOException
+    {
+        clear();
+    }
+
+    @Override
+    public void free( MemoryBuffer memoryBuffer )
+    {
+        memoryBuffer.free();
+        memoryBuffers.remove( memoryBuffer );
+    }
+
+    @Override
+    public MemoryBuffer allocate( int size )
+    {
+        long baseAddress = unsafe.allocateMemory( size );
+        UnsafeMemoryBuffer memoryBuffer = new UnsafeMemoryBuffer( baseAddress, size );
+        memoryBuffers.add( memoryBuffer );
+        return memoryBuffer;
+    }
+
+    @Override
+    public void clear()
+    {
+        Iterator<UnsafeMemoryBuffer> iterator = memoryBuffers.iterator();
+        while ( iterator.hasNext() )
+        {
+            UnsafeMemoryBuffer memoryBuffer = iterator.next();
+            unsafe.setMemory( memoryBuffer.baseAddress, memoryBuffer.capacity, (byte) 0 );
+            iterator.remove();
+        }
+    }
+
+    @Override
+    public int getCapacity()
+    {
+        long capacity = 0;
+        Iterator<UnsafeMemoryBuffer> iterator = memoryBuffers.iterator();
+        while ( iterator.hasNext() )
+        {
+            capacity += iterator.next().capacity;
+        }
+        return (int) capacity;
+    }
+
+    @Override
+    public int getNumber()
+    {
+        return number;
+    }
+
+    private class UnsafeMemoryBuffer
+        extends AbstractMemoryBuffer
+    {
+
+        private final long baseAddress;
+
+        private final long capacity;
+
+        private ByteOrder byteOrder = ByteOrder.BIG_ENDIAN;
+
+        private UnsafeMemoryBuffer( long baseAddress, long capacity )
+        {
+            this.baseAddress = baseAddress;
+            this.capacity = capacity;
+        }
+
+        @Override
+        public long capacity()
+        {
+            return capacity;
+        }
+
+        @Override
+        public long maxCapacity()
+        {
+            return capacity;
+        }
+
+        @Override
+        public boolean growing()
+        {
+            return false;
+        }
+
+        @Override
+        public ByteOrder byteOrder()
+        {
+            return byteOrder;
+        }
+
+        @Override
+        public void byteOrder( ByteOrder byteOrder )
+        {
+            this.byteOrder = byteOrder;
+        }
+
+        @Override
+        public void free()
+        {
+            unsafe.freeMemory( baseAddress );
+        }
+
+        @Override
+        public void clear()
+        {
+            unsafe.setMemory( baseAddress, capacity, (byte) 0 );
+            writerIndex = 0;
+            readerIndex = 0;
+        }
+
+        @Override
+        protected void writeByte( long offset, byte value )
+        {
+            unsafe.putByte( baseAddress + offset, value );
+            writerIndex++;
+        }
+
+        @Override
+        protected byte readByte( long offset )
+        {
+            byte value = unsafe.getByte( baseAddress + offset );
+            readerIndex++;
+            return value;
+        }
+
+        @Override
+        public short readShort()
+        {
+            short value = unsafe.getShort( baseAddress + readerIndex );
+            readerIndex += 2;
+            return value;
+        }
+
+        @Override
+        public char readChar()
+        {
+            char value = unsafe.getChar( baseAddress + readerIndex );
+            readerIndex += 2;
+            return value;
+        }
+
+        @Override
+        public int readInt()
+        {
+            int value = unsafe.getInt( baseAddress + readerIndex );
+            readerIndex += 4;
+            return value;
+        }
+
+        @Override
+        public long readLong()
+        {
+            long value = unsafe.getLong( baseAddress + readerIndex );
+            readerIndex += 8;
+            return value;
+        }
+
+        @Override
+        public float readFloat()
+        {
+            float value = unsafe.getFloat( baseAddress + readerIndex );
+            readerIndex += 4;
+            return value;
+        }
+
+        @Override
+        public double readDouble()
+        {
+            double value = unsafe.getDouble( baseAddress + readerIndex );
+            readerIndex += 8;
+            return value;
+        }
+
+        @Override
+        public void writeShort( short value )
+        {
+            unsafe.putShort( baseAddress + writerIndex, value );
+            writerIndex += 2;
+        }
+
+        @Override
+        public void writeChar( char value )
+        {
+            unsafe.putChar( baseAddress + writerIndex, value );
+            writerIndex += 2;
+        }
+
+        @Override
+        public void writeInt( int value )
+        {
+            unsafe.putInt( baseAddress + writerIndex, value );
+            writerIndex += 4;
+        }
+
+        @Override
+        public void writeLong( long value )
+        {
+            unsafe.putLong( baseAddress + writerIndex, value );
+            writerIndex += 8;
+        }
+
+        @Override
+        public void writeFloat( float value )
+        {
+            unsafe.putFloat( baseAddress + writerIndex, value );
+            writerIndex += 4;
+        }
+
+        @Override
+        public void writeDouble( double value )
+        {
+            unsafe.putDouble( baseAddress + writerIndex, value );
+            writerIndex += 8;
+        }
+
+    }
+
+}
diff --git a/directmemory-cache/src/main/java/org/apache/directmemory/memory/allocator/MergingByteBufferAllocatorImpl.java b/directmemory-cache/src/main/java/org/apache/directmemory/memory/allocator/MergingByteBufferAllocatorImpl.java
index 9055b9a..1770ce5 100644
--- a/directmemory-cache/src/main/java/org/apache/directmemory/memory/allocator/MergingByteBufferAllocatorImpl.java
+++ b/directmemory-cache/src/main/java/org/apache/directmemory/memory/allocator/MergingByteBufferAllocatorImpl.java
@@ -38,7 +38,7 @@
 import java.util.concurrent.locks.ReentrantLock;
 
 /**
- * {@link ByteBufferAllocator} implementation with {@link ByteBuffer} merging capabilities.
+ * {@link Allocator} implementation with {@link ByteBuffer} merging capabilities.
  * <p/>
  * {@link ByteBuffer}s are wrapped into an {@link LinkedByteBuffer}, and when a {@link ByteBuffer} is freed,
  * lookup is done to the neighbor to check if they are also free, in which case they are merged.
@@ -377,11 +377,8 @@
 
     private class MergingNioMemoryBuffer extends NioMemoryBuffer {
 
-        private final LinkedByteBuffer linkedBuffer;
-
         MergingNioMemoryBuffer(LinkedByteBuffer linkedBuffer) {
             super(linkedBuffer.buffer);
-            this.linkedBuffer = linkedBuffer;
         }
 
         @Override
diff --git a/directmemory-cache/src/main/java/org/apache/directmemory/memory/allocator/SlabByteBufferAllocatorImpl.java b/directmemory-cache/src/main/java/org/apache/directmemory/memory/allocator/SlabByteBufferAllocatorImpl.java
index fb69ee4..eb78522 100644
--- a/directmemory-cache/src/main/java/org/apache/directmemory/memory/allocator/SlabByteBufferAllocatorImpl.java
+++ b/directmemory-cache/src/main/java/org/apache/directmemory/memory/allocator/SlabByteBufferAllocatorImpl.java
@@ -31,7 +31,7 @@
 
 
 /**
- * {@link ByteBufferAllocator} implementation that uses {@link FixedSizeByteBufferAllocatorImpl}
+ * {@link Allocator} implementation that uses {@link FixedSizeByteBufferAllocatorImpl}
  * of different size to allocate best matching's size {@link ByteBuffer}
  *
  * @since 0.6
diff --git a/directmemory-cache/src/main/java/org/apache/directmemory/memory/allocator/UnsafeUtils.java b/directmemory-cache/src/main/java/org/apache/directmemory/memory/allocator/UnsafeUtils.java
new file mode 100644
index 0000000..c8323c2
--- /dev/null
+++ b/directmemory-cache/src/main/java/org/apache/directmemory/memory/allocator/UnsafeUtils.java
@@ -0,0 +1,54 @@
+/*
+ * 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.directmemory.memory.allocator;
+
+import java.lang.reflect.Field;
+
+@SuppressWarnings( "restriction" )
+public final class UnsafeUtils
+{
+
+    private static final sun.misc.Unsafe UNSAFE;
+
+    static
+    {
+        sun.misc.Unsafe unsafe;
+        try
+        {
+            Field unsafeField = sun.misc.Unsafe.class.getDeclaredField( "theUnsafe" );
+            unsafeField.setAccessible( true );
+            unsafe = (sun.misc.Unsafe) unsafeField.get( null );
+        }
+        catch ( Exception e )
+        {
+            unsafe = null;
+        }
+
+        UNSAFE = unsafe;
+    }
+
+    private UnsafeUtils()
+    {
+    }
+
+    public static sun.misc.Unsafe getUnsafe()
+    {
+        return UNSAFE;
+    }
+}
diff --git a/directmemory-cache/src/main/java/org/apache/directmemory/memory/buffer/AbstractMemoryBuffer.java b/directmemory-cache/src/main/java/org/apache/directmemory/memory/buffer/AbstractMemoryBuffer.java
index b25a686..56e98cd 100644
--- a/directmemory-cache/src/main/java/org/apache/directmemory/memory/buffer/AbstractMemoryBuffer.java
+++ b/directmemory-cache/src/main/java/org/apache/directmemory/memory/buffer/AbstractMemoryBuffer.java
@@ -313,6 +313,19 @@
         this.writerIndex = writerIndex;
     }
 
+    protected void rangeCheck( long offset )
+    {
+        if ( offset < 0 )
+        {
+            throw new IndexOutOfBoundsException( String.format( "Offset %s is below 0", offset ) );
+        }
+        if ( offset >= maxCapacity() )
+        {
+            throw new IndexOutOfBoundsException( String.format( "Offset %s is higher than maximum legal index ",
+                                                                offset, ( maxCapacity() - 1 ) ) );
+        }
+    }
+
     protected abstract void writeByte( long offset, byte value );
 
     protected abstract byte readByte( long offset );
diff --git a/directmemory-cache/src/test/java/org/apache/directmemory/memory/RoundRobinAllocationPolicyTest.java b/directmemory-cache/src/test/java/org/apache/directmemory/memory/RoundRobinAllocationPolicyTest.java
index 59a1611..ef97913 100644
--- a/directmemory-cache/src/test/java/org/apache/directmemory/memory/RoundRobinAllocationPolicyTest.java
+++ b/directmemory-cache/src/test/java/org/apache/directmemory/memory/RoundRobinAllocationPolicyTest.java
@@ -28,7 +28,7 @@
 import java.util.ArrayList;
 import java.util.List;
 
-import org.apache.directmemory.memory.allocator.ByteBufferAllocator;
+import org.apache.directmemory.memory.allocator.Allocator;
 import org.apache.directmemory.memory.buffer.MemoryBuffer;
 import org.junit.Before;
 import org.junit.Test;
@@ -44,7 +44,7 @@
 
     private static final int NUMBER_OF_BUFFERS = 4;
 
-    List<ByteBufferAllocator> allocators;
+    List<Allocator> allocators;
 
     RoundRobinAllocationPolicy allocationPolicy;
 
@@ -52,7 +52,7 @@
     public void initAllocationPolicy()
     {
 
-        allocators = new ArrayList<ByteBufferAllocator>();
+        allocators = new ArrayList<Allocator>();
 
         for ( int i = 0; i < NUMBER_OF_BUFFERS; i++ )
         {
@@ -111,7 +111,7 @@
      * Dummy {@link OffHeapMemoryBuffer} that do nothing.
      */
     private static class DummyByteBufferAllocator
-        implements ByteBufferAllocator
+        implements Allocator
     {
 
         @Override
diff --git a/directmemory-cache/src/test/java/org/apache/directmemory/memory/SlabMemoryManagerServiceTest.java b/directmemory-cache/src/test/java/org/apache/directmemory/memory/SlabMemoryManagerServiceTest.java
index f32e8c2..8b2a212 100644
--- a/directmemory-cache/src/test/java/org/apache/directmemory/memory/SlabMemoryManagerServiceTest.java
+++ b/directmemory-cache/src/test/java/org/apache/directmemory/memory/SlabMemoryManagerServiceTest.java
@@ -22,7 +22,7 @@
 import java.util.Collection;
 import java.util.HashSet;
 
-import org.apache.directmemory.memory.allocator.ByteBufferAllocator;
+import org.apache.directmemory.memory.allocator.Allocator;
 import org.apache.directmemory.memory.allocator.FixedSizeByteBufferAllocatorImpl;
 import org.apache.directmemory.memory.allocator.SlabByteBufferAllocatorImpl;
 import org.junit.Test;
@@ -37,7 +37,7 @@
         final MemoryManagerService<Object> mms = new MemoryManagerServiceImpl<Object>() {
 
             @Override
-            protected ByteBufferAllocator instanciateByteBufferAllocator( int allocatorNumber, int size )
+            protected Allocator instanciateByteBufferAllocator( int allocatorNumber, int size )
             {
                 Collection<FixedSizeByteBufferAllocatorImpl> slabs = new HashSet<FixedSizeByteBufferAllocatorImpl>();
                 
diff --git a/directmemory-cache/src/test/java/org/apache/directmemory/memory/Starter.java b/directmemory-cache/src/test/java/org/apache/directmemory/memory/Starter.java
index 903d21d..9dd3fc2 100644
--- a/directmemory-cache/src/test/java/org/apache/directmemory/memory/Starter.java
+++ b/directmemory-cache/src/test/java/org/apache/directmemory/memory/Starter.java
@@ -21,7 +21,7 @@
 
 import org.apache.directmemory.measures.Ram;
 import org.apache.directmemory.memory.MemoryManager;
-import org.apache.directmemory.memory.allocator.ByteBufferAllocator;
+import org.apache.directmemory.memory.allocator.Allocator;
 import org.apache.directmemory.memory.allocator.MergingByteBufferAllocatorImpl;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -70,7 +70,7 @@
         logger.info( "************************************************" );
     }
     
-    public void dump( ByteBufferAllocator mem )
+    public void dump( Allocator mem )
     {
         logger.info( "off-heap - buffer: " + mem.getNumber() );
         logger.info( "off-heap - allocated: " + Ram.inMb( mem.getCapacity() ) );
@@ -82,7 +82,7 @@
     
     public void rawInsert( int megabytes, int howMany )
     {
-        ByteBufferAllocator allocator = new MergingByteBufferAllocatorImpl( 1, megabytes * 1024 * 1024 );
+        Allocator allocator = new MergingByteBufferAllocatorImpl( 1, megabytes * 1024 * 1024 );
         assertNotNull( allocator );
         int size = allocator.getCapacity() / ( howMany );
         size -= size / 100 * 1;
diff --git a/directmemory-cache/src/test/java/org/apache/directmemory/memory/allocator/FixedSizeByteBufferAllocatorImplTest.java b/directmemory-cache/src/test/java/org/apache/directmemory/memory/allocator/FixedSizeByteBufferAllocatorImplTest.java
index 28a8107..6c0b99d 100644
--- a/directmemory-cache/src/test/java/org/apache/directmemory/memory/allocator/FixedSizeByteBufferAllocatorImplTest.java
+++ b/directmemory-cache/src/test/java/org/apache/directmemory/memory/allocator/FixedSizeByteBufferAllocatorImplTest.java
@@ -30,7 +30,7 @@
     public void allocationTest()
     {
         
-        ByteBufferAllocator allocator = new FixedSizeByteBufferAllocatorImpl( 0, 5000, 256, 1 );
+        Allocator allocator = new FixedSizeByteBufferAllocatorImpl( 0, 5000, 256, 1 );
         
         MemoryBuffer bf1 = allocator.allocate( 250 );
         Assert.assertEquals( 256, bf1.maxCapacity() );
@@ -61,7 +61,7 @@
     public void releaseTest()
     {
         
-        ByteBufferAllocator allocator = new FixedSizeByteBufferAllocatorImpl( 0, 1000, 256, 1 );
+        Allocator allocator = new FixedSizeByteBufferAllocatorImpl( 0, 1000, 256, 1 );
         
         MemoryBuffer bf1 = allocator.allocate( 250 );
         Assert.assertEquals( 256, bf1.maxCapacity() );
@@ -98,7 +98,7 @@
     public void allocateAndFreeTest()
     {
         
-        ByteBufferAllocator allocator = new FixedSizeByteBufferAllocatorImpl( 0, 1000, 256, 1 );
+        Allocator allocator = new FixedSizeByteBufferAllocatorImpl( 0, 1000, 256, 1 );
         
         for (int i = 0; i < 1000; i++)
         {
diff --git a/directmemory-cache/src/test/java/org/apache/directmemory/memory/allocator/MergingByteBufferAllocatorImplTest.java b/directmemory-cache/src/test/java/org/apache/directmemory/memory/allocator/MergingByteBufferAllocatorImplTest.java
index d9a06bc..2a63b5f 100644
--- a/directmemory-cache/src/test/java/org/apache/directmemory/memory/allocator/MergingByteBufferAllocatorImplTest.java
+++ b/directmemory-cache/src/test/java/org/apache/directmemory/memory/allocator/MergingByteBufferAllocatorImplTest.java
@@ -32,7 +32,7 @@
     public void allocationTest()
     {
         
-        ByteBufferAllocator allocator = new MergingByteBufferAllocatorImpl( 0, 5000 );
+        Allocator allocator = new MergingByteBufferAllocatorImpl( 0, 5000 );
         
         MemoryBuffer bf1 = allocator.allocate( 250 );
         Assert.assertEquals( 250, bf1.maxCapacity() );
@@ -68,7 +68,7 @@
     public void releaseTest()
     {
         
-        ByteBufferAllocator allocator = new MergingByteBufferAllocatorImpl( 0, 1000 );
+        Allocator allocator = new MergingByteBufferAllocatorImpl( 0, 1000 );
         
         MemoryBuffer bf1 = allocator.allocate( 250 );
         Assert.assertEquals( 250, bf1.maxCapacity() );
@@ -98,7 +98,7 @@
     public void allocateAndFreeTest()
     {
         
-        ByteBufferAllocator allocator = new MergingByteBufferAllocatorImpl( 0, 1000 );
+        Allocator allocator = new MergingByteBufferAllocatorImpl( 0, 1000 );
         
         for (int i = 0; i < 1000; i++)
         {
@@ -120,7 +120,7 @@
     public void allocationWithoutSplittingPointerTest()
     {
         
-        ByteBufferAllocator allocator = new MergingByteBufferAllocatorImpl( 0, 200 );
+        Allocator allocator = new MergingByteBufferAllocatorImpl( 0, 200 );
         
         MemoryBuffer bf1 = allocator.allocate( 180 );
         Assert.assertEquals( 200, bf1.maxCapacity() );
diff --git a/directmemory-cache/src/test/java/org/apache/directmemory/memory/allocator/SlabByteBufferAllocatorImplTest.java b/directmemory-cache/src/test/java/org/apache/directmemory/memory/allocator/SlabByteBufferAllocatorImplTest.java
index 4d464e7..666e004 100644
--- a/directmemory-cache/src/test/java/org/apache/directmemory/memory/allocator/SlabByteBufferAllocatorImplTest.java
+++ b/directmemory-cache/src/test/java/org/apache/directmemory/memory/allocator/SlabByteBufferAllocatorImplTest.java
@@ -40,7 +40,7 @@
         slabs.add( new FixedSizeByteBufferAllocatorImpl( 3, 1024, 1024, 1 ) );
         
         
-        ByteBufferAllocator allocator = new SlabByteBufferAllocatorImpl( 0, slabs, false );
+        Allocator allocator = new SlabByteBufferAllocatorImpl( 0, slabs, false );
         
         MemoryBuffer bf1 = allocator.allocate( 250 );
         Assert.assertEquals( 256, bf1.maxCapacity() );