You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by ds...@apache.org on 2015/12/07 20:12:34 UTC

incubator-geode git commit: GEODE-622: add OffHeapStorage unit test coverage

Repository: incubator-geode
Updated Branches:
  refs/heads/develop e414a4933 -> 812d51c4d


GEODE-622: add OffHeapStorage unit test coverage

Removed unused OFF_HEAP_TOTAL_SIZE system property.


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

Branch: refs/heads/develop
Commit: 812d51c4d71b3163e9c26cf2d3046bd8267ebe69
Parents: e414a49
Author: Darrel Schneider <ds...@pivotal.io>
Authored: Wed Dec 2 15:06:34 2015 -0800
Committer: Darrel Schneider <ds...@pivotal.io>
Committed: Mon Dec 7 11:09:30 2015 -0800

----------------------------------------------------------------------
 .../internal/offheap/OffHeapStorage.java        |  29 +--
 .../offheap/SimpleMemoryAllocatorImpl.java      |  22 +-
 .../offheap/OffHeapStorageJUnitTest.java        | 202 +++++++++++++++++++
 3 files changed, 219 insertions(+), 34 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/812d51c4/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapStorage.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapStorage.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapStorage.java
index ef584f1..82cbfeb 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapStorage.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapStorage.java
@@ -147,6 +147,7 @@ public class OffHeapStorage implements OffHeapMemoryStats {
         result = MAX_SLAB_SIZE;
       }
     }
+    assert result > 0 && result <= MAX_SLAB_SIZE && result <= offHeapMemorySize;
     return result;
   }
   
@@ -175,14 +176,6 @@ public class OffHeapStorage implements OffHeapMemoryStats {
    * @return MemoryAllocator for off-heap storage
    */
   public static MemoryAllocator createOffHeapStorage(LogWriter lw, StatisticsFactory sf, long offHeapMemorySize, DistributedSystem system) {
-    // TODO: delete this block of code after tests are changed to use new config
-    if (offHeapMemorySize == 0 && !Boolean.getBoolean(InternalLocator.FORCE_LOCATOR_DM_TYPE)) {
-      String offHeapConfig = System.getProperty("gemfire.OFF_HEAP_TOTAL_SIZE");
-      if (offHeapConfig != null && !offHeapConfig.equals("")) {
-        offHeapMemorySize = parseLongWithUnits(offHeapConfig, 0L, 1024*1024);
-      }
-    }
-    
     MemoryAllocator result;
     if (offHeapMemorySize == 0 || Boolean.getBoolean(InternalLocator.FORCE_LOCATOR_DM_TYPE)) {
       // Checking the FORCE_LOCATOR_DM_TYPE is a quick hack to keep our locator from allocating off heap memory.
@@ -199,15 +192,6 @@ public class OffHeapStorage implements OffHeapMemoryStats {
       
       // determine off-heap and slab sizes
       final long maxSlabSize = calcMaxSlabSize(offHeapMemorySize);
-      assert maxSlabSize > 0;
-      
-      // validate sizes
-      if (maxSlabSize > MAX_SLAB_SIZE) {
-        throw new IllegalArgumentException("gemfire.OFF_HEAP_SLAB_SIZE of value " + offHeapMemorySize + " exceeds maximum value of " + MAX_SLAB_SIZE);
-      }
-      if (maxSlabSize > offHeapMemorySize) {
-        throw new IllegalArgumentException("The off heap slab size (which is " + maxSlabSize + "; set it with gemfire.OFF_HEAP_SLAB_SIZE) must be less than or equal to the total size (which is " + offHeapMemorySize + "; set it with gemfire.OFF_HEAP_SLAB_SIZE).");
-      }
       
       final int slabCount = calcSlabCount(maxSlabSize, offHeapMemorySize);
 
@@ -222,9 +206,10 @@ public class OffHeapStorage implements OffHeapMemoryStats {
   }
   
   private static final long MAX_SLAB_SIZE = Integer.MAX_VALUE;
-  private static final long MIN_SLAB_SIZE = 1024;
+  static final long MIN_SLAB_SIZE = 1024;
 
-  private static int calcSlabCount(long maxSlabSize, long offHeapMemorySize) {
+  // non-private for unit test access
+  static int calcSlabCount(long maxSlabSize, long offHeapMemorySize) {
     long result = offHeapMemorySize / maxSlabSize;
     if ((offHeapMemorySize % maxSlabSize) >= MIN_SLAB_SIZE) {
       result++;
@@ -430,13 +415,13 @@ public class OffHeapStorage implements OffHeapMemoryStats {
         if (this.ids == null) {
           return;
         }
-        final InternalDistributedSystem dsToDisconnect = this.ids;
-        this.ids = null; // set null to prevent memory leak after closure!
-        
         if (stayConnectedOnOutOfOffHeapMemory) {
           return;
         }
         
+        final InternalDistributedSystem dsToDisconnect = this.ids;
+        this.ids = null; // set null to prevent memory leak after closure!
+        
         if (dsToDisconnect.getDistributionManager().getRootCause() == null) {
           dsToDisconnect.getDistributionManager().setRootCause(cause);
         }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/812d51c4/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/SimpleMemoryAllocatorImpl.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/SimpleMemoryAllocatorImpl.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/SimpleMemoryAllocatorImpl.java
index dfd05c6..d053797 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/SimpleMemoryAllocatorImpl.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/SimpleMemoryAllocatorImpl.java
@@ -25,12 +25,13 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Set;
 import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicReference;
+
 import org.apache.logging.log4j.Logger;
 
 import com.gemstone.gemfire.LogWriter;
 import com.gemstone.gemfire.cache.CacheClosedException;
 import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.RegionService;
 import com.gemstone.gemfire.internal.cache.BucketRegion;
 import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
 import com.gemstone.gemfire.internal.cache.LocalRegion;
@@ -100,7 +101,6 @@ public final class SimpleMemoryAllocatorImpl implements MemoryAllocator, MemoryI
   private volatile MemoryUsageListener[] memoryUsageListeners = new MemoryUsageListener[0];
   
   private static SimpleMemoryAllocatorImpl singleton = null;
-  private static final AtomicReference<Thread> asyncCleanupThread = new AtomicReference<>();
   final ChunkFactory chunkFactory;
   
   public static SimpleMemoryAllocatorImpl getAllocator() {
@@ -297,8 +297,8 @@ public final class SimpleMemoryAllocatorImpl implements MemoryAllocator, MemoryI
   public List<Chunk> getLostChunks() {
     List<Chunk> liveChunks = this.freeList.getLiveChunks();
     List<Chunk> regionChunks = getRegionLiveChunks();
-    Set liveChunksSet = new HashSet(liveChunks);
-    Set regionChunksSet = new HashSet(regionChunks);
+    Set<Chunk> liveChunksSet = new HashSet<>(liveChunks);
+    Set<Chunk> regionChunksSet = new HashSet<>(regionChunks);
     liveChunksSet.removeAll(regionChunksSet);
     return new ArrayList<Chunk>(liveChunksSet);
   }
@@ -308,23 +308,22 @@ public final class SimpleMemoryAllocatorImpl implements MemoryAllocator, MemoryI
    */
   private List<Chunk> getRegionLiveChunks() {
     ArrayList<Chunk> result = new ArrayList<Chunk>();
-    GemFireCacheImpl gfc = GemFireCacheImpl.getInstance();
+    RegionService gfc = GemFireCacheImpl.getInstance();
     if (gfc != null) {
-      Iterator rootIt = gfc.rootRegions().iterator();
+      Iterator<Region<?,?>> rootIt = gfc.rootRegions().iterator();
       while (rootIt.hasNext()) {
-        Region rr = (Region) rootIt.next();
+        Region<?,?> rr = rootIt.next();
         getRegionLiveChunks(rr, result);
-        Iterator srIt = rr.subregions(true).iterator();
+        Iterator<Region<?,?>> srIt = rr.subregions(true).iterator();
         while (srIt.hasNext()) {
-          Region sr = (Region)srIt.next();
-          getRegionLiveChunks(sr, result);
+          getRegionLiveChunks(srIt.next(), result);
         }
       }
     }
     return result;
   }
 
-  private void getRegionLiveChunks(Region r, List<Chunk> result) {
+  private void getRegionLiveChunks(Region<?,?> r, List<Chunk> result) {
     if (r.getAttributes().getOffHeap()) {
 
       if (r instanceof PartitionedRegion) {
@@ -375,7 +374,6 @@ public final class SimpleMemoryAllocatorImpl implements MemoryAllocator, MemoryI
     return result;
   }
   
-  @SuppressWarnings("unused")
   public static void debugLog(String msg, boolean logStack) {
     if (logStack) {
       logger.info(msg, new RuntimeException(msg));

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/812d51c4/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapStorageJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapStorageJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapStorageJUnitTest.java
index 8b61ab0..de21487 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapStorageJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapStorageJUnitTest.java
@@ -17,14 +17,29 @@
 package com.gemstone.gemfire.internal.offheap;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
+import java.io.File;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
+import com.gemstone.gemfire.OutOfOffHeapMemoryException;
+import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.cache.CacheFactory;
+import com.gemstone.gemfire.distributed.DistributedSystem;
+import com.gemstone.gemfire.distributed.internal.DistributionStats;
+import com.gemstone.gemfire.distributed.internal.InternalLocator;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
+import com.jayway.awaitility.Awaitility;
 
 @Category(UnitTest.class)
 public class OffHeapStorageJUnitTest {
@@ -76,4 +91,191 @@ public class OffHeapStorageJUnitTest {
     assertEquals(GIGABYTE, OffHeapStorage.parseOffHeapMemorySize("1g"));
     assertEquals(Integer.MAX_VALUE * GIGABYTE, OffHeapStorage.parseOffHeapMemorySize("" + Integer.MAX_VALUE + "g"));
   }
+  @Test
+  public void testCalcMaxSlabSize() {
+    assertEquals(100, OffHeapStorage.calcMaxSlabSize(100L));
+    assertEquals(Integer.MAX_VALUE, OffHeapStorage.calcMaxSlabSize(Long.MAX_VALUE));
+    try {
+      System.setProperty("gemfire.OFF_HEAP_SLAB_SIZE", "99");
+      assertEquals(99*1024*1024, OffHeapStorage.calcMaxSlabSize(100L*1024*1024));
+      System.setProperty("gemfire.OFF_HEAP_SLAB_SIZE", "88m");
+      assertEquals(88*1024*1024, OffHeapStorage.calcMaxSlabSize(100L*1024*1024));
+      System.setProperty("gemfire.OFF_HEAP_SLAB_SIZE", "77M");
+      assertEquals(77*1024*1024, OffHeapStorage.calcMaxSlabSize(100L*1024*1024));
+      System.setProperty("gemfire.OFF_HEAP_SLAB_SIZE", "1g");
+      assertEquals(1*1024*1024*1024, OffHeapStorage.calcMaxSlabSize(2L*1024*1024*1024));
+      System.setProperty("gemfire.OFF_HEAP_SLAB_SIZE", "1G");
+      assertEquals(1L*1024*1024*1024, OffHeapStorage.calcMaxSlabSize(2L*1024*1024*1024+1));
+      System.setProperty("gemfire.OFF_HEAP_SLAB_SIZE", "foobarG");
+      try {
+        OffHeapStorage.calcMaxSlabSize(100);
+        fail("expected IllegalArgumentException");
+      } catch (IllegalArgumentException expected) {
+      }
+      System.setProperty("gemfire.OFF_HEAP_SLAB_SIZE", "");
+      assertEquals(100, OffHeapStorage.calcMaxSlabSize(100L));
+      assertEquals(Integer.MAX_VALUE, OffHeapStorage.calcMaxSlabSize(Long.MAX_VALUE));
+    } finally {
+      System.clearProperty("gemfire.OFF_HEAP_SLAB_SIZE");
+    }
+  }
+  @Test
+  public void testCreateOffHeapStorage() {
+    System.setProperty(InternalLocator.FORCE_LOCATOR_DM_TYPE, "true");
+    try {
+      assertEquals(null, OffHeapStorage.createOffHeapStorage(null, null, 0, null));
+    } finally {
+      System.clearProperty(InternalLocator.FORCE_LOCATOR_DM_TYPE);
+    }
+    // TODO: mock the StatiticsFactory and InternalDistributedSystem that createOffHeapStorage require
+    Cache c = new CacheFactory().set("mcast-port", "0").create();
+    try {
+      try {
+        OffHeapStorage.createOffHeapStorage(null, c.getDistributedSystem(), OffHeapStorage.MIN_SLAB_SIZE-1, c.getDistributedSystem());
+      } catch (IllegalArgumentException expected) {
+        expected.getMessage().equals("The amount of off heap memory must be at least " + OffHeapStorage.MIN_SLAB_SIZE + " but it was set to " + (OffHeapStorage.MIN_SLAB_SIZE-1));
+      }
+      try {
+        OffHeapStorage.createOffHeapStorage(null, c.getDistributedSystem(), OffHeapStorage.MIN_SLAB_SIZE, null);
+      } catch (IllegalArgumentException expected) {
+        expected.getMessage().equals("InternalDistributedSystem is null");
+      }
+      MemoryAllocator ma = OffHeapStorage.createOffHeapStorage(null, c.getDistributedSystem(), 1024*1024, c.getDistributedSystem());
+      try {
+        OffHeapMemoryStats stats = ma.getStats();
+        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.getFragmentation());
+        assertEquals(1, stats.getFragments());
+        assertEquals(1024*1024, stats.getLargestFragment());
+        assertEquals(0, stats.getObjects());
+        assertEquals(0, stats.getReads());
+
+        stats.incFreeMemory(100);
+        assertEquals(1024*1024+100, stats.getFreeMemory());
+        stats.incFreeMemory(-100);
+        assertEquals(1024*1024, stats.getFreeMemory());
+
+        stats.incMaxMemory(100);
+        assertEquals(1024*1024+100, stats.getMaxMemory());
+        stats.incMaxMemory(-100);
+        assertEquals(1024*1024, stats.getMaxMemory());
+
+        stats.incUsedMemory(100);
+        assertEquals(100, stats.getUsedMemory());
+        stats.incUsedMemory(-100);
+        assertEquals(0, stats.getUsedMemory());
+
+        stats.incObjects(100);
+        assertEquals(100, stats.getObjects());
+        stats.incObjects(-100);
+        assertEquals(0, stats.getObjects());
+
+        stats.incReads();
+        assertEquals(1, stats.getReads());
+
+        stats.setFragmentation(100);
+        assertEquals(100, stats.getFragmentation());
+        stats.setFragmentation(0);
+        assertEquals(0, stats.getFragmentation());
+
+        stats.setFragments(2);
+        assertEquals(2, stats.getFragments());
+        stats.setFragments(1);
+        assertEquals(1, stats.getFragments());
+
+        stats.setLargestFragment(100);
+        assertEquals(100, stats.getLargestFragment());
+        stats.setLargestFragment(1024*1024);
+        assertEquals(1024*1024, stats.getLargestFragment());
+
+        boolean originalEnableClockStats = DistributionStats.enableClockStats;
+        DistributionStats.enableClockStats = true;
+        try {
+          long start = stats.startCompaction();
+          while (stats.startCompaction() == start) {
+            Thread.yield();
+          }
+          stats.endCompaction(start);
+          assertEquals(1, stats.getCompactions());
+          assertTrue(stats.getCompactionTime() > 0);
+        } finally {
+          DistributionStats.enableClockStats = originalEnableClockStats;
+        }
+
+        stats.incObjects(100);
+        stats.incUsedMemory(100);
+        stats.setFragmentation(100);
+        OffHeapStorage ohs = (OffHeapStorage) stats;
+        ohs.initialize(new NullOffHeapMemoryStats());
+        assertEquals(0, stats.getFreeMemory());
+        assertEquals(0, stats.getMaxMemory());
+        assertEquals(0, stats.getUsedMemory());
+        assertEquals(0, stats.getCompactions());
+        assertEquals(0, stats.getCompactionTime());
+        assertEquals(0, stats.getFragmentation());
+        assertEquals(0, stats.getFragments());
+        assertEquals(0, stats.getLargestFragment());
+        assertEquals(0, stats.getObjects());
+        assertEquals(0, stats.getReads());
+        System.setProperty(OffHeapStorage.STAY_CONNECTED_ON_OUTOFOFFHEAPMEMORY_PROPERTY, "true");
+        try {
+          try {
+            ma.allocate(1024*1024+1, null);
+            fail("expected OutOfOffHeapMemoryException");
+          } catch (OutOfOffHeapMemoryException expected) {
+          }
+          assertTrue(c.getDistributedSystem().isConnected());
+          try {
+            ma.allocate(1024*1024+1, null);
+            fail("expected OutOfOffHeapMemoryException");
+          } catch (OutOfOffHeapMemoryException expected) {
+          }
+          assertTrue(c.getDistributedSystem().isConnected());
+        } finally {
+          System.clearProperty(OffHeapStorage.STAY_CONNECTED_ON_OUTOFOFFHEAPMEMORY_PROPERTY);
+        }
+        try {
+          ma.allocate(1024*1024+1, null);
+          fail("expected OutOfOffHeapMemoryException");
+        } catch (OutOfOffHeapMemoryException expected) {
+        }
+        try {
+          ma.allocate(1024*1024+1, null);
+          fail("expected OutOfOffHeapMemoryException");
+        } catch (OutOfOffHeapMemoryException expected) {
+        }
+        Awaitility.await().atMost(5, TimeUnit.SECONDS).until(() -> {
+          return !c.getDistributedSystem().isConnected();
+        });
+
+      } finally {
+        System.setProperty(SimpleMemoryAllocatorImpl.FREE_OFF_HEAP_MEMORY_PROPERTY, "true");
+        try {
+          ma.close();
+        } finally {
+          System.clearProperty(SimpleMemoryAllocatorImpl.FREE_OFF_HEAP_MEMORY_PROPERTY);
+        }
+      }
+   } finally {
+      c.close();
+    }
+  }
+  @Test
+  public void testCalcSlabCount() {
+    final long MSS = OffHeapStorage.MIN_SLAB_SIZE;
+    assertEquals(100, OffHeapStorage.calcSlabCount(MSS*4, MSS*4*100));
+    assertEquals(100, OffHeapStorage.calcSlabCount(MSS*4, (MSS*4*100) + (MSS-1)));
+    assertEquals(101, OffHeapStorage.calcSlabCount(MSS*4, (MSS*4*100) + MSS));
+    assertEquals(Integer.MAX_VALUE, OffHeapStorage.calcSlabCount(MSS, MSS * Integer.MAX_VALUE));
+    assertEquals(Integer.MAX_VALUE, OffHeapStorage.calcSlabCount(MSS, (MSS * Integer.MAX_VALUE) + MSS-1));
+    try {
+      OffHeapStorage.calcSlabCount(MSS, (((long)MSS) * Integer.MAX_VALUE) + MSS);
+      fail("Expected IllegalArgumentException");
+    } catch (IllegalArgumentException expected) {
+    }
+  }
 }