You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by mc...@apache.org on 2019/01/30 18:47:24 UTC

[geode] branch develop updated: GEODE-6304: Refactor memory monitor to properly reset tolerance counter (#3102)

This is an automated email from the ASF dual-hosted git repository.

mcmellawatt pushed a commit to branch develop
in repository https://gitbox.apache.org/repos/asf/geode.git


The following commit(s) were added to refs/heads/develop by this push:
     new 546abdd  GEODE-6304: Refactor memory monitor to properly reset tolerance counter (#3102)
546abdd is described below

commit 546abdd173afe13c01e148900c2b9114dc5bc4cc
Author: Ryan McMahon <rm...@pivotal.io>
AuthorDate: Wed Jan 30 10:47:03 2019 -0800

    GEODE-6304: Refactor memory monitor to properly reset tolerance counter (#3102)
    
    The tolerance counter was not properly being reset between
    non-consecutive EVICTION and CRITICAL events.  To fix this and make it
    more understandable, the threshold logic was combined with the logic to
    compute the next state in the MemoryThreshold state machine.  The
    counter is now reset between non-consecutive EVICTION and CRITICAL
    events.
---
 .../management/MemoryThresholdsDUnitTest.java      | 136 +++++++-------
 .../ResourceManagerWithQueryMonitorDUnitTest.java  |  10 +-
 .../cache/PartitionedRegionEvictionDUnitTest.java  |   5 +-
 .../internal/cache/eviction/EvictionDUnitTest.java |   2 +-
 .../cache/control/MemoryMonitorJUnitTest.java      | 128 ++++++-------
 .../internal/cache/control/HeapMemoryMonitor.java  | 120 ++++---------
 .../internal/cache/control/MemoryThresholds.java   |  42 ++++-
 .../cache/control/HeapMemoryMonitorTest.java       | 197 ++++++++++++++++++++-
 .../java/org/apache/geode/test/fake/Fakes.java     |   4 +
 .../geode/cache/lucene/EvictionDUnitTest.java      |   5 +-
 10 files changed, 414 insertions(+), 235 deletions(-)

diff --git a/geode-core/src/distributedTest/java/org/apache/geode/cache/management/MemoryThresholdsDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/cache/management/MemoryThresholdsDUnitTest.java
index c11b575..c186b24 100644
--- a/geode-core/src/distributedTest/java/org/apache/geode/cache/management/MemoryThresholdsDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/cache/management/MemoryThresholdsDUnitTest.java
@@ -471,7 +471,7 @@ public class MemoryThresholdsDUnitTest extends ClientServerTestCase {
       public Object call() throws Exception {
         InternalCache gfCache = getCache();
         getCache().getLogger().fine(addExpectedExString);
-        gfCache.getInternalResourceManager().getHeapMonitor().updateStateAndSendEvent(950);
+        gfCache.getInternalResourceManager().getHeapMonitor().updateStateAndSendEvent(950, "test");
         getCache().getLogger().fine(removeExpectedExString);
         return null;
       }
@@ -491,7 +491,7 @@ public class MemoryThresholdsDUnitTest extends ClientServerTestCase {
       public Object call() throws Exception {
         InternalCache gfCache = getCache();
         getCache().getLogger().fine(addExpectedBelow);
-        gfCache.getInternalResourceManager().getHeapMonitor().updateStateAndSendEvent(850);
+        gfCache.getInternalResourceManager().getHeapMonitor().updateStateAndSendEvent(850, "test");
         getCache().getLogger().fine(removeExpectedBelow);
         return null;
       }
@@ -508,7 +508,7 @@ public class MemoryThresholdsDUnitTest extends ClientServerTestCase {
       @Override
       public Object call() throws Exception {
         InternalCache gfCache = getCache();
-        gfCache.getInternalResourceManager().getHeapMonitor().updateStateAndSendEvent(840);
+        gfCache.getInternalResourceManager().getHeapMonitor().updateStateAndSendEvent(840, "test");
         return null;
       }
     });
@@ -524,7 +524,7 @@ public class MemoryThresholdsDUnitTest extends ClientServerTestCase {
       @Override
       public Object call() throws Exception {
         InternalCache gfCache = getCache();
-        gfCache.getInternalResourceManager().getHeapMonitor().updateStateAndSendEvent(750);
+        gfCache.getInternalResourceManager().getHeapMonitor().updateStateAndSendEvent(750, "test");
         return null;
       }
     });
@@ -541,7 +541,7 @@ public class MemoryThresholdsDUnitTest extends ClientServerTestCase {
       @Override
       public Object call() throws Exception {
         InternalCache gfCache = getCache();
-        gfCache.getInternalResourceManager().getHeapMonitor().updateStateAndSendEvent(950);
+        gfCache.getInternalResourceManager().getHeapMonitor().updateStateAndSendEvent(950, "test");
         return null;
       }
     });
@@ -557,7 +557,7 @@ public class MemoryThresholdsDUnitTest extends ClientServerTestCase {
       @Override
       public Object call() throws Exception {
         InternalCache gfCache = getCache();
-        gfCache.getInternalResourceManager().getHeapMonitor().updateStateAndSendEvent(750);
+        gfCache.getInternalResourceManager().getHeapMonitor().updateStateAndSendEvent(750, "test");
         return null;
       }
     });
@@ -574,7 +574,7 @@ public class MemoryThresholdsDUnitTest extends ClientServerTestCase {
       @Override
       public Object call() throws Exception {
         InternalCache gfCache = getCache();
-        gfCache.getInternalResourceManager().getHeapMonitor().updateStateAndSendEvent(850);
+        gfCache.getInternalResourceManager().getHeapMonitor().updateStateAndSendEvent(850, "test");
         return null;
       }
     });
@@ -1499,7 +1499,7 @@ public class MemoryThresholdsDUnitTest extends ClientServerTestCase {
       public Object call() throws Exception {
         getCache().getLogger().fine(addExpectedExString);
         ((GemFireCacheImpl) getCache()).getInternalResourceManager().getHeapMonitor()
-            .updateStateAndSendEvent(950);
+            .updateStateAndSendEvent(950, "test");
         HeapMemoryMonitor.setTestBytesUsedForThresholdSet(950);
         getCache().getLogger().fine(removeExpectedExString);
         return null;
@@ -1514,7 +1514,7 @@ public class MemoryThresholdsDUnitTest extends ClientServerTestCase {
         getCache().getLogger().fine(addExpectedBelow);
         HeapMemoryMonitor.setTestBytesUsedForThresholdSet(850);
         ((GemFireCacheImpl) getCache()).getInternalResourceManager().getHeapMonitor()
-            .updateStateAndSendEvent(850);
+            .updateStateAndSendEvent(850, "test");
         getCache().getLogger().fine(removeExpectedBelow);
         return null;
       }
@@ -1527,7 +1527,7 @@ public class MemoryThresholdsDUnitTest extends ClientServerTestCase {
       public Object call() throws Exception {
         getCache().getLogger().fine(addExpectedBelow);
         ((GemFireCacheImpl) getCache()).getInternalResourceManager().getHeapMonitor()
-            .updateStateAndSendEvent(750);
+            .updateStateAndSendEvent(750, "test");
         getCache().getLogger().fine(removeExpectedBelow);
         return null;
       }
@@ -1746,7 +1746,7 @@ public class MemoryThresholdsDUnitTest extends ClientServerTestCase {
     public Object call() throws Exception {
       InternalResourceManager irm = ((GemFireCacheImpl) getCache()).getInternalResourceManager();
       // Reset CRITICAL_UP by informing all that heap usage is now 1 byte (0 would disable).
-      irm.getHeapMonitor().updateStateAndSendEvent(1);
+      irm.getHeapMonitor().updateStateAndSendEvent(1, "test");
       Set<ResourceListener> listeners = irm.getResourceListeners(ResourceType.HEAP_MEMORY);
       Iterator<ResourceListener> it = listeners.iterator();
       while (it.hasNext()) {
@@ -1943,7 +1943,7 @@ public class MemoryThresholdsDUnitTest extends ClientServerTestCase {
                                                                                    // 10%
         assertTrue(fakeHeapUsage > 0);
         assertTrue(fakeHeapUsage <= fakeHeapMaxSize);
-        hmm.updateStateAndSendEvent(fakeHeapUsage);
+        hmm.updateStateAndSendEvent(fakeHeapUsage, "test");
         getCache().getLogger().fine(removeExpectedExString);
 
         assertTrue(hmm.getState().isCritical());
@@ -1961,7 +1961,7 @@ public class MemoryThresholdsDUnitTest extends ClientServerTestCase {
                                                                                    // by 30%
         assertTrue(fakeHeapMaxSize > 0);
         getCache().getLogger().fine(addExpectedBelow);
-        hmm.updateStateAndSendEvent(fakeHeapUsage);
+        hmm.updateStateAndSendEvent(fakeHeapUsage, "test");
         getCache().getLogger().fine(removeExpectedBelow);
         assertFalse(hmm.getState().isCritical());
         {
@@ -2110,7 +2110,7 @@ public class MemoryThresholdsDUnitTest extends ClientServerTestCase {
                                                                                                // 10%
             assertTrue(newfakeHeapUsage > 0);
             assertTrue(newfakeHeapUsage <= fakeHeapMaxSize);
-            hmm.updateStateAndSendEvent(newfakeHeapUsage);
+            hmm.updateStateAndSendEvent(newfakeHeapUsage, "test");
             assertTrue(hmm.getState().isCritical());
             {
               Integer k = new Integer(2);
@@ -2122,7 +2122,7 @@ public class MemoryThresholdsDUnitTest extends ClientServerTestCase {
                                                                                           // by 30%
             assertTrue(fakeHeapMaxSize > 0);
             getCache().getLogger().fine(addExpectedBelow);
-            hmm.updateStateAndSendEvent(newfakeHeapUsage);
+            hmm.updateStateAndSendEvent(newfakeHeapUsage, "test");
             getCache().getLogger().fine(removeExpectedBelow);
             assertFalse(hmm.getState().isCritical());
             {
@@ -2175,7 +2175,7 @@ public class MemoryThresholdsDUnitTest extends ClientServerTestCase {
                                                                                            // by 10%
         assertTrue(newfakeHeapUsage > 0);
         assertTrue(newfakeHeapUsage <= fakeHeapMaxSize);
-        hmm.updateStateAndSendEvent(newfakeHeapUsage);
+        hmm.updateStateAndSendEvent(newfakeHeapUsage, "test");
         assertTrue(hmm.getState().isCritical());
         {
           Integer k = new Integer(5);
@@ -2189,7 +2189,7 @@ public class MemoryThresholdsDUnitTest extends ClientServerTestCase {
                                                                                       // 30%
         assertTrue(fakeHeapMaxSize > 0);
         getCache().getLogger().fine(addExpectedBelow);
-        hmm.updateStateAndSendEvent(newfakeHeapUsage);
+        hmm.updateStateAndSendEvent(newfakeHeapUsage, "test");
         getCache().getLogger().fine(removeExpectedBelow);
         assertFalse(hmm.getState().isCritical());
         {
@@ -2314,7 +2314,7 @@ public class MemoryThresholdsDUnitTest extends ClientServerTestCase {
             assertTrue(newfakeHeapUsage <= fakeHeapMaxSize);
             HeapMemoryMonitor hmm =
                 ((InternalResourceManager) getCache().getResourceManager()).getHeapMonitor();
-            hmm.updateStateAndSendEvent(newfakeHeapUsage);
+            hmm.updateStateAndSendEvent(newfakeHeapUsage, "test");
             assertTrue(hmm.getState().isCritical());
             final Integer k = new Integer(2); // reload with same key again and again
             final Integer expectedInvocations3 = new Integer(expectedInvocations.getAndIncrement());
@@ -2361,7 +2361,7 @@ public class MemoryThresholdsDUnitTest extends ClientServerTestCase {
                                                                                               // 30%
             assertTrue(fakeHeapMaxSize > 0);
             getCache().getLogger().fine(addExpectedBelow);
-            hmm.updateStateAndSendEvent(newfakeHeapUsage);
+            hmm.updateStateAndSendEvent(newfakeHeapUsage, "test");
             getCache().getLogger().fine(removeExpectedBelow);
             assertFalse(hmm.getState().isCritical());
             Integer k = new Integer(3); // same key as previously used, this time is should stick
@@ -2396,7 +2396,7 @@ public class MemoryThresholdsDUnitTest extends ClientServerTestCase {
                                                                                            // by 10%
         assertTrue(newfakeHeapUsage > 0);
         assertTrue(newfakeHeapUsage <= fakeHeapMaxSize);
-        hmm.updateStateAndSendEvent(newfakeHeapUsage);
+        hmm.updateStateAndSendEvent(newfakeHeapUsage, "test");
         getCache().getLogger().fine(removeExpectedExString);
         assertTrue(hmm.getState().isCritical());
         k = new Integer(5);
@@ -2411,7 +2411,7 @@ public class MemoryThresholdsDUnitTest extends ClientServerTestCase {
                                                                                       // 30%
         assertTrue(fakeHeapMaxSize > 0);
         getCache().getLogger().fine(addExpectedBelow);
-        hmm.updateStateAndSendEvent(newfakeHeapUsage);
+        hmm.updateStateAndSendEvent(newfakeHeapUsage, "test");
         getCache().getLogger().fine(removeExpectedBelow);
         assertFalse(hmm.getState().isCritical());
         return expectedInvocations10;
@@ -2504,63 +2504,75 @@ public class MemoryThresholdsDUnitTest extends ClientServerTestCase {
 
   @Test
   public void testCriticalMemoryEventTolerance() {
-    final Host host = Host.getHost(0);
-    final VM vm = host.getVM(0);
-    vm.invoke(new SerializableCallable() {
-      @Override
-      public Object call() throws Exception {
-        int defaultTolerance = 1;
-        HeapMemoryMonitor.setTestDisableMemoryUpdates(false);
-        GemFireCacheImpl cache = (GemFireCacheImpl) getCache();
-        InternalResourceManager irm = cache.getInternalResourceManager();
-        HeapMemoryMonitor hmm = irm.getHeapMonitor();
-        hmm.setTestMaxMemoryBytes(100);
-        HeapMemoryMonitor.setTestBytesUsedForThresholdSet(1);
-        irm.setCriticalHeapPercentage(95);
-        for (int i = 0; i < defaultTolerance; i++) {
-          hmm.updateStateAndSendEvent(96);
-          assertFalse(hmm.getState().isCritical());
-        }
-        getCache().getLogger().fine(addExpectedExString);
-        hmm.updateStateAndSendEvent(96);
-        assertTrue(hmm.getState().isCritical());
-        getCache().getLogger().fine(removeExpectedExString);
-        getCache().getLogger().fine(addExpectedBelow);
-        hmm.updateStateAndSendEvent(92);
-        getCache().getLogger().fine(removeExpectedBelow);
-        assertFalse(hmm.getState().isCritical());
-        HeapMemoryMonitor.setTestDisableMemoryUpdates(true);
-        return null;
-      }
-    });
+    testMemoryEventTolerance(true);
   }
 
   @Test
   public void testEvictionMemoryEventTolerance() {
+    testMemoryEventTolerance(false);
+  }
+
+  private void testMemoryEventTolerance(boolean isCritical) {
     final Host host = Host.getHost(0);
     final VM vm = host.getVM(0);
     vm.invoke(new SerializableCallable() {
       @Override
       public Object call() throws Exception {
         HeapMemoryMonitor.setTestDisableMemoryUpdates(false);
-        String vendor = System.getProperty("java.vendor");
-        boolean isSun = (vendor.contains("Sun") || vendor.contains("Oracle"));
-        int defaultTolerance = isSun ? 1 : 5;
         GemFireCacheImpl cache = (GemFireCacheImpl) getCache();
         InternalResourceManager irm = cache.getInternalResourceManager();
         HeapMemoryMonitor hmm = irm.getHeapMonitor();
         hmm.setTestMaxMemoryBytes(100);
         HeapMemoryMonitor.setTestBytesUsedForThresholdSet(1);
-        irm.setEvictionHeapPercentage(50);
-        for (int i = 0; i < defaultTolerance; i++) {
-          hmm.updateStateAndSendEvent(55);
-          assertFalse(hmm.getState().isEviction());
+
+        if (isCritical) {
+          irm.setCriticalHeapPercentage(95);
+        } else {
+          irm.setEvictionHeapPercentage(50);
+        }
+
+        int previousMemoryStateChangeTolerance = hmm.getMemoryStateChangeTolerance();
+        final int criticalBytesUsed = 96;
+        final int evictionBytesUsed = 55;
+        final int memoryStateChangeTolerance = 3;
+
+        try {
+          hmm.setMemoryStateChangeTolerance(memoryStateChangeTolerance);
+
+          for (int i = 0; i < memoryStateChangeTolerance; i++) {
+            if (isCritical) {
+              hmm.updateStateAndSendEvent(criticalBytesUsed, "test");
+              assertFalse(hmm.getState().isCritical());
+            } else {
+              hmm.updateStateAndSendEvent(evictionBytesUsed, "test");
+              assertFalse(hmm.getState().isEviction());
+            }
+          }
+          if (isCritical) {
+            // Adding expected strings so we do not fail the
+            // test prematurely
+            getCache().getLogger().fine(addExpectedExString);
+            hmm.updateStateAndSendEvent(criticalBytesUsed, "test");
+            assertTrue(hmm.getState().isCritical());
+            getCache().getLogger().fine(removeExpectedExString);
+            getCache().getLogger().fine(addExpectedBelow);
+            final int belowCriticalBytes = 92;
+            hmm.updateStateAndSendEvent(belowCriticalBytes, "test");
+            getCache().getLogger().fine(removeExpectedBelow);
+            assertFalse(hmm.getState().isCritical());
+          } else {
+            hmm.updateStateAndSendEvent(evictionBytesUsed, "test");
+            assertTrue(hmm.getState().isEviction());
+            final int belowEvictionBytes = 45;
+            hmm.updateStateAndSendEvent(belowEvictionBytes, "test");
+            assertFalse(hmm.getState().isEviction());
+
+          }
+
+          HeapMemoryMonitor.setTestDisableMemoryUpdates(true);
+        } finally {
+          hmm.setMemoryStateChangeTolerance(previousMemoryStateChangeTolerance);
         }
-        hmm.updateStateAndSendEvent(55);
-        assertTrue(hmm.getState().isEviction());
-        hmm.updateStateAndSendEvent(45);
-        assertFalse(hmm.getState().isEviction());
-        HeapMemoryMonitor.setTestDisableMemoryUpdates(true);
         return null;
       }
     });
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/cache/query/dunit/ResourceManagerWithQueryMonitorDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/cache/query/dunit/ResourceManagerWithQueryMonitorDUnitTest.java
index 0a71172..7076acf 100755
--- a/geode-core/src/distributedTest/java/org/apache/geode/cache/query/dunit/ResourceManagerWithQueryMonitorDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/cache/query/dunit/ResourceManagerWithQueryMonitorDUnitTest.java
@@ -119,7 +119,7 @@ public class ResourceManagerWithQueryMonitorDUnitTest extends ClientServerTestCa
     public Object call() throws Exception {
       InternalResourceManager irm = ((GemFireCacheImpl) getCache()).getInternalResourceManager();
       // Reset CRITICAL_UP by informing all that heap usage is now 1 byte (0 would disable).
-      irm.getHeapMonitor().updateStateAndSendEvent(NORMAL_HEAP_USED);
+      irm.getHeapMonitor().updateStateAndSendEvent(NORMAL_HEAP_USED, "test");
       Set<ResourceListener> listeners = irm.getResourceListeners(ResourceType.HEAP_MEMORY);
       Iterator<ResourceListener> it = listeners.iterator();
       while (it.hasNext()) {
@@ -1043,7 +1043,7 @@ public class ResourceManagerWithQueryMonitorDUnitTest extends ClientServerTestCa
       public void run2() {
         InternalResourceManager resourceManager =
             (InternalResourceManager) getCache().getResourceManager();
-        resourceManager.getHeapMonitor().updateStateAndSendEvent(CRITICAL_HEAP_USED);
+        resourceManager.getHeapMonitor().updateStateAndSendEvent(CRITICAL_HEAP_USED, "test");
       }
     });
   }
@@ -1054,7 +1054,7 @@ public class ResourceManagerWithQueryMonitorDUnitTest extends ClientServerTestCa
       public void run2() {
         InternalResourceManager resourceManager =
             (InternalResourceManager) getCache().getResourceManager();
-        resourceManager.getHeapMonitor().updateStateAndSendEvent(NORMAL_HEAP_USED);
+        resourceManager.getHeapMonitor().updateStateAndSendEvent(NORMAL_HEAP_USED, "test");
       }
     });
   }
@@ -1336,7 +1336,7 @@ public class ResourceManagerWithQueryMonitorDUnitTest extends ClientServerTestCa
           if (count++ == numObjectsBeforeCancel) {
             InternalResourceManager resourceManager =
                 (InternalResourceManager) getCache().getResourceManager();
-            resourceManager.getHeapMonitor().updateStateAndSendEvent(CRITICAL_HEAP_USED);
+            resourceManager.getHeapMonitor().updateStateAndSendEvent(CRITICAL_HEAP_USED, "test");
             triggeredOOME = true;
           }
           break;
@@ -1356,7 +1356,7 @@ public class ResourceManagerWithQueryMonitorDUnitTest extends ClientServerTestCa
           if (triggeredOOME == false) {
             InternalResourceManager resourceManager =
                 (InternalResourceManager) getCache().getResourceManager();
-            resourceManager.getHeapMonitor().updateStateAndSendEvent(CRITICAL_HEAP_USED);
+            resourceManager.getHeapMonitor().updateStateAndSendEvent(CRITICAL_HEAP_USED, "test");
             triggeredOOME = true;
             try {
               Thread.sleep(1000);
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/PartitionedRegionEvictionDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/PartitionedRegionEvictionDUnitTest.java
index e00bb87..c5a6d85 100755
--- a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/PartitionedRegionEvictionDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/PartitionedRegionEvictionDUnitTest.java
@@ -49,7 +49,6 @@ import org.apache.geode.cache.SubscriptionAttributes;
 import org.apache.geode.cache.util.CacheListenerAdapter;
 import org.apache.geode.cache.util.ObjectSizer;
 import org.apache.geode.cache30.CacheSerializableRunnable;
-import org.apache.geode.distributed.internal.DistributionConfig;
 import org.apache.geode.internal.OSProcess;
 import org.apache.geode.internal.cache.control.HeapMemoryMonitor;
 import org.apache.geode.internal.cache.control.InternalResourceManager.ResourceType;
@@ -203,20 +202,18 @@ public class PartitionedRegionEvictionDUnitTest extends JUnit4CacheTestCase {
   protected void raiseFakeNotification() {
     ((GemFireCacheImpl) getCache()).getHeapEvictor().setTestAbortAfterLoopCount(1);
     HeapMemoryMonitor.setTestDisableMemoryUpdates(true);
-    System.setProperty(DistributionConfig.GEMFIRE_PREFIX + "memoryEventTolerance", "0");
 
     setEvictionPercentage(85);
     HeapMemoryMonitor hmm =
         ((GemFireCacheImpl) getCache()).getInternalResourceManager().getHeapMonitor();
     hmm.setTestMaxMemoryBytes(100);
 
-    hmm.updateStateAndSendEvent(90);
+    hmm.updateStateAndSendEvent(90, "test");
   }
 
   protected void cleanUpAfterFakeNotification() {
     ((GemFireCacheImpl) getCache()).getHeapEvictor().setTestAbortAfterLoopCount(Integer.MAX_VALUE);
     HeapMemoryMonitor.setTestDisableMemoryUpdates(false);
-    System.clearProperty(DistributionConfig.GEMFIRE_PREFIX + "memoryEventTolerance");
   }
 
   @Test
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/eviction/EvictionDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/eviction/EvictionDUnitTest.java
index 0e13301..8084523 100644
--- a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/eviction/EvictionDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/eviction/EvictionDUnitTest.java
@@ -239,7 +239,7 @@ public class EvictionDUnitTest {
     } else {
       HeapMemoryMonitor hmm = cache.getInternalResourceManager().getHeapMonitor();
       hmm.setTestMaxMemoryBytes(100);
-      hmm.updateStateAndSendEvent(90);
+      hmm.updateStateAndSendEvent(90, "test");
     }
 
     int entrySize = ENTRY_SIZE + 100;
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/control/MemoryMonitorJUnitTest.java b/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/control/MemoryMonitorJUnitTest.java
index c655d79..f5aa7d0 100644
--- a/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/control/MemoryMonitorJUnitTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/control/MemoryMonitorJUnitTest.java
@@ -114,7 +114,7 @@ public class MemoryMonitorJUnitTest {
         internalManager.getResourceListeners(ResourceType.HEAP_MEMORY);
     assertEquals(10 + SYSTEM_LISTENERS, heapListeners.size());
 
-    heapMonitor.updateStateAndSendEvent(700);
+    heapMonitor.updateStateAndSendEvent(700, "test");
     assertEquals(0, internalManager.getStats().getEvictionStartEvents());
     for (ResourceListener listener : heapListeners) {
       if (listener instanceof TestMemoryThresholdListener) {
@@ -123,7 +123,7 @@ public class MemoryMonitorJUnitTest {
     }
 
     // make sure listeners are invoked
-    heapMonitor.updateStateAndSendEvent(870);
+    heapMonitor.updateStateAndSendEvent(870, "test");
     assertEquals(0, internalManager.getStats().getEvictionStopEvents());
     assertEquals(1, internalManager.getStats().getEvictionStartEvents());
     assertEquals(0, internalManager.getStats().getHeapCriticalEvents());
@@ -135,7 +135,7 @@ public class MemoryMonitorJUnitTest {
     }
 
     // make sure same event is not triggered twice
-    heapMonitor.updateStateAndSendEvent(880);
+    heapMonitor.updateStateAndSendEvent(880, "test");
     assertEquals(0, internalManager.getStats().getEvictionStopEvents());
     assertEquals(1, internalManager.getStats().getEvictionStartEvents());
     assertEquals(0, internalManager.getStats().getHeapCriticalEvents());
@@ -160,10 +160,10 @@ public class MemoryMonitorJUnitTest {
     irm.addResourceListener(ResourceType.HEAP_MEMORY, listener);
 
     hmm.setTestMaxMemoryBytes(1000);
-    hmm.updateStateAndSendEvent(870);
+    hmm.updateStateAndSendEvent(870, "test");
     assertEquals(0, irm.getStats().getHeapCriticalEvents());
-    hmm.updateStateAndSendEvent(950);
-    hmm.updateStateAndSendEvent(770);
+    hmm.updateStateAndSendEvent(950, "test");
+    hmm.updateStateAndSendEvent(770, "test");
     assertEquals(0, irm.getStats().getHeapCriticalEvents());
     assertEquals(0, irm.getStats().getEvictionStartEvents());
 
@@ -174,7 +174,7 @@ public class MemoryMonitorJUnitTest {
     assertEquals(0, irm.getStats().getEvictionStartEvents());
     assertEquals(0, irm.getStats().getHeapCriticalEvents());
 
-    hmm.updateStateAndSendEvent(870);
+    hmm.updateStateAndSendEvent(870, "test");
     assertEquals(1, listener.getEvictionThresholdCalls());
     assertEquals(1, irm.getStats().getEvictionStartEvents());
     assertEquals(0, irm.getStats().getHeapCriticalEvents());
@@ -185,7 +185,7 @@ public class MemoryMonitorJUnitTest {
     assertEquals(1, irm.getStats().getEvictionStartEvents());
     assertEquals(0, irm.getStats().getHeapCriticalEvents());
 
-    hmm.updateStateAndSendEvent(870);
+    hmm.updateStateAndSendEvent(870, "test");
     assertEquals(1, irm.getStats().getEvictionStartEvents());
     assertEquals(0, irm.getStats().getHeapCriticalEvents());
 
@@ -194,7 +194,7 @@ public class MemoryMonitorJUnitTest {
     assertEquals(1, irm.getStats().getEvictionStartEvents());
     assertEquals(0, irm.getStats().getHeapCriticalEvents());
 
-    hmm.updateStateAndSendEvent(970);
+    hmm.updateStateAndSendEvent(970, "test");
     assertEquals(1, irm.getStats().getHeapCriticalEvents());
     assertEquals(0, listener.getEvictionThresholdCalls());
     assertEquals(1, irm.getStats().getEvictionStartEvents());
@@ -346,7 +346,7 @@ public class MemoryMonitorJUnitTest {
     assertEquals(2 + addSubregion + SYSTEM_LISTENERS,
         internalManager.getResourceListeners(ResourceType.HEAP_MEMORY).size());
 
-    heapMonitor.updateStateAndSendEvent(97);
+    heapMonitor.updateStateAndSendEvent(97, "test");
     assertEquals(1, listener.getCriticalThresholdCalls());
     boolean caughtException = false;
     try {
@@ -359,7 +359,7 @@ public class MemoryMonitorJUnitTest {
       throw new AssertionError("An expected exception was not thrown");
     }
     // make region healthy
-    heapMonitor.updateStateAndSendEvent(91);
+    heapMonitor.updateStateAndSendEvent(91, "test");
     // try the puts again
     try {
       region.put("key-1", "value-2");
@@ -458,28 +458,28 @@ public class MemoryMonitorJUnitTest {
     assertEquals(0, listener.getAllCalls());
     // test EVICTION, CRITICAL, EVICTION, NORMAL cycle
     for (int i = 0; i < 3; i++) {
-      hmm.updateStateAndSendEvent(82); // EVICTION
+      hmm.updateStateAndSendEvent(82, "test"); // EVICTION
       assertEquals(i * 4 + 1, listener.getAllCalls());
       assertEquals((i * 3) + 1, listener.getEvictionThresholdCalls());
       assertEquals(i + 1, irm.getStats().getEvictionStartEvents());
       assertEquals(82, listener.getCurrentHeapPercentage());
       assertEquals(2, listener.getBytesFromThreshold());
 
-      hmm.updateStateAndSendEvent(92); // CRITICAL
+      hmm.updateStateAndSendEvent(92, "test"); // CRITICAL
       assertEquals(i * 4 + 2, listener.getAllCalls());
       assertEquals(i + 1, listener.getCriticalThresholdCalls());
       assertEquals(i + 1, irm.getStats().getHeapCriticalEvents());
       assertEquals(92, listener.getCurrentHeapPercentage());
       assertEquals(2, listener.getBytesFromThreshold());
 
-      hmm.updateStateAndSendEvent(85); // EVICTION
+      hmm.updateStateAndSendEvent(85, "test"); // EVICTION
       assertEquals(i * 4 + 3, listener.getAllCalls());
       assertEquals((i * 3) + 3, listener.getEvictionThresholdCalls());
       assertEquals(i + 1, irm.getStats().getHeapSafeEvents());
       assertEquals(85, listener.getCurrentHeapPercentage());
       assertEquals(5, listener.getBytesFromThreshold());
 
-      hmm.updateStateAndSendEvent(76); // NORMAL
+      hmm.updateStateAndSendEvent(76, "test"); // NORMAL
       assertEquals(i * 4 + 4, listener.getAllCalls());
       assertEquals(i + 1, listener.getNormalCalls());
       assertEquals(i + 1, irm.getStats().getEvictionStopEvents());
@@ -488,8 +488,8 @@ public class MemoryMonitorJUnitTest {
     }
     listener.resetThresholdCalls();
 
-    // test EVICTION to CRITICAL back to EVICTION
-    hmm.updateStateAndSendEvent(95); // CRITICAL
+    // test CRITICAL back to NORMAL
+    hmm.updateStateAndSendEvent(95, "test"); // CRITICAL
     assertEquals(1, listener.getEvictionThresholdCalls());
     assertEquals(1, listener.getCriticalThresholdCalls());
     assertEquals(4, irm.getStats().getHeapCriticalEvents());
@@ -497,7 +497,7 @@ public class MemoryMonitorJUnitTest {
     assertEquals(1, listener.getAllCalls());
     assertEquals(95, listener.getCurrentHeapPercentage());
     assertEquals(5, listener.getBytesFromThreshold());
-    hmm.updateStateAndSendEvent(75); // EVICTION
+    hmm.updateStateAndSendEvent(75, "test"); // NORMAL
     assertEquals(1, listener.getNormalCalls());
     assertEquals(1, listener.getEvictionThresholdCalls());
     assertEquals(1, listener.getCriticalThresholdCalls());
@@ -508,47 +508,47 @@ public class MemoryMonitorJUnitTest {
 
     // generate many events in threshold thickness for eviction threshold
     for (int i = 0; i < 5; i++) {
-      hmm.updateStateAndSendEvent(82); // EVICTION
+      hmm.updateStateAndSendEvent(82, "test"); // EVICTION
       assertEquals(1, listener.getEvictionThresholdCalls());
       assertEquals((i * 2) + 1, listener.getAllCalls());
       assertEquals(82, listener.getCurrentHeapPercentage());
       assertEquals(2, listener.getBytesFromThreshold());
-      hmm.updateStateAndSendEvent(79); // EVICTION THICKNESS
+      hmm.updateStateAndSendEvent(79, "test"); // EVICTION THICKNESS
     }
     listener.resetThresholdCalls();
     // generate many events in threshold thickness for critical threshold
     for (int i = 0; i < 5; i++) {
-      hmm.updateStateAndSendEvent(92); // CRITICAL
+      hmm.updateStateAndSendEvent(92, "test"); // CRITICAL
       assertEquals(1, listener.getCriticalThresholdCalls());
       assertEquals((i * 2) + 1, listener.getAllCalls());
       assertEquals(92, listener.getCurrentHeapPercentage());
       assertEquals(2, listener.getBytesFromThreshold());
-      hmm.updateStateAndSendEvent(89); // CRITICAL THICKNESS
+      hmm.updateStateAndSendEvent(89, "test"); // CRITICAL THICKNESS
     }
-    hmm.updateStateAndSendEvent(75);
+    hmm.updateStateAndSendEvent(75, "test");
     listener.resetThresholdCalls();
 
     // generate many events around threshold thickness for eviction threshold
     for (int i = 1; i < 6; i++) {
-      hmm.updateStateAndSendEvent(82); // EVICTION
+      hmm.updateStateAndSendEvent(82, "test"); // EVICTION
       assertEquals(i, listener.getEvictionThresholdCalls());
       assertEquals(82, listener.getCurrentHeapPercentage());
       assertEquals(2, listener.getBytesFromThreshold());
-      hmm.updateStateAndSendEvent(77); // NORMAL
+      hmm.updateStateAndSendEvent(77, "test"); // NORMAL
       assertEquals(i, listener.getNormalCalls());
       assertEquals(77, listener.getCurrentHeapPercentage());
       assertEquals(3, listener.getBytesFromThreshold());
       assertEquals(i * 2, listener.getAllCalls());
     }
-    hmm.updateStateAndSendEvent(87); // EVICTION
+    hmm.updateStateAndSendEvent(87, "test"); // EVICTION
     listener.resetThresholdCalls();
     // generate many events around threshold thickness for critical threshold
     for (int i = 1; i < 6; i++) {
-      hmm.updateStateAndSendEvent(92); // CRITICAL
+      hmm.updateStateAndSendEvent(92, "test"); // CRITICAL
       assertEquals(i, listener.getCriticalThresholdCalls());
       assertEquals(92, listener.getCurrentHeapPercentage());
       assertEquals(2, listener.getBytesFromThreshold());
-      hmm.updateStateAndSendEvent(87); // EVICTION
+      hmm.updateStateAndSendEvent(87, "test"); // EVICTION
       assertEquals(i * 2, listener.getEvictionThresholdCalls());
       assertEquals(87, listener.getCurrentHeapPercentage());
       assertEquals(3, listener.getBytesFromThreshold());
@@ -557,19 +557,19 @@ public class MemoryMonitorJUnitTest {
     listener.resetThresholdCalls();
 
     // from CRITICAL drop to EVICTION THICKNESS, and then to NORMAL
-    hmm.updateStateAndSendEvent(96); // CRITICAL
+    hmm.updateStateAndSendEvent(96, "test"); // CRITICAL
     assertEquals(1, listener.getCriticalThresholdCalls());
     assertEquals(1, listener.getAllCalls());
     assertEquals(6, listener.getBytesFromThreshold());
     assertEquals(96, listener.getCurrentHeapPercentage());
     listener.resetThresholdCalls();
-    hmm.updateStateAndSendEvent(79); // EVICTION THICKNESS
+    hmm.updateStateAndSendEvent(79, "test"); // EVICTION THICKNESS
     assertEquals(1, listener.getEvictionThresholdCalls());
     assertEquals(1, listener.getAllCalls());
     assertEquals(11, listener.getBytesFromThreshold());
     assertEquals(79, listener.getCurrentHeapPercentage());
     listener.resetThresholdCalls();
-    hmm.updateStateAndSendEvent(77); // NORMAL
+    hmm.updateStateAndSendEvent(77, "test"); // NORMAL
     assertEquals(1, listener.getNormalCalls());
     assertEquals(1, listener.getAllCalls());
     assertEquals(3, listener.getBytesFromThreshold());
@@ -594,11 +594,11 @@ public class MemoryMonitorJUnitTest {
     // make sure that both thresholds are enabled, disable one threshold, make sure
     // events for the other are delivered, enable the threshold
     // eviction threshold
-    hmm.updateStateAndSendEvent(82); // EVICTION
+    hmm.updateStateAndSendEvent(82, "test"); // EVICTION
     assertEquals(1, listener.getEvictionThresholdCalls());
     assertEquals(1, listener.getAllCalls());
     assertEquals(1, irm.getStats().getEvictionStartEvents());
-    hmm.updateStateAndSendEvent(92); // CRITICAL
+    hmm.updateStateAndSendEvent(92, "test"); // CRITICAL
     assertEquals(1, listener.getCriticalThresholdCalls());
     assertEquals(2, listener.getAllCalls());
     assertEquals(1, irm.getStats().getHeapCriticalEvents());
@@ -613,16 +613,16 @@ public class MemoryMonitorJUnitTest {
     assertEquals(0, irm.getStats().getEvictionThreshold());
     assertEquals(1, irm.getStats().getEvictionStopEvents());
 
-    hmm.updateStateAndSendEvent(75); // EVICTION_DISABLED
+    hmm.updateStateAndSendEvent(75, "test"); // EVICTION_DISABLED
     assertEquals(1, listener.getNormalCalls());
     assertEquals(0, listener.getEvictionThresholdCalls());
     assertEquals(1, irm.getStats().getEvictionStopEvents());
     assertEquals(1, irm.getStats().getHeapSafeEvents());
-    hmm.updateStateAndSendEvent(85); // EVICTION_DISABLED
+    hmm.updateStateAndSendEvent(85, "test"); // EVICTION_DISABLED
     assertEquals(0, listener.getEvictionThresholdCalls());
     assertEquals(1, irm.getStats().getEvictionStartEvents());
     assertEquals(2, listener.getAllCalls());
-    hmm.updateStateAndSendEvent(92); // CRITICAL
+    hmm.updateStateAndSendEvent(92, "test"); // CRITICAL
     assertEquals(3, listener.getAllCalls());
     assertEquals(2, listener.getCriticalThresholdCalls());
     listener.resetThresholdCalls();
@@ -630,18 +630,18 @@ public class MemoryMonitorJUnitTest {
     HeapMemoryMonitor.setTestBytesUsedForThresholdSet(92);
     irm.setEvictionHeapPercentage(80f);
     assertEquals(1, listener.getEvictionThresholdCalls());
-    hmm.updateStateAndSendEvent(84); // EVICTION
+    hmm.updateStateAndSendEvent(84, "test"); // EVICTION
     assertEquals(2, listener.getAllCalls()); // EVICTION_CRITICAL+EVICTION
-    hmm.updateStateAndSendEvent(77); // NORMAL
+    hmm.updateStateAndSendEvent(77, "test"); // NORMAL
     assertEquals(1, listener.getNormalCalls());
     assertEquals(3, listener.getAllCalls());
-    hmm.updateStateAndSendEvent(82); // EVICTION
+    hmm.updateStateAndSendEvent(82, "test"); // EVICTION
     assertEquals(3, listener.getEvictionThresholdCalls());
     assertEquals(4, listener.getAllCalls());
-    hmm.updateStateAndSendEvent(91); // CRITICAL
+    hmm.updateStateAndSendEvent(91, "test"); // CRITICAL
     assertEquals(2, listener.getCriticalThresholdCalls());
     assertEquals(5, listener.getAllCalls());
-    hmm.updateStateAndSendEvent(87); // EVICTION
+    hmm.updateStateAndSendEvent(87, "test"); // EVICTION
     assertEquals(5, listener.getEvictionThresholdCalls());
     assertEquals(3, irm.getStats().getHeapSafeEvents());
     assertEquals(6, listener.getAllCalls());
@@ -655,17 +655,17 @@ public class MemoryMonitorJUnitTest {
     assertEquals(1, listener.getCriticalDisabledCalls());
     assertEquals(0, irm.getStats().getCriticalThreshold());
 
-    hmm.updateStateAndSendEvent(92); // NO EVENT
+    hmm.updateStateAndSendEvent(92, "test"); // NO EVENT
     assertEquals(0, listener.getCriticalThresholdCalls());
     assertEquals(3, irm.getStats().getHeapCriticalEvents());
     assertEquals(2, listener.getAllCalls());
-    hmm.updateStateAndSendEvent(89); // NO EVENT
+    hmm.updateStateAndSendEvent(89, "test"); // NO EVENT
     assertEquals(3, irm.getStats().getHeapSafeEvents());
     assertEquals(3, listener.getAllCalls());
-    hmm.updateStateAndSendEvent(77); // NORMAL
+    hmm.updateStateAndSendEvent(77, "test"); // NORMAL
     assertEquals(1, listener.getNormalCalls());
     assertEquals(4, listener.getAllCalls());
-    hmm.updateStateAndSendEvent(93); // EVICTION
+    hmm.updateStateAndSendEvent(93, "test"); // EVICTION
     assertEquals(2, listener.getEvictionThresholdCalls());
     assertEquals(5, listener.getAllCalls());
     listener.resetThresholdCalls();
@@ -673,18 +673,18 @@ public class MemoryMonitorJUnitTest {
     HeapMemoryMonitor.setTestBytesUsedForThresholdSet(93);
     irm.setCriticalHeapPercentage(90f); // forced event
     assertEquals(1, listener.getAllCalls());
-    hmm.updateStateAndSendEvent(92); // NO EVENT
+    hmm.updateStateAndSendEvent(92, "test"); // NO EVENT
     assertEquals(1, listener.getCriticalThresholdCalls());
     assertEquals(2, listener.getAllCalls());
-    hmm.updateStateAndSendEvent(89); // CRITICAL THICKNESS
+    hmm.updateStateAndSendEvent(89, "test"); // CRITICAL THICKNESS
     assertEquals(1, listener.getEvictionThresholdCalls());
-    hmm.updateStateAndSendEvent(87); // EVICTION
+    hmm.updateStateAndSendEvent(87, "test"); // EVICTION
     assertEquals(2, listener.getEvictionThresholdCalls());
     assertEquals(4, listener.getAllCalls());
-    hmm.updateStateAndSendEvent(77); // NORMAL
+    hmm.updateStateAndSendEvent(77, "test"); // NORMAL
     assertEquals(1, listener.getNormalCalls());
     assertEquals(5, listener.getAllCalls());
-    hmm.updateStateAndSendEvent(83); // EVICTION
+    hmm.updateStateAndSendEvent(83, "test"); // EVICTION
     assertEquals(3, listener.getEvictionThresholdCalls());
     assertEquals(6, listener.getAllCalls());
     listener.resetThresholdCalls();
@@ -697,9 +697,9 @@ public class MemoryMonitorJUnitTest {
     irm.setEvictionHeapPercentage(0f);
     assertEquals(1, listener.getEvictionDisabledCalls());
 
-    hmm.updateStateAndSendEvent(95); // NO EVENT
-    hmm.updateStateAndSendEvent(87); // NO EVENT
-    hmm.updateStateAndSendEvent(77); // NO EVENT
+    hmm.updateStateAndSendEvent(95, "test"); // NO EVENT
+    hmm.updateStateAndSendEvent(87, "test"); // NO EVENT
+    hmm.updateStateAndSendEvent(77, "test"); // NO EVENT
     assertEquals(5, listener.getAllCalls()); // the two DISABLE calls
     listener.resetThresholdCalls();
 
@@ -707,23 +707,23 @@ public class MemoryMonitorJUnitTest {
     HeapMemoryMonitor.setTestBytesUsedForThresholdSet(77);
     irm.setEvictionHeapPercentage(80f);
     assertEquals(0, listener.getEvictionThresholdCalls());
-    hmm.updateStateAndSendEvent(88); // EVICTION
+    hmm.updateStateAndSendEvent(88, "test"); // EVICTION
     assertEquals(1, listener.getEvictionThresholdCalls());
     assertEquals(2, listener.getAllCalls());
-    hmm.updateStateAndSendEvent(92); // NO EVENT
+    hmm.updateStateAndSendEvent(92, "test"); // NO EVENT
     assertEquals(3, listener.getAllCalls());
-    hmm.updateStateAndSendEvent(77); // NORMAL
+    hmm.updateStateAndSendEvent(77, "test"); // NORMAL
     assertEquals(2, listener.getNormalCalls());
-    hmm.updateStateAndSendEvent(98); // EVICTION
+    hmm.updateStateAndSendEvent(98, "test"); // EVICTION
     assertEquals(2, listener.getEvictionThresholdCalls());
     assertEquals(5, listener.getAllCalls());
     HeapMemoryMonitor.setTestBytesUsedForThresholdSet(98);
     irm.setEvictionHeapPercentage(0f); // resets old state
     listener.resetThresholdCalls();
 
-    hmm.updateStateAndSendEvent(87); // NO EVENT
-    hmm.updateStateAndSendEvent(77); // NO EVENT
-    hmm.updateStateAndSendEvent(85); // NO EVENT
+    hmm.updateStateAndSendEvent(87, "test"); // NO EVENT
+    hmm.updateStateAndSendEvent(77, "test"); // NO EVENT
+    hmm.updateStateAndSendEvent(85, "test"); // NO EVENT
     assertEquals(3, listener.getAllCalls());
 
     // enable critical, verify that forced event is not generated
@@ -731,15 +731,15 @@ public class MemoryMonitorJUnitTest {
     irm.setCriticalHeapPercentage(90f);
     assertEquals(0, listener.getCriticalThresholdCalls());
     assertEquals(4, listener.getAllCalls());
-    hmm.updateStateAndSendEvent(94); // CRITICAL
+    hmm.updateStateAndSendEvent(94, "test"); // CRITICAL
     assertEquals(5, listener.getAllCalls());
-    hmm.updateStateAndSendEvent(77); // NORMAL
+    hmm.updateStateAndSendEvent(77, "test"); // NORMAL
     assertEquals(2, listener.getNormalCalls());
     assertEquals(6, listener.getAllCalls());
-    hmm.updateStateAndSendEvent(87); // NO EVENT
+    hmm.updateStateAndSendEvent(87, "test"); // NO EVENT
     assertEquals(6, listener.getAllCalls());
     assertEquals(1, listener.getCriticalThresholdCalls());
-    hmm.updateStateAndSendEvent(85); // NORMAL
+    hmm.updateStateAndSendEvent(85, "test"); // NORMAL
     assertEquals(2, listener.getNormalCalls());
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/control/HeapMemoryMonitor.java b/geode-core/src/main/java/org/apache/geode/internal/cache/control/HeapMemoryMonitor.java
index 684b63e..3047670 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/control/HeapMemoryMonitor.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/control/HeapMemoryMonitor.java
@@ -83,22 +83,6 @@ public class HeapMemoryMonitor implements NotificationListener, MemoryMonitor {
   // Listener for heap memory usage as reported by the Cache stats.
   private final LocalStatListener statListener = new LocalHeapStatListener();
 
-  /*
-   * Number of eviction or critical state changes that have to occur before the event is delivered.
-   * This was introduced because we saw sudden memory usage spikes in jrockit VM.
-   */
-  private static final int memoryStateChangeTolerance;
-  static {
-    String vendor = System.getProperty("java.vendor");
-    if (vendor.contains("Sun") || vendor.contains("Oracle")) {
-      memoryStateChangeTolerance =
-          Integer.getInteger(DistributionConfig.GEMFIRE_PREFIX + "memoryEventTolerance", 1);
-    } else {
-      memoryStateChangeTolerance =
-          Integer.getInteger(DistributionConfig.GEMFIRE_PREFIX + "memoryEventTolerance", 5);
-    }
-  }
-
   // JVM MXBean used to report changes in heap memory usage
   private static final MemoryPoolMXBean tenuredMemoryPoolMXBean;
   static {
@@ -146,16 +130,11 @@ public class HeapMemoryMonitor implements NotificationListener, MemoryMonitor {
   private volatile MemoryState currentState = MemoryState.DISABLED;
 
   // Set when startMonitoring() and stopMonitoring() are called
-  private Boolean started = false;
+  boolean started = false;
 
   // Set to true when setEvictionThreshold(...) is called.
   private boolean hasEvictionThreshold = false;
 
-  // Only change state when these counters exceed {@link
-  // HeapMemoryMonitor#memoryStateChangeTolerance}
-  private int criticalToleranceCounter;
-  private int evictionToleranceCounter;
-
   private final InternalResourceManager resourceManager;
   private final ResourceAdvisor resourceAdvisor;
   private final InternalCache cache;
@@ -232,6 +211,14 @@ public class HeapMemoryMonitor implements NotificationListener, MemoryMonitor {
     return builder.toString();
   }
 
+  public void setMemoryStateChangeTolerance(int memoryStateChangeTolerance) {
+    thresholds.setMemoryStateChangeTolerance(memoryStateChangeTolerance);
+  }
+
+  public int getMemoryStateChangeTolerance() {
+    return thresholds.getMemoryStateChangeTolerance();
+  }
+
   /**
    * Monitoring is done using a combination of data from the JVM and statistics collected from the
    * cache. A usage threshold is set on the MemoryMXBean of the JVM to get notifications when the
@@ -452,18 +439,18 @@ public class HeapMemoryMonitor implements NotificationListener, MemoryMonitor {
    */
   public void updateStateAndSendEvent() {
     updateStateAndSendEvent(
-        testBytesUsedForThresholdSet != -1 ? testBytesUsedForThresholdSet : getBytesUsed());
+        testBytesUsedForThresholdSet != -1 ? testBytesUsedForThresholdSet : getBytesUsed(),
+        "notification");
   }
 
   /**
    * Compare the number of bytes used to the thresholds. If necessary, change the state and send an
    * event for the state change.
    *
-   * Public for testing.
-   *
    * @param bytesUsed Number of bytes of heap memory currently used.
+   * @param eventOrigin Indicates where the event originated e.g. notification vs polling
    */
-  public void updateStateAndSendEvent(long bytesUsed) {
+  public void updateStateAndSendEvent(long bytesUsed, String eventOrigin) {
     this.stats.changeTenuredHeapUsed(bytesUsed);
     synchronized (this) {
       MemoryState oldState = this.mostRecentEvent.getState();
@@ -471,16 +458,14 @@ public class HeapMemoryMonitor implements NotificationListener, MemoryMonitor {
       if (oldState != newState) {
         setUsageThresholdOnMXBean(bytesUsed);
 
-        if (!skipEventDueToToleranceLimits(oldState, newState)) {
-          this.currentState = newState;
+        this.currentState = newState;
 
-          MemoryEvent event = new MemoryEvent(ResourceType.HEAP_MEMORY, oldState, newState,
-              this.cache.getMyId(), bytesUsed, true, this.thresholds);
+        MemoryEvent event = new MemoryEvent(ResourceType.HEAP_MEMORY, oldState, newState,
+            this.cache.getMyId(), bytesUsed, true, this.thresholds);
 
-          this.upcomingEvent.set(event);
-          processLocalEvent(event);
-          updateStatsFromEvent(event);
-        }
+        this.upcomingEvent.set(event);
+        processLocalEvent(event, eventOrigin);
+        updateStatsFromEvent(event);
 
         // The state didn't change. However, if the state isn't normal and the
         // number of bytes used changed, then go ahead and send the event
@@ -489,7 +474,7 @@ public class HeapMemoryMonitor implements NotificationListener, MemoryMonitor {
         MemoryEvent event = new MemoryEvent(ResourceType.HEAP_MEMORY, oldState, newState,
             this.cache.getMyId(), bytesUsed, true, this.thresholds);
         this.upcomingEvent.set(event);
-        processLocalEvent(event);
+        processLocalEvent(event, eventOrigin);
       }
     }
   }
@@ -582,49 +567,6 @@ public class HeapMemoryMonitor implements NotificationListener, MemoryMonitor {
   }
 
   /**
-   * To avoid memory spikes in jrockit, we only deliver events if we receive more than
-   * {@link HeapMemoryMonitor#memoryStateChangeTolerance} of the same state change.
-   *
-   * @return True if an event should be skipped, false otherwise.
-   */
-  private boolean skipEventDueToToleranceLimits(MemoryState oldState, MemoryState newState) {
-    if (testDisableMemoryUpdates) {
-      return false;
-    }
-
-    if (newState.isEviction() && !oldState.isEviction()) {
-      this.evictionToleranceCounter++;
-      this.criticalToleranceCounter = 0;
-      if (this.evictionToleranceCounter <= memoryStateChangeTolerance) {
-        if (logger.isDebugEnabled()) {
-          logger.debug("State " + newState + " ignored. toleranceCounter:"
-              + this.evictionToleranceCounter + " MEMORY_EVENT_TOLERANCE:"
-              + memoryStateChangeTolerance);
-        }
-        return true;
-      }
-    } else if (newState.isCritical()) {
-      this.criticalToleranceCounter++;
-      this.evictionToleranceCounter = 0;
-      if (this.criticalToleranceCounter <= memoryStateChangeTolerance) {
-        if (logger.isDebugEnabled()) {
-          logger.debug("State " + newState + " ignored. toleranceCounter:"
-              + this.criticalToleranceCounter + " MEMORY_EVENT_TOLERANCE:"
-              + memoryStateChangeTolerance);
-        }
-        return true;
-      }
-    } else {
-      this.criticalToleranceCounter = 0;
-      this.evictionToleranceCounter = 0;
-      if (logger.isDebugEnabled()) {
-        logger.debug("TOLERANCE counters reset");
-      }
-    }
-    return false;
-  }
-
-  /**
    * Returns the number of bytes of memory reported by the tenured pool as currently in use.
    */
   @Override
@@ -644,8 +586,9 @@ public class HeapMemoryMonitor implements NotificationListener, MemoryMonitor {
    * Package private for testing.
    *
    * @param event Event to process.
+   * @param eventOrigin Indicates where the event originated e.g. notification vs polling
    */
-  synchronized void processLocalEvent(MemoryEvent event) {
+  synchronized void processLocalEvent(MemoryEvent event, String eventOrigin) {
     assert event.isLocal();
 
     if (logger.isDebugEnabled()) {
@@ -654,16 +597,16 @@ public class HeapMemoryMonitor implements NotificationListener, MemoryMonitor {
 
     if (event.getState().isCritical() && !event.getPreviousState().isCritical()) {
       this.cache.getLogger().error(
-          String.format("Member: %s above %s critical threshold",
-              new Object[] {event.getMember(), "heap"}));
+          String.format("Member: %s above %s critical threshold. Event generated via %s.",
+              event.getMember(), "heap", eventOrigin));
       if (!this.cache.isQueryMonitorDisabledForLowMemory()) {
         this.cache.getQueryMonitor().setLowMemory(true, event.getBytesUsed());
       }
 
     } else if (!event.getState().isCritical() && event.getPreviousState().isCritical()) {
       this.cache.getLogger().error(
-          String.format("Member: %s below %s critical threshold",
-              new Object[] {event.getMember(), "heap"}));
+          String.format("Member: %s below %s critical threshold. Event generated via %s.",
+              event.getMember(), "heap", eventOrigin));
       if (!this.cache.isQueryMonitorDisabledForLowMemory()) {
         this.cache.getQueryMonitor().setLowMemory(false, event.getBytesUsed());
       }
@@ -671,10 +614,10 @@ public class HeapMemoryMonitor implements NotificationListener, MemoryMonitor {
 
     if (event.getState().isEviction() && !event.getPreviousState().isEviction()) {
       this.cache.getLogger().info(String.format("Member: %s above %s eviction threshold",
-          new Object[] {event.getMember(), "heap"}));
+          event.getMember(), "heap"));
     } else if (!event.getState().isEviction() && event.getPreviousState().isEviction()) {
       this.cache.getLogger().info(String.format("Member: %s below %s eviction threshold",
-          new Object[] {event.getMember(), "heap"}));
+          event.getMember(), "heap"));
     }
 
     if (logger.isDebugEnabled()) {
@@ -817,7 +760,7 @@ public class HeapMemoryMonitor implements NotificationListener, MemoryMonitor {
           @Override
           public void run() {
             if (!testDisableMemoryUpdates) {
-              updateStateAndSendEvent(usedBytes);
+              updateStateAndSendEvent(usedBytes, "polling");
             }
           }
         });
@@ -838,8 +781,7 @@ public class HeapMemoryMonitor implements NotificationListener, MemoryMonitor {
   @Override
   public String toString() {
     return "HeapMemoryMonitor [thresholds=" + this.thresholds + ", mostRecentEvent="
-        + this.mostRecentEvent + ", criticalToleranceCounter=" + this.criticalToleranceCounter
-        + ", evictionToleranceCounter=" + this.evictionToleranceCounter + "]";
+        + this.mostRecentEvent + "]";
   }
 
   /**
@@ -853,7 +795,7 @@ public class HeapMemoryMonitor implements NotificationListener, MemoryMonitor {
         return;
       }
       try {
-        updateStateAndSendEvent(getBytesUsed());
+        updateStateAndSendEvent(getBytesUsed(), "polling");
       } catch (Exception e) {
         HeapMemoryMonitor.logger.debug("Poller Thread caught exception:", e);
       }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/control/MemoryThresholds.java b/geode-core/src/main/java/org/apache/geode/internal/cache/control/MemoryThresholds.java
index 248ceb4..d5e6ec4 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/control/MemoryThresholds.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/control/MemoryThresholds.java
@@ -127,6 +127,17 @@ public class MemoryThresholds {
   // Number of bytes used below which memory will leave the eviction state
   private final long evictionThresholdClearBytes;
 
+  /*
+   * Number of eviction or critical state changes that have to occur before the event is delivered.
+   * The default is 0 so we will change states immediately by default.
+   */
+  private static int memoryStateChangeTolerance =
+      Integer.getInteger(DistributionConfig.GEMFIRE_PREFIX + "memoryEventTolerance", 0);
+
+  // Only change state when these counters exceed {@link
+  // HeapMemoryMonitor#memoryStateChangeTolerance}
+  private transient int toleranceCounter;
+
   MemoryThresholds(long maxMemoryBytes) {
     this(maxMemoryBytes, DEFAULT_CRITICAL_PERCENTAGE, DEFAULT_EVICTION_PERCENTAGE);
   }
@@ -180,13 +191,14 @@ public class MemoryThresholds {
     if (this.evictionThreshold != 0 && this.criticalThreshold != 0) {
       if (bytesUsed < this.evictionThresholdClearBytes
           || (!oldState.isEviction() && bytesUsed < this.evictionThresholdBytes)) {
+        toleranceCounter = 0;
         return MemoryState.NORMAL;
       }
       if (bytesUsed < this.criticalThresholdClearBytes
           || (!oldState.isCritical() && bytesUsed < this.criticalThresholdBytes)) {
-        return MemoryState.EVICTION;
+        return checkToleranceAndGetNextState(MemoryState.EVICTION, oldState);
       }
-      return MemoryState.EVICTION_CRITICAL;
+      return checkToleranceAndGetNextState(MemoryState.EVICTION_CRITICAL, oldState);
     }
 
     // Are both eviction and critical thresholds disabled?
@@ -198,18 +210,20 @@ public class MemoryThresholds {
     if (this.evictionThreshold == 0) {
       if (bytesUsed < this.criticalThresholdClearBytes
           || (!oldState.isCritical() && bytesUsed < this.criticalThresholdBytes)) {
+        toleranceCounter = 0;
         return MemoryState.EVICTION_DISABLED;
       }
-      return MemoryState.EVICTION_DISABLED_CRITICAL;
+      return checkToleranceAndGetNextState(MemoryState.EVICTION_DISABLED_CRITICAL, oldState);
     }
 
     // Just the eviction threshold is enabled
     if (bytesUsed < this.evictionThresholdClearBytes
         || (!oldState.isEviction() && bytesUsed < this.evictionThresholdBytes)) {
+      toleranceCounter = 0;
       return MemoryState.CRITICAL_DISABLED;
     }
 
-    return MemoryState.EVICTION_CRITICAL_DISABLED;
+    return checkToleranceAndGetNextState(MemoryState.EVICTION_CRITICAL_DISABLED, oldState);
   }
 
   @Override
@@ -261,6 +275,14 @@ public class MemoryThresholds {
     return this.evictionThreshold > 0.0f;
   }
 
+  void setMemoryStateChangeTolerance(int memoryStateChangeTolerance) {
+    MemoryThresholds.memoryStateChangeTolerance = memoryStateChangeTolerance;
+  }
+
+  int getMemoryStateChangeTolerance() {
+    return MemoryThresholds.memoryStateChangeTolerance;
+  }
+
   /**
    * Generate a Thresholds object from data available from the DataInput
    *
@@ -284,4 +306,16 @@ public class MemoryThresholds {
     out.writeFloat(this.criticalThreshold);
     out.writeFloat(this.evictionThreshold);
   }
+
+  /**
+   * To avoid memory spikes in JVMs susceptible to bad heap memory
+   * reads/outliers, we only deliver events if we receive more than
+   * memoryStateChangeTolerance of the same state change.
+   *
+   * @return New state if above tolerance, old state if below
+   */
+  private MemoryState checkToleranceAndGetNextState(MemoryState newState, MemoryState oldState) {
+    return memoryStateChangeTolerance > 0
+        && toleranceCounter++ < memoryStateChangeTolerance ? oldState : newState;
+  }
 }
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/control/HeapMemoryMonitorTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/control/HeapMemoryMonitorTest.java
index 0539b03..9dd8ab5 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/control/HeapMemoryMonitorTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/control/HeapMemoryMonitorTest.java
@@ -17,12 +17,14 @@ package org.apache.geode.internal.cache.control;
 import static org.assertj.core.api.Assertions.assertThat;
 import static org.assertj.core.api.Assertions.assertThatThrownBy;
 import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.spy;
 import static org.mockito.Mockito.when;
 
 import java.util.Collections;
 import java.util.HashSet;
 import java.util.Set;
 
+import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 
@@ -32,6 +34,7 @@ import org.apache.geode.distributed.DistributedMember;
 import org.apache.geode.distributed.DistributedSystem;
 import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
 import org.apache.geode.internal.cache.InternalCache;
+import org.apache.geode.test.fake.Fakes;
 
 public class HeapMemoryMonitorTest {
 
@@ -41,12 +44,19 @@ public class HeapMemoryMonitorTest {
   private DistributedMember member;
   private InternalDistributedMember myself;
   private ResourceAdvisor resourceAdvisor;
+  private int previousMemoryStateChangeTolerance;
+  private static final int memoryStateChangeTolerance = 3;
   private static final String LOW_MEMORY_REGEX =
       "Function: null cannot be executed because the members.*are running low on memory";
+  private static final int criticalUsedBytes = 95;
+  private static final int evictionUsedBytes = 85;
+  private static final int normalUsedBytes = 60;
+  private static final int testMemoryEventTolerance = 3;
+
 
   @Before
   public void setup() {
-    InternalCache internalCache = mock(InternalCache.class);
+    InternalCache internalCache = Fakes.cache();
     DistributedSystem distributedSystem = mock(DistributedSystem.class);
     function = mock(Function.class);
     member = mock(InternalDistributedMember.class);
@@ -57,7 +67,10 @@ public class HeapMemoryMonitorTest {
     when(internalCache.getDistributionAdvisor()).thenReturn(resourceAdvisor);
     when(internalCache.getMyId()).thenReturn(myself);
 
-    heapMonitor = new HeapMemoryMonitor(null, internalCache, null);
+    heapMonitor = new HeapMemoryMonitor(mock(InternalResourceManager.class), internalCache,
+        mock(ResourceManagerStats.class));
+    previousMemoryStateChangeTolerance = heapMonitor.getMemoryStateChangeTolerance();
+    heapMonitor.setMemoryStateChangeTolerance(memoryStateChangeTolerance);
     memberSet = new HashSet<>();
     memberSet.add(member);
     heapMonitor.setMostRecentEvent(new MemoryEvent(InternalResourceManager.ResourceType.HEAP_MEMORY,
@@ -65,6 +78,11 @@ public class HeapMemoryMonitorTest {
         true, null)); // myself is not critical
   }
 
+  @After
+  public void cleanup() {
+    heapMonitor.setMemoryStateChangeTolerance(previousMemoryStateChangeTolerance);
+  }
+
   // ========== tests for getHeapCriticalMembersFrom ==========
   @Test
   public void getHeapCriticalMembersFrom_WithEmptyCriticalMembersReturnsEmptySet() {
@@ -248,7 +266,182 @@ public class HeapMemoryMonitorTest {
         .isExactlyInstanceOf(LowMemoryException.class).hasMessageMatching(LOW_MEMORY_REGEX);
   }
 
+  // ========== tests for updateStateAndSendEvent ==========
+  @Test
+  public void updateStateAndSendEvent_ThrashingShouldNotChangeState_CriticalAndEvictionEnabled() {
+    // Initialize the most recent state to NORMAL
+    setupHeapMonitorThresholds(true, true);
+
+    // If we thrash between CRITICAL_EVICTION and NORMAL, we don't expect a state transition
+    // to happen because we have a memoryStateChangeTolerance of 3 in this test. We only expect
+    // a state transition if threshold value + 1 consecutive events have been received above
+    // the critical threshold.
+    sendAlternatingEventsAndAssertState(MemoryThresholds.MemoryState.NORMAL);
+  }
+
+  @Test
+  public void updateStateAndSendEvent_ThrashingShouldNotChangeState_CriticalOnlyEnabled() {
+    // Initialize the most recent state to NORMAL
+    setupHeapMonitorThresholds(false, true);
+
+    // If we thrash between EVICTION_DISABLED_CRITICAL and NORMAL, we don't expect a state
+    // transition
+    // to happen because we have a memoryStateChangeTolerance of 3 in this test. We only expect
+    // a state transition if threshold value + 1 consecutive events have been received above
+    // the critical threshold.
+    sendAlternatingEventsAndAssertState(MemoryThresholds.MemoryState.EVICTION_DISABLED);
+  }
+
+  @Test
+  public void updateStateAndSendEvent_ThrashingShouldNotChangeState_EvictionOnlyEnabled() {
+    // Initialize the most recent state to NORMAL
+    setupHeapMonitorThresholds(true, false);
+
+    // If we thrash between EVICTION_CRITICAL_DISABLED and NORMAL, we don't expect a state
+    // transition
+    // to happen because we have a memoryStateChangeTolerance of 3 in this test. We only expect
+    // a state transition if threshold value + 1 consecutive events have been received above
+    // the critical threshold.
+    sendAlternatingEventsAndAssertState(MemoryThresholds.MemoryState.CRITICAL_DISABLED);
+  }
+
+  @Test
+  public void updateStateAndSendEvent_AboveCriticalMoreThanEventTolerance() {
+    setupHeapMonitorThresholds(true, true);
+
+    // It will take 4 consecutive events above the critical threshold to cause a state transition
+    // given our memoryStateChangeTolerance of 3 in this test.
+    sendEventAndAssertState(criticalUsedBytes, testMemoryEventTolerance,
+        MemoryThresholds.MemoryState.NORMAL);
+    sendEventAndAssertState(criticalUsedBytes, 1, MemoryThresholds.MemoryState.EVICTION_CRITICAL);
+  }
+
+  @Test
+  public void updateStateAndSendEvent_AboveCriticalTwoEventsThenAboveEviction() {
+    // Initialize the most recent state to NORMAL
+    setupHeapMonitorThresholds(true, true);
+
+    // The first three events are above the CRITICAL threshold and will count towards the
+    // memoryStateChangeTolerance of 3, but the last event is only above the eviction
+    // threshold so we expect the state transition to be from NORMAL to EVICTION.
+    sendEventAndAssertState(criticalUsedBytes, testMemoryEventTolerance,
+        MemoryThresholds.MemoryState.NORMAL);
+    sendEventAndAssertState(evictionUsedBytes, 1, MemoryThresholds.MemoryState.EVICTION);
+  }
+
+  @Test
+  public void updateStateAndSendEvent_ThreeEvictionsThenCriticalTransitionEvictionCritical() {
+    // Initialize the most recent state to NORMAL
+    setupHeapMonitorThresholds(true, true);
+
+    // The first three events are above the EVICTION threshold and will count towards the
+    // memoryStateChangeTolerance of 3, but the last event is only above the eviction
+    // threshold so we expect the state transition to be from NORMAL to EVICTION_CRITICAL.
+    sendEventAndAssertState(evictionUsedBytes, testMemoryEventTolerance,
+        MemoryThresholds.MemoryState.NORMAL);
+    sendEventAndAssertState(criticalUsedBytes, 1, MemoryThresholds.MemoryState.EVICTION_CRITICAL);
+  }
+
+  @Test
+  public void updateStateAndSendEvent_EvictionDisabledTransitionToCritical() {
+    // In this test, the EVICTION threshold is disabled, so we'd expect a transition from
+    // EVICTION_DISABLED to EVICTION_DISABLED_CRITICAL after the memoryStateChangeTolerance
+    // of 3 is exceeded.
+    setupHeapMonitorThresholds(false, true);
+
+    sendEventAndAssertState(criticalUsedBytes, testMemoryEventTolerance,
+        MemoryThresholds.MemoryState.EVICTION_DISABLED);
+    sendEventAndAssertState(criticalUsedBytes, 1,
+        MemoryThresholds.MemoryState.EVICTION_DISABLED_CRITICAL);
+  }
+
+  @Test
+  public void updateStateAndSendEvent_CriticalDisabledTransitionToEviction() {
+    // In this test, the CRITICAL threshold is disabled, so we'd expect a transition from
+    // CRITICAL_DISABLED to EVICTION_CRITICAL_DISABLED after the memoryStateChangeTolerance
+    // of 3 is exceeded.
+    setupHeapMonitorThresholds(true, false);
+
+    // It should take 4 above critical events for the state transition to take effect, because
+    // our memory state change tolerance is set to 3 for this test
+    sendEventAndAssertState(evictionUsedBytes, testMemoryEventTolerance,
+        MemoryThresholds.MemoryState.CRITICAL_DISABLED);
+    sendEventAndAssertState(evictionUsedBytes, 1,
+        MemoryThresholds.MemoryState.EVICTION_CRITICAL_DISABLED);
+  }
+
+  @Test
+  public void updateStateAndSendEvent_TogglingBetweenEvictionAndCritical_StatesTransition() {
+    setupHeapMonitorThresholds(true, true);
+
+    sendEventAndAssertState(criticalUsedBytes, testMemoryEventTolerance,
+        MemoryThresholds.MemoryState.NORMAL);
+    // Once in the EVICTION state, the transition between EVICTION and CRITICAL should not
+    // depend on the threshold counter
+    sendEventAndAssertState(evictionUsedBytes, 1, MemoryThresholds.MemoryState.EVICTION);
+    sendEventAndAssertState(criticalUsedBytes, 1, MemoryThresholds.MemoryState.EVICTION_CRITICAL);
+    sendEventAndAssertState(evictionUsedBytes, 1, MemoryThresholds.MemoryState.EVICTION);
+    sendEventAndAssertState(criticalUsedBytes, 1, MemoryThresholds.MemoryState.EVICTION_CRITICAL);
+  }
+
+  @Test
+  public void updateStateAndSendEvent_NormalToCriticalToNormalToCritical_ThresholdReset() {
+    setupHeapMonitorThresholds(true, true);
+
+    sendEventAndAssertState(criticalUsedBytes, testMemoryEventTolerance,
+        MemoryThresholds.MemoryState.NORMAL);
+    sendEventAndAssertState(criticalUsedBytes, 1, MemoryThresholds.MemoryState.EVICTION_CRITICAL);
+    sendEventAndAssertState(normalUsedBytes, 1, MemoryThresholds.MemoryState.NORMAL);
+    // Threshold counter should have been reset, so we need thre more events in the CRITICAL range
+    // to trigger a state transition
+    sendEventAndAssertState(criticalUsedBytes, testMemoryEventTolerance,
+        MemoryThresholds.MemoryState.NORMAL);
+    sendEventAndAssertState(criticalUsedBytes, 1, MemoryThresholds.MemoryState.EVICTION_CRITICAL);
+  }
+
   // ========== private methods ==========
+  private void setupHeapMonitorThresholds(boolean enableEviction, boolean enableCritical) {
+    // Initialize the most recent state to NORMAL
+    heapMonitor = spy(heapMonitor);
+
+    // This will prevent the polling monitor from firing and causing state transitions. We
+    // want complete control over the state transitions in this test.
+    heapMonitor.started = true;
+
+    HeapMemoryMonitor.setTestBytesUsedForThresholdSet(50);
+    heapMonitor.setTestMaxMemoryBytes(100);
+
+    if (enableCritical) {
+      heapMonitor.setCriticalThreshold(90f);
+    }
+
+    if (enableEviction) {
+      heapMonitor.setEvictionThreshold(80f);
+    }
+  }
+
+  private void sendEventAndAssertState(int bytesUsed, int numEvents,
+      MemoryThresholds.MemoryState expectedState) {
+    for (int i = 0; i < numEvents; ++i) {
+      heapMonitor.updateStateAndSendEvent(bytesUsed, "test");
+      assertThat(heapMonitor.getState()).isEqualByComparingTo(expectedState);
+    }
+  }
+
+  private void sendAlternatingEventsAndAssertState(MemoryThresholds.MemoryState expectedState) {
+    // testMemoryEventTolerance * 2 is somewhat arbitrary - we just want to test that we don't
+    // change states after exceeding the tolerance value if the events alternate between critical
+    // and normal used bytes.
+    for (int i = 0; i < testMemoryEventTolerance * 2; ++i) {
+      // Alternate between normal bytes and critical bytes using modular arithmetic
+      if (i % 2 == 0) {
+        sendEventAndAssertState(normalUsedBytes, 1, expectedState);
+      } else {
+        sendEventAndAssertState(criticalUsedBytes, 1, expectedState);
+      }
+    }
+  }
+
   private void getHeapCriticalMembersFrom_returnsEmptySet(Set adviseCriticalMembers, Set argSet) {
     when(resourceAdvisor.adviseCriticalMembers()).thenReturn(adviseCriticalMembers);
 
diff --git a/geode-junit/src/main/java/org/apache/geode/test/fake/Fakes.java b/geode-junit/src/main/java/org/apache/geode/test/fake/Fakes.java
index 74095c2..ba5522c 100644
--- a/geode-junit/src/main/java/org/apache/geode/test/fake/Fakes.java
+++ b/geode-junit/src/main/java/org/apache/geode/test/fake/Fakes.java
@@ -30,6 +30,7 @@ import org.apache.geode.cache.Cache;
 import org.apache.geode.cache.DataPolicy;
 import org.apache.geode.cache.Region;
 import org.apache.geode.cache.RegionAttributes;
+import org.apache.geode.cache.query.internal.QueryMonitor;
 import org.apache.geode.distributed.internal.ClusterDistributionManager;
 import org.apache.geode.distributed.internal.DSClock;
 import org.apache.geode.distributed.internal.DistributionConfig;
@@ -78,6 +79,7 @@ public class Fakes {
     LogWriter logger = mock(LogWriter.class);
     Statistics stats = mock(Statistics.class);
     TXManagerImpl txManager = mock(TXManagerImpl.class);
+    QueryMonitor queryMonitor = mock(QueryMonitor.class);
 
     InternalDistributedMember member;
     member = new InternalDistributedMember("localhost", 5555);
@@ -96,6 +98,8 @@ public class Fakes {
     when(cache.createPdxInstanceFactory(any())).thenReturn(pdxInstanceFactory);
     when(cache.getPdxRegistry()).thenReturn(pdxRegistryMock);
     when(cache.getTxManager()).thenReturn(txManager);
+    when(cache.getLogger()).thenReturn(logger);
+    when(cache.getQueryMonitor()).thenReturn(queryMonitor);
 
     when(system.getDistributedMember()).thenReturn(member);
     when(system.getConfig()).thenReturn(config);
diff --git a/geode-lucene/src/distributedTest/java/org/apache/geode/cache/lucene/EvictionDUnitTest.java b/geode-lucene/src/distributedTest/java/org/apache/geode/cache/lucene/EvictionDUnitTest.java
index 541de1a..7b41433 100644
--- a/geode-lucene/src/distributedTest/java/org/apache/geode/cache/lucene/EvictionDUnitTest.java
+++ b/geode-lucene/src/distributedTest/java/org/apache/geode/cache/lucene/EvictionDUnitTest.java
@@ -32,7 +32,6 @@ import org.junit.runner.RunWith;
 
 import org.apache.geode.cache.Cache;
 import org.apache.geode.cache.Region;
-import org.apache.geode.distributed.internal.DistributionConfig;
 import org.apache.geode.internal.cache.GemFireCacheImpl;
 import org.apache.geode.internal.cache.PartitionedRegion;
 import org.apache.geode.internal.cache.control.HeapMemoryMonitor;
@@ -132,7 +131,6 @@ public class EvictionDUnitTest extends LuceneQueriesAccessorBase {
   protected void raiseFakeNotification() {
     ((GemFireCacheImpl) getCache()).getHeapEvictor().setTestAbortAfterLoopCount(1);
     HeapMemoryMonitor.setTestDisableMemoryUpdates(true);
-    System.setProperty(DistributionConfig.GEMFIRE_PREFIX + "memoryEventTolerance", "0");
 
     getCache().getResourceManager()
         .setEvictionHeapPercentage(EVICTION_HEAP_PERCENTAGE_FAKE_NOTIFICATION);
@@ -140,13 +138,12 @@ public class EvictionDUnitTest extends LuceneQueriesAccessorBase {
         ((GemFireCacheImpl) getCache()).getInternalResourceManager().getHeapMonitor();
     heapMemoryMonitor.setTestMaxMemoryBytes(TEST_MAX_MEMORY);
 
-    heapMemoryMonitor.updateStateAndSendEvent(MEMORY_USED_FAKE_NOTIFICATION);
+    heapMemoryMonitor.updateStateAndSendEvent(MEMORY_USED_FAKE_NOTIFICATION, "test");
   }
 
   protected void cleanUpAfterFakeNotification() {
     ((GemFireCacheImpl) getCache()).getHeapEvictor().setTestAbortAfterLoopCount(Integer.MAX_VALUE);
     HeapMemoryMonitor.setTestDisableMemoryUpdates(false);
-    System.clearProperty(DistributionConfig.GEMFIRE_PREFIX + "memoryEventTolerance");
   }
 
 }