You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by ji...@apache.org on 2016/03/30 00:25:47 UTC

[17/50] [abbrv] incubator-geode git commit: GEODE-1101: rename SimpleMemoryAllocator to MemoryAllocator

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/82faa8af/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/SimpleMemoryAllocatorFillPatternJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/SimpleMemoryAllocatorFillPatternJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/SimpleMemoryAllocatorFillPatternJUnitTest.java
deleted file mode 100644
index c61f2f4..0000000
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/SimpleMemoryAllocatorFillPatternJUnitTest.java
+++ /dev/null
@@ -1,183 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package com.gemstone.gemfire.internal.offheap;
-
-import static org.junit.Assert.*;
-import static com.googlecode.catchexception.CatchException.*;
-
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
-
-import junit.framework.TestCase;
-
-/**
- * Tests fill pattern validation for the {@link SimpleMemoryAllocatorImpl}.
- * @author rholmes
- */
-@Category(UnitTest.class)
-public class SimpleMemoryAllocatorFillPatternJUnitTest {
-  
-  /** Size of single test slab.*/
-  private static final int SLAB_SIZE = 1024 * 1024 * 50;
-  
-  /** Canned data for write operations. */
-  private static final byte[] WRITE_BYTES = new String("Some string data.").getBytes();
-  
-  /** Chunk size for basic huge allocation test. */
-  private static final int HUGE_CHUNK_SIZE = 1024 * 200;
-  
-  /** The number of chunks to allocate in order to force compaction. */
-  private static final int COMPACTION_CHUNKS = 3;
-  
-  /** Our slab size divided in three (with some padding for safety). */
-  private static final int COMPACTION_CHUNK_SIZE = (SLAB_SIZE / COMPACTION_CHUNKS) - 1024;
-  
-  /** This should force compaction when allocated. */
-  private static final int FORCE_COMPACTION_CHUNK_SIZE = COMPACTION_CHUNK_SIZE * 2;
-
-  /** Our test victim. */
-  private SimpleMemoryAllocatorImpl allocator = null;
-  
-  /** Our test victim's memory slab. */
-  private SlabImpl slab = null;
-
-  /**
-   * Enables fill validation and creates the test victim.
-   */
-  @Before
-  public void setUp() throws Exception {
-    System.setProperty("gemfire.validateOffHeapWithFill", "true");
-    this.slab = new SlabImpl(SLAB_SIZE);
-    this.allocator = SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new SlabImpl[]{this.slab});
-  }
-
-  /**
-   * Frees off heap memory.
-   */
-  @After
-  public void tearDown() throws Exception {
-    SimpleMemoryAllocatorImpl.freeOffHeapMemory();
-    System.clearProperty("gemfire.validateOffHeapWithFill");
-  }
-
-  /**
-   * This tests the fill pattern for a single tiny Chunk allocation.
-   * @throws Exception
-   */
-  @Test
-  public void testFillPatternBasicForTinyAllocations() throws Exception {
-    doFillPatternBasic(1024);
-  }
-  
-  /**
-   * This tests the fill pattern for a single huge Chunk allocation.
-   * @throws Exception
-   */
-  @Test
-  public void testFillPatternBasicForHugeAllocations() throws Exception {
-    doFillPatternBasic(HUGE_CHUNK_SIZE);
-  }
-  
-  private void doFillPatternBasic(final int chunkSize) {
-    /*
-     * Pull a chunk off the fragment.  This will have no fill because
-     * it is a "fresh" chunk.
-     */
-    OffHeapStoredObject chunk = (OffHeapStoredObject) this.allocator.allocate(chunkSize);
-
-    /*
-     * Chunk should have valid fill from initial fragment allocation.
-     */
-    chunk.validateFill();
-         
-    // "Dirty" the chunk so the release has something to fill over
-    chunk.writeDataBytes(OffHeapStoredObject.MIN_CHUNK_SIZE + 1, WRITE_BYTES);
-
-    // This should free the Chunk (ref count == 1)
-    chunk.release();
-
-    /*
-     * This chunk should have a fill because it was reused from the
-     * free list (assuming no fragmentation at this point...)
-     */
-    chunk = (OffHeapStoredObject) this.allocator.allocate(chunkSize);
-    
-    // Make sure we have a fill this time
-    chunk.validateFill();
-    
-    // Give the fill code something to write over during the release
-    chunk.writeDataBytes(OffHeapStoredObject.MIN_CHUNK_SIZE + 1, WRITE_BYTES);
-    chunk.release();
-
-    // Again, make sure the release implemented the fill
-    chunk.validateFill();
-
-    // "Dirty up" the free chunk
-    chunk.writeDataBytes(OffHeapStoredObject.MIN_CHUNK_SIZE + 1, WRITE_BYTES);
-    
-    catchException(chunk).validateFill();
-    assertTrue(caughtException() instanceof IllegalStateException);
-    assertEquals("Fill pattern violated for chunk " + chunk.getAddress() + " with size " + chunk.getSize(), caughtException().getMessage());
-    
-  }
-
-  /**
-   * This tests that fill validation is working properly on newly created fragments after
-   * a compaction.
-   * @throws Exception
-   */
-  @Test
-  public void testFillPatternAfterCompaction() throws Exception {
-    /*
-     * Stores our allocated memory.
-     */
-    OffHeapStoredObject[] allocatedChunks = new OffHeapStoredObject[COMPACTION_CHUNKS];
-    
-    /*
-     * Use up most of our memory
-     * Our memory looks like [      ][      ][      ]
-     */
-    for(int i =0;i < allocatedChunks.length;++i) {
-      allocatedChunks[i] = (OffHeapStoredObject) this.allocator.allocate(COMPACTION_CHUNK_SIZE);
-      allocatedChunks[i].validateFill();
-    }
-
-    /*
-     * Release some of our allocated chunks.
-     */
-    for(int i=0;i < 2;++i) {
-      allocatedChunks[i].release();
-      allocatedChunks[i].validateFill();      
-    }
-    
-    /*
-     * Now, allocate another chunk that is slightly larger than one of
-     * our initial chunks.  This should force a compaction causing our
-     * memory to look like [            ][      ].
-     */
-    OffHeapStoredObject slightlyLargerChunk = (OffHeapStoredObject) this.allocator.allocate(FORCE_COMPACTION_CHUNK_SIZE);
-    
-    /*
-     * Make sure the compacted memory has the fill validation.
-     */
-    slightlyLargerChunk.validateFill();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/82faa8af/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/SimpleMemoryAllocatorJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/SimpleMemoryAllocatorJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/SimpleMemoryAllocatorJUnitTest.java
deleted file mode 100644
index 135aba2..0000000
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/SimpleMemoryAllocatorJUnitTest.java
+++ /dev/null
@@ -1,594 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package com.gemstone.gemfire.internal.offheap;
-
-import static org.junit.Assert.*;
-
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.concurrent.atomic.AtomicReference;
-
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.contrib.java.lang.system.RestoreSystemProperties;
-import org.junit.experimental.categories.Category;
-
-import com.gemstone.gemfire.OutOfOffHeapMemoryException;
-import com.gemstone.gemfire.cache.CacheClosedException;
-import com.gemstone.gemfire.internal.logging.NullLogWriter;
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
-
-@Category(UnitTest.class)
-public class SimpleMemoryAllocatorJUnitTest {
-  @Rule
-  public final RestoreSystemProperties restoreSystemProperties = new RestoreSystemProperties();
-
-  private static int round(int multiple, int v) {
-    return ((v+multiple-1)/multiple)*multiple;
-  }
-  @Test
-  public void testNullGetAllocator() {
-    try {
-      SimpleMemoryAllocatorImpl.getAllocator();
-      fail("expected CacheClosedException");
-    } catch (CacheClosedException expected) {
-    }
-  }
-  @Test
-  public void testConstructor() {
-    try {
-      SimpleMemoryAllocatorImpl.createForUnitTest(null, null, null);
-      fail("expected IllegalArgumentException");
-    } catch (IllegalArgumentException expected) {
-    }
-  }
-  @Test
-  public void testCreate() {
-    System.setProperty(SimpleMemoryAllocatorImpl.FREE_OFF_HEAP_MEMORY_PROPERTY, "false");
-    {
-      NullOutOfOffHeapMemoryListener listener = new NullOutOfOffHeapMemoryListener();
-      NullOffHeapMemoryStats stats = new NullOffHeapMemoryStats();
-      try {
-        SimpleMemoryAllocatorImpl.createForUnitTest(listener, stats, 10, 950, 100, new SlabFactory() {
-     @Override
-     public Slab create(int size) {
-        throw new OutOfMemoryError("expected");
-     }
-    });
-      } catch (OutOfMemoryError expected) {
-      }
-      assertTrue(listener.isClosed());
-      assertTrue(stats.isClosed());
-     }
-    {
-      NullOutOfOffHeapMemoryListener listener = new NullOutOfOffHeapMemoryListener();
-      NullOffHeapMemoryStats stats = new NullOffHeapMemoryStats();
-      int MAX_SLAB_SIZE = 100;
-      try {
-        SlabFactory factory = new SlabFactory() {
-          private int createCount = 0;
-          @Override
-          public Slab create(int size) {
-            createCount++;
-            if (createCount == 1) {
-              return new SlabImpl(size);
-            } else {
-              throw new OutOfMemoryError("expected");
-            }
-          }
-        };
-        SimpleMemoryAllocatorImpl.createForUnitTest(listener, stats, 10, 950, MAX_SLAB_SIZE, factory);
-      } catch (OutOfMemoryError expected) {
-      }
-      assertTrue(listener.isClosed());
-      assertTrue(stats.isClosed());
-    }
-    {
-      NullOutOfOffHeapMemoryListener listener = new NullOutOfOffHeapMemoryListener();
-      NullOffHeapMemoryStats stats = new NullOffHeapMemoryStats();
-      SlabFactory factory = new SlabFactory() {
-        @Override
-        public Slab create(int size) {
-          return new SlabImpl(size);
-        }
-      };
-      MemoryAllocator ma = 
-        SimpleMemoryAllocatorImpl.createForUnitTest(listener, stats, 10, 950, 100, factory);
-      try {
-        assertFalse(listener.isClosed());
-        assertFalse(stats.isClosed());
-        ma.close();
-        assertTrue(listener.isClosed());
-        assertFalse(stats.isClosed());
-        listener = new NullOutOfOffHeapMemoryListener();
-        NullOffHeapMemoryStats stats2 = new NullOffHeapMemoryStats();
-        {
-          SlabImpl slab = new SlabImpl(1024);
-          try {
-            SimpleMemoryAllocatorImpl.createForUnitTest(listener, stats2, new SlabImpl[]{slab});
-          } catch (IllegalStateException expected) {
-            assertTrue("unexpected message: " + expected.getMessage(), 
-                expected.getMessage().equals("attempted to reuse existing off-heap memory even though new off-heap memory was allocated"));
-          } finally {
-            slab.free();
-          }
-          assertFalse(stats.isClosed());
-          assertTrue(listener.isClosed());
-          assertTrue(stats2.isClosed());
-        }
-        listener = new NullOutOfOffHeapMemoryListener();
-        stats2 = new NullOffHeapMemoryStats();
-        MemoryAllocator ma2 = SimpleMemoryAllocatorImpl.createForUnitTest(listener, stats2, 10, 950, 100, factory);
-        assertSame(ma, ma2);
-        assertTrue(stats.isClosed());
-        assertFalse(listener.isClosed());
-        assertFalse(stats2.isClosed());
-        stats = stats2;
-        ma.close();
-        assertTrue(listener.isClosed());
-        assertFalse(stats.isClosed());
-      } finally {
-        SimpleMemoryAllocatorImpl.freeOffHeapMemory();
-      }
-      assertTrue(stats.isClosed());
-    }
-  }
-  @Test
-  public void testBasics() {
-    int BATCH_SIZE = 1;
-    int TINY_MULTIPLE = FreeListManager.TINY_MULTIPLE;
-    int HUGE_MULTIPLE = FreeListManager.HUGE_MULTIPLE;
-    int perObjectOverhead = OffHeapStoredObject.HEADER_SIZE;
-    int maxTiny = FreeListManager.MAX_TINY-perObjectOverhead;
-    int minHuge = maxTiny+1;
-    int TOTAL_MEM = (maxTiny+perObjectOverhead)*BATCH_SIZE /*+ (maxBig+perObjectOverhead)*BATCH_SIZE*/ + round(TINY_MULTIPLE, minHuge+1+perObjectOverhead)*BATCH_SIZE + (TINY_MULTIPLE+perObjectOverhead)*BATCH_SIZE /*+ (MIN_BIG_SIZE+perObjectOverhead)*BATCH_SIZE*/ + round(TINY_MULTIPLE, minHuge+perObjectOverhead+1);
-    SlabImpl slab = new SlabImpl(TOTAL_MEM);
-    try {
-      SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new SlabImpl[]{slab});
-      assertEquals(TOTAL_MEM, ma.getFreeMemory());
-      assertEquals(TOTAL_MEM, ma.freeList.getFreeFragmentMemory());
-      assertEquals(0, ma.freeList.getFreeTinyMemory());
-      assertEquals(0, ma.freeList.getFreeHugeMemory());
-      StoredObject tinymc = ma.allocate(maxTiny);
-      assertEquals(TOTAL_MEM-round(TINY_MULTIPLE, maxTiny+perObjectOverhead), ma.getFreeMemory());
-      assertEquals(round(TINY_MULTIPLE, maxTiny+perObjectOverhead)*(BATCH_SIZE-1), ma.freeList.getFreeTinyMemory());
-      StoredObject hugemc = ma.allocate(minHuge);
-      assertEquals(TOTAL_MEM-round(TINY_MULTIPLE, minHuge+perObjectOverhead)/*-round(BIG_MULTIPLE, maxBig+perObjectOverhead)*/-round(TINY_MULTIPLE, maxTiny+perObjectOverhead), ma.getFreeMemory());
-      long freeSlab = ma.freeList.getFreeFragmentMemory();
-      long oldFreeHugeMemory = ma.freeList.getFreeHugeMemory();
-      assertEquals(round(TINY_MULTIPLE, minHuge+perObjectOverhead)*(BATCH_SIZE-1), oldFreeHugeMemory);
-      hugemc.release();
-      assertEquals(round(TINY_MULTIPLE, minHuge+perObjectOverhead), ma.freeList.getFreeHugeMemory()-oldFreeHugeMemory);
-      assertEquals(TOTAL_MEM/*-round(BIG_MULTIPLE, maxBig+perObjectOverhead)*/-round(TINY_MULTIPLE, maxTiny+perObjectOverhead), ma.getFreeMemory());
-      assertEquals(TOTAL_MEM-round(TINY_MULTIPLE, maxTiny+perObjectOverhead), ma.getFreeMemory());
-      long oldFreeTinyMemory = ma.freeList.getFreeTinyMemory();
-      tinymc.release();
-      assertEquals(round(TINY_MULTIPLE, maxTiny+perObjectOverhead), ma.freeList.getFreeTinyMemory()-oldFreeTinyMemory);
-      assertEquals(TOTAL_MEM, ma.getFreeMemory());
-      // now lets reallocate from the free lists
-      tinymc = ma.allocate(maxTiny);
-      assertEquals(oldFreeTinyMemory, ma.freeList.getFreeTinyMemory());
-      assertEquals(TOTAL_MEM-round(TINY_MULTIPLE, maxTiny+perObjectOverhead), ma.getFreeMemory());
-      hugemc = ma.allocate(minHuge);
-      assertEquals(oldFreeHugeMemory, ma.freeList.getFreeHugeMemory());
-      assertEquals(TOTAL_MEM-round(TINY_MULTIPLE, minHuge+perObjectOverhead)/*-round(BIG_MULTIPLE, maxBig+perObjectOverhead)*/-round(TINY_MULTIPLE, maxTiny+perObjectOverhead), ma.getFreeMemory());
-      hugemc.release();
-      assertEquals(round(TINY_MULTIPLE, minHuge+perObjectOverhead), ma.freeList.getFreeHugeMemory()-oldFreeHugeMemory);
-      assertEquals(TOTAL_MEM/*-round(BIG_MULTIPLE, maxBig+perObjectOverhead)*/-round(TINY_MULTIPLE, maxTiny+perObjectOverhead), ma.getFreeMemory());
-      assertEquals(TOTAL_MEM-round(TINY_MULTIPLE, maxTiny+perObjectOverhead), ma.getFreeMemory());
-      tinymc.release();
-      assertEquals(round(TINY_MULTIPLE, maxTiny+perObjectOverhead), ma.freeList.getFreeTinyMemory()-oldFreeTinyMemory);
-      assertEquals(TOTAL_MEM, ma.getFreeMemory());
-      // None of the reallocates should have come from the slab.
-      assertEquals(freeSlab, ma.freeList.getFreeFragmentMemory());
-      tinymc = ma.allocate(1);
-      assertEquals(round(TINY_MULTIPLE, 1+perObjectOverhead), tinymc.getSize());
-      assertEquals(freeSlab-(round(TINY_MULTIPLE, 1+perObjectOverhead)*BATCH_SIZE), ma.freeList.getFreeFragmentMemory());
-      freeSlab = ma.freeList.getFreeFragmentMemory();
-      tinymc.release();
-      assertEquals(round(TINY_MULTIPLE, maxTiny+perObjectOverhead)+(round(TINY_MULTIPLE, 1+perObjectOverhead)*BATCH_SIZE), ma.freeList.getFreeTinyMemory()-oldFreeTinyMemory);
-      
-      hugemc = ma.allocate(minHuge+1);
-      assertEquals(round(TINY_MULTIPLE, minHuge+1+perObjectOverhead), hugemc.getSize());
-      assertEquals(round(TINY_MULTIPLE, minHuge+perObjectOverhead)*(BATCH_SIZE-1), ma.freeList.getFreeHugeMemory());
-      hugemc.release();
-      assertEquals(round(TINY_MULTIPLE, minHuge+perObjectOverhead)*BATCH_SIZE, ma.freeList.getFreeHugeMemory());
-      hugemc = ma.allocate(minHuge);
-      assertEquals(round(TINY_MULTIPLE, minHuge+perObjectOverhead)*(BATCH_SIZE-1), ma.freeList.getFreeHugeMemory());
-      if (BATCH_SIZE > 1) {
-        StoredObject hugemc2 = ma.allocate(minHuge);
-        assertEquals(round(TINY_MULTIPLE, minHuge+perObjectOverhead)*(BATCH_SIZE-2), ma.freeList.getFreeHugeMemory());
-        hugemc2.release();
-        assertEquals(round(TINY_MULTIPLE, minHuge+perObjectOverhead)*(BATCH_SIZE-1), ma.freeList.getFreeHugeMemory());
-      }
-      hugemc.release();
-      assertEquals(round(TINY_MULTIPLE, minHuge+perObjectOverhead)*BATCH_SIZE, ma.freeList.getFreeHugeMemory());
-      // now that we do compaction the following allocate works.
-      hugemc = ma.allocate(minHuge + HUGE_MULTIPLE + HUGE_MULTIPLE-1);
-    } finally {
-      SimpleMemoryAllocatorImpl.freeOffHeapMemory();
-    }
-  }
-  
-  @Test
-  public void testChunkCreateDirectByteBuffer() {
-    SlabImpl slab = new SlabImpl(1024*1024);
-    try {
-      SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new SlabImpl[]{slab});
-      ByteBuffer bb = ByteBuffer.allocate(1024);
-      for (int i=0; i < 1024; i++) {
-        bb.put((byte) i);
-      }
-      bb.position(0);
-      OffHeapStoredObject c = (OffHeapStoredObject) ma.allocateAndInitialize(bb.array(), false, false);
-      assertEquals(1024, c.getDataSize());
-      if (!Arrays.equals(bb.array(), c.getRawBytes())) {
-        fail("arrays are not equal. Expected " + Arrays.toString(bb.array()) + " but found: " + Arrays.toString(c.getRawBytes()));
-      }
-      ByteBuffer dbb = c.createDirectByteBuffer();
-      assertEquals(true, dbb.isDirect());
-      assertEquals(bb, dbb);
-    } finally {
-      SimpleMemoryAllocatorImpl.freeOffHeapMemory();
-    }
-  }
-  
-  @Test
-  public void testDebugLog() {
-    SimpleMemoryAllocatorImpl.debugLog("test debug log", false);
-    SimpleMemoryAllocatorImpl.debugLog("test debug log", true);
-  }
-  @Test
-  public void testGetLostChunks() {
-    SlabImpl slab = new SlabImpl(1024*1024);
-    try {
-      SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new SlabImpl[]{slab});
-      assertEquals(Collections.emptyList(), ma.getLostChunks());
-    } finally {
-      SimpleMemoryAllocatorImpl.freeOffHeapMemory();
-    }
-  }
-  @Test
-  public void testFindSlab() {
-    final int SLAB_SIZE = 1024*1024;
-    SlabImpl slab = new SlabImpl(SLAB_SIZE);
-    try {
-      SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new SlabImpl[]{slab});
-      assertEquals(0, ma.findSlab(slab.getMemoryAddress()));
-      assertEquals(0, ma.findSlab(slab.getMemoryAddress()+SLAB_SIZE-1));
-      try {
-        ma.findSlab(slab.getMemoryAddress()-1);
-        fail("expected IllegalStateException");
-      } catch (IllegalStateException expected) {
-      }
-      try {
-        ma.findSlab(slab.getMemoryAddress()+SLAB_SIZE);
-        fail("expected IllegalStateException");
-      } catch (IllegalStateException expected) {
-      }
-    } finally {
-      SimpleMemoryAllocatorImpl.freeOffHeapMemory();
-    }
-  }
-  @Test
-  public void testValidateAddressAndSize() {
-    final int SLAB_SIZE = 1024*1024;
-    SlabImpl slab = new SlabImpl(SLAB_SIZE);
-    try {
-      SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new SlabImpl[]{slab});
-      try {
-        SimpleMemoryAllocatorImpl.validateAddress(0L);
-        fail("expected IllegalStateException");
-      } catch (IllegalStateException expected) {
-        assertEquals("Unexpected exception message: " + expected.getMessage(), true, expected.getMessage().contains("addr was smaller than expected"));
-      }
-      try {
-        SimpleMemoryAllocatorImpl.validateAddress(1L);
-        fail("expected IllegalStateException");
-      } catch (IllegalStateException expected) {
-        assertEquals("Unexpected exception message: " + expected.getMessage(), true, expected.getMessage().contains("Valid addresses must be in one of the following ranges:"));
-      }
-      SimpleMemoryAllocatorImpl.validateAddressAndSizeWithinSlab(slab.getMemoryAddress(), SLAB_SIZE, false);
-      SimpleMemoryAllocatorImpl.validateAddressAndSizeWithinSlab(slab.getMemoryAddress(), SLAB_SIZE, true);
-      SimpleMemoryAllocatorImpl.validateAddressAndSizeWithinSlab(slab.getMemoryAddress(), -1, true);
-      try {
-        SimpleMemoryAllocatorImpl.validateAddressAndSizeWithinSlab(slab.getMemoryAddress()-1, SLAB_SIZE, true);
-        fail("expected IllegalStateException");
-      } catch (IllegalStateException expected) {
-        assertEquals("Unexpected exception message: " + expected.getMessage(), true, expected.getMessage().equals(" address 0x" + Long.toString(slab.getMemoryAddress()-1, 16) + " does not address the original slab memory"));
-      }
-      try {
-        SimpleMemoryAllocatorImpl.validateAddressAndSizeWithinSlab(slab.getMemoryAddress(), SLAB_SIZE+1, true);
-        fail("expected IllegalStateException");
-      } catch (IllegalStateException expected) {
-        assertEquals("Unexpected exception message: " + expected.getMessage(), true, expected.getMessage().equals(" address 0x" + Long.toString(slab.getMemoryAddress()+SLAB_SIZE, 16) + " does not address the original slab memory"));
-      }
-    } finally {
-      SimpleMemoryAllocatorImpl.freeOffHeapMemory();
-    }
-  }
-  @Test
-  public void testMemoryInspection() {
-    final int SLAB_SIZE = 1024*1024;
-    SlabImpl slab = new SlabImpl(SLAB_SIZE);
-    try {
-      SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new SlabImpl[]{slab});
-      MemoryInspector inspector = ma.getMemoryInspector();
-      assertNotNull(inspector);
-      assertEquals(null, inspector.getFirstBlock());
-      assertEquals(Collections.emptyList(), inspector.getSnapshot());
-      assertEquals(Collections.emptyList(), inspector.getAllocatedBlocks());
-      assertEquals(null, inspector.getBlockAfter(null));
-      inspector.createSnapshot();
-      // call this twice for code coverage
-      inspector.createSnapshot();
-      try {
-        assertEquals(inspector.getAllBlocks(), inspector.getSnapshot());
-        MemoryBlock firstBlock = inspector.getFirstBlock();
-        assertNotNull(firstBlock);
-        assertEquals(1024*1024, firstBlock.getBlockSize());
-        assertEquals("N/A", firstBlock.getDataType());
-        assertEquals(-1, firstBlock.getFreeListId());
-        assertTrue(firstBlock.getAddress() > 0);
-        assertNull(firstBlock.getNextBlock());
-        assertEquals(0, firstBlock.getRefCount());
-        assertEquals(0, firstBlock.getSlabId());
-        assertEquals(MemoryBlock.State.UNUSED, firstBlock.getState());
-        assertFalse(firstBlock.isCompressed());
-        assertFalse(firstBlock.isSerialized());
-        assertEquals(null, inspector.getBlockAfter(firstBlock));
-      } finally {
-        inspector.clearSnapshot();
-      }
-    } finally {
-      SimpleMemoryAllocatorImpl.freeOffHeapMemory();
-    }
-  }
-
-  @Test
-  public void testClose() {
-    System.setProperty(SimpleMemoryAllocatorImpl.FREE_OFF_HEAP_MEMORY_PROPERTY, "false");
-    SlabImpl slab = new SlabImpl(1024*1024);
-    boolean freeSlab = true;
-    SlabImpl[] slabs = new SlabImpl[]{slab};
-    try {
-      SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), slabs);
-      ma.close();
-      ma.close();
-      System.setProperty(SimpleMemoryAllocatorImpl.FREE_OFF_HEAP_MEMORY_PROPERTY, "true");
-      try {
-        ma = SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), slabs);
-        ma.close();
-        freeSlab = false;
-        ma.close();
-      } finally {
-        System.clearProperty(SimpleMemoryAllocatorImpl.FREE_OFF_HEAP_MEMORY_PROPERTY);
-      }
-    } finally {
-      if (freeSlab) {
-        SimpleMemoryAllocatorImpl.freeOffHeapMemory();
-      }
-    }
-    
-  }
-  
-  @Test
-  public void testCompaction() {
-    final int perObjectOverhead = OffHeapStoredObject.HEADER_SIZE;
-    final int BIG_ALLOC_SIZE = 150000;
-    final int SMALL_ALLOC_SIZE = BIG_ALLOC_SIZE/2;
-    final int TOTAL_MEM = BIG_ALLOC_SIZE;
-    SlabImpl slab = new SlabImpl(TOTAL_MEM);
-    try {
-      SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new SlabImpl[]{slab});
-      StoredObject bmc = ma.allocate(BIG_ALLOC_SIZE-perObjectOverhead);
-      try {
-        StoredObject smc = ma.allocate(SMALL_ALLOC_SIZE-perObjectOverhead);
-        fail("Expected out of memory");
-      } catch (OutOfOffHeapMemoryException expected) {
-      }
-      bmc.release();
-      assertEquals(TOTAL_MEM, ma.freeList.getFreeMemory());
-      StoredObject smc1 = ma.allocate(SMALL_ALLOC_SIZE-perObjectOverhead);
-      StoredObject smc2 = ma.allocate(SMALL_ALLOC_SIZE-perObjectOverhead);
-      smc2.release();
-      assertEquals(TOTAL_MEM-SMALL_ALLOC_SIZE, ma.freeList.getFreeMemory());
-      try {
-        bmc = ma.allocate(BIG_ALLOC_SIZE-perObjectOverhead);
-        fail("Expected out of memory");
-      } catch (OutOfOffHeapMemoryException expected) {
-      }
-      smc1.release();
-      assertEquals(TOTAL_MEM, ma.freeList.getFreeMemory());
-      bmc = ma.allocate(BIG_ALLOC_SIZE-perObjectOverhead);
-      bmc.release();
-      assertEquals(TOTAL_MEM, ma.freeList.getFreeMemory());
-      ArrayList<StoredObject> mcs = new ArrayList<StoredObject>();
-      for (int i=0; i < BIG_ALLOC_SIZE/(8+perObjectOverhead); i++) {
-        mcs.add(ma.allocate(8));
-      }
-      checkMcs(mcs);
-      assertEquals(0, ma.freeList.getFreeMemory());
-      try {
-        ma.allocate(8);
-        fail("expected out of memory");
-      } catch (OutOfOffHeapMemoryException expected) {
-      }
-      mcs.remove(0).release(); // frees 8+perObjectOverhead
-      assertEquals(8+perObjectOverhead, ma.freeList.getFreeMemory());
-      mcs.remove(0).release(); // frees 8+perObjectOverhead
-      assertEquals((8+perObjectOverhead)*2, ma.freeList.getFreeMemory());
-      ma.allocate(16).release(); // allocates and frees 16+perObjectOverhead; still have perObjectOverhead
-      assertEquals((8+perObjectOverhead)*2, ma.freeList.getFreeMemory());
-      mcs.remove(0).release(); // frees 8+perObjectOverhead
-      assertEquals((8+perObjectOverhead)*3, ma.freeList.getFreeMemory());
-      mcs.remove(0).release(); // frees 8+perObjectOverhead
-      assertEquals((8+perObjectOverhead)*4, ma.freeList.getFreeMemory());
-      // At this point I should have 8*4 + perObjectOverhead*4 of free memory
-      ma.allocate(8*4+perObjectOverhead*3).release();
-      assertEquals((8+perObjectOverhead)*4, ma.freeList.getFreeMemory());
-      mcs.remove(0).release(); // frees 8+perObjectOverhead
-      assertEquals((8+perObjectOverhead)*5, ma.freeList.getFreeMemory());
-      // At this point I should have 8*5 + perObjectOverhead*5 of free memory
-      try {
-        ma.allocate((8*5+perObjectOverhead*4)+1);
-        fail("expected out of memory");
-      } catch (OutOfOffHeapMemoryException expected) {
-      }
-      mcs.remove(0).release(); // frees 8+perObjectOverhead
-      assertEquals((8+perObjectOverhead)*6, ma.freeList.getFreeMemory());
-      checkMcs(mcs);
-      // At this point I should have 8*6 + perObjectOverhead*6 of free memory
-      StoredObject mc24 = ma.allocate(24);
-      checkMcs(mcs);
-      assertEquals((8+perObjectOverhead)*6 - (24+perObjectOverhead), ma.freeList.getFreeMemory());
-      // At this point I should have 8*3 + perObjectOverhead*5 of free memory
-      StoredObject mc16 = ma.allocate(16);
-      checkMcs(mcs);
-      assertEquals((8+perObjectOverhead)*6 - (24+perObjectOverhead) - (16+perObjectOverhead), ma.freeList.getFreeMemory());
-      // At this point I should have 8*1 + perObjectOverhead*4 of free memory
-      mcs.add(ma.allocate(8));
-      checkMcs(mcs);
-      assertEquals((8+perObjectOverhead)*6 - (24+perObjectOverhead) - (16+perObjectOverhead) - (8+perObjectOverhead), ma.freeList.getFreeMemory());
-      // At this point I should have 8*0 + perObjectOverhead*3 of free memory
-      StoredObject mcDO = ma.allocate(perObjectOverhead*2);
-      checkMcs(mcs);
-      // At this point I should have 8*0 + perObjectOverhead*0 of free memory
-      assertEquals(0, ma.freeList.getFreeMemory());
-      try {
-        ma.allocate(1);
-        fail("expected out of memory");
-      } catch (OutOfOffHeapMemoryException expected) {
-      }
-      checkMcs(mcs);
-      assertEquals(0, ma.freeList.getFreeMemory());
-      mcDO.release();
-      assertEquals((perObjectOverhead*3), ma.freeList.getFreeMemory());
-      mcs.remove(mcs.size()-1).release();
-      assertEquals((perObjectOverhead*3)+(8+perObjectOverhead), ma.freeList.getFreeMemory());
-      mc16.release();
-      assertEquals((perObjectOverhead*3)+(8+perObjectOverhead)+(16+perObjectOverhead), ma.freeList.getFreeMemory());
-      mc24.release();
-      assertEquals((perObjectOverhead*3)+(8+perObjectOverhead)+(16+perObjectOverhead)+(24+perObjectOverhead), ma.freeList.getFreeMemory());
-      
-      long freeMem = ma.freeList.getFreeMemory();
-      for (StoredObject mc: mcs) {
-        mc.release();
-        assertEquals(freeMem+(8+perObjectOverhead), ma.freeList.getFreeMemory());
-        freeMem += (8+perObjectOverhead);
-      }
-      mcs.clear();
-      assertEquals(TOTAL_MEM, ma.freeList.getFreeMemory());
-      bmc = ma.allocate(BIG_ALLOC_SIZE-perObjectOverhead);
-      bmc.release();
-    } finally {
-      SimpleMemoryAllocatorImpl.freeOffHeapMemory();
-    }
-  }
-  
-  long expectedMemoryUsage;
-  boolean memoryUsageEventReceived;
-  @Test
-  public void testUsageEventListener() {
-    final int perObjectOverhead = OffHeapStoredObject.HEADER_SIZE;
-    final int SMALL_ALLOC_SIZE = 1000;
-    SlabImpl slab = new SlabImpl(3000);
-    try {
-      SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new SlabImpl[]{slab});
-      MemoryUsageListener listener = new MemoryUsageListener() {
-        @Override
-        public void updateMemoryUsed(final long bytesUsed) {
-          SimpleMemoryAllocatorJUnitTest.this.memoryUsageEventReceived = true;
-          assertEquals(SimpleMemoryAllocatorJUnitTest.this.expectedMemoryUsage, bytesUsed);
-        }
-      };
-      ma.addMemoryUsageListener(listener);
-      
-      this.expectedMemoryUsage = SMALL_ALLOC_SIZE;
-      this.memoryUsageEventReceived = false;
-      StoredObject smc = ma.allocate(SMALL_ALLOC_SIZE-perObjectOverhead);
-      assertEquals(true, this.memoryUsageEventReceived);
-      
-      this.expectedMemoryUsage = SMALL_ALLOC_SIZE * 2;
-      this.memoryUsageEventReceived = false;
-      smc = ma.allocate(SMALL_ALLOC_SIZE-perObjectOverhead);
-      assertEquals(true, this.memoryUsageEventReceived);
-      
-      MemoryUsageListener unaddedListener = new MemoryUsageListener() {
-        @Override
-        public void updateMemoryUsed(final long bytesUsed) {
-          throw new IllegalStateException("Should never be called");
-        }
-      };
-      ma.removeMemoryUsageListener(unaddedListener);
-      
-      ma.removeMemoryUsageListener(listener);
-      
-      ma.removeMemoryUsageListener(unaddedListener);
-
-      this.expectedMemoryUsage = SMALL_ALLOC_SIZE * 2;
-      this.memoryUsageEventReceived = false;
-      smc = ma.allocate(SMALL_ALLOC_SIZE-perObjectOverhead);
-      assertEquals(false, this.memoryUsageEventReceived);
-      
-    } finally {
-      SimpleMemoryAllocatorImpl.freeOffHeapMemory();
-    }
-  }
-  private void checkMcs(ArrayList<StoredObject> mcs) {
-    for (StoredObject mc: mcs) {
-      assertEquals(8+8, mc.getSize());
-    }
-  }
-  
-  @Test
-  public void testOutOfOffHeapMemory() {
-    final int perObjectOverhead = OffHeapStoredObject.HEADER_SIZE;
-    final int BIG_ALLOC_SIZE = 150000;
-    final int SMALL_ALLOC_SIZE = BIG_ALLOC_SIZE/2;
-    final int TOTAL_MEM = BIG_ALLOC_SIZE;
-    final SlabImpl slab = new SlabImpl(TOTAL_MEM);
-    final AtomicReference<OutOfOffHeapMemoryException> ooom = new AtomicReference<OutOfOffHeapMemoryException>();
-    final OutOfOffHeapMemoryListener oooml = new OutOfOffHeapMemoryListener() {
-      @Override
-      public void outOfOffHeapMemory(OutOfOffHeapMemoryException cause) {
-        ooom.set(cause);
-      }
-      @Override
-      public void close() {
-      }
-    };
-    try {
-      SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.createForUnitTest(oooml, new NullOffHeapMemoryStats(), new SlabImpl[]{slab});
-      // make a big allocation
-      StoredObject bmc = ma.allocate(BIG_ALLOC_SIZE-perObjectOverhead);
-      assertNull(ooom.get());
-      // drive the ma to ooom with small allocations
-      try {
-        StoredObject smc = ma.allocate(SMALL_ALLOC_SIZE-perObjectOverhead);
-        fail("Expected out of memory");
-      } catch (OutOfOffHeapMemoryException expected) {
-      }
-      assertNotNull(ooom.get());
-      assertTrue(ooom.get().getMessage().contains("Out of off-heap memory. Could not allocate size of "));
-    } finally {
-      SimpleMemoryAllocatorImpl.freeOffHeapMemory();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/82faa8af/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/TinyMemoryBlockJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/TinyMemoryBlockJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/TinyMemoryBlockJUnitTest.java
index d444865..681bec0 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/TinyMemoryBlockJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/TinyMemoryBlockJUnitTest.java
@@ -39,7 +39,7 @@ import com.gemstone.gemfire.test.junit.categories.UnitTest;
 @Category(UnitTest.class)
 public class TinyMemoryBlockJUnitTest {
 
-  private SimpleMemoryAllocatorImpl ma;
+  private MemoryAllocatorImpl ma;
   private OutOfOffHeapMemoryListener ooohml;
   private OffHeapMemoryStats stats;
 
@@ -50,7 +50,7 @@ public class TinyMemoryBlockJUnitTest {
   };
 
   private static class TestableFreeListManager extends FreeListManager {
-    TestableFreeListManager(SimpleMemoryAllocatorImpl ma, final Slab[] slabs) {
+    TestableFreeListManager(MemoryAllocatorImpl ma, final Slab[] slabs) {
       super (ma, slabs);
     }
   }
@@ -73,12 +73,12 @@ public class TinyMemoryBlockJUnitTest {
   public void setUp() throws Exception {
     ooohml = mock(OutOfOffHeapMemoryListener.class);
     stats = mock(OffHeapMemoryStats.class);
-    ma = (SimpleMemoryAllocatorImpl) SimpleMemoryAllocatorImpl.createForUnitTest(ooohml, stats, slabs);
+    ma = (MemoryAllocatorImpl) MemoryAllocatorImpl.createForUnitTest(ooohml, stats, slabs);
   }
 
   @After
   public void tearDown() throws Exception {
-    SimpleMemoryAllocatorImpl.freeOffHeapMemory();
+    MemoryAllocatorImpl.freeOffHeapMemory();
   }
 
   protected Object getValue() {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/82faa8af/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/TxReleasesOffHeapOnCloseJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/TxReleasesOffHeapOnCloseJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/TxReleasesOffHeapOnCloseJUnitTest.java
index 2bc5759..d8999fe 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/TxReleasesOffHeapOnCloseJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/TxReleasesOffHeapOnCloseJUnitTest.java
@@ -49,7 +49,7 @@ public class TxReleasesOffHeapOnCloseJUnitTest {
   @Test
   public void testTxReleasesOffHeapOnClose() {
     createCache();
-    SimpleMemoryAllocatorImpl sma = SimpleMemoryAllocatorImpl.getAllocator();
+    MemoryAllocatorImpl sma = MemoryAllocatorImpl.getAllocator();
     RegionFactory rf = cache.createRegionFactory();
     rf.setOffHeap(true);
     Region r = rf.create("testTxReleasesOffHeapOnClose");

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/82faa8af/geode-core/src/test/java/com/gemstone/gemfire/pdx/OffHeapByteBufferByteSourceJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/pdx/OffHeapByteBufferByteSourceJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/pdx/OffHeapByteBufferByteSourceJUnitTest.java
index 8380f57..28e0439 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/pdx/OffHeapByteBufferByteSourceJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/pdx/OffHeapByteBufferByteSourceJUnitTest.java
@@ -23,7 +23,7 @@ import java.nio.ByteBuffer;
 import org.junit.experimental.categories.Category;
 
 import com.gemstone.gemfire.internal.offheap.OffHeapStoredObject;
-import com.gemstone.gemfire.internal.offheap.SimpleMemoryAllocatorImpl;
+import com.gemstone.gemfire.internal.offheap.MemoryAllocatorImpl;
 import com.gemstone.gemfire.internal.offheap.StoredObject;
 import com.gemstone.gemfire.internal.tcp.ByteBufferInputStream.ByteSource;
 import com.gemstone.gemfire.internal.tcp.ByteBufferInputStream.ByteSourceFactory;
@@ -34,7 +34,7 @@ public class OffHeapByteBufferByteSourceJUnitTest extends OffHeapByteSourceJUnit
   
   @Override
   protected ByteSource createByteSource(byte[] bytes) {
-    StoredObject so = SimpleMemoryAllocatorImpl.getAllocator().allocateAndInitialize(bytes, false, false);
+    StoredObject so = MemoryAllocatorImpl.getAllocator().allocateAndInitialize(bytes, false, false);
     if (so instanceof OffHeapStoredObject) {
       OffHeapStoredObject c = (OffHeapStoredObject) so;
       ByteBuffer bb = c.createDirectByteBuffer();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/82faa8af/geode-core/src/test/java/com/gemstone/gemfire/pdx/OffHeapByteSourceJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/pdx/OffHeapByteSourceJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/pdx/OffHeapByteSourceJUnitTest.java
index 2111f79..7946b7e 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/pdx/OffHeapByteSourceJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/pdx/OffHeapByteSourceJUnitTest.java
@@ -23,7 +23,7 @@ import org.junit.experimental.categories.Category;
 import com.gemstone.gemfire.internal.offheap.OffHeapStoredObject;
 import com.gemstone.gemfire.internal.offheap.NullOffHeapMemoryStats;
 import com.gemstone.gemfire.internal.offheap.NullOutOfOffHeapMemoryListener;
-import com.gemstone.gemfire.internal.offheap.SimpleMemoryAllocatorImpl;
+import com.gemstone.gemfire.internal.offheap.MemoryAllocatorImpl;
 import com.gemstone.gemfire.internal.offheap.StoredObject;
 import com.gemstone.gemfire.internal.offheap.SlabImpl;
 import com.gemstone.gemfire.internal.tcp.ByteBufferInputStream.ByteSource;
@@ -36,12 +36,12 @@ public class OffHeapByteSourceJUnitTest extends ByteSourceJUnitTest {
 
   @Before
   public void setUp() throws Exception {
-    SimpleMemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new SlabImpl[]{new SlabImpl(1024*1024)});
+    MemoryAllocatorImpl.createForUnitTest(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new SlabImpl[]{new SlabImpl(1024*1024)});
   }
 
   @After
   public void tearDown() throws Exception {
-    SimpleMemoryAllocatorImpl.freeOffHeapMemory();
+    MemoryAllocatorImpl.freeOffHeapMemory();
   }
 
   @Override
@@ -51,7 +51,7 @@ public class OffHeapByteSourceJUnitTest extends ByteSourceJUnitTest {
   
   @Override
   protected ByteSource createByteSource(byte[] bytes) {
-    StoredObject so = SimpleMemoryAllocatorImpl.getAllocator().allocateAndInitialize(bytes, false, false);
+    StoredObject so = MemoryAllocatorImpl.getAllocator().allocateAndInitialize(bytes, false, false);
     if (so instanceof OffHeapStoredObject) {
       // bypass the factory to make sure that OffHeapByteSource is tested
       return new OffHeapByteSource(so);