You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by es...@apache.org on 2018/09/21 00:14:57 UTC

[geode] branch feature/GEODE-5748-1 created (now e530564)

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

eshu11 pushed a change to branch feature/GEODE-5748-1
in repository https://gitbox.apache.org/repos/asf/geode.git.


      at e530564  GEODE-5748: Hold a write lock during cleanUpAfterFailedGII

This branch includes the following new commits:

     new e530564  GEODE-5748: Hold a write lock during cleanUpAfterFailedGII

The 1 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.



[geode] 01/01: GEODE-5748: Hold a write lock during cleanUpAfterFailedGII

Posted by es...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

eshu11 pushed a commit to branch feature/GEODE-5748-1
in repository https://gitbox.apache.org/repos/asf/geode.git

commit e53056488650cec35df45210e5d44a629720828f
Author: eshu <es...@pivotal.io>
AuthorDate: Thu Sep 20 17:12:52 2018 -0700

    GEODE-5748: Hold a write lock during cleanUpAfterFailedGII
    
     * Hold write lock when cleanUpAfterFailedGII.
     * Hold read lock when cache operation is performed on the farside if region is not yet initialized.
     * Also hold read lock when transaction is performed on the farside if region not initialized yet.
---
 .../geode/internal/cache/AbstractRegionMap.java    |  31 ++++-
 .../apache/geode/internal/cache/BucketRegion.java  |   5 +
 .../geode/internal/cache/DistributedRegion.java    |  46 ++++++-
 .../geode/internal/cache/InternalRegion.java       |   6 +
 .../geode/internal/cache/map/RegionMapDestroy.java |   4 +
 .../geode/internal/cache/map/RegionMapPut.java     |   4 +
 .../internal/cache/AbstractRegionMapTest.java      | 143 ++++++++++++++++++++-
 .../internal/cache/DistributedRegionTest.java      |  32 +++--
 .../internal/cache/map/RegionMapDestroyTest.java   |  36 ++++++
 .../geode/internal/cache/map/RegionMapPutTest.java |  36 ++++++
 10 files changed, 318 insertions(+), 25 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 bc1ca9b..565d2ed 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
@@ -1038,6 +1038,7 @@ public abstract class AbstractRegionMap
     final boolean hasRemoteOrigin = !txId.getMemberId().equals(owner.getMyId());
     boolean callbackEventAddedToPending = false;
     IndexManager oqlIndexManager = owner.getIndexManager();
+    boolean locked = owner.lockWhenRegionIsInitializing();
     try {
       RegionEntry re = getEntry(key);
       if (re != null) {
@@ -1291,6 +1292,10 @@ public abstract class AbstractRegionMap
     } catch (DiskAccessException dae) {
       owner.handleDiskAccessException(dae);
       throw dae;
+    } finally {
+      if (locked) {
+        owner.unlockWhenRegionIsInitializing();
+      }
     }
   }
 
@@ -1337,6 +1342,7 @@ public abstract class AbstractRegionMap
     boolean clearOccured = false;
     DiskRegion dr = owner.getDiskRegion();
     boolean ownerIsInitialized = owner.isInitialized();
+    boolean locked = false;
     try {
       // Fix for Bug #44431. We do NOT want to update the region and wait
       // later for index INIT as region.clear() can cause inconsistency if
@@ -1346,6 +1352,7 @@ public abstract class AbstractRegionMap
         oqlIndexManager.waitForIndexInit();
       }
       lockForCacheModification(owner, event);
+      locked = owner.lockWhenRegionIsInitializing();
       try {
         try {
           if (forceNewEntry || forceCallbacks) {
@@ -1710,6 +1717,9 @@ public abstract class AbstractRegionMap
       }
     } finally {
       releaseCacheModificationLock(owner, event);
+      if (locked) {
+        owner.unlockWhenRegionIsInitializing();
+      }
     }
 
   }
@@ -1770,6 +1780,7 @@ public abstract class AbstractRegionMap
     }
 
     lockForCacheModification(owner, event);
+    boolean locked = owner.lockWhenRegionIsInitializing();
 
     try {
       RegionEntry re = getEntry(event.getKey());
@@ -1802,6 +1813,9 @@ public abstract class AbstractRegionMap
       throw dae;
     } finally {
       releaseCacheModificationLock(owner, event);
+      if (locked) {
+        owner.unlockWhenRegionIsInitializing();
+      }
       if (dr != null) {
         dr.removeClearCountReference();
       }
@@ -1831,6 +1845,7 @@ public abstract class AbstractRegionMap
     if (oqlIndexManager != null) {
       oqlIndexManager.waitForIndexInit();
     }
+    boolean locked = owner.lockWhenRegionIsInitializing();
     try {
       if (forceNewEntry) {
         boolean opCompleted = false;
@@ -2042,6 +2057,9 @@ public abstract class AbstractRegionMap
       if (oqlIndexManager != null) {
         oqlIndexManager.countDownIndexUpdaters();
       }
+      if (locked) {
+        owner.unlockWhenRegionIsInitializing();
+      }
     }
   }
 
@@ -2112,9 +2130,16 @@ public abstract class AbstractRegionMap
       callbackEvent.makeSerializedNewValue();
       txHandleWANEvent(owner, callbackEvent, txEntryState);
     }
-    RegionMapCommitPut commitPut = new RegionMapCommitPut(this, owner, callbackEvent, putOp,
-        didDestroy, txId, txEvent, pendingCallbacks, txEntryState);
-    commitPut.put();
+    boolean locked = owner.lockWhenRegionIsInitializing();
+    try {
+      RegionMapCommitPut commitPut = new RegionMapCommitPut(this, owner, callbackEvent, putOp,
+          didDestroy, txId, txEvent, pendingCallbacks, txEntryState);
+      commitPut.put();
+    } finally {
+      if (locked) {
+        owner.unlockWhenRegionIsInitializing();
+      }
+    }
   }
 
   private void txHandleWANEvent(final LocalRegion owner, EntryEventImpl callbackEvent,
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/BucketRegion.java b/geode-core/src/main/java/org/apache/geode/internal/cache/BucketRegion.java
index 0fb745d..4dbed7c 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/BucketRegion.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/BucketRegion.java
@@ -1341,8 +1341,10 @@ public class BucketRegion extends DistributedRegion implements Bucket {
 
     InternalRegion internalRegion = event.getRegion();
     AbstractRegionMap arm = ((AbstractRegionMap) internalRegion.getRegionMap());
+    boolean locked = false;
     try {
       arm.lockForCacheModification(internalRegion, event);
+      locked = internalRegion.lockWhenRegionIsInitializing();
       beginLocalWrite(event);
       try {
         if (!hasSeenEvent(event)) {
@@ -1366,6 +1368,9 @@ public class BucketRegion extends DistributedRegion implements Bucket {
       }
     } finally {
       arm.releaseCacheModificationLock(event.getRegion(), event);
+      if (locked) {
+        internalRegion.unlockWhenRegionIsInitializing();
+      }
     }
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/DistributedRegion.java b/geode-core/src/main/java/org/apache/geode/internal/cache/DistributedRegion.java
index 488b1fa..c6c8a43 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/DistributedRegion.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/DistributedRegion.java
@@ -31,6 +31,7 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.locks.Condition;
 import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
 
 import org.apache.logging.log4j.Logger;
 
@@ -181,6 +182,7 @@ public class DistributedRegion extends LocalRegion implements InternalDistribute
    * Lock to prevent multiple threads on this member from performing a clear at the same time.
    */
   private final Object clearLock = new Object();
+  private final ReentrantReadWriteLock failedGIIClearLock = new ReentrantReadWriteLock(true);
 
   private static final AtomicBoolean loggedNetworkPartitionWarning = new AtomicBoolean(false);
 
@@ -1332,10 +1334,7 @@ public class DistributedRegion extends LocalRegion implements InternalDistribute
     }
 
     if (!getRegionMap().isEmpty()) {
-      RegionVersionVector rvv = getVersionVector();
-      if (rvv != null) {
-        rvv.lockForClear(getFullPath(), getDistributionManager(), getMyId());
-      }
+      lockFailedGIIClearWriteLock();
       try {
         closeEntries();
         if (getDiskRegion() != null) {
@@ -1355,13 +1354,41 @@ public class DistributedRegion extends LocalRegion implements InternalDistribute
           }
         }
       } finally {
-        if (rvv != null) {
-          rvv.unlockForClear(getMyId());
-        }
+        unlockFailedGIIClearWriteLock();
       }
     }
   }
 
+  void lockFailedGIIClearWriteLock() {
+    failedGIIClearLock.writeLock().lock();
+  }
+
+  void unlockFailedGIIClearWriteLock() {
+    failedGIIClearLock.writeLock().unlock();
+  }
+
+  void lockFailedGIIClearReadLock() {
+    failedGIIClearLock.readLock().lock();
+  }
+
+  void unlockFailedGIIClearReadLock() {
+    failedGIIClearLock.readLock().unlock();
+  }
+
+  @Override
+  public boolean lockWhenRegionIsInitializing() {
+    if (!isInitialized()) {
+      lockFailedGIIClearReadLock();
+      return true;
+    }
+    return false;
+  }
+
+  @Override
+  public void unlockWhenRegionIsInitializing() {
+    unlockFailedGIIClearReadLock();
+  }
+
   private void initMembershipRoles() {
     synchronized (this.advisorListener) {
       // hold sync to prevent listener from changing initial members
@@ -1853,8 +1880,10 @@ public class DistributedRegion extends LocalRegion implements InternalDistribute
   void basicUpdateEntryVersion(EntryEventImpl event) throws EntryNotFoundException {
     InternalRegion internalRegion = event.getRegion();
     AbstractRegionMap regionMap = (AbstractRegionMap) internalRegion.getRegionMap();
+    boolean locked = false;
     try {
       regionMap.lockForCacheModification(internalRegion, event);
+      locked = internalRegion.lockWhenRegionIsInitializing();
       try {
         if (!hasSeenEvent(event)) {
           super.basicUpdateEntryVersion(event);
@@ -1866,6 +1895,9 @@ public class DistributedRegion extends LocalRegion implements InternalDistribute
       }
     } finally {
       regionMap.releaseCacheModificationLock(internalRegion, event);
+      if (locked) {
+        internalRegion.unlockWhenRegionIsInitializing();
+      }
     }
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/InternalRegion.java b/geode-core/src/main/java/org/apache/geode/internal/cache/InternalRegion.java
index d21d130..caeb9f9 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/InternalRegion.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/InternalRegion.java
@@ -411,4 +411,10 @@ public interface InternalRegion extends Region, HasCachePerfStats, RegionEntryCo
   MemoryThresholdInfo getAtomicThresholdInfo();
 
   InternalCache getInternalCache();
+
+  default boolean lockWhenRegionIsInitializing() {
+    return false;
+  };
+
+  default void unlockWhenRegionIsInitializing() {};
 }
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 39b65de..5a49fef 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
@@ -105,6 +105,7 @@ public class RegionMapDestroy {
     }
 
     cacheModificationLock.lockForCacheModification(internalRegion, event);
+    boolean locked = internalRegion.lockWhenRegionIsInitializing();
     try {
 
       while (retry) {
@@ -173,6 +174,9 @@ public class RegionMapDestroy {
 
     } finally {
       cacheModificationLock.releaseCacheModificationLock(internalRegion, event);
+      if (locked) {
+        internalRegion.unlockWhenRegionIsInitializing();
+      }
     }
     return false;
   }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/map/RegionMapPut.java b/geode-core/src/main/java/org/apache/geode/internal/cache/map/RegionMapPut.java
index 50dc4fa..813f717 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/map/RegionMapPut.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/map/RegionMapPut.java
@@ -146,10 +146,14 @@ public class RegionMapPut extends AbstractRegionMapPut {
   @Override
   protected void runWhileLockedForCacheModification(Runnable r) {
     cacheModificationLock.lockForCacheModification(getOwner(), getEvent());
+    boolean locked = getOwner().lockWhenRegionIsInitializing();
     try {
       r.run();
     } finally {
       cacheModificationLock.releaseCacheModificationLock(getOwner(), getEvent());
+      if (locked) {
+        getOwner().unlockWhenRegionIsInitializing();
+      }
     }
   }
 
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 390a3b3..212a1b8 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
@@ -25,6 +25,7 @@ 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.RETURNS_DEEP_STUBS;
 import static org.mockito.Mockito.doThrow;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.never;
@@ -1186,15 +1187,22 @@ public class AbstractRegionMapTest {
 
     protected TestableAbstractRegionMap(boolean withConcurrencyChecks,
         ConcurrentMapWithReusableEntries map, RegionEntryFactory factory) {
-      this(withConcurrencyChecks, map, factory, null);
+      this(withConcurrencyChecks, false, map, factory, null);
     }
 
     protected TestableAbstractRegionMap(boolean withConcurrencyChecks,
         ConcurrentMapWithReusableEntries map, RegionEntryFactory factory,
         RegionEntry regionEntryForGetEntry) {
+      this(withConcurrencyChecks, false, map, factory, regionEntryForGetEntry);
+    }
+
+    protected TestableAbstractRegionMap(boolean withConcurrencyChecks, boolean isDistributedRegion,
+        ConcurrentMapWithReusableEntries map, RegionEntryFactory factory,
+        RegionEntry regionEntryForGetEntry) {
       super(null);
       this.regionEntryForGetEntry = regionEntryForGetEntry;
-      LocalRegion owner = mock(LocalRegion.class);
+      LocalRegion owner = isDistributedRegion ? mock(DistributedRegion.class, RETURNS_DEEP_STUBS)
+          : mock(LocalRegion.class);
       CachePerfStats cachePerfStats = mock(CachePerfStats.class);
       when(owner.getCachePerfStats()).thenReturn(cachePerfStats);
       when(owner.getConcurrencyChecksEnabled()).thenReturn(withConcurrencyChecks);
@@ -1268,19 +1276,30 @@ public class AbstractRegionMapTest {
    */
   private static class TxTestableAbstractRegionMap extends AbstractRegionMap {
 
-    protected TxTestableAbstractRegionMap() {
+    protected TxTestableAbstractRegionMap(boolean isInitialized) {
       super(null);
-      LocalRegion owner = mock(LocalRegion.class);
+      InternalRegion owner;
+      if (isInitialized) {
+        owner = mock(LocalRegion.class);
+        when(owner.isInitialized()).thenReturn(true);
+      } else {
+        owner = mock(DistributedRegion.class);
+        when(owner.isInitialized()).thenReturn(false);
+      }
       KeyInfo keyInfo = mock(KeyInfo.class);
       when(keyInfo.getKey()).thenReturn(KEY);
       when(owner.getKeyInfo(eq(KEY), any(), any())).thenReturn(keyInfo);
       when(owner.getMyId()).thenReturn(mock(InternalDistributedMember.class));
       when(owner.getCache()).thenReturn(mock(InternalCache.class));
       when(owner.isAllEvents()).thenReturn(true);
-      when(owner.isInitialized()).thenReturn(true);
       when(owner.shouldNotifyBridgeClients()).thenReturn(true);
+      when(owner.lockWhenRegionIsInitializing()).thenCallRealMethod();
       initialize(owner, new Attributes(), null, false);
     }
+
+    protected TxTestableAbstractRegionMap() {
+      this(true);
+    }
   }
 
   @Test
@@ -1361,6 +1380,120 @@ public class AbstractRegionMapTest {
         false);
   }
 
+  @Test
+  public void txApplyPutDoesNotLockWhenRegionIsInitialized() {
+    AbstractRegionMap arm = new TxTestableAbstractRegionMap();
+    TXId txId = mock(TXId.class, RETURNS_DEEP_STUBS);
+    EventID eventId = mock(EventID.class);
+    TXRmtEvent txRmtEvent = mock(TXRmtEvent.class);
+
+    arm.txApplyPut(Operation.UPDATE, KEY, "", false, txId, txRmtEvent, eventId, null,
+        new ArrayList<>(), null, null, null, null, 1);
+
+    verify(arm._getOwner()).lockWhenRegionIsInitializing();
+    assertThat(arm._getOwner().lockWhenRegionIsInitializing()).isFalse();
+    verify(arm._getOwner(), never()).unlockWhenRegionIsInitializing();
+  }
+
+  @Test
+  public void txApplyPutLockWhenRegionIsInitializing() {
+    AbstractRegionMap arm = new TxTestableAbstractRegionMap(false);
+    TXId txId = mock(TXId.class, RETURNS_DEEP_STUBS);
+    EventID eventId = mock(EventID.class);
+    TXRmtEvent txRmtEvent = mock(TXRmtEvent.class);
+
+    arm.txApplyPut(Operation.UPDATE, KEY, "", false, txId, txRmtEvent, eventId, null,
+        new ArrayList<>(), null, null, null, null, 1);
+
+    verify(arm._getOwner()).lockWhenRegionIsInitializing();
+    assertThat(arm._getOwner().lockWhenRegionIsInitializing()).isTrue();
+    verify(arm._getOwner()).unlockWhenRegionIsInitializing();
+  }
+
+  @Test
+  public void txApplyDestroyDoesNotLockWhenRegionIsInitialized() {
+    AbstractRegionMap arm = new TxTestableAbstractRegionMap();
+    TXId txId = mock(TXId.class, RETURNS_DEEP_STUBS);
+
+    arm.txApplyDestroy(KEY, txId, null, false, false, null, null, null, new ArrayList<>(), null,
+        null, true, null, null, 0);
+
+    verify(arm._getOwner()).lockWhenRegionIsInitializing();
+    assertThat(arm._getOwner().lockWhenRegionIsInitializing()).isFalse();
+    verify(arm._getOwner(), never()).unlockWhenRegionIsInitializing();
+  }
+
+  @Test
+  public void txApplyDestroyLockWhenRegionIsInitializing() {
+    AbstractRegionMap arm = new TxTestableAbstractRegionMap(false);
+    TXId txId = mock(TXId.class, RETURNS_DEEP_STUBS);
+
+    arm.txApplyDestroy(KEY, txId, null, false, false, null, null, null, new ArrayList<>(), null,
+        null, true, null, null, 0);
+
+    verify(arm._getOwner()).lockWhenRegionIsInitializing();
+    assertThat(arm._getOwner().lockWhenRegionIsInitializing()).isTrue();
+    verify(arm._getOwner()).unlockWhenRegionIsInitializing();
+  }
+
+  @Test
+  public void txApplyInvalidateDoesNotLockWhenRegionIsInitialized() {
+    AbstractRegionMap arm = new TxTestableAbstractRegionMap();
+    TXId txId = mock(TXId.class, RETURNS_DEEP_STUBS);
+
+    arm.txApplyInvalidate(new Object(), Token.INVALID, false,
+        txId, mock(TXRmtEvent.class), false,
+        mock(EventID.class), null, new ArrayList<EntryEventImpl>(), null, null, null, null, 1);
+
+    verify(arm._getOwner()).lockWhenRegionIsInitializing();
+    assertThat(arm._getOwner().lockWhenRegionIsInitializing()).isFalse();
+    verify(arm._getOwner(), never()).unlockWhenRegionIsInitializing();
+  }
+
+  @Test
+  public void txApplyInvalidateLockWhenRegionIsInitializing() {
+    AbstractRegionMap arm = new TxTestableAbstractRegionMap(false);
+    TXId txId = mock(TXId.class, RETURNS_DEEP_STUBS);
+
+    arm.txApplyInvalidate(new Object(), Token.INVALID, false,
+        txId, mock(TXRmtEvent.class), false,
+        mock(EventID.class), null, new ArrayList<EntryEventImpl>(), null, null, null, null, 1);
+
+    verify(arm._getOwner()).lockWhenRegionIsInitializing();
+    assertThat(arm._getOwner().lockWhenRegionIsInitializing()).isTrue();
+    verify(arm._getOwner()).unlockWhenRegionIsInitializing();
+  }
+
+  @Test
+  public void invalidateDoesNotLockWhenRegionIsInitialized() {
+    TestableAbstractRegionMap arm = new TestableAbstractRegionMap(false, true,
+        mock(ConcurrentMapWithReusableEntries.class), mock(RegionEntryFactory.class),
+        mock(RegionEntry.class));
+    EntryEventImpl event = createEventForInvalidate(arm._getOwner());
+    when(arm._getOwner().isInitialized()).thenReturn(true);
+    when(arm._getOwner().lockWhenRegionIsInitializing()).thenCallRealMethod();
+    arm.invalidate(event, false, false, false);
+
+    verify(arm._getOwner()).lockWhenRegionIsInitializing();
+    assertThat(arm._getOwner().lockWhenRegionIsInitializing()).isFalse();
+    verify(arm._getOwner(), never()).unlockWhenRegionIsInitializing();
+  }
+
+  @Test
+  public void invalidateLocksWhenRegionIsInitializing() {
+    TestableAbstractRegionMap arm = new TestableAbstractRegionMap(false, true,
+        mock(ConcurrentMapWithReusableEntries.class), mock(RegionEntryFactory.class),
+        mock(RegionEntry.class));
+    EntryEventImpl event = createEventForInvalidate(arm._getOwner());
+    when(arm._getOwner().isInitialized()).thenReturn(false);
+    when(arm._getOwner().lockWhenRegionIsInitializing()).thenCallRealMethod();
+    arm.invalidate(event, false, false, false);
+
+    verify(arm._getOwner()).lockWhenRegionIsInitializing();
+    assertThat(arm._getOwner().lockWhenRegionIsInitializing()).isTrue();
+    verify(arm._getOwner()).unlockWhenRegionIsInitializing();
+  }
+
   private static class TxNoRegionEntryTestableAbstractRegionMap
       extends TxTestableAbstractRegionMap {
     @Override
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/DistributedRegionTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/DistributedRegionTest.java
index cf27546..a256012 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/DistributedRegionTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/DistributedRegionTest.java
@@ -15,7 +15,6 @@
 package org.apache.geode.internal.cache;
 
 import static org.assertj.core.api.Assertions.assertThat;
-import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.Matchers.anyObject;
 import static org.mockito.Matchers.eq;
 import static org.mockito.Mockito.RETURNS_DEEP_STUBS;
@@ -27,8 +26,6 @@ import static org.mockito.Mockito.when;
 
 import org.junit.Test;
 
-import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
-import org.apache.geode.internal.cache.versions.RegionVersionVector;
 
 
 public class DistributedRegionTest {
@@ -49,21 +46,17 @@ public class DistributedRegionTest {
   @Test
   public void cleanUpAfterFailedGIIHoldsLockForClear() {
     DistributedRegion distributedRegion = mock(DistributedRegion.class, RETURNS_DEEP_STUBS);
-    RegionVersionVector regionVersionVector = mock(RegionVersionVector.class);
     RegionMap regionMap = mock(RegionMap.class);
-    InternalDistributedMember member = mock(InternalDistributedMember.class);
 
     doCallRealMethod().when(distributedRegion).cleanUpAfterFailedGII(false);
-    when(distributedRegion.getVersionVector()).thenReturn(regionVersionVector);
     when(distributedRegion.getRegionMap()).thenReturn(regionMap);
     when(regionMap.isEmpty()).thenReturn(false);
-    when(distributedRegion.getMyId()).thenReturn(member);
 
     distributedRegion.cleanUpAfterFailedGII(false);
 
-    verify(regionVersionVector).lockForClear(any(), any(), eq(member));
+    verify(distributedRegion).lockFailedGIIClearWriteLock();
     verify(distributedRegion).closeEntries();
-    verify(regionVersionVector).unlockForClear(eq(member));
+    verify(distributedRegion).unlockFailedGIIClearWriteLock();
   }
 
   @Test
@@ -77,8 +70,27 @@ public class DistributedRegionTest {
 
     distributedRegion.cleanUpAfterFailedGII(true);
 
-
     verify(diskRegion).resetRecoveredEntries(eq(distributedRegion));
     verify(distributedRegion, never()).closeEntries();
   }
+
+  @Test
+  public void lockHeldWhenRegionIsNotInitialized() {
+    DistributedRegion distributedRegion = mock(DistributedRegion.class);
+    doCallRealMethod().when(distributedRegion).lockWhenRegionIsInitializing();
+    when(distributedRegion.isInitialized()).thenReturn(false);
+
+    assertThat(distributedRegion.lockWhenRegionIsInitializing()).isTrue();
+    verify(distributedRegion).lockFailedGIIClearReadLock();
+  }
+
+  @Test
+  public void lockNotHeldWhenRegionIsInitialized() {
+    DistributedRegion distributedRegion = mock(DistributedRegion.class);
+    doCallRealMethod().when(distributedRegion).lockWhenRegionIsInitializing();
+    when(distributedRegion.isInitialized()).thenReturn(true);
+
+    assertThat(distributedRegion.lockWhenRegionIsInitializing()).isFalse();
+    verify(distributedRegion, never()).lockFailedGIIClearReadLock();
+  }
 }
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 f02468d..4fdab13 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
@@ -21,6 +21,7 @@ import static org.mockito.ArgumentMatchers.anyBoolean;
 import static org.mockito.ArgumentMatchers.anyInt;
 import static org.mockito.ArgumentMatchers.eq;
 import static org.mockito.ArgumentMatchers.same;
+import static org.mockito.Mockito.RETURNS_DEEP_STUBS;
 import static org.mockito.Mockito.doThrow;
 import static org.mockito.Mockito.inOrder;
 import static org.mockito.Mockito.mock;
@@ -42,6 +43,7 @@ import org.apache.geode.cache.EntryNotFoundException;
 import org.apache.geode.cache.Operation;
 import org.apache.geode.cache.query.internal.index.IndexManager;
 import org.apache.geode.internal.cache.CachePerfStats;
+import org.apache.geode.internal.cache.DistributedRegion;
 import org.apache.geode.internal.cache.EntryEventImpl;
 import org.apache.geode.internal.cache.InternalRegion;
 import org.apache.geode.internal.cache.RegionClearedException;
@@ -1404,6 +1406,40 @@ public class RegionMapDestroyTest {
     verifyInvokedDestroyMethodsOnRegion(false);
   }
 
+  @Test
+  public void destroyDoesNotLockGIIClearLockWhenRegionIsInitialized()
+      throws Exception {
+    DistributedRegion region = mock(DistributedRegion.class, RETURNS_DEEP_STUBS);
+    when(region.isInitialized()).thenReturn(true);
+    when(region.lockWhenRegionIsInitializing()).thenCallRealMethod();
+    RegionMapDestroy mapDestroy =
+        new RegionMapDestroy(region, regionMap, mock(CacheModificationLock.class));
+
+    mapDestroy.destroy(event, inTokenMode, duringRI, cacheWrite, isEviction,
+        expectedOldValue, removeRecoveredEntry);
+
+    verify(region).lockWhenRegionIsInitializing();
+    assertThat(region.lockWhenRegionIsInitializing()).isFalse();
+    verify(region, never()).unlockWhenRegionIsInitializing();
+  }
+
+  @Test
+  public void destroyLockGIIClearLockWhenRegionIsInitializing()
+      throws Exception {
+    DistributedRegion region = mock(DistributedRegion.class, RETURNS_DEEP_STUBS);
+    when(region.isInitialized()).thenReturn(false);
+    when(region.lockWhenRegionIsInitializing()).thenCallRealMethod();
+    RegionMapDestroy mapDestroy =
+        new RegionMapDestroy(region, regionMap, mock(CacheModificationLock.class));
+
+    mapDestroy.destroy(event, inTokenMode, duringRI, cacheWrite, isEviction,
+        expectedOldValue, removeRecoveredEntry);
+
+    verify(region).lockWhenRegionIsInitializing();
+    assertThat(region.lockWhenRegionIsInitializing()).isTrue();
+    verify(region).unlockWhenRegionIsInitializing();
+  }
+
   ///////////////////// given methods /////////////////////////////
 
   private void givenConcurrencyChecks(boolean enabled) {
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/map/RegionMapPutTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/map/RegionMapPutTest.java
index e525785..7f518dd 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/map/RegionMapPutTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/map/RegionMapPutTest.java
@@ -44,6 +44,7 @@ import org.apache.geode.cache.Operation;
 import org.apache.geode.cache.RegionAttributes;
 import org.apache.geode.cache.Scope;
 import org.apache.geode.internal.cache.CachePerfStats;
+import org.apache.geode.internal.cache.DistributedRegion;
 import org.apache.geode.internal.cache.EntryEventImpl;
 import org.apache.geode.internal.cache.EntryEventSerialization;
 import org.apache.geode.internal.cache.ImageState;
@@ -944,6 +945,41 @@ public class RegionMapPutTest {
         eq(true), eq(ifNew), eq(ifOld), eq(expectedOldValue), eq(requireOldValue));
   }
 
+  @Test
+  public void runWileLockedForCacheModificationDoesNotLockGIIClearLockWhenRegionIsInitialized()
+      throws Exception {
+    DistributedRegion region = mock(DistributedRegion.class);
+    when(region.isInitialized()).thenReturn(true);
+    when(region.lockWhenRegionIsInitializing()).thenCallRealMethod();
+    RegionMapPut regionMapPut = new RegionMapPut(focusedRegionMap, region, cacheModificationLock,
+        entryEventSerialization, event, ifNew, ifOld, overwriteDestroyed, requireOldValue,
+        expectedOldValue);
+
+    regionMapPut.runWhileLockedForCacheModification(() -> {
+    });
+
+    verify(region).lockWhenRegionIsInitializing();
+    assertThat(region.lockWhenRegionIsInitializing()).isFalse();
+    verify(region, never()).unlockWhenRegionIsInitializing();
+  }
+
+  @Test
+  public void runWileLockedForCacheModificationLockGIIClearLockWhenRegionIsInitializing() {
+    DistributedRegion region = mock(DistributedRegion.class);
+    when(region.isInitialized()).thenReturn(false);
+    when(region.lockWhenRegionIsInitializing()).thenCallRealMethod();
+    RegionMapPut regionMapPut = new RegionMapPut(focusedRegionMap, region, cacheModificationLock,
+        entryEventSerialization, event, ifNew, ifOld, overwriteDestroyed, requireOldValue,
+        expectedOldValue);
+
+    regionMapPut.runWhileLockedForCacheModification(() -> {
+    });
+
+    verify(region).lockWhenRegionIsInitializing();
+    assertThat(region.lockWhenRegionIsInitializing()).isTrue();
+    verify(region).unlockWhenRegionIsInitializing();
+  }
+
   private void givenAnOperationThatDoesNotGuaranteeOldValue() {
     when(event.getOperation()).thenReturn(Operation.UPDATE);
   }