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 2020/06/11 00:41:29 UTC

[GitHub] [geode] aaronlindsey opened a new pull request #5236: GEODE-8241: Locator observes locator-wait-time

aaronlindsey opened a new pull request #5236:
URL: https://github.com/apache/geode/pull/5236


   In the case where a locator starts up and is unable to connect to any
   other locators, it may decide to become the membership coordinator even
   if locator-wait-time has not elapsed.
   
   This change addresses this issue by requiring a locator to wait for
   locator-wait-time before deciding to become the coordinator.
   
   Co-authored-by: Aaron Lindsey <al...@vmware.com>
   Co-Authored-By: Vincent Ford <vf...@pivotal.io>
   Co-authored-by: Bill Burcham <bb...@pivotal.io>
   


----------------------------------------------------------------
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.

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



[GitHub] [geode] aaronlindsey commented on a change in pull request #5236: GEODE-8241: Locator observes locator-wait-time

Posted by GitBox <gi...@apache.org>.
aaronlindsey commented on a change in pull request #5236:
URL: https://github.com/apache/geode/pull/5236#discussion_r440451304



##########
File path: geode-membership/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/MembershipIntegrationTest.java
##########
@@ -172,6 +181,92 @@ public void secondMembershipCanJoinUsingTheSecondLocatorToStart()
     stop(locator2, locator1);
   }
 
+  @Test
+  public void secondMembershipPausesForLocatorWaitTime()
+      throws IOException, MemberStartupException {
+
+    /*
+     * Start a locator for the coordinator (membership) so we have a port for it.
+     *
+     * Its locator-wait-time is set to 0 so it eventually (soon after membership is started) forms a
+     * distributed system and becomes a coordinator.
+     */
+
+    final MembershipLocator<MemberIdentifier> coordinatorLocator = createLocator(0);
+    coordinatorLocator.start();
+    final int coordinatorLocatorPort = coordinatorLocator.getPort();
+
+    final Membership<MemberIdentifier> coordinatorMembership =
+        createMembership(coordinatorLocator, coordinatorLocatorPort);
+
+    /*
+     * We have not even started the membership yet — connection attempts will certainly fail until
+     * we do. This is a bit like the locator (host) not being present in DNS (yet).
+     */
+
+    /*
+     * Start a second locator and membership trying to join via the coordinator (membership) that
+     * hasn't yet started behind the port.
+     *
+     * Set its locator-wait-time so it'll not become a coordinator right away, allowing time for the
+     * other member to start and become a coordinator.
+     *
+     * Calculate the locator-wait-time to be greater than the minimum wait time for connecting to a
+     * locator.
+     */
+
+    final MembershipLocator<MemberIdentifier> lateJoiningLocator = createLocator(0);
+    lateJoiningLocator.start();
+    final int lateJoiningLocatorPort = lateJoiningLocator.getPort();
+
+    final int[] lateJoiningMembershipLocatorPorts =
+        new int[] {coordinatorLocatorPort, lateJoiningLocatorPort};
+
+    final Duration minimumJoinWaitTime = Duration
+        .ofMillis(JOIN_RETRY_SLEEP + FIND_LOCATOR_RETRY_SLEEP) // amount of sleep time per retry
+        .multipliedBy(lateJoiningMembershipLocatorPorts.length * 2); // expected number of retries
+    final int locatorWaitTime = (int) (3 * minimumJoinWaitTime.getSeconds());
+
+    final MembershipConfig lateJoiningMembershipConfig =
+        createMembershipConfig(true, locatorWaitTime, lateJoiningMembershipLocatorPorts);
+    final Membership<MemberIdentifier> lateJoiningMembership =
+        createMembership(lateJoiningMembershipConfig, lateJoiningLocator);
+
+    CompletableFuture<Void> lateJoiningMembershipStartup = executorServiceRule.runAsync(() -> {
+      try {
+        start(lateJoiningMembership);
+      } catch (MemberStartupException e) {
+        throw new RuntimeException(e);
+      }
+    });
+
+    /*
+     * Now start the coordinator (membership), after waiting longer than the minimum wait time for
+     * connecting to a locator but shorter than the locator-wait-time.
+     */
+
+    CompletableFuture<Void> coordinatorMembershipStartup = executorServiceRule.runAsync(() -> {
+      try {
+        Thread.sleep(2 * minimumJoinWaitTime.toMillis());

Review comment:
       Again, `minimumJoinWaitTime` is the minimum amount of time a locator will wait to become a coordinator. We are just trying to make sure we sleep for longer than the minimum so we can detect whether the `lateJoiningMembership` is waiting for the full `locator-wait-time` and not just the minimum wait time.




----------------------------------------------------------------
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.

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



[GitHub] [geode] aaronlindsey commented on pull request #5236: GEODE-8241: Locator observes locator-wait-time

Posted by GitBox <gi...@apache.org>.
aaronlindsey commented on pull request #5236:
URL: https://github.com/apache/geode/pull/5236#issuecomment-642340466


   FWIW, I ran the test 100x on my machine and it passed each time. When I reverted the change to `GMSJoinLeave.java`, I saw the test fail about 30x consistently.


----------------------------------------------------------------
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.

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



[GitHub] [geode] aaronlindsey commented on pull request #5236: GEODE-8241: Locator observes locator-wait-time

Posted by GitBox <gi...@apache.org>.
aaronlindsey commented on pull request #5236:
URL: https://github.com/apache/geode/pull/5236#issuecomment-646223094


   I rebased on develop to pick up the fix for the failing `AcceptanceTestOpenJDK11` check.


----------------------------------------------------------------
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.

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



[GitHub] [geode] bschuchardt commented on a change in pull request #5236: GEODE-8241: Locator observes locator-wait-time

Posted by GitBox <gi...@apache.org>.
bschuchardt commented on a change in pull request #5236:
URL: https://github.com/apache/geode/pull/5236#discussion_r438832696



##########
File path: geode-membership/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/MembershipIntegrationTest.java
##########
@@ -172,6 +179,91 @@ public void secondMembershipCanJoinUsingTheSecondLocatorToStart()
     stop(locator2, locator1);
   }
 
+  @Test
+  public void secondMembershipPausesForLocatorWaitTime()
+      throws IOException, MemberStartupException {
+
+    /*
+     * Start a locator for the coordinator (membership) so we have a port for it.
+     *
+     * Its locator-wait-time is set to 0 so it eventually (soon after membership is started) forms a
+     * distributed system and becomes a coordinator.
+     */
+
+    final MembershipLocator<MemberIdentifier> coordinatorLocator = createLocator(0);
+    coordinatorLocator.start();
+    final int coordinatorLocatorPort = coordinatorLocator.getPort();
+
+    final Membership<MemberIdentifier> coordinatorMembership =
+        createMembership(coordinatorLocator, coordinatorLocatorPort);
+
+    /*
+     * We have not even started the membership yet — connection attempts will certainly fail until
+     * we do. This is a bit like the locator (host) not being present in DNS (yet).
+     */
+
+    /*
+     * Start a second locator and membership trying to join via the coordinator (membership) that
+     * hasn't yet started behind the port.
+     *
+     * Set its locator-wait-time so it'll not become a coordinator right away, allowing time for the
+     * other member to start and become a coordinator.
+     *
+     * Calculate the locator-wait-time to be greater than the minimum wait time for connecting to a
+     * locator.
+     */
+
+    final MembershipLocator<MemberIdentifier> lateJoiningLocator = createLocator(0);
+    lateJoiningLocator.start();
+    final int lateJoiningLocatorPort = lateJoiningLocator.getPort();
+
+    final int[] lateJoiningMembershipLocatorPorts =
+        new int[] {coordinatorLocatorPort, lateJoiningLocatorPort};
+
+    final Duration minimumJoinWaitTime = Duration
+        .ofMillis(2_000) // expected amount of sleep time per loop in GMSJoinLeave.join()
+        .multipliedBy(lateJoiningMembershipLocatorPorts.length * 2); // expected number of loops
+    final int locatorWaitTime = (int) (3 * minimumJoinWaitTime.getSeconds());
+
+    final MembershipConfig lateJoiningMembershipConfig =
+        createMembershipConfig(true, locatorWaitTime, lateJoiningMembershipLocatorPorts);
+    final Membership<MemberIdentifier> lateJoiningMembership =
+        createMembership(lateJoiningMembershipConfig, lateJoiningLocator);
+
+    CompletableFuture<Void> lateJoiningMembershipStartup = executorServiceRule.runAsync(() -> {
+      try {
+        start(lateJoiningMembership);
+      } catch (MemberStartupException e) {
+        e.printStackTrace();

Review comment:
       There are a couple of these auto-generated try/catch statements in the test.  If they're failure conditions shouldn't the test fail?  If they're not, what good are these stack traces?




----------------------------------------------------------------
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.

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



[GitHub] [geode] aaronlindsey merged pull request #5236: GEODE-8241: Locator observes locator-wait-time

Posted by GitBox <gi...@apache.org>.
aaronlindsey merged pull request #5236:
URL: https://github.com/apache/geode/pull/5236


   


----------------------------------------------------------------
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.

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



[GitHub] [geode] aaronlindsey commented on pull request #5236: GEODE-8241: Locator observes locator-wait-time

Posted by GitBox <gi...@apache.org>.
aaronlindsey commented on pull request #5236:
URL: https://github.com/apache/geode/pull/5236#issuecomment-645616125


   I've been running this test in the background while I work and so far it has passed 300x in a row without failing. I will continue letting it run until it reaches 1000 iterations.


----------------------------------------------------------------
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.

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



[GitHub] [geode] aaronlindsey commented on pull request #5236: GEODE-8241: Locator observes locator-wait-time

Posted by GitBox <gi...@apache.org>.
aaronlindsey commented on pull request #5236:
URL: https://github.com/apache/geode/pull/5236#issuecomment-646219134


   The failing `AcceptanceTestOpenJDK11` check is also failing on develop.


----------------------------------------------------------------
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.

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



[GitHub] [geode] aaronlindsey commented on pull request #5236: GEODE-8241: Locator observes locator-wait-time

Posted by GitBox <gi...@apache.org>.
aaronlindsey commented on pull request #5236:
URL: https://github.com/apache/geode/pull/5236#issuecomment-644401196


   > Once those are answered, I feel like we are still left with a test that will be flaky because of all the timing dependencies here.
   
   I understand your concern. We tried to choose the 2x and 3x values so that there is a sufficient buffer between (a) the minimum wait time, (b) the time at which the coordinator membership becomes "reachable", and (c) the locator wait time. We saw that with these values the test did pass and fail reliably. We also tried writing a test using `GMSJoinLeaveJUnitTest` but it had the same type of timing dependencies.
   
   The difficulty here is that we're trying to write "black box" style tests for a timeout. I don't see a way to avoid the timing dependencies without a large refactor, which I was trying to avoid since this is otherwise a small change with very little risk.


----------------------------------------------------------------
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.

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



[GitHub] [geode] aaronlindsey commented on a change in pull request #5236: GEODE-8241: Locator observes locator-wait-time

Posted by GitBox <gi...@apache.org>.
aaronlindsey commented on a change in pull request #5236:
URL: https://github.com/apache/geode/pull/5236#discussion_r440448569



##########
File path: geode-membership/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/MembershipIntegrationTest.java
##########
@@ -172,6 +181,92 @@ public void secondMembershipCanJoinUsingTheSecondLocatorToStart()
     stop(locator2, locator1);
   }
 
+  @Test
+  public void secondMembershipPausesForLocatorWaitTime()
+      throws IOException, MemberStartupException {
+
+    /*
+     * Start a locator for the coordinator (membership) so we have a port for it.
+     *
+     * Its locator-wait-time is set to 0 so it eventually (soon after membership is started) forms a
+     * distributed system and becomes a coordinator.
+     */
+
+    final MembershipLocator<MemberIdentifier> coordinatorLocator = createLocator(0);
+    coordinatorLocator.start();
+    final int coordinatorLocatorPort = coordinatorLocator.getPort();
+
+    final Membership<MemberIdentifier> coordinatorMembership =
+        createMembership(coordinatorLocator, coordinatorLocatorPort);
+
+    /*
+     * We have not even started the membership yet — connection attempts will certainly fail until
+     * we do. This is a bit like the locator (host) not being present in DNS (yet).
+     */
+
+    /*
+     * Start a second locator and membership trying to join via the coordinator (membership) that
+     * hasn't yet started behind the port.
+     *
+     * Set its locator-wait-time so it'll not become a coordinator right away, allowing time for the
+     * other member to start and become a coordinator.
+     *
+     * Calculate the locator-wait-time to be greater than the minimum wait time for connecting to a
+     * locator.
+     */
+
+    final MembershipLocator<MemberIdentifier> lateJoiningLocator = createLocator(0);
+    lateJoiningLocator.start();
+    final int lateJoiningLocatorPort = lateJoiningLocator.getPort();
+
+    final int[] lateJoiningMembershipLocatorPorts =
+        new int[] {coordinatorLocatorPort, lateJoiningLocatorPort};
+
+    final Duration minimumJoinWaitTime = Duration
+        .ofMillis(JOIN_RETRY_SLEEP + FIND_LOCATOR_RETRY_SLEEP) // amount of sleep time per retry
+        .multipliedBy(lateJoiningMembershipLocatorPorts.length * 2); // expected number of retries

Review comment:
       It comes from [this line](https://github.com/apache/geode/blob/e5426f79870e3217fc7c66572d2dddf44dbddd43/geode-membership/src/main/java/org/apache/geode/distributed/internal/membership/gms/membership/GMSJoinLeave.java#L327). I could assign the `2` to a constant and reference that if it would be clearer.




----------------------------------------------------------------
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.

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



[GitHub] [geode] aaronlindsey commented on a change in pull request #5236: GEODE-8241: Locator observes locator-wait-time

Posted by GitBox <gi...@apache.org>.
aaronlindsey commented on a change in pull request #5236:
URL: https://github.com/apache/geode/pull/5236#discussion_r441823437



##########
File path: geode-membership/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/MembershipIntegrationTest.java
##########
@@ -172,6 +181,92 @@ public void secondMembershipCanJoinUsingTheSecondLocatorToStart()
     stop(locator2, locator1);
   }
 
+  @Test
+  public void secondMembershipPausesForLocatorWaitTime()
+      throws IOException, MemberStartupException {
+
+    /*
+     * Start a locator for the coordinator (membership) so we have a port for it.
+     *
+     * Its locator-wait-time is set to 0 so it eventually (soon after membership is started) forms a
+     * distributed system and becomes a coordinator.
+     */
+
+    final MembershipLocator<MemberIdentifier> coordinatorLocator = createLocator(0);
+    coordinatorLocator.start();
+    final int coordinatorLocatorPort = coordinatorLocator.getPort();
+
+    final Membership<MemberIdentifier> coordinatorMembership =
+        createMembership(coordinatorLocator, coordinatorLocatorPort);
+
+    /*
+     * We have not even started the membership yet — connection attempts will certainly fail until
+     * we do. This is a bit like the locator (host) not being present in DNS (yet).
+     */
+
+    /*
+     * Start a second locator and membership trying to join via the coordinator (membership) that
+     * hasn't yet started behind the port.
+     *
+     * Set its locator-wait-time so it'll not become a coordinator right away, allowing time for the
+     * other member to start and become a coordinator.
+     *
+     * Calculate the locator-wait-time to be greater than the minimum wait time for connecting to a
+     * locator.
+     */
+
+    final MembershipLocator<MemberIdentifier> lateJoiningLocator = createLocator(0);
+    lateJoiningLocator.start();
+    final int lateJoiningLocatorPort = lateJoiningLocator.getPort();
+
+    final int[] lateJoiningMembershipLocatorPorts =
+        new int[] {coordinatorLocatorPort, lateJoiningLocatorPort};
+
+    final Duration minimumJoinWaitTime = Duration
+        .ofMillis(JOIN_RETRY_SLEEP + FIND_LOCATOR_RETRY_SLEEP) // amount of sleep time per retry
+        .multipliedBy(lateJoiningMembershipLocatorPorts.length * 2); // expected number of retries
+    final int locatorWaitTime = (int) (3 * minimumJoinWaitTime.getSeconds());
+
+    final MembershipConfig lateJoiningMembershipConfig =
+        createMembershipConfig(true, locatorWaitTime, lateJoiningMembershipLocatorPorts);
+    final Membership<MemberIdentifier> lateJoiningMembership =
+        createMembership(lateJoiningMembershipConfig, lateJoiningLocator);
+
+    CompletableFuture<Void> lateJoiningMembershipStartup = executorServiceRule.runAsync(() -> {
+      try {
+        start(lateJoiningMembership);
+      } catch (MemberStartupException e) {
+        throw new RuntimeException(e);
+      }
+    });
+
+    /*
+     * Now start the coordinator (membership), after waiting longer than the minimum wait time for
+     * connecting to a locator but shorter than the locator-wait-time.
+     */
+
+    CompletableFuture<Void> coordinatorMembershipStartup = executorServiceRule.runAsync(() -> {
+      try {
+        Thread.sleep(2 * minimumJoinWaitTime.toMillis());

Review comment:
       I added a comment to the code in the latest commit.

##########
File path: geode-membership/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/MembershipIntegrationTest.java
##########
@@ -172,6 +181,92 @@ public void secondMembershipCanJoinUsingTheSecondLocatorToStart()
     stop(locator2, locator1);
   }
 
+  @Test
+  public void secondMembershipPausesForLocatorWaitTime()
+      throws IOException, MemberStartupException {
+
+    /*
+     * Start a locator for the coordinator (membership) so we have a port for it.
+     *
+     * Its locator-wait-time is set to 0 so it eventually (soon after membership is started) forms a
+     * distributed system and becomes a coordinator.
+     */
+
+    final MembershipLocator<MemberIdentifier> coordinatorLocator = createLocator(0);
+    coordinatorLocator.start();
+    final int coordinatorLocatorPort = coordinatorLocator.getPort();
+
+    final Membership<MemberIdentifier> coordinatorMembership =
+        createMembership(coordinatorLocator, coordinatorLocatorPort);
+
+    /*
+     * We have not even started the membership yet — connection attempts will certainly fail until
+     * we do. This is a bit like the locator (host) not being present in DNS (yet).
+     */
+
+    /*
+     * Start a second locator and membership trying to join via the coordinator (membership) that
+     * hasn't yet started behind the port.
+     *
+     * Set its locator-wait-time so it'll not become a coordinator right away, allowing time for the
+     * other member to start and become a coordinator.
+     *
+     * Calculate the locator-wait-time to be greater than the minimum wait time for connecting to a
+     * locator.
+     */
+
+    final MembershipLocator<MemberIdentifier> lateJoiningLocator = createLocator(0);
+    lateJoiningLocator.start();
+    final int lateJoiningLocatorPort = lateJoiningLocator.getPort();
+
+    final int[] lateJoiningMembershipLocatorPorts =
+        new int[] {coordinatorLocatorPort, lateJoiningLocatorPort};
+
+    final Duration minimumJoinWaitTime = Duration
+        .ofMillis(JOIN_RETRY_SLEEP + FIND_LOCATOR_RETRY_SLEEP) // amount of sleep time per retry
+        .multipliedBy(lateJoiningMembershipLocatorPorts.length * 2); // expected number of retries
+    final int locatorWaitTime = (int) (3 * minimumJoinWaitTime.getSeconds());

Review comment:
       I added a comment to the code in the latest commit.




----------------------------------------------------------------
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.

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



[GitHub] [geode] aaronlindsey commented on pull request #5236: GEODE-8241: Locator observes locator-wait-time

Posted by GitBox <gi...@apache.org>.
aaronlindsey commented on pull request #5236:
URL: https://github.com/apache/geode/pull/5236#issuecomment-646043919


   @Bill @bschuchardt with the previous changes I noticed a failure after 400-something runs. The chosen `locatorWaitTime` was not quite long enough. So I greatly increased the `locatorWaitTime` in the latest commit and then I was able to run the test 1000x successfully. I'm pretty confident that after the latest commit the chance of flakiness is negligible.


----------------------------------------------------------------
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.

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



[GitHub] [geode] Bill commented on a change in pull request #5236: GEODE-8241: Locator observes locator-wait-time

Posted by GitBox <gi...@apache.org>.
Bill commented on a change in pull request #5236:
URL: https://github.com/apache/geode/pull/5236#discussion_r440341432



##########
File path: geode-membership/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/MembershipIntegrationTest.java
##########
@@ -172,6 +181,92 @@ public void secondMembershipCanJoinUsingTheSecondLocatorToStart()
     stop(locator2, locator1);
   }
 
+  @Test
+  public void secondMembershipPausesForLocatorWaitTime()
+      throws IOException, MemberStartupException {
+
+    /*
+     * Start a locator for the coordinator (membership) so we have a port for it.
+     *
+     * Its locator-wait-time is set to 0 so it eventually (soon after membership is started) forms a
+     * distributed system and becomes a coordinator.
+     */
+
+    final MembershipLocator<MemberIdentifier> coordinatorLocator = createLocator(0);
+    coordinatorLocator.start();
+    final int coordinatorLocatorPort = coordinatorLocator.getPort();
+
+    final Membership<MemberIdentifier> coordinatorMembership =
+        createMembership(coordinatorLocator, coordinatorLocatorPort);
+
+    /*
+     * We have not even started the membership yet — connection attempts will certainly fail until
+     * we do. This is a bit like the locator (host) not being present in DNS (yet).
+     */
+
+    /*
+     * Start a second locator and membership trying to join via the coordinator (membership) that
+     * hasn't yet started behind the port.
+     *
+     * Set its locator-wait-time so it'll not become a coordinator right away, allowing time for the
+     * other member to start and become a coordinator.
+     *
+     * Calculate the locator-wait-time to be greater than the minimum wait time for connecting to a
+     * locator.
+     */
+
+    final MembershipLocator<MemberIdentifier> lateJoiningLocator = createLocator(0);
+    lateJoiningLocator.start();
+    final int lateJoiningLocatorPort = lateJoiningLocator.getPort();
+
+    final int[] lateJoiningMembershipLocatorPorts =
+        new int[] {coordinatorLocatorPort, lateJoiningLocatorPort};
+
+    final Duration minimumJoinWaitTime = Duration
+        .ofMillis(JOIN_RETRY_SLEEP + FIND_LOCATOR_RETRY_SLEEP) // amount of sleep time per retry
+        .multipliedBy(lateJoiningMembershipLocatorPorts.length * 2); // expected number of retries
+    final int locatorWaitTime = (int) (3 * minimumJoinWaitTime.getSeconds());

Review comment:
       where does `3` come from?

##########
File path: geode-membership/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/MembershipIntegrationTest.java
##########
@@ -172,6 +181,92 @@ public void secondMembershipCanJoinUsingTheSecondLocatorToStart()
     stop(locator2, locator1);
   }
 
+  @Test
+  public void secondMembershipPausesForLocatorWaitTime()
+      throws IOException, MemberStartupException {
+
+    /*
+     * Start a locator for the coordinator (membership) so we have a port for it.
+     *
+     * Its locator-wait-time is set to 0 so it eventually (soon after membership is started) forms a
+     * distributed system and becomes a coordinator.
+     */
+
+    final MembershipLocator<MemberIdentifier> coordinatorLocator = createLocator(0);
+    coordinatorLocator.start();
+    final int coordinatorLocatorPort = coordinatorLocator.getPort();
+
+    final Membership<MemberIdentifier> coordinatorMembership =
+        createMembership(coordinatorLocator, coordinatorLocatorPort);
+
+    /*
+     * We have not even started the membership yet — connection attempts will certainly fail until
+     * we do. This is a bit like the locator (host) not being present in DNS (yet).
+     */
+
+    /*
+     * Start a second locator and membership trying to join via the coordinator (membership) that
+     * hasn't yet started behind the port.
+     *
+     * Set its locator-wait-time so it'll not become a coordinator right away, allowing time for the
+     * other member to start and become a coordinator.
+     *
+     * Calculate the locator-wait-time to be greater than the minimum wait time for connecting to a
+     * locator.
+     */
+
+    final MembershipLocator<MemberIdentifier> lateJoiningLocator = createLocator(0);
+    lateJoiningLocator.start();
+    final int lateJoiningLocatorPort = lateJoiningLocator.getPort();
+
+    final int[] lateJoiningMembershipLocatorPorts =
+        new int[] {coordinatorLocatorPort, lateJoiningLocatorPort};
+
+    final Duration minimumJoinWaitTime = Duration
+        .ofMillis(JOIN_RETRY_SLEEP + FIND_LOCATOR_RETRY_SLEEP) // amount of sleep time per retry
+        .multipliedBy(lateJoiningMembershipLocatorPorts.length * 2); // expected number of retries

Review comment:
       where does `2` come from? even if you don't want to explicitly reference an internal constant, it'd be nice to document where that constant lives

##########
File path: geode-membership/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/MembershipIntegrationTest.java
##########
@@ -172,6 +181,92 @@ public void secondMembershipCanJoinUsingTheSecondLocatorToStart()
     stop(locator2, locator1);
   }
 
+  @Test
+  public void secondMembershipPausesForLocatorWaitTime()
+      throws IOException, MemberStartupException {
+
+    /*
+     * Start a locator for the coordinator (membership) so we have a port for it.
+     *
+     * Its locator-wait-time is set to 0 so it eventually (soon after membership is started) forms a
+     * distributed system and becomes a coordinator.
+     */
+
+    final MembershipLocator<MemberIdentifier> coordinatorLocator = createLocator(0);
+    coordinatorLocator.start();
+    final int coordinatorLocatorPort = coordinatorLocator.getPort();
+
+    final Membership<MemberIdentifier> coordinatorMembership =
+        createMembership(coordinatorLocator, coordinatorLocatorPort);
+
+    /*
+     * We have not even started the membership yet — connection attempts will certainly fail until
+     * we do. This is a bit like the locator (host) not being present in DNS (yet).
+     */
+
+    /*
+     * Start a second locator and membership trying to join via the coordinator (membership) that
+     * hasn't yet started behind the port.
+     *
+     * Set its locator-wait-time so it'll not become a coordinator right away, allowing time for the
+     * other member to start and become a coordinator.
+     *
+     * Calculate the locator-wait-time to be greater than the minimum wait time for connecting to a
+     * locator.
+     */
+
+    final MembershipLocator<MemberIdentifier> lateJoiningLocator = createLocator(0);
+    lateJoiningLocator.start();
+    final int lateJoiningLocatorPort = lateJoiningLocator.getPort();
+
+    final int[] lateJoiningMembershipLocatorPorts =
+        new int[] {coordinatorLocatorPort, lateJoiningLocatorPort};
+
+    final Duration minimumJoinWaitTime = Duration
+        .ofMillis(JOIN_RETRY_SLEEP + FIND_LOCATOR_RETRY_SLEEP) // amount of sleep time per retry
+        .multipliedBy(lateJoiningMembershipLocatorPorts.length * 2); // expected number of retries
+    final int locatorWaitTime = (int) (3 * minimumJoinWaitTime.getSeconds());
+
+    final MembershipConfig lateJoiningMembershipConfig =
+        createMembershipConfig(true, locatorWaitTime, lateJoiningMembershipLocatorPorts);
+    final Membership<MemberIdentifier> lateJoiningMembership =
+        createMembership(lateJoiningMembershipConfig, lateJoiningLocator);
+
+    CompletableFuture<Void> lateJoiningMembershipStartup = executorServiceRule.runAsync(() -> {
+      try {
+        start(lateJoiningMembership);
+      } catch (MemberStartupException e) {
+        throw new RuntimeException(e);
+      }
+    });
+
+    /*
+     * Now start the coordinator (membership), after waiting longer than the minimum wait time for
+     * connecting to a locator but shorter than the locator-wait-time.
+     */
+
+    CompletableFuture<Void> coordinatorMembershipStartup = executorServiceRule.runAsync(() -> {
+      try {
+        Thread.sleep(2 * minimumJoinWaitTime.toMillis());

Review comment:
       where does this `2` come from?




----------------------------------------------------------------
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.

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



[GitHub] [geode] aaronlindsey commented on pull request #5236: GEODE-8241: Locator observes locator-wait-time

Posted by GitBox <gi...@apache.org>.
aaronlindsey commented on pull request #5236:
URL: https://github.com/apache/geode/pull/5236#issuecomment-642338908


   @vfordpivotal please review


----------------------------------------------------------------
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.

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



[GitHub] [geode] aaronlindsey commented on a change in pull request #5236: GEODE-8241: Locator observes locator-wait-time

Posted by GitBox <gi...@apache.org>.
aaronlindsey commented on a change in pull request #5236:
URL: https://github.com/apache/geode/pull/5236#discussion_r438938194



##########
File path: geode-membership/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/MembershipIntegrationTest.java
##########
@@ -172,6 +179,91 @@ public void secondMembershipCanJoinUsingTheSecondLocatorToStart()
     stop(locator2, locator1);
   }
 
+  @Test
+  public void secondMembershipPausesForLocatorWaitTime()
+      throws IOException, MemberStartupException {
+
+    /*
+     * Start a locator for the coordinator (membership) so we have a port for it.
+     *
+     * Its locator-wait-time is set to 0 so it eventually (soon after membership is started) forms a
+     * distributed system and becomes a coordinator.
+     */
+
+    final MembershipLocator<MemberIdentifier> coordinatorLocator = createLocator(0);
+    coordinatorLocator.start();
+    final int coordinatorLocatorPort = coordinatorLocator.getPort();
+
+    final Membership<MemberIdentifier> coordinatorMembership =
+        createMembership(coordinatorLocator, coordinatorLocatorPort);
+
+    /*
+     * We have not even started the membership yet — connection attempts will certainly fail until
+     * we do. This is a bit like the locator (host) not being present in DNS (yet).
+     */
+
+    /*
+     * Start a second locator and membership trying to join via the coordinator (membership) that
+     * hasn't yet started behind the port.
+     *
+     * Set its locator-wait-time so it'll not become a coordinator right away, allowing time for the
+     * other member to start and become a coordinator.
+     *
+     * Calculate the locator-wait-time to be greater than the minimum wait time for connecting to a
+     * locator.
+     */
+
+    final MembershipLocator<MemberIdentifier> lateJoiningLocator = createLocator(0);
+    lateJoiningLocator.start();
+    final int lateJoiningLocatorPort = lateJoiningLocator.getPort();
+
+    final int[] lateJoiningMembershipLocatorPorts =
+        new int[] {coordinatorLocatorPort, lateJoiningLocatorPort};
+
+    final Duration minimumJoinWaitTime = Duration
+        .ofMillis(2_000) // expected amount of sleep time per loop in GMSJoinLeave.join()
+        .multipliedBy(lateJoiningMembershipLocatorPorts.length * 2); // expected number of loops
+    final int locatorWaitTime = (int) (3 * minimumJoinWaitTime.getSeconds());
+
+    final MembershipConfig lateJoiningMembershipConfig =
+        createMembershipConfig(true, locatorWaitTime, lateJoiningMembershipLocatorPorts);
+    final Membership<MemberIdentifier> lateJoiningMembership =
+        createMembership(lateJoiningMembershipConfig, lateJoiningLocator);
+
+    CompletableFuture<Void> lateJoiningMembershipStartup = executorServiceRule.runAsync(() -> {
+      try {
+        start(lateJoiningMembership);
+      } catch (MemberStartupException e) {
+        e.printStackTrace();

Review comment:
       Updated in latest commits




----------------------------------------------------------------
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.

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



[GitHub] [geode] aaronlindsey commented on a change in pull request #5236: GEODE-8241: Locator observes locator-wait-time

Posted by GitBox <gi...@apache.org>.
aaronlindsey commented on a change in pull request #5236:
URL: https://github.com/apache/geode/pull/5236#discussion_r440451304



##########
File path: geode-membership/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/MembershipIntegrationTest.java
##########
@@ -172,6 +181,92 @@ public void secondMembershipCanJoinUsingTheSecondLocatorToStart()
     stop(locator2, locator1);
   }
 
+  @Test
+  public void secondMembershipPausesForLocatorWaitTime()
+      throws IOException, MemberStartupException {
+
+    /*
+     * Start a locator for the coordinator (membership) so we have a port for it.
+     *
+     * Its locator-wait-time is set to 0 so it eventually (soon after membership is started) forms a
+     * distributed system and becomes a coordinator.
+     */
+
+    final MembershipLocator<MemberIdentifier> coordinatorLocator = createLocator(0);
+    coordinatorLocator.start();
+    final int coordinatorLocatorPort = coordinatorLocator.getPort();
+
+    final Membership<MemberIdentifier> coordinatorMembership =
+        createMembership(coordinatorLocator, coordinatorLocatorPort);
+
+    /*
+     * We have not even started the membership yet — connection attempts will certainly fail until
+     * we do. This is a bit like the locator (host) not being present in DNS (yet).
+     */
+
+    /*
+     * Start a second locator and membership trying to join via the coordinator (membership) that
+     * hasn't yet started behind the port.
+     *
+     * Set its locator-wait-time so it'll not become a coordinator right away, allowing time for the
+     * other member to start and become a coordinator.
+     *
+     * Calculate the locator-wait-time to be greater than the minimum wait time for connecting to a
+     * locator.
+     */
+
+    final MembershipLocator<MemberIdentifier> lateJoiningLocator = createLocator(0);
+    lateJoiningLocator.start();
+    final int lateJoiningLocatorPort = lateJoiningLocator.getPort();
+
+    final int[] lateJoiningMembershipLocatorPorts =
+        new int[] {coordinatorLocatorPort, lateJoiningLocatorPort};
+
+    final Duration minimumJoinWaitTime = Duration
+        .ofMillis(JOIN_RETRY_SLEEP + FIND_LOCATOR_RETRY_SLEEP) // amount of sleep time per retry
+        .multipliedBy(lateJoiningMembershipLocatorPorts.length * 2); // expected number of retries
+    final int locatorWaitTime = (int) (3 * minimumJoinWaitTime.getSeconds());
+
+    final MembershipConfig lateJoiningMembershipConfig =
+        createMembershipConfig(true, locatorWaitTime, lateJoiningMembershipLocatorPorts);
+    final Membership<MemberIdentifier> lateJoiningMembership =
+        createMembership(lateJoiningMembershipConfig, lateJoiningLocator);
+
+    CompletableFuture<Void> lateJoiningMembershipStartup = executorServiceRule.runAsync(() -> {
+      try {
+        start(lateJoiningMembership);
+      } catch (MemberStartupException e) {
+        throw new RuntimeException(e);
+      }
+    });
+
+    /*
+     * Now start the coordinator (membership), after waiting longer than the minimum wait time for
+     * connecting to a locator but shorter than the locator-wait-time.
+     */
+
+    CompletableFuture<Void> coordinatorMembershipStartup = executorServiceRule.runAsync(() -> {
+      try {
+        Thread.sleep(2 * minimumJoinWaitTime.toMillis());

Review comment:
       Again, `minimumJoinWaitTime` is the minimum amount of time a locator will wait to become a coordinator. We are just trying to make sure we sleep for longer than the minimum but shorter than the `locator-wait-time` so we can detect whether the `lateJoiningMembership` is waiting for the full `locator-wait-time` and not just the minimum wait time.




----------------------------------------------------------------
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.

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



[GitHub] [geode] aaronlindsey commented on a change in pull request #5236: GEODE-8241: Locator observes locator-wait-time

Posted by GitBox <gi...@apache.org>.
aaronlindsey commented on a change in pull request #5236:
URL: https://github.com/apache/geode/pull/5236#discussion_r438938334



##########
File path: geode-membership/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/MembershipIntegrationTest.java
##########
@@ -172,6 +179,91 @@ public void secondMembershipCanJoinUsingTheSecondLocatorToStart()
     stop(locator2, locator1);
   }
 
+  @Test
+  public void secondMembershipPausesForLocatorWaitTime()
+      throws IOException, MemberStartupException {
+
+    /*
+     * Start a locator for the coordinator (membership) so we have a port for it.
+     *
+     * Its locator-wait-time is set to 0 so it eventually (soon after membership is started) forms a
+     * distributed system and becomes a coordinator.
+     */
+
+    final MembershipLocator<MemberIdentifier> coordinatorLocator = createLocator(0);
+    coordinatorLocator.start();
+    final int coordinatorLocatorPort = coordinatorLocator.getPort();
+
+    final Membership<MemberIdentifier> coordinatorMembership =
+        createMembership(coordinatorLocator, coordinatorLocatorPort);
+
+    /*
+     * We have not even started the membership yet — connection attempts will certainly fail until
+     * we do. This is a bit like the locator (host) not being present in DNS (yet).
+     */
+
+    /*
+     * Start a second locator and membership trying to join via the coordinator (membership) that
+     * hasn't yet started behind the port.
+     *
+     * Set its locator-wait-time so it'll not become a coordinator right away, allowing time for the
+     * other member to start and become a coordinator.
+     *
+     * Calculate the locator-wait-time to be greater than the minimum wait time for connecting to a
+     * locator.
+     */
+
+    final MembershipLocator<MemberIdentifier> lateJoiningLocator = createLocator(0);
+    lateJoiningLocator.start();
+    final int lateJoiningLocatorPort = lateJoiningLocator.getPort();
+
+    final int[] lateJoiningMembershipLocatorPorts =
+        new int[] {coordinatorLocatorPort, lateJoiningLocatorPort};
+
+    final Duration minimumJoinWaitTime = Duration
+        .ofMillis(2_000) // expected amount of sleep time per loop in GMSJoinLeave.join()

Review comment:
       Updated in latest commits




----------------------------------------------------------------
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.

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



[GitHub] [geode] aaronlindsey commented on a change in pull request #5236: GEODE-8241: Locator observes locator-wait-time

Posted by GitBox <gi...@apache.org>.
aaronlindsey commented on a change in pull request #5236:
URL: https://github.com/apache/geode/pull/5236#discussion_r438937359



##########
File path: geode-membership/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/MembershipIntegrationTest.java
##########
@@ -172,6 +179,91 @@ public void secondMembershipCanJoinUsingTheSecondLocatorToStart()
     stop(locator2, locator1);
   }
 
+  @Test
+  public void secondMembershipPausesForLocatorWaitTime()
+      throws IOException, MemberStartupException {
+
+    /*
+     * Start a locator for the coordinator (membership) so we have a port for it.
+     *
+     * Its locator-wait-time is set to 0 so it eventually (soon after membership is started) forms a
+     * distributed system and becomes a coordinator.
+     */
+
+    final MembershipLocator<MemberIdentifier> coordinatorLocator = createLocator(0);
+    coordinatorLocator.start();
+    final int coordinatorLocatorPort = coordinatorLocator.getPort();
+
+    final Membership<MemberIdentifier> coordinatorMembership =
+        createMembership(coordinatorLocator, coordinatorLocatorPort);
+
+    /*
+     * We have not even started the membership yet — connection attempts will certainly fail until
+     * we do. This is a bit like the locator (host) not being present in DNS (yet).
+     */
+
+    /*
+     * Start a second locator and membership trying to join via the coordinator (membership) that
+     * hasn't yet started behind the port.
+     *
+     * Set its locator-wait-time so it'll not become a coordinator right away, allowing time for the
+     * other member to start and become a coordinator.
+     *
+     * Calculate the locator-wait-time to be greater than the minimum wait time for connecting to a
+     * locator.
+     */
+
+    final MembershipLocator<MemberIdentifier> lateJoiningLocator = createLocator(0);
+    lateJoiningLocator.start();
+    final int lateJoiningLocatorPort = lateJoiningLocator.getPort();
+
+    final int[] lateJoiningMembershipLocatorPorts =
+        new int[] {coordinatorLocatorPort, lateJoiningLocatorPort};
+
+    final Duration minimumJoinWaitTime = Duration
+        .ofMillis(2_000) // expected amount of sleep time per loop in GMSJoinLeave.join()

Review comment:
       It seems like it would be better to reference the same constants that are used in `GMSJoinLeave` instead of hard-coding this value.




----------------------------------------------------------------
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.

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



[GitHub] [geode] aaronlindsey commented on a change in pull request #5236: GEODE-8241: Locator observes locator-wait-time

Posted by GitBox <gi...@apache.org>.
aaronlindsey commented on a change in pull request #5236:
URL: https://github.com/apache/geode/pull/5236#discussion_r440447385



##########
File path: geode-membership/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/MembershipIntegrationTest.java
##########
@@ -172,6 +181,92 @@ public void secondMembershipCanJoinUsingTheSecondLocatorToStart()
     stop(locator2, locator1);
   }
 
+  @Test
+  public void secondMembershipPausesForLocatorWaitTime()
+      throws IOException, MemberStartupException {
+
+    /*
+     * Start a locator for the coordinator (membership) so we have a port for it.
+     *
+     * Its locator-wait-time is set to 0 so it eventually (soon after membership is started) forms a
+     * distributed system and becomes a coordinator.
+     */
+
+    final MembershipLocator<MemberIdentifier> coordinatorLocator = createLocator(0);
+    coordinatorLocator.start();
+    final int coordinatorLocatorPort = coordinatorLocator.getPort();
+
+    final Membership<MemberIdentifier> coordinatorMembership =
+        createMembership(coordinatorLocator, coordinatorLocatorPort);
+
+    /*
+     * We have not even started the membership yet — connection attempts will certainly fail until
+     * we do. This is a bit like the locator (host) not being present in DNS (yet).
+     */
+
+    /*
+     * Start a second locator and membership trying to join via the coordinator (membership) that
+     * hasn't yet started behind the port.
+     *
+     * Set its locator-wait-time so it'll not become a coordinator right away, allowing time for the
+     * other member to start and become a coordinator.
+     *
+     * Calculate the locator-wait-time to be greater than the minimum wait time for connecting to a
+     * locator.
+     */
+
+    final MembershipLocator<MemberIdentifier> lateJoiningLocator = createLocator(0);
+    lateJoiningLocator.start();
+    final int lateJoiningLocatorPort = lateJoiningLocator.getPort();
+
+    final int[] lateJoiningMembershipLocatorPorts =
+        new int[] {coordinatorLocatorPort, lateJoiningLocatorPort};
+
+    final Duration minimumJoinWaitTime = Duration
+        .ofMillis(JOIN_RETRY_SLEEP + FIND_LOCATOR_RETRY_SLEEP) // amount of sleep time per retry
+        .multipliedBy(lateJoiningMembershipLocatorPorts.length * 2); // expected number of retries
+    final int locatorWaitTime = (int) (3 * minimumJoinWaitTime.getSeconds());

Review comment:
       There is a minimum amount of time a locator will wait to become the coordinator, regardless of the value for `locator-wait-time`. The variable, `minimumJoinWaitTime`, is meant to capture that value. By setting `locator-wait-time` to 3x the `minimumJoinWaitTime`, we are just trying to make sure the `locator-wait-time` is sufficiently larger than the minimum so we can reliably detect whether the `lateJoiningMembership` is waiting for the full `locator-wait-time` and not just the minimum wait time.




----------------------------------------------------------------
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.

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



[GitHub] [geode] vfordpivotal commented on pull request #5236: GEODE-8241: Locator observes locator-wait-time

Posted by GitBox <gi...@apache.org>.
vfordpivotal commented on pull request #5236:
URL: https://github.com/apache/geode/pull/5236#issuecomment-642423913


   I can further support this code change by having run this change multiple times in another test context and it addressed our start-up issues around start-up of locators when DNS resolution was slow. I feel very confident this code change addresses our issues, but would like additional feedback on the test code provided.


----------------------------------------------------------------
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.

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



[GitHub] [geode] aaronlindsey commented on a change in pull request #5236: GEODE-8241: Locator observes locator-wait-time

Posted by GitBox <gi...@apache.org>.
aaronlindsey commented on a change in pull request #5236:
URL: https://github.com/apache/geode/pull/5236#discussion_r441823755



##########
File path: geode-membership/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/MembershipIntegrationTest.java
##########
@@ -172,6 +181,92 @@ public void secondMembershipCanJoinUsingTheSecondLocatorToStart()
     stop(locator2, locator1);
   }
 
+  @Test
+  public void secondMembershipPausesForLocatorWaitTime()
+      throws IOException, MemberStartupException {
+
+    /*
+     * Start a locator for the coordinator (membership) so we have a port for it.
+     *
+     * Its locator-wait-time is set to 0 so it eventually (soon after membership is started) forms a
+     * distributed system and becomes a coordinator.
+     */
+
+    final MembershipLocator<MemberIdentifier> coordinatorLocator = createLocator(0);
+    coordinatorLocator.start();
+    final int coordinatorLocatorPort = coordinatorLocator.getPort();
+
+    final Membership<MemberIdentifier> coordinatorMembership =
+        createMembership(coordinatorLocator, coordinatorLocatorPort);
+
+    /*
+     * We have not even started the membership yet — connection attempts will certainly fail until
+     * we do. This is a bit like the locator (host) not being present in DNS (yet).
+     */
+
+    /*
+     * Start a second locator and membership trying to join via the coordinator (membership) that
+     * hasn't yet started behind the port.
+     *
+     * Set its locator-wait-time so it'll not become a coordinator right away, allowing time for the
+     * other member to start and become a coordinator.
+     *
+     * Calculate the locator-wait-time to be greater than the minimum wait time for connecting to a
+     * locator.
+     */
+
+    final MembershipLocator<MemberIdentifier> lateJoiningLocator = createLocator(0);
+    lateJoiningLocator.start();
+    final int lateJoiningLocatorPort = lateJoiningLocator.getPort();
+
+    final int[] lateJoiningMembershipLocatorPorts =
+        new int[] {coordinatorLocatorPort, lateJoiningLocatorPort};
+
+    final Duration minimumJoinWaitTime = Duration
+        .ofMillis(JOIN_RETRY_SLEEP + FIND_LOCATOR_RETRY_SLEEP) // amount of sleep time per retry
+        .multipliedBy(lateJoiningMembershipLocatorPorts.length * 2); // expected number of retries

Review comment:
       I extracted a method for calculating the minimum number of retries before becoming coordinator in the latest commit.




----------------------------------------------------------------
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.

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



[GitHub] [geode] aaronlindsey commented on pull request #5236: GEODE-8241: Locator observes locator-wait-time

Posted by GitBox <gi...@apache.org>.
aaronlindsey commented on pull request #5236:
URL: https://github.com/apache/geode/pull/5236#issuecomment-645549135


   @Bill what do you think of mocking out calls to `System.currentTimeMillis()` in `GMSJoinLeave`? Allowing the test to control the clock might allow a more deterministic test. I'm not sure if it would work, though, because there are a lot of usages of `System.currentTimeMillis()` in `GMSJoinLeave` that our test doesn't care about which would also be affected.


----------------------------------------------------------------
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.

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