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:57 UTC

[27/50] [abbrv] incubator-geode git commit: GEODE-1017: change off-memory from compaction to defragmentation

GEODE-1017: change off-memory from compaction to defragmentation

 Replaced references of compaction with `defragmentation`, to better
 convey the intent behind the operation.


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

Branch: refs/heads/feature/GEODE-17-3
Commit: 155f87d5ff3b8b5936376c0dce7aba4e214b3672
Parents: 67f0e2c
Author: Sai Boorlagadda <sb...@pivotal.io>
Authored: Wed Mar 16 13:07:49 2016 -0700
Committer: Sai Boorlagadda <sb...@pivotal.io>
Committed: Thu Mar 17 10:07:21 2016 -0700

----------------------------------------------------------------------
 .../gemfire/internal/offheap/Fragment.java      |  2 +-
 .../internal/offheap/FreeListManager.java       | 36 +++++------
 .../internal/offheap/MemoryAllocatorImpl.java   |  2 +-
 .../internal/offheap/OffHeapMemoryStats.java    |  8 +--
 .../internal/offheap/OffHeapStorage.java        | 58 ++++++++---------
 .../internal/beans/MemberMBeanBridge.java       |  2 +-
 .../internal/offheap/FreeListManagerTest.java   | 66 ++++++++++----------
 .../MemoryAllocatorFillPatternJUnitTest.java    | 24 +++----
 .../offheap/MemoryAllocatorJUnitTest.java       |  4 +-
 .../offheap/NullOffHeapMemoryStats.java         |  8 +--
 .../internal/offheap/OffHeapRegionBase.java     |  4 +-
 .../offheap/OffHeapStorageJUnitTest.java        | 18 +++---
 12 files changed, 116 insertions(+), 116 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/155f87d5/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/Fragment.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/Fragment.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/Fragment.java
index 0ea6cf8..b4f827b 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/Fragment.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/Fragment.java
@@ -24,7 +24,7 @@ import java.util.concurrent.atomic.AtomicIntegerFieldUpdater;
  * at the end. The freeIdx keeps track of the first byte of free memory in
  * the fragment.
  * The base memory address and the total size of a fragment never change.
- * During compaction fragments go away and are recreated.
+ * During defragmentation fragments go away and are recreated.
  * 
  * @author darrel
  *

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/155f87d5/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/FreeListManager.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/FreeListManager.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/FreeListManager.java
index c943a7e..6de7be9 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/FreeListManager.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/FreeListManager.java
@@ -223,7 +223,7 @@ public class FreeListManager {
           return result;
         }
       }
-    } while (compact(chunkSize));
+    } while (defragment(chunkSize));
     // We tried all the fragments and didn't find any free memory.
     logOffHeapState(chunkSize);
     final OutOfOffHeapMemoryException failure = new OutOfOffHeapMemoryException("Out of off-heap memory. Could not allocate size of " + chunkSize);
@@ -240,7 +240,7 @@ public class FreeListManager {
 
   void logOffHeapState(Logger lw, int chunkSize) {
     OffHeapMemoryStats stats = this.ma.getStats();
-    lw.info("OutOfOffHeapMemory allocating size of " + chunkSize + ". allocated=" + this.allocatedSize.get() + " compactions=" + this.compactCount.get() + " objects=" + stats.getObjects() + " free=" + stats.getFreeMemory() + " fragments=" + stats.getFragments() + " largestFragment=" + stats.getLargestFragment() + " fragmentation=" + stats.getFragmentation());
+    lw.info("OutOfOffHeapMemory allocating size of " + chunkSize + ". allocated=" + this.allocatedSize.get() + " defragmentations=" + this.defragmentationCount.get() + " objects=" + stats.getObjects() + " free=" + stats.getFreeMemory() + " fragments=" + stats.getFragments() + " largestFragment=" + stats.getLargestFragment() + " fragmentation=" + stats.getFragmentation());
     logFragmentState(lw);
     logTinyState(lw);
     logHugeState(lw);
@@ -268,7 +268,7 @@ public class FreeListManager {
     }
   }
 
-  protected final AtomicInteger compactCount = new AtomicInteger();
+  protected final AtomicInteger defragmentationCount = new AtomicInteger();
   /*
    * Set this to "true" to perform data integrity checks on allocated and reused Chunks.  This may clobber 
    * performance so turn on only when necessary.
@@ -302,24 +302,24 @@ public class FreeListManager {
   }
   public final static int MAX_TINY = TINY_MULTIPLE*TINY_FREE_LIST_COUNT;
   /**
-   * Compacts memory and returns true if enough memory to allocate chunkSize
+   * Defragments memory and returns true if enough memory to allocate chunkSize
    * is freed. Otherwise returns false;
    * TODO OFFHEAP: what should be done about contiguous chunks that end up being bigger than 2G?
    * Currently if we are given slabs bigger than 2G or that just happen to be contiguous and add
-   * up to 2G then the compactor may unify them together into a single Chunk and our 32-bit chunkSize
+   * up to 2G then the FreeListManager may unify them together into a single Chunk and our 32-bit chunkSize
    * field will overflow. This code needs to detect this and just create a chunk of 2G and then start
    * a new one.
    * Or to prevent it from happening we could just check the incoming slabs and throw away a few bytes
    * to keep them from being contiguous.
    */
-  boolean compact(int chunkSize) {
-    final long startCompactionTime = this.ma.getStats().startCompaction();
-    final int countPreSync = this.compactCount.get();
-    afterCompactCountFetched();
+  boolean defragment(int chunkSize) {
+    final long startDefragmentationTime = this.ma.getStats().startDefragmentation();
+    final int countPreSync = this.defragmentationCount.get();
+    afterDefragmentationCountFetched();
     try {
       synchronized (this) {
-        if (this.compactCount.get() != countPreSync) {
-          // someone else did a compaction while we waited on the sync.
+        if (this.defragmentationCount.get() != countPreSync) {
+          // someone else did a defragmentation while we waited on the sync.
           // So just return true causing the caller to retry the allocation.
           return true;
         }
@@ -432,8 +432,8 @@ public class FreeListManager {
 
         fillFragments();
 
-        // Signal any waiters that a compaction happened.
-        this.compactCount.incrementAndGet();
+        // Signal any waiters that a defragmentation happened.
+        this.defragmentationCount.incrementAndGet();
 
         this.ma.getStats().setLargestFragment(largestFragment);
         this.ma.getStats().setFragments(tmp.size());        
@@ -442,14 +442,14 @@ public class FreeListManager {
         return result;
       } // sync
     } finally {
-      this.ma.getStats().endCompaction(startCompactionTime);
+      this.ma.getStats().endDefragmentation(startDefragmentationTime);
     }
   }
 
   /**
    * Unit tests override this method to get better test coverage
    */
-  protected void afterCompactCountFetched() {
+  protected void afterDefragmentationCountFetched() {
   }
   
   static void verifyOffHeapAlignment(int tinyMultiple) {
@@ -519,7 +519,7 @@ public class FreeListManager {
         diff = f.getSize() - offset;
       } while (diff >= OffHeapStoredObject.MIN_CHUNK_SIZE && !f.allocate(offset, offset+diff));
       if (diff < OffHeapStoredObject.MIN_CHUNK_SIZE) {
-        // If diff > 0 then that memory will be lost during compaction.
+        // If diff > 0 then that memory will be lost during defragmentation.
         // This should never happen since we keep the sizes rounded
         // based on MIN_CHUNK_SIZE.
         assert diff == 0;
@@ -531,7 +531,7 @@ public class FreeListManager {
       result.offer(chunkAddr);
     }
     // All the fragments have been turned in to chunks so now clear them
-    // The compaction will create new fragments.
+    // The defragmentation will create new fragments.
     this.fragmentList.clear();
     if (!result.isEmpty()) {
       l.add(result);
@@ -567,7 +567,7 @@ public class FreeListManager {
     try {
       fragment = this.fragmentList.get(fragIdx);
     } catch (IndexOutOfBoundsException ignore) {
-      // A concurrent compaction can cause this.
+      // A concurrent defragmentation can cause this.
       return null;
     }
     boolean retryFragment;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/155f87d5/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/MemoryAllocatorImpl.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/MemoryAllocatorImpl.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/MemoryAllocatorImpl.java
index 2050dd4..9135efd 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/MemoryAllocatorImpl.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/MemoryAllocatorImpl.java
@@ -47,7 +47,7 @@ import com.gemstone.gemfire.internal.offheap.annotations.Unretained;
  * We also keep lists of any chunk that have been allocated and freed.
  * An allocation will always try to find a chunk in a free list that is a close fit to the requested size.
  * If no close fits exist then it allocates the next slice from the front of one the original large chunks.
- * If we can not find enough free memory then all the existing free memory is compacted.
+ * If we can not find enough free memory then all the existing free memory is defragmented.
  * If we still do not have enough to make the allocation an exception is thrown.
  * 
  * @author darrel

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/155f87d5/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapMemoryStats.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapMemoryStats.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapMemoryStats.java
index 790e43d..f19d509 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapMemoryStats.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapMemoryStats.java
@@ -33,8 +33,8 @@ public interface OffHeapMemoryStats {
   public void incReads();
   public void setFragments(long value);
   public void setLargestFragment(int value);
-  public long startCompaction();
-  public void endCompaction(long start);
+  public long startDefragmentation();
+  public void endDefragmentation(long start);
   public void setFragmentation(int value);
   
   public long getFreeMemory();
@@ -42,11 +42,11 @@ public interface OffHeapMemoryStats {
   public long getUsedMemory();
   public long getReads();
   public int getObjects();
-  public int getCompactions();
+  public int getDefragmentations();
   public long getFragments();
   public int getLargestFragment();
   public int getFragmentation();
-  public long getCompactionTime();
+  public long getDefragmentationTime();
   
   public Statistics getStats();
   public void close();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/155f87d5/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapStorage.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapStorage.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapStorage.java
index 2bdcfba..0976259 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapStorage.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapStorage.java
@@ -56,10 +56,10 @@ public class OffHeapStorage implements OffHeapMemoryStats {
   private static final int usedMemoryId;
   private static final int objectsId;
   private static final int readsId;
-  private static final int compactionsId;
+  private static final int defragmentationId;
   private static final int fragmentsId;
   private static final int largestFragmentId;
-  private static final int compactionTimeId;
+  private static final int defragmentationTimeId;
   private static final int fragmentationId;
   // NOTE!!!! When adding new stats make sure and update the initialize method on this class
   
@@ -68,19 +68,19 @@ public class OffHeapStorage implements OffHeapMemoryStats {
     final StatisticsTypeFactory f = StatisticsTypeFactoryImpl.singleton();
     
     final String usedMemoryDesc = "The amount of off-heap memory, in bytes, that is being used to store data.";
-    final String compactionsDesc = "The total number of times off-heap memory has been compacted.";
-    final String compactionTimeDesc = "The total time spent compacting off-heap memory.";
-    final String fragmentationDesc = "The percentage of off-heap free memory that is fragmented.  Updated every time a compaction is performed.";
-    final String fragmentsDesc = "The number of fragments of free off-heap memory. Updated every time a compaction is done.";
+    final String defragmentationDesc = "The total number of times off-heap memory has been defragmented.";
+    final String defragmentationTimeDesc = "The total time spent defragmenting off-heap memory.";
+    final String fragmentationDesc = "The percentage of off-heap free memory that is fragmented.  Updated every time a defragmentation is performed.";
+    final String fragmentsDesc = "The number of fragments of free off-heap memory. Updated every time a defragmentation is done.";
     final String freeMemoryDesc = "The amount of off-heap memory, in bytes, that is not being used.";
-    final String largestFragmentDesc = "The largest fragment of memory found by the last compaction of off heap memory. Updated every time a compaction is done.";
+    final String largestFragmentDesc = "The largest fragment of memory found by the last defragmentation of off heap memory. Updated every time a defragmentation is done.";
     final String objectsDesc = "The number of objects stored in off-heap memory.";
     final String readsDesc = "The total number of reads of off-heap memory. Only reads of a full object increment this statistic. If only a part of the object is read this statistic is not incremented.";
     final String maxMemoryDesc = "The maximum amount of off-heap memory, in bytes. This is the amount of memory allocated at startup and does not change.";
 
     final String usedMemory = "usedMemory";
-    final String compactions = "compactions";
-    final String compactionTime = "compactionTime";
+    final String defragmentations = "defragmentations";
+    final String defragmentationTime = "defragmentationTime";
     final String fragmentation = "fragmentation";
     final String fragments = "fragments";
     final String freeMemory = "freeMemory";
@@ -94,8 +94,8 @@ public class OffHeapStorage implements OffHeapMemoryStats {
         statsTypeDescription,
         new StatisticDescriptor[] {
             f.createLongGauge(usedMemory, usedMemoryDesc, "bytes"),
-            f.createIntCounter(compactions, compactionsDesc, "compactions"),
-            f.createLongCounter(compactionTime, compactionTimeDesc, "nanoseconds", false),
+            f.createIntCounter(defragmentations, defragmentationDesc, "defragmentations"),
+            f.createLongCounter(defragmentationTime, defragmentationTimeDesc, "nanoseconds", false),
             f.createIntGauge(fragmentation, fragmentationDesc, "percentage"),
             f.createLongGauge(fragments, fragmentsDesc, "fragments"),
             f.createLongGauge(freeMemory, freeMemoryDesc, "bytes"),
@@ -107,8 +107,8 @@ public class OffHeapStorage implements OffHeapMemoryStats {
     );
     
     usedMemoryId = statsType.nameToId(usedMemory);
-    compactionsId = statsType.nameToId(compactions);
-    compactionTimeId = statsType.nameToId(compactionTime);
+    defragmentationId = statsType.nameToId(defragmentations);
+    defragmentationTimeId = statsType.nameToId(defragmentationTime);
     fragmentationId = statsType.nameToId(fragmentation);
     fragmentsId = statsType.nameToId(fragments);
     freeMemoryId = statsType.nameToId(freeMemory);
@@ -281,13 +281,13 @@ public class OffHeapStorage implements OffHeapMemoryStats {
     return this.stats.getLong(readsId);
   }
 
-  private void incCompactions() {
-    this.stats.incInt(compactionsId, 1);
+  private void incDefragmentations() {
+    this.stats.incInt(defragmentationId, 1);
   }
 
   @Override
-  public int getCompactions() {
-    return this.stats.getInt(compactionsId);
+  public int getDefragmentations() {
+    return this.stats.getInt(defragmentationId);
   }
 
   @Override
@@ -311,21 +311,21 @@ public class OffHeapStorage implements OffHeapMemoryStats {
   }
   
   @Override
-  public long startCompaction() {
+  public long startDefragmentation() {
     return DistributionStats.getStatTime();
   }
   
   @Override
-  public void endCompaction(long start) {
-    incCompactions();
+  public void endDefragmentation(long start) {
+    incDefragmentations();
     if (DistributionStats.enableClockStats) {
-      stats.incLong(compactionTimeId, DistributionStats.getStatTime()-start);
+      stats.incLong(defragmentationTimeId, DistributionStats.getStatTime()-start);
     }
   }  
   
   @Override
-  public long getCompactionTime() {
-    return stats.getLong(compactionTimeId);
+  public long getDefragmentationTime() {
+    return stats.getLong(defragmentationTimeId);
   }
 
   @Override
@@ -354,21 +354,21 @@ public class OffHeapStorage implements OffHeapMemoryStats {
     setUsedMemory(oldStats.getUsedMemory());
     setObjects(oldStats.getObjects());
     setReads(oldStats.getReads());
-    setCompactions(oldStats.getCompactions());
+    setDefragmentations(oldStats.getDefragmentations());
     setFragments(oldStats.getFragments());
     setLargestFragment(oldStats.getLargestFragment());
-    setCompactionTime(oldStats.getCompactionTime());
+    setDefragmentationTime(oldStats.getDefragmentationTime());
     setFragmentation(oldStats.getFragmentation());
     
     oldStats.close();
   }
 
-  private void setCompactionTime(long value) {
-    stats.setLong(compactionTimeId, value);
+  private void setDefragmentationTime(long value) {
+    stats.setLong(defragmentationTimeId, value);
   }
 
-  private void setCompactions(int value) {
-    this.stats.setInt(compactionsId, value);
+  private void setDefragmentations(int value) {
+    this.stats.setInt(defragmentationId, value);
   }
 
   private void setReads(long value) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/155f87d5/geode-core/src/main/java/com/gemstone/gemfire/management/internal/beans/MemberMBeanBridge.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/beans/MemberMBeanBridge.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/beans/MemberMBeanBridge.java
index 61e328d..6c7a44f 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/beans/MemberMBeanBridge.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/beans/MemberMBeanBridge.java
@@ -1919,7 +1919,7 @@ public class MemberMBeanBridge {
     OffHeapMemoryStats stats = getOffHeapStats();
     
     if(null != stats) {
-      compactionTime = stats.getCompactionTime();
+      compactionTime = stats.getDefragmentationTime();
     }
     
     return compactionTime;            

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/155f87d5/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/FreeListManagerTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/FreeListManagerTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/FreeListManagerTest.java
index 950d90b..28ff3ac 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/FreeListManagerTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/FreeListManagerTest.java
@@ -278,7 +278,7 @@ public class FreeListManagerTest {
   }
   
   @Test
-  public void compactWithLargeChunkSizeReturnsFalse() {
+  public void defragmentWithLargeChunkSizeReturnsFalse() {
     int SMALL_SLAB = 16;
     int MEDIUM_SLAB = 128;
     Slab slab = new SlabImpl(DEFAULT_SLAB_SIZE);
@@ -295,12 +295,12 @@ public class FreeListManagerTest {
     for (OffHeapStoredObject c: chunks) {
       OffHeapStoredObject.release(c.getAddress(), this.freeListManager);
     }
-    this.freeListManager.firstCompact = false;
-    assertThat(this.freeListManager.compact(DEFAULT_SLAB_SIZE+1)).isFalse();
+    this.freeListManager.firstDefragmentation = false;
+    assertThat(this.freeListManager.defragment(DEFAULT_SLAB_SIZE+1)).isFalse();
   }
   
   @Test
-  public void compactWithChunkSizeOfMaxSlabReturnsTrue() {
+  public void defragmentWithChunkSizeOfMaxSlabReturnsTrue() {
     int SMALL_SLAB = 16;
     int MEDIUM_SLAB = 128;
     Slab slab = new SlabImpl(DEFAULT_SLAB_SIZE);
@@ -318,12 +318,12 @@ public class FreeListManagerTest {
       OffHeapStoredObject.release(c.getAddress(), this.freeListManager);
     }
     
-    assertThat(this.freeListManager.compact(DEFAULT_SLAB_SIZE)).isTrue();
+    assertThat(this.freeListManager.defragment(DEFAULT_SLAB_SIZE)).isTrue();
     //assertThat(this.freeListManager.getFragmentList()).hasSize(4); // TODO intermittently fails because Fragments may be merged
   }
   
   @Test
-  public void compactWithLiveChunks() {
+  public void defragmentWithLiveChunks() {
     int SMALL_SLAB = 16;
     int MEDIUM_SLAB = 128;
     Slab slab = new SlabImpl(DEFAULT_SLAB_SIZE);
@@ -341,22 +341,22 @@ public class FreeListManagerTest {
       OffHeapStoredObject.release(c.getAddress(), this.freeListManager);
     }
     
-    assertThat(this.freeListManager.compact(DEFAULT_SLAB_SIZE/2)).isTrue();
+    assertThat(this.freeListManager.defragment(DEFAULT_SLAB_SIZE/2)).isTrue();
   }
   
   @Test
-  public void compactAfterAllocatingAll() {
+  public void defragmentAfterAllocatingAll() {
     setUpSingleSlabManager();
     OffHeapStoredObject c = freeListManager.allocate(DEFAULT_SLAB_SIZE-8);
-    this.freeListManager.firstCompact = false;
-    assertThat(this.freeListManager.compact(1)).isFalse();
-    // call compact twice for extra code coverage
-    assertThat(this.freeListManager.compact(1)).isFalse();
+    this.freeListManager.firstDefragmentation = false;
+    assertThat(this.freeListManager.defragment(1)).isFalse();
+    // call defragmen twice for extra code coverage
+    assertThat(this.freeListManager.defragment(1)).isFalse();
     assertThat(this.freeListManager.getFragmentList()).isEmpty();
   }
   
   @Test
-  public void afterAllocatingAllOneSizeCompactToAllocateDifferentSize() {
+  public void afterAllocatingAllOneSizeDefragmentToAllocateDifferentSize() {
     setUpSingleSlabManager();
     ArrayList<OffHeapStoredObject> chunksToFree = new ArrayList<>();
     ArrayList<OffHeapStoredObject> chunksToFreeLater = new ArrayList<>();
@@ -394,42 +394,42 @@ public class FreeListManagerTest {
     OffHeapStoredObject.release(c4.getAddress(), freeListManager);
     OffHeapStoredObject.release(mediumChunk1.getAddress(), freeListManager);
     OffHeapStoredObject.release(mediumChunk2.getAddress(), freeListManager);
-    this.freeListManager.firstCompact = false;
-    assertThat(freeListManager.compact(DEFAULT_SLAB_SIZE-(ALLOCATE_COUNT*32))).isFalse();
+    this.freeListManager.firstDefragmentation = false;
+    assertThat(freeListManager.defragment(DEFAULT_SLAB_SIZE-(ALLOCATE_COUNT*32))).isFalse();
     for (int i=0; i < ((256*2)/96); i++) {
       OffHeapStoredObject.release(chunksToFreeLater.get(i).getAddress(), freeListManager);
     }
-    assertThat(freeListManager.compact(DEFAULT_SLAB_SIZE-(ALLOCATE_COUNT*32))).isTrue();
+    assertThat(freeListManager.defragment(DEFAULT_SLAB_SIZE-(ALLOCATE_COUNT*32))).isTrue();
   }
   
   @Test
-  public void afterAllocatingAndFreeingCompact() {
+  public void afterAllocatingAndFreeingDefragment() {
     int slabSize = 1024*3;
     setUpSingleSlabManager(slabSize);
     OffHeapStoredObject bigChunk1 = freeListManager.allocate(slabSize/3-8);
     OffHeapStoredObject bigChunk2 = freeListManager.allocate(slabSize/3-8);
     OffHeapStoredObject bigChunk3 = freeListManager.allocate(slabSize/3-8);
-    this.freeListManager.firstCompact = false;
-    assertThat(freeListManager.compact(1)).isFalse();
+    this.freeListManager.firstDefragmentation = false;
+    assertThat(freeListManager.defragment(1)).isFalse();
     OffHeapStoredObject.release(bigChunk3.getAddress(), freeListManager);
     OffHeapStoredObject.release(bigChunk2.getAddress(), freeListManager);
     OffHeapStoredObject.release(bigChunk1.getAddress(), freeListManager);
-    assertThat(freeListManager.compact(slabSize)).isTrue();
+    assertThat(freeListManager.defragment(slabSize)).isTrue();
   }
   
   @Test
-  public void compactWithEmptyTinyFreeList() {
+  public void defragmentWithEmptyTinyFreeList() {
     setUpSingleSlabManager();
     Fragment originalFragment = this.freeListManager.getFragmentList().get(0);
     OffHeapStoredObject c = freeListManager.allocate(16);
     OffHeapStoredObject.release(c.getAddress(), this.freeListManager);
     c = freeListManager.allocate(16);
-    this.freeListManager.firstCompact = false;
-    assertThat(this.freeListManager.compact(1)).isTrue();
+    this.freeListManager.firstDefragmentation = false;
+    assertThat(this.freeListManager.defragment(1)).isTrue();
     assertThat(this.freeListManager.getFragmentList()).hasSize(1);
-    Fragment compactedFragment = this.freeListManager.getFragmentList().get(0);
-    assertThat(compactedFragment.getSize()).isEqualTo(originalFragment.getSize()-(16+8));
-    assertThat(compactedFragment.getAddress()).isEqualTo(originalFragment.getAddress()+(16+8));
+    Fragment defragmentedFragment = this.freeListManager.getFragmentList().get(0);
+    assertThat(defragmentedFragment.getSize()).isEqualTo(originalFragment.getSize()-(16+8));
+    assertThat(defragmentedFragment.getAddress()).isEqualTo(originalFragment.getAddress()+(16+8));
   }
   
   @Test
@@ -445,7 +445,7 @@ public class FreeListManagerTest {
     this.freeListManager.allocate(DEFAULT_SLAB_SIZE-8-(OffHeapStoredObject.MIN_CHUNK_SIZE-1));
     this.freeListManager.allocate(MEDIUM_SLAB-8-(OffHeapStoredObject.MIN_CHUNK_SIZE-1));
     
-    assertThat(this.freeListManager.compact(SMALL_SLAB)).isTrue();
+    assertThat(this.freeListManager.defragment(SMALL_SLAB)).isTrue();
   }
  @Test
   public void maxAllocationUsesAllMemory() {
@@ -861,13 +861,13 @@ public class FreeListManagerTest {
       return super.createFreeListForEmptySlot(freeLists, idx);
     }
 
-    public boolean firstCompact = true;
+    public boolean firstDefragmentation = true;
     @Override
-    protected void afterCompactCountFetched() {
-      if (this.firstCompact) {
-        this.firstCompact = false;
-        // Force compact into thinking a concurrent compaction happened.
-        this.compactCount.incrementAndGet();
+    protected void afterDefragmentationCountFetched() {
+      if (this.firstDefragmentation) {
+        this.firstDefragmentation = false;
+        // Force defragmentation into thinking a concurrent defragmentation happened.
+        this.defragmentationCount.incrementAndGet();
       }
     }
     

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/155f87d5/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/MemoryAllocatorFillPatternJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/MemoryAllocatorFillPatternJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/MemoryAllocatorFillPatternJUnitTest.java
index f1d223d..7036187 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/MemoryAllocatorFillPatternJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/MemoryAllocatorFillPatternJUnitTest.java
@@ -44,14 +44,14 @@ public class MemoryAllocatorFillPatternJUnitTest {
   /** 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;
+  /** The number of chunks to allocate in order to force defragmentation. */
+  private static final int DEFRAGMENTATION_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;
+  private static final int DEFRAGMENTATION_CHUNK_SIZE = (SLAB_SIZE / DEFRAGMENTATION_CHUNKS) - 1024;
   
-  /** This should force compaction when allocated. */
-  private static final int FORCE_COMPACTION_CHUNK_SIZE = COMPACTION_CHUNK_SIZE * 2;
+  /** This should force defragmentation when allocated. */
+  private static final int FORCE_DEFRAGMENTATION_CHUNK_SIZE = DEFRAGMENTATION_CHUNK_SIZE * 2;
 
   /** Our test victim. */
   private MemoryAllocatorImpl allocator = null;
@@ -141,22 +141,22 @@ public class MemoryAllocatorFillPatternJUnitTest {
 
   /**
    * This tests that fill validation is working properly on newly created fragments after
-   * a compaction.
+   * a defragmentation.
    * @throws Exception
    */
   @Test
-  public void testFillPatternAfterCompaction() throws Exception {
+  public void testFillPatternAfterDefragmentation() throws Exception {
     /*
      * Stores our allocated memory.
      */
-    OffHeapStoredObject[] allocatedChunks = new OffHeapStoredObject[COMPACTION_CHUNKS];
+    OffHeapStoredObject[] allocatedChunks = new OffHeapStoredObject[DEFRAGMENTATION_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] = (OffHeapStoredObject) this.allocator.allocate(DEFRAGMENTATION_CHUNK_SIZE);
       allocatedChunks[i].validateFill();
     }
 
@@ -170,13 +170,13 @@ public class MemoryAllocatorFillPatternJUnitTest {
     
     /*
      * Now, allocate another chunk that is slightly larger than one of
-     * our initial chunks.  This should force a compaction causing our
+     * our initial chunks.  This should force a defragmentation causing our
      * memory to look like [            ][      ].
      */
-    OffHeapStoredObject slightlyLargerChunk = (OffHeapStoredObject) this.allocator.allocate(FORCE_COMPACTION_CHUNK_SIZE);
+    OffHeapStoredObject slightlyLargerChunk = (OffHeapStoredObject) this.allocator.allocate(FORCE_DEFRAGMENTATION_CHUNK_SIZE);
     
     /*
-     * Make sure the compacted memory has the fill validation.
+     * Make sure the defragmented memory has the fill validation.
      */
     slightlyLargerChunk.validateFill();
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/155f87d5/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/MemoryAllocatorJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/MemoryAllocatorJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/MemoryAllocatorJUnitTest.java
index 7639f8d..582e8b9 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/MemoryAllocatorJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/MemoryAllocatorJUnitTest.java
@@ -219,7 +219,7 @@ public class MemoryAllocatorJUnitTest {
       }
       hugemc.release();
       assertEquals(round(TINY_MULTIPLE, minHuge+perObjectOverhead)*BATCH_SIZE, ma.freeList.getFreeHugeMemory());
-      // now that we do compaction the following allocate works.
+      // now that we do defragmentation the following allocate works.
       hugemc = ma.allocate(minHuge + HUGE_MULTIPLE + HUGE_MULTIPLE-1);
     } finally {
       MemoryAllocatorImpl.freeOffHeapMemory();
@@ -389,7 +389,7 @@ public class MemoryAllocatorJUnitTest {
   }
   
   @Test
-  public void testCompaction() {
+  public void testDefragmentation() {
     final int perObjectOverhead = OffHeapStoredObject.HEADER_SIZE;
     final int BIG_ALLOC_SIZE = 150000;
     final int SMALL_ALLOC_SIZE = BIG_ALLOC_SIZE/2;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/155f87d5/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/NullOffHeapMemoryStats.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/NullOffHeapMemoryStats.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/NullOffHeapMemoryStats.java
index 88bab77..d5ce58a 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/NullOffHeapMemoryStats.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/NullOffHeapMemoryStats.java
@@ -60,7 +60,7 @@ public class NullOffHeapMemoryStats implements OffHeapMemoryStats {
     return 0;
   }
   @Override
-  public int getCompactions() {
+  public int getDefragmentations() {
     return 0;
   }
   @Override
@@ -78,11 +78,11 @@ public class NullOffHeapMemoryStats implements OffHeapMemoryStats {
     return 0;
   }
   @Override
-  public long startCompaction() {
+  public long startDefragmentation() {
     return 0;
   }
   @Override
-  public void endCompaction(long start) {
+  public void endDefragmentation(long start) {
   }
   @Override
   public void setFragmentation(int value) {
@@ -96,7 +96,7 @@ public class NullOffHeapMemoryStats implements OffHeapMemoryStats {
     return null;
   }
   @Override
-  public long getCompactionTime() {
+  public long getDefragmentationTime() {
     return 0;
   }
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/155f87d5/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapRegionBase.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapRegionBase.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapRegionBase.java
index fb1aa41..02673a2 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapRegionBase.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapRegionBase.java
@@ -99,8 +99,8 @@ public abstract class OffHeapRegionBase {
       assertEquals(offHeapSize, ma.getFreeMemory());
       assertEquals(0, ma.getUsedMemory());
       // do an allocation larger than the slab size
-      // TODO: currently the compact will product slabs bigger than the max slab size
-      // (see the todo comment on compact() in FreeListManager).
+      // TODO: currently the defragment will produce slabs bigger than the max slab size
+      // (see the todo comment on defragment() in FreeListManager).
       // So we request 20m here since that it the total size.
       try {
         ma.allocate(1024*1024*20);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/155f87d5/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapStorageJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapStorageJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapStorageJUnitTest.java
index d30d4c4..77b1d8d 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapStorageJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapStorageJUnitTest.java
@@ -167,8 +167,8 @@ public class OffHeapStorageJUnitTest {
       assertEquals(1024*1024, stats.getFreeMemory());
       assertEquals(1024*1024, stats.getMaxMemory());
       assertEquals(0, stats.getUsedMemory());
-      assertEquals(0, stats.getCompactions());
-      assertEquals(0, stats.getCompactionTime());
+      assertEquals(0, stats.getDefragmentations());
+      assertEquals(0, stats.getDefragmentationTime());
       assertEquals(0, stats.getFragmentation());
       assertEquals(1, stats.getFragments());
       assertEquals(1024*1024, stats.getLargestFragment());
@@ -216,13 +216,13 @@ public class OffHeapStorageJUnitTest {
       boolean originalEnableClockStats = DistributionStats.enableClockStats;
       DistributionStats.enableClockStats = true;
       try {
-        long start = stats.startCompaction();
-        while (stats.startCompaction() == start) {
+        long start = stats.startDefragmentation();
+        while (stats.startDefragmentation() == start) {
           Thread.yield();
         }
-        stats.endCompaction(start);
-        assertEquals(1, stats.getCompactions());
-        assertTrue(stats.getCompactionTime() > 0);
+        stats.endDefragmentation(start);
+        assertEquals(1, stats.getDefragmentations());
+        assertTrue(stats.getDefragmentationTime() > 0);
       } finally {
         DistributionStats.enableClockStats = originalEnableClockStats;
       }
@@ -235,8 +235,8 @@ public class OffHeapStorageJUnitTest {
       assertEquals(0, stats.getFreeMemory());
       assertEquals(0, stats.getMaxMemory());
       assertEquals(0, stats.getUsedMemory());
-      assertEquals(0, stats.getCompactions());
-      assertEquals(0, stats.getCompactionTime());
+      assertEquals(0, stats.getDefragmentations());
+      assertEquals(0, stats.getDefragmentationTime());
       assertEquals(0, stats.getFragmentation());
       assertEquals(0, stats.getFragments());
       assertEquals(0, stats.getLargestFragment());