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 2018/06/01 17:42:50 UTC

[geode] branch develop updated: GEODE-5265: fix dataStoreEntryCount statistic (#2006)

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

dschneider 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 44e5d36  GEODE-5265: fix dataStoreEntryCount statistic (#2006)
44e5d36 is described below

commit 44e5d36d46216c4005c77619cce1f9cd47aa3ebf
Author: Darrel Schneider <ds...@pivotal.io>
AuthorDate: Fri Jun 1 10:41:47 2018 -0700

    GEODE-5265: fix dataStoreEntryCount statistic (#2006)
    
    * tested and fixed updateRecoveredEntry's dataStoreEntryCount code
    * tested and fixed initialImagePut's dataStoreEntryCount code
    * tested and fixed txApplyDestroy's dataStoreEntryCount code
    * tested and fixed RegionMapDestroy's dataStoreEntryCount code
---
 .../geode/internal/cache/AbstractRegionMap.java    |  23 ++-
 .../geode/internal/cache/map/RegionMapDestroy.java |   5 +-
 .../internal/cache/AbstractRegionMapTest.java      | 212 +++++++++++++++++++++
 .../internal/cache/map/RegionMapDestroyTest.java   | 131 +++++++++++++
 4 files changed, 364 insertions(+), 7 deletions(-)

diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/AbstractRegionMap.java b/geode-core/src/main/java/org/apache/geode/internal/cache/AbstractRegionMap.java
index ef4f2eb..1b03d84 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/AbstractRegionMap.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/AbstractRegionMap.java
@@ -767,6 +767,7 @@ public abstract class AbstractRegionMap
       try {
         if (_isOwnerALocalRegion()) {
           boolean oldValueWasTombstone = re.isTombstone();
+          boolean oldIsDestroyedOrRemoved = re.isDestroyedOrRemoved();
           if (oldValueWasTombstone) {
             // when a tombstone is to be overwritten, unschedule it first
             _getOwner().unscheduleTombstone(re);
@@ -783,8 +784,10 @@ public abstract class AbstractRegionMap
                                            // value for the cache
           if (re.isTombstone()) {
             _getOwner().scheduleTombstone(re, re.getVersionStamp().asVersionTag());
-            _getOwner().updateSizeOnRemove(key, oldSize);
-          } else if (oldValueWasTombstone) {
+            if (!oldIsDestroyedOrRemoved) {
+              _getOwner().updateSizeOnRemove(key, oldSize);
+            }
+          } else if (oldIsDestroyedOrRemoved) {
             _getOwner().updateSizeOnCreate(key, _getOwner().calculateRegionEntryValueSize(re));
           } else {
             _getOwner().updateSizeOnPut(key, oldSize,
@@ -870,6 +873,7 @@ public abstract class AbstractRegionMap
                     }
                   }
                   final boolean oldIsTombstone = oldRe.isTombstone();
+                  final boolean oldIsDestroyedOrRemoved = oldRe.isDestroyedOrRemoved();
                   final int oldSize = owner.calculateRegionEntryValueSize(oldRe);
                   try {
                     result = oldRe.initialImagePut(owner, lastModified, newValue, wasRecovered,
@@ -884,7 +888,9 @@ public abstract class AbstractRegionMap
                         }
                       }
                       if (newValue == Token.TOMBSTONE) {
-                        owner.updateSizeOnRemove(key, oldSize);
+                        if (!oldIsDestroyedOrRemoved) {
+                          owner.updateSizeOnRemove(key, oldSize);
+                        }
                         if (owner.getServerProxy() == null
                             && owner.getVersionVector().isTombstoneTooOld(
                                 entryVersion.getMemberID(), entryVersion.getRegionVersion())) {
@@ -1065,6 +1071,7 @@ public abstract class AbstractRegionMap
             if (!re.isRemoved() || re.isTombstone()) {
               Object oldValue = re.getValueInVM(owner);
               final int oldSize = owner.calculateRegionEntryValueSize(re);
+              final boolean wasDestroyedOrRemoved = re.isDestroyedOrRemoved();
               // Create an entry event only if the calling context is
               // a receipt of a TXCommitMessage AND there are callbacks installed
               // for this region
@@ -1112,7 +1119,9 @@ public abstract class AbstractRegionMap
                     }
                   }
                   EntryLogger.logTXDestroy(_getOwnerObject(), key);
-                  owner.updateSizeOnRemove(key, oldSize);
+                  if (!wasDestroyedOrRemoved) {
+                    owner.updateSizeOnRemove(key, oldSize);
+                  }
                 } catch (RegionClearedException rce) {
                   clearOccured = true;
                 }
@@ -1185,6 +1194,7 @@ public abstract class AbstractRegionMap
                       }
                       int oldSize = 0;
                       boolean wasTombstone = oldRe.isTombstone();
+                      boolean wasDestroyedOrRemoved = oldRe.isDestroyedOrRemoved();
                       {
                         if (!wasTombstone) {
                           oldSize = owner.calculateRegionEntryValueSize(oldRe);
@@ -1197,7 +1207,9 @@ public abstract class AbstractRegionMap
                       if (wasTombstone) {
                         owner.unscheduleTombstone(oldRe);
                       }
-                      owner.updateSizeOnRemove(oldRe.getKey(), oldSize);
+                      if (!wasDestroyedOrRemoved) {
+                        owner.updateSizeOnRemove(oldRe.getKey(), oldSize);
+                      }
                       owner.txApplyDestroyPart2(oldRe, oldRe.getKey(), inTokenMode,
                           false /* Clear Conflicting with the operation */);
                       lruEntryDestroy(oldRe);
@@ -1245,7 +1257,6 @@ public abstract class AbstractRegionMap
                   callbackEventAddedToPending = true;
                 }
                 EntryLogger.logTXDestroy(_getOwnerObject(), key);
-                owner.updateSizeOnCreate(newRe.getKey(), 0);
                 if (owner.getConcurrencyChecksEnabled() && callbackEvent.getVersionTag() != null) {
                   newRe.makeTombstone(owner, callbackEvent.getVersionTag());
                 } else if (!inTokenMode) {
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/map/RegionMapDestroy.java b/geode-core/src/main/java/org/apache/geode/internal/cache/map/RegionMapDestroy.java
index 085c4b4..6a34c50 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/map/RegionMapDestroy.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/map/RegionMapDestroy.java
@@ -720,11 +720,14 @@ public class RegionMapDestroy {
       EntryNotFoundException, RegionClearedException {
     focusedRegionMap.processVersionTag(re, event);
     final int oldSize = internalRegion.calculateRegionEntryValueSize(re);
+    final boolean wasRemoved = re.isDestroyedOrRemoved();
     boolean retVal = re.destroy(event.getRegion(), event, inTokenMode, cacheWrite, expectedOldValue,
         forceDestroy, removeRecoveredEntry);
     if (retVal) {
       EntryLogger.logDestroy(event);
-      internalRegion.updateSizeOnRemove(event.getKey(), oldSize);
+      if (!wasRemoved) {
+        internalRegion.updateSizeOnRemove(event.getKey(), oldSize);
+      }
     }
     return retVal;
   }
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/AbstractRegionMapTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/AbstractRegionMapTest.java
index d8957ed..d546ddb 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/AbstractRegionMapTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/AbstractRegionMapTest.java
@@ -22,6 +22,7 @@ import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.ArgumentMatchers.anyBoolean;
+import static org.mockito.ArgumentMatchers.anyInt;
 import static org.mockito.ArgumentMatchers.anyLong;
 import static org.mockito.ArgumentMatchers.eq;
 import static org.mockito.Mockito.doThrow;
@@ -44,13 +45,17 @@ import org.apache.geode.cache.EvictionAttributes;
 import org.apache.geode.cache.Operation;
 import org.apache.geode.cache.Scope;
 import org.apache.geode.cache.TransactionId;
+import org.apache.geode.cache.client.internal.ServerRegionProxy;
 import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
+import org.apache.geode.internal.cache.entries.DiskEntry.RecoveredEntry;
 import org.apache.geode.internal.cache.eviction.EvictableEntry;
 import org.apache.geode.internal.cache.eviction.EvictionController;
 import org.apache.geode.internal.cache.eviction.EvictionCounters;
 import org.apache.geode.internal.cache.tier.sockets.ClientProxyMembershipID;
 import org.apache.geode.internal.cache.versions.RegionVersionVector;
 import org.apache.geode.internal.cache.versions.VersionHolder;
+import org.apache.geode.internal.cache.versions.VersionSource;
+import org.apache.geode.internal.cache.versions.VersionStamp;
 import org.apache.geode.internal.cache.versions.VersionTag;
 import org.apache.geode.internal.util.concurrent.ConcurrentMapWithReusableEntries;
 import org.apache.geode.internal.util.concurrent.CustomEntryConcurrentHashMap;
@@ -711,6 +716,196 @@ public class AbstractRegionMapTest {
     }
   }
 
+  @Test
+  public void updateRecoveredEntry_givenExistingDestroyedOrRemovedAndSettingToTombstone_neverCallsUpdateSizeOnRemove() {
+    RecoveredEntry recoveredEntry = mock(RecoveredEntry.class);
+    RegionEntry regionEntry = mock(RegionEntry.class);
+    when(regionEntry.isTombstone()).thenReturn(false).thenReturn(true);
+    when(regionEntry.isDestroyedOrRemoved()).thenReturn(true);
+    when(regionEntry.getVersionStamp()).thenReturn(mock(VersionStamp.class));
+    TestableAbstractRegionMap arm = new TestableAbstractRegionMap(false, null, null, regionEntry);
+
+    arm.updateRecoveredEntry(KEY, recoveredEntry);
+
+    verify(arm._getOwner(), never()).updateSizeOnRemove(any(), anyInt());
+  }
+
+  @Test
+  public void updateRecoveredEntry_givenExistingRemovedNonTombstone_neverCallsUpdateSizeOnRemove() {
+    RecoveredEntry recoveredEntry = mock(RecoveredEntry.class);
+    RegionEntry regionEntry = mock(RegionEntry.class);
+    when(regionEntry.isRemoved()).thenReturn(true);
+    when(regionEntry.isTombstone()).thenReturn(false);
+    TestableAbstractRegionMap arm = new TestableAbstractRegionMap(false, null, null, regionEntry);
+
+    arm.updateRecoveredEntry(KEY, recoveredEntry);
+
+    verify(arm._getOwner(), never()).updateSizeOnRemove(any(), anyInt());
+  }
+
+  @Test
+  public void updateRecoveredEntry_givenNoExistingEntry_neverCallsUpdateSizeOnRemove() {
+    RecoveredEntry recoveredEntry = mock(RecoveredEntry.class);
+    TestableAbstractRegionMap arm = new TestableAbstractRegionMap(false, null, null, null);
+
+    arm.updateRecoveredEntry(KEY, recoveredEntry);
+
+    verify(arm._getOwner(), never()).updateSizeOnRemove(any(), anyInt());
+  }
+
+  @Test
+  public void updateRecoveredEntry_givenExistingNonTombstoneAndSettingToTombstone_callsUpdateSizeOnRemove() {
+    RecoveredEntry recoveredEntry = mock(RecoveredEntry.class);
+    RegionEntry regionEntry = mock(RegionEntry.class);
+    when(regionEntry.isTombstone()).thenReturn(false).thenReturn(true);
+    when(regionEntry.getVersionStamp()).thenReturn(mock(VersionStamp.class));
+    TestableAbstractRegionMap arm = new TestableAbstractRegionMap(false, null, null, regionEntry);
+
+    arm.updateRecoveredEntry(KEY, recoveredEntry);
+
+    verify(arm._getOwner(), times(1)).updateSizeOnRemove(eq(KEY), anyInt());
+  }
+
+  @Test
+  public void initialImagePut_givenPutIfAbsentReturningDestroyedOrRemovedEntry_neverCallsUpdateSizeOnRemove()
+      throws RegionClearedException {
+    ConcurrentMapWithReusableEntries map = mock(ConcurrentMapWithReusableEntries.class);
+    RegionEntry entry = mock(RegionEntry.class);
+    when(entry.isDestroyedOrRemoved()).thenReturn(true);
+    when(entry.initialImagePut(any(), anyLong(), any(), anyBoolean(), anyBoolean()))
+        .thenReturn(true);
+    VersionStamp versionStamp = mock(VersionStamp.class);
+    when(entry.getVersionStamp()).thenReturn(versionStamp);
+    when(versionStamp.asVersionTag()).thenReturn(mock(VersionTag.class));
+    when(map.putIfAbsent(eq(KEY), any())).thenReturn(entry).thenReturn(null);
+    TestableAbstractRegionMap arm = new TestableAbstractRegionMap(false, map, null);
+    when(arm._getOwner().getConcurrencyChecksEnabled()).thenReturn(true);
+    when(arm._getOwner().getServerProxy()).thenReturn(mock(ServerRegionProxy.class));
+    VersionTag versionTag = mock(VersionTag.class);
+    when(versionTag.getMemberID()).thenReturn(mock(VersionSource.class));
+
+    arm.initialImagePut(KEY, 0, Token.TOMBSTONE, false, false, versionTag, null, false);
+
+    verify(arm._getOwner(), never()).updateSizeOnRemove(any(), anyInt());
+  }
+
+  @Test
+  public void initialImagePut_givenPutIfAbsentReturningNonTombstone_callsUpdateSizeOnRemove()
+      throws RegionClearedException {
+    ConcurrentMapWithReusableEntries map = mock(ConcurrentMapWithReusableEntries.class);
+    RegionEntry entry = mock(RegionEntry.class);
+    when(entry.isTombstone()).thenReturn(false);
+    when(entry.isDestroyedOrRemoved()).thenReturn(false);
+    when(entry.initialImagePut(any(), anyLong(), any(), anyBoolean(), anyBoolean()))
+        .thenReturn(true);
+    VersionStamp versionStamp = mock(VersionStamp.class);
+    when(entry.getVersionStamp()).thenReturn(versionStamp);
+    when(versionStamp.asVersionTag()).thenReturn(mock(VersionTag.class));
+    when(map.putIfAbsent(eq(KEY), any())).thenReturn(entry).thenReturn(null);
+    TestableAbstractRegionMap arm = new TestableAbstractRegionMap(false, map, null);
+    when(arm._getOwner().getConcurrencyChecksEnabled()).thenReturn(true);
+    when(arm._getOwner().getServerProxy()).thenReturn(mock(ServerRegionProxy.class));
+    VersionTag versionTag = mock(VersionTag.class);
+    when(versionTag.getMemberID()).thenReturn(mock(VersionSource.class));
+
+    arm.initialImagePut(KEY, 0, Token.TOMBSTONE, false, false, versionTag, null, false);
+
+    verify(arm._getOwner(), times(1)).updateSizeOnRemove(eq(KEY), anyInt());
+  }
+
+  @Test
+  public void txApplyDestroy_givenExistingDestroyedOrRemovedEntry_neverCallsUpdateSizeOnRemove() {
+    RegionEntry regionEntry = mock(RegionEntry.class);
+    when(regionEntry.isTombstone()).thenReturn(false);
+    when(regionEntry.isDestroyedOrRemoved()).thenReturn(true);
+    when(regionEntry.getVersionStamp()).thenReturn(mock(VersionStamp.class));
+    TXId txId = mock(TXId.class);
+    when(txId.getMemberId()).thenReturn(mock(InternalDistributedMember.class));
+    TestableAbstractRegionMap arm = new TestableAbstractRegionMap(false, null, null, regionEntry);
+
+    arm.txApplyDestroy(KEY, txId, null, false, false, null, null, null, new ArrayList<>(), null,
+        null, false, null, null, 0);
+
+    verify(arm._getOwner(), never()).updateSizeOnRemove(any(), anyInt());
+  }
+
+  @Test
+  public void txApplyDestroy_givenExistingNonTombstone_callsUpdateSizeOnRemove() {
+    RegionEntry regionEntry = mock(RegionEntry.class);
+    when(regionEntry.isTombstone()).thenReturn(false);
+    when(regionEntry.getVersionStamp()).thenReturn(mock(VersionStamp.class));
+    TXId txId = mock(TXId.class);
+    when(txId.getMemberId()).thenReturn(mock(InternalDistributedMember.class));
+    TestableAbstractRegionMap arm = new TestableAbstractRegionMap(false, null, null, regionEntry);
+
+    arm.txApplyDestroy(KEY, txId, null, false, false, null, null, null, new ArrayList<>(), null,
+        null, false, null, null, 0);
+
+    verify(arm._getOwner(), times(1)).updateSizeOnRemove(eq(KEY), anyInt());
+  }
+
+  @Test
+  public void txApplyDestroy_givenPutIfAbsentReturningDestroyedOrRemovedEntry_neverCallsUpdateSizeOnRemove()
+      throws RegionClearedException {
+    ConcurrentMapWithReusableEntries map = mock(ConcurrentMapWithReusableEntries.class);
+    RegionEntry entry = mock(RegionEntry.class);
+    when(entry.isTombstone()).thenReturn(false);
+    when(entry.isDestroyedOrRemoved()).thenReturn(true);
+    VersionStamp versionStamp = mock(VersionStamp.class);
+    when(entry.getVersionStamp()).thenReturn(versionStamp);
+    when(versionStamp.asVersionTag()).thenReturn(mock(VersionTag.class));
+    when(map.putIfAbsent(eq(KEY), any())).thenReturn(entry).thenReturn(null);
+    TXId txId = mock(TXId.class);
+    when(txId.getMemberId()).thenReturn(mock(InternalDistributedMember.class));
+    TestableAbstractRegionMap arm = new TestableAbstractRegionMap(false, map, null);
+    when(arm._getOwner().getConcurrencyChecksEnabled()).thenReturn(true);
+
+    arm.txApplyDestroy(KEY, txId, null, false, false, null, null, null, new ArrayList<>(), null,
+        null, false, null, null, 0);
+
+    verify(arm._getOwner(), never()).updateSizeOnRemove(any(), anyInt());
+  }
+
+  @Test
+  public void txApplyDestroy_givenPutIfAbsentReturningNonTombstone_callsUpdateSizeOnRemove()
+      throws RegionClearedException {
+    ConcurrentMapWithReusableEntries map = mock(ConcurrentMapWithReusableEntries.class);
+    RegionEntry entry = mock(RegionEntry.class);
+    when(entry.getKey()).thenReturn(KEY);
+    when(entry.isTombstone()).thenReturn(false);
+    VersionStamp versionStamp = mock(VersionStamp.class);
+    when(entry.getVersionStamp()).thenReturn(versionStamp);
+    when(versionStamp.asVersionTag()).thenReturn(mock(VersionTag.class));
+    when(map.putIfAbsent(eq(KEY), any())).thenReturn(entry).thenReturn(null);
+    TXId txId = mock(TXId.class);
+    when(txId.getMemberId()).thenReturn(mock(InternalDistributedMember.class));
+    TestableAbstractRegionMap arm = new TestableAbstractRegionMap(false, map, null);
+    when(arm._getOwner().getConcurrencyChecksEnabled()).thenReturn(true);
+
+    arm.txApplyDestroy(KEY, txId, null, false, false, null, null, null, new ArrayList<>(), null,
+        null, false, null, null, 0);
+
+    verify(arm._getOwner(), times(1)).updateSizeOnRemove(eq(KEY), anyInt());
+  }
+
+  @Test
+  public void txApplyDestroy_givenFactory_neverCallsUpdateSizeOnRemove()
+      throws RegionClearedException {
+    ConcurrentMapWithReusableEntries map = mock(ConcurrentMapWithReusableEntries.class);
+    RegionEntry entry = mock(RegionEntry.class);
+    TXId txId = mock(TXId.class);
+    when(txId.getMemberId()).thenReturn(mock(InternalDistributedMember.class));
+    RegionEntryFactory factory = mock(RegionEntryFactory.class);
+    when(factory.createEntry(any(), any(), any())).thenReturn(entry);
+    TestableAbstractRegionMap arm = new TestableAbstractRegionMap(false, map, factory);
+    when(arm._getOwner().getConcurrencyChecksEnabled()).thenReturn(true);
+
+    arm.txApplyDestroy(KEY, txId, null, false, false, null, null, null, new ArrayList<>(), null,
+        null, false, null, null, 0);
+
+    verify(arm._getOwner(), never()).updateSizeOnCreate(any(), anyInt());
+  }
+
   private EntryEventImpl createEventForInvalidate(LocalRegion lr) {
     when(lr.getKeyInfo(KEY)).thenReturn(new KeyInfo(KEY, null, null));
     return EntryEventImpl.create(lr, Operation.INVALIDATE, KEY, false, null, true, false);
@@ -743,6 +938,7 @@ public class AbstractRegionMapTest {
    * TestableAbstractRegionMap
    */
   private static class TestableAbstractRegionMap extends AbstractRegionMap {
+    private final RegionEntry regionEntryForGetEntry;
 
     protected TestableAbstractRegionMap() {
       this(false);
@@ -754,7 +950,14 @@ public class AbstractRegionMapTest {
 
     protected TestableAbstractRegionMap(boolean withConcurrencyChecks,
         ConcurrentMapWithReusableEntries map, RegionEntryFactory factory) {
+      this(withConcurrencyChecks, map, factory, null);
+    }
+
+    protected TestableAbstractRegionMap(boolean withConcurrencyChecks,
+        ConcurrentMapWithReusableEntries map, RegionEntryFactory factory,
+        RegionEntry regionEntryForGetEntry) {
       super(null);
+      this.regionEntryForGetEntry = regionEntryForGetEntry;
       LocalRegion owner = mock(LocalRegion.class);
       CachePerfStats cachePerfStats = mock(CachePerfStats.class);
       when(owner.getCachePerfStats()).thenReturn(cachePerfStats);
@@ -771,6 +974,15 @@ public class AbstractRegionMapTest {
         setEntryFactory(factory);
       }
     }
+
+    @Override
+    public RegionEntry getEntry(Object key) {
+      if (this.regionEntryForGetEntry != null) {
+        return this.regionEntryForGetEntry;
+      } else {
+        return super.getEntry(key);
+      }
+    }
   }
 
 
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/map/RegionMapDestroyTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/map/RegionMapDestroyTest.java
index bd59c5d..3837420 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/map/RegionMapDestroyTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/map/RegionMapDestroyTest.java
@@ -18,6 +18,7 @@ import static org.assertj.core.api.Assertions.assertThat;
 import static org.assertj.core.api.Assertions.assertThatThrownBy;
 import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.ArgumentMatchers.anyBoolean;
+import static org.mockito.ArgumentMatchers.anyInt;
 import static org.mockito.ArgumentMatchers.eq;
 import static org.mockito.Mockito.doThrow;
 import static org.mockito.Mockito.mock;
@@ -248,6 +249,18 @@ public class RegionMapDestroyTest {
   }
 
   @Test
+  public void destroyWithEmptyRegionInTokenModeNeverCallsUpdateSizeOnRemove() {
+    givenConcurrencyChecks(false);
+    givenEmptyRegionMap();
+    givenInTokenMode();
+
+    assertThat(arm.destroy(event, inTokenMode, duringRI, cacheWrite, isEviction, expectedOldValue,
+        removeRecoveredEntry)).isTrue();
+
+    verify(arm._getOwner(), never()).updateSizeOnRemove(any(), anyInt());
+  }
+
+  @Test
   public void destroyWithEmptyRegionInTokenModeWithRegionClearedExceptionDoesDestroy()
       throws Exception {
     givenConcurrencyChecks(false);
@@ -289,6 +302,19 @@ public class RegionMapDestroyTest {
   }
 
   @Test
+  public void evictDestroyWithExistingTombstoneInTokenModeNeverCallsUpdateSizeOnRemove() {
+    givenConcurrencyChecks(true);
+    givenEviction();
+    givenExistingEntry(Token.TOMBSTONE);
+    givenInTokenMode();
+
+    assertThat(arm.destroy(event, inTokenMode, duringRI, cacheWrite, isEviction, expectedOldValue,
+        removeRecoveredEntry)).isTrue();
+
+    verify(arm._getOwner(), never()).updateSizeOnRemove(any(), anyInt());
+  }
+
+  @Test
   public void evictDestroyWithExistingTombstoneInUseByTransactionInTokenModeDoesNothing()
       throws RegionClearedException {
     givenConcurrencyChecks(true);
@@ -336,6 +362,22 @@ public class RegionMapDestroyTest {
     validateInvokedDestroyMethodsOnRegion(false);
   }
 
+
+  @Test
+  public void destroyWithConcurrentChangeFromNullToValidRetriesAndCallsUpdateSizeOnRemove()
+      throws RegionClearedException {
+    givenConcurrencyChecks(true);
+    givenEvictionWithMockedEntryMap();
+    givenExistingEvictableEntry("value");
+
+    when(entryMap.get(KEY)).thenReturn(null).thenReturn(evictableEntry);
+
+    assertThat(arm.destroy(event, inTokenMode, duringRI, cacheWrite, isEviction, expectedOldValue,
+        removeRecoveredEntry)).isTrue();
+
+    verify(arm._getOwner(), times(1)).updateSizeOnRemove(any(), anyInt());
+  }
+
   @Test
   public void destroyInTokenModeWithConcurrentChangeFromNullToRemovePhase2RetriesAndDoesDestroy()
       throws RegionClearedException {
@@ -363,6 +405,29 @@ public class RegionMapDestroyTest {
     validateInvokedDestroyMethodsOnRegion(false);
   }
 
+
+  @Test
+  public void destroyInTokenModeWithConcurrentChangeFromNullToRemovePhase2RetriesAndNeverCallsUpdateSizeOnRemove()
+      throws RegionClearedException {
+    givenConcurrencyChecks(true);
+    givenEvictionWithMockedEntryMap();
+    givenInTokenMode();
+
+    when(evictableEntry.isRemovedPhase2()).thenReturn(true);
+    when(evictableEntry.destroy(any(), any(), anyBoolean(), anyBoolean(), any(), anyBoolean(),
+        anyBoolean())).thenReturn(true);
+    when(entryMap.get(KEY)).thenReturn(null);
+    when(entryMap.putIfAbsent(eq(KEY), any())).thenReturn(evictableEntry).thenReturn(null);
+
+    // isEviction is false despite having eviction enabled
+    isEviction = false;
+
+    assertThat(arm.destroy(event, inTokenMode, duringRI, cacheWrite, isEviction, expectedOldValue,
+        removeRecoveredEntry)).isTrue();
+
+    verify(arm._getOwner(), never()).updateSizeOnRemove(any(), anyInt());
+  }
+
   @Test
   public void destroyWithConcurrentChangeFromTombstoneToValidRetriesAndDoesDestroy()
       throws RegionClearedException {
@@ -380,6 +445,21 @@ public class RegionMapDestroyTest {
   }
 
   @Test
+  public void destroyWithConcurrentChangeFromTombstoneToValidRetriesAndCallsUpdateSizeOnRemove()
+      throws RegionClearedException {
+    givenConcurrencyChecks(true);
+    givenEvictionWithMockedEntryMap();
+    givenExistingEvictableEntryWithMockedIsTombstone();
+
+    isEviction = false;
+
+    assertThat(arm.destroy(event, inTokenMode, duringRI, cacheWrite, isEviction, expectedOldValue,
+        removeRecoveredEntry)).isTrue();
+
+    verify(arm._getOwner(), times(1)).updateSizeOnRemove(any(), anyInt());
+  }
+
+  @Test
   public void destroyOfExistingEntryInTokenModeAddsAToken() {
     givenConcurrencyChecks(false);
     givenEmptyRegionMap();
@@ -394,6 +474,19 @@ public class RegionMapDestroyTest {
   }
 
   @Test
+  public void destroyOfExistingEntryInTokenModeCallsUpdateSizeOnRemove() {
+    givenConcurrencyChecks(false);
+    givenEmptyRegionMap();
+    givenExistingEntry();
+    givenInTokenMode();
+
+    assertThat(arm.destroy(event, inTokenMode, duringRI, cacheWrite, isEviction, expectedOldValue,
+        removeRecoveredEntry)).isTrue();
+
+    verify(arm._getOwner(), times(1)).updateSizeOnRemove(any(), anyInt());
+  }
+
+  @Test
   public void destroyOfExistingTombstoneInTokenModeWithConcurrencyChecksDoesNothing() {
     givenConcurrencyChecks(true);
     givenEmptyRegionMap();
@@ -409,6 +502,19 @@ public class RegionMapDestroyTest {
   }
 
   @Test
+  public void destroyOfExistingTombstoneInTokenModeWithConcurrencyChecksNeverCallsUpdateSizeOnRemove() {
+    givenConcurrencyChecks(true);
+    givenEmptyRegionMap();
+    givenExistingEntryWithTokenAndVersionTag(Token.TOMBSTONE);
+    givenInTokenMode();
+
+    assertThat(arm.destroy(event, inTokenMode, duringRI, cacheWrite, isEviction, expectedOldValue,
+        removeRecoveredEntry)).isTrue();
+
+    verify(arm._getOwner(), never()).updateSizeOnRemove(any(), anyInt());
+  }
+
+  @Test
   public void destroyOfExistingTombstoneWithConcurrencyChecksThrowsEntryNotFound() {
     givenConcurrencyChecks(true);
     givenEmptyRegionMap();
@@ -433,6 +539,19 @@ public class RegionMapDestroyTest {
   }
 
   @Test
+  public void destroyOfExistingTombstoneWithConcurrencyChecksAndRemoveRecoveredEntryNeverCallsUpdateSizeOnRemove() {
+    givenConcurrencyChecks(true);
+    givenEmptyRegionMap();
+    givenExistingEntryWithTokenAndVersionTag(Token.TOMBSTONE);
+    givenRemoveRecoveredEntry();
+
+    assertThat(arm.destroy(event, inTokenMode, duringRI, cacheWrite, isEviction, expectedOldValue,
+        removeRecoveredEntry)).isTrue();
+
+    verify(arm._getOwner(), never()).updateSizeOnRemove(any(), anyInt());
+  }
+
+  @Test
   public void destroyOfExistingRemovePhase2WithConcurrencyChecksAndRemoveRecoveredEntryDoesRetryAndThrowsEntryNotFound() {
     givenConcurrencyChecks(true);
     givenEmptyRegionMap();
@@ -456,6 +575,18 @@ public class RegionMapDestroyTest {
     validateInvokedDestroyMethodsOnRegion(false);
   }
 
+  @Test
+  public void destroyOfExistingEntryCallsUpdateSizeOnRemove() {
+    givenConcurrencyChecks(false);
+    givenEmptyRegionMap();
+    givenExistingEntry();
+
+    assertThat(arm.destroy(event, inTokenMode, duringRI, cacheWrite, isEviction, expectedOldValue,
+        removeRecoveredEntry)).isTrue();
+
+    verify(arm._getOwner(), times(1)).updateSizeOnRemove(any(), anyInt());
+  }
+
   /**
    * This might be a bug. It seems like we should have created a tombstone but we have no version
    * tag so that might be the cause of this bug.

-- 
To stop receiving notification emails like this one, please contact
dschneider@apache.org.