You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@geode.apache.org by GitBox <gi...@apache.org> on 2022/04/20 22:58:57 UTC

[GitHub] [geode] DonalEvans commented on a diff in pull request #7596: GEODE-10242: Generate tail key after all child regions become primary

DonalEvans commented on code in PR #7596:
URL: https://github.com/apache/geode/pull/7596#discussion_r854578032


##########
geode-core/src/main/java/org/apache/geode/internal/cache/ColocationHelper.java:
##########
@@ -377,6 +378,26 @@ public static Map<String, LocalDataSet> getColocatedLocalDataSetsForBuckets(
    */
   public static List<PartitionedRegion> getColocatedChildRegions(
       PartitionedRegion partitionedRegion) {
+    List<PartitionedRegion> colocatedChildRegions =
+        getColocatedChildRegions(partitionedRegion, true, false);
+
+    // Fix for 44484 - Make the list of colocated child regions
+    // is always in the same order on all nodes.
+    Collections.sort(colocatedChildRegions, (o1, o2) -> {

Review Comment:
   Since `colocatedChildRegions` is a list, this can be simplified to:
   ```
       colocatedChildRegions.sort((o1, o2) -> {
   ```



##########
geode-core/src/main/java/org/apache/geode/internal/cache/ColocationHelper.java:
##########
@@ -377,6 +378,26 @@ public static Map<String, LocalDataSet> getColocatedLocalDataSetsForBuckets(
    */
   public static List<PartitionedRegion> getColocatedChildRegions(
       PartitionedRegion partitionedRegion) {
+    List<PartitionedRegion> colocatedChildRegions =
+        getColocatedChildRegions(partitionedRegion, true, false);
+
+    // Fix for 44484 - Make the list of colocated child regions
+    // is always in the same order on all nodes.
+    Collections.sort(colocatedChildRegions, (o1, o2) -> {
+      if (o1.isShadowPR() == o2.isShadowPR()) {
+        return o1.getFullPath().compareTo(o2.getFullPath());
+      }
+      if (o1.isShadowPR()) {
+        return 1;
+      }
+      return -1;
+    });
+    return colocatedChildRegions;
+  }
+
+  @NotNull
+  private static List<PartitionedRegion> getColocatedChildRegions(

Review Comment:
   This method does not sort the child regions, so the returned list when calling `getColocatedChildRegions(partitionedRegion)` will not be the same as calling `getColocatedChildRegions(partitionedRegion, true, false)`. Would it be better to move the sorting into this method, since the one-argument version of the method calls this one anyway?



##########
geode-core/src/main/java/org/apache/geode/internal/cache/BucketRegion.java:
##########
@@ -629,6 +642,76 @@ public void handleWANEvent(EntryEventImpl event) {
     }
   }
 
+  boolean needToWaitForColocatedBucketsBecomePrimary() {
+    if (hasChildRegion()) {
+      synchronized (this) {
+        return notPrimary;
+      }
+    }
+    return false;
+  }
+
+  boolean hasChildRegion() {
+    return ColocationHelper.getFirstColocatedNonShadowChildRegions(partitionedRegion).size() > 0;
+  }
+
+  void waitForAllChildColocatedBucketsBecomePrimary() {
+    synchronized (allChildBucketsBecomePrimaryLock) {
+      if (!alreadyInWaitForAllChildBucketsToBecomePrimary) {
+        alreadyInWaitForAllChildBucketsToBecomePrimary = true;
+        executeCheckIfAllChildBucketsBecomePrimary();
+      }
+      while (!allChildBucketsBecomePrimary && getBucketAdvisor().isPrimary()) {
+        // if no longer primary, no need to wait as the operation should fail
+        // with PrimaryBucketException later.
+        try {
+          allChildBucketsBecomePrimaryLock.wait(10);
+        } catch (InterruptedException e) {
+          Thread.currentThread().interrupt();
+          getCache().getCancelCriterion().checkCancelInProgress(e);
+        }
+      }
+    }
+  }
+
+  void executeCheckIfAllChildBucketsBecomePrimary() {
+    if (waitForAllChildBucketsToBecomePrimaryExecutor == null) {
+      waitForAllChildBucketsToBecomePrimaryExecutor =
+          LoggingExecutors.newSingleThreadExecutor("CheckPrimaryForColocation", true);
+    }
+    waitForAllChildBucketsToBecomePrimaryExecutor
+        .execute(this::checkIfAllChildBucketsBecomePrimary);
+  }
+
+  void checkIfAllChildBucketsBecomePrimary() {
+    try {
+      while (!getBucketAdvisor().checkIfAllColocatedChildBucketsBecomePrimary()) {
+        if (!getBucketAdvisor().isPrimary()) {
+          // This parent bucket is no longer a primary, no need to wait.
+          return;
+        }
+      }
+      synchronized (allChildBucketsBecomePrimaryLock) {
+        allChildBucketsBecomePrimary = true;
+        notPrimary = false;

Review Comment:
   We seem to always set these two fields together, and they're always the opposite value of eachother. Are both needed?



##########
geode-core/src/main/java/org/apache/geode/internal/cache/BucketRegion.java:
##########
@@ -235,6 +237,12 @@ AtomicLong5 getEventSeqNum() {
     return eventSeqNum;
   }
 
+  boolean notPrimary = true;
+  volatile boolean allChildBucketsBecomePrimary = false;
+  private Object allChildBucketsBecomePrimaryLock = new Object();

Review Comment:
   This should be `final`



##########
geode-core/src/test/java/org/apache/geode/internal/cache/BucketRegionTest.java:
##########
@@ -715,4 +722,178 @@ public void txHandleWANEventDoesNotCallHandleWANEventIfParallelWanNotEnabled() {
 
     verify(bucketRegion, never()).handleWANEvent(event);
   }
+
+  @Test
+  public void needToWaitForColocatedBucketsBecomePrimaryReturnsTrueIfHasChildRegionAndWasNotPrimary() {
+    BucketRegion bucketRegion =
+        spy(new BucketRegion(regionName, regionAttributes, partitionedRegion,
+            cache, internalRegionArgs, disabledClock()));
+    doReturn(true).when(bucketRegion).hasChildRegion();
+
+    assertThat(bucketRegion.needToWaitForColocatedBucketsBecomePrimary()).isTrue();
+  }
+
+  @Test
+  public void needToWaitForColocatedBucketsBecomePrimaryReturnsFalseIfHasChildRegionAndWasPrimary() {
+    BucketRegion bucketRegion =
+        spy(new BucketRegion(regionName, regionAttributes, partitionedRegion,
+            cache, internalRegionArgs, disabledClock()));
+    doReturn(true).when(bucketRegion).hasChildRegion();
+    bucketRegion.notPrimary = false;
+
+    assertThat(bucketRegion.needToWaitForColocatedBucketsBecomePrimary()).isFalse();
+  }
+
+  @Test
+  public void needToWaitForColocatedBucketsBecomePrimaryReturnsFalseIfNoChildRegion() {
+    BucketRegion bucketRegion =
+        spy(new BucketRegion(regionName, regionAttributes, partitionedRegion,
+            cache, internalRegionArgs, disabledClock()));
+    doReturn(false).when(bucketRegion).hasChildRegion();
+
+    assertThat(bucketRegion.needToWaitForColocatedBucketsBecomePrimary()).isFalse();
+  }
+
+  @Test
+  public void handleWANEventDoesNotWaitForAllChildColocatedBucketsBecomePrimaryIfNoNeedToWait() {
+    BucketRegion bucketRegion =
+        spy(new BucketRegion(regionName, regionAttributes, partitionedRegion,
+            cache, internalRegionArgs, disabledClock()));
+    when(bucketAdvisor.isPrimary()).thenReturn(true);
+    doReturn(false).when(bucketRegion).needToWaitForColocatedBucketsBecomePrimary();
+
+    when(partitionedRegion.getTotalNumberOfBuckets()).thenReturn(4);
+    doReturn(0).when(bucketRegion).getId();
+
+    bucketRegion.handleWANEvent(event);
+
+    verify(bucketRegion, never()).waitForAllChildColocatedBucketsBecomePrimary();
+    verify(event).setTailKey(4L);
+  }
+
+  @Test
+  public void handleWANEventWaitsForAllChildColocatedBucketsBecomePrimaryIfWasNotPrimary() {
+    BucketRegion bucketRegion =
+        spy(new BucketRegion(regionName, regionAttributes, partitionedRegion,
+            cache, internalRegionArgs, disabledClock()));
+    when(bucketAdvisor.isPrimary()).thenReturn(true);
+    doReturn(true).when(bucketRegion).hasChildRegion();
+    doNothing().when(bucketRegion).waitForAllChildColocatedBucketsBecomePrimary();
+    when(partitionedRegion.getTotalNumberOfBuckets()).thenReturn(4);
+    doReturn(0).when(bucketRegion).getId();
+
+    bucketRegion.handleWANEvent(event);
+
+    verify(bucketRegion).waitForAllChildColocatedBucketsBecomePrimary();
+    verify(event).setTailKey(4L);
+  }
+
+  @Test
+  public void handleWANEventSetNotPrimaryIfWasPrimaryAndNoLongerAPrimary() {
+    BucketRegion bucketRegion =
+        spy(new BucketRegion(regionName, regionAttributes, partitionedRegion,
+            cache, internalRegionArgs, disabledClock()));
+    when(bucketAdvisor.isPrimary()).thenReturn(false);
+    bucketRegion.notPrimary = false;
+
+    bucketRegion.handleWANEvent(event);
+
+    verify(bucketRegion).setNotPrimaryIfNecessary();
+    assertThat(bucketRegion.notPrimary).isTrue();
+    assertThat(bucketRegion.allChildBucketsBecomePrimary).isFalse();
+  }
+
+  @Test
+  public void handleWANEventDoesNotSetNotPrimaryIfWasNotPrimary() {
+    BucketRegion bucketRegion =
+        spy(new BucketRegion(regionName, regionAttributes, partitionedRegion,
+            cache, internalRegionArgs, disabledClock()));
+    when(bucketAdvisor.isPrimary()).thenReturn(false);
+    bucketRegion.notPrimary = true;
+
+    bucketRegion.handleWANEvent(event);
+
+    verify(bucketRegion, never()).setNotPrimaryIfNecessary();
+  }
+
+  @Test
+  public void onlyOneThreadWillExecuteCheckIfAllChildBucketsBecomePrimary() throws Exception {
+    BucketRegion bucketRegion =
+        spy(new BucketRegion(regionName, regionAttributes, partitionedRegion,
+            cache, internalRegionArgs, disabledClock()));
+    doNothing().when(bucketRegion).executeCheckIfAllChildBucketsBecomePrimary();
+
+    Future<?> future = executor.submit(bucketRegion::waitForAllChildColocatedBucketsBecomePrimary);
+    Future<?> future2 = executor.submit(bucketRegion::waitForAllChildColocatedBucketsBecomePrimary);
+
+    future.get();
+    future2.get();

Review Comment:
   If I remove the synchronization from `waitForAllChildColocatedBucketsBecomePrimary()`, this test still passes, even when run 1000 times, so I think that without something to ensure that both threads are trying to execute the synchronized block at the same time, this test can't guarantee that it's testing correct synchronization, just that successive calls to `bucketRegion.waitForAllChildColocatedBucketsBecomePrimary()` do not result in `executeCheckIfAllChildBucketsBecomePrimary` being called twice.



##########
geode-core/src/test/java/org/apache/geode/internal/cache/BucketAdvisorTest.java:
##########
@@ -317,4 +372,163 @@ public void testGetAllHostingMembersReturnsMemberWhenBucketAdvisorHasTwoProfiles
 
     assertThat(bucketAdvisor.adviseInitialized().size()).isEqualTo(1);
   }
+
+  @Test
+  void checkIfAllColocatedChildBucketsBecomePrimaryRetrunsTrueIfAllChildBucketsArePrimary() {

Review Comment:
   Typo here, should be "PrimaryReturnsTrue"



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@geode.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org