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 2021/02/24 20:23:44 UTC

[GitHub] [geode] albertogpz opened a new pull request #6052: GEODE-8971: Add grace period when stopping gateway sender with group-…

albertogpz opened a new pull request #6052:
URL: https://github.com/apache/geode/pull/6052


   …transaction-events enabled
   
   When the gateway sender is stopped there is a high probability that batches with
   incomplete transactions are sent even if group-transaction-events is enabled.
   
   The reason is that once the stop command reaches the gateway sender, it immediately
   stops queueing events, and this could happen in the middle of receiving events for the
   same transaction. If this is the case, some events for the transaction may have
   reached the queue right before the stop command was received and the rest of events
   for that transaction would not make it to the queue (they would be dropped) because
   they arrived right after the stop command was received at the gateway sender.
   
   One way to solve this issue to allow for a grace period when the gateway sender
   is stopped during which only events that belong to transactions already
   in the queue are queued. Any other event received by the gateway sender
   during this grace period would be dropped. After this grace period
   the gateway sender will be effectively stopped.
   
   Thank you for submitting a contribution to Apache Geode.
   
   In order to streamline the review of the contribution we ask you
   to ensure the following steps have been taken:
   
   ### For all changes:
   - [ ] Is there a JIRA ticket associated with this PR? Is it referenced in the commit message?
   
   - [ ] Has your PR been rebased against the latest commit within the target branch (typically `develop`)?
   
   - [ ] Is your initial contribution a single, squashed commit?
   
   - [ ] Does `gradlew build` run cleanly?
   
   - [ ] Have you written or updated unit tests to verify your changes?
   
   - [ ] If adding new dependencies to the code, are these dependencies licensed in a way that is compatible for inclusion under [ASF 2.0](http://www.apache.org/legal/resolved.html#category-a)?
   
   ### Note:
   Please ensure that once the PR is submitted, check Concourse for build issues and
   submit an update to your PR as soon as possible. If you need help, please send an
   email to dev@geode.apache.org.
   


----------------------------------------------------------------
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] albertogpz commented on a change in pull request #6052: GEODE-8971: Add grace period when stopping gateway sender with group-…

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



##########
File path: geode-core/src/main/java/org/apache/geode/internal/cache/wan/serial/SerialGatewaySenderQueue.java
##########
@@ -862,19 +877,35 @@ EventsAndLastKey getElementsMatching(Predicate condition, Predicate stopConditio
         continue;
       }
 
-      if (condition.test(object)) {
-        elementsMatching.add(object);
-        peekedIds.add(currentKey);
-        extraPeekedIds.add(currentKey);
-        lastKey = currentKey;
+      if (condition.test((InternalGatewayQueueEvent) object)) {

Review comment:
       ok




----------------------------------------------------------------
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] albertogpz commented on a change in pull request #6052: GEODE-8971: Add grace period when stopping gateway sender with group-…

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



##########
File path: geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/serial/SerialWANStatsDUnitTest.java
##########
@@ -348,7 +349,276 @@ public void testReplicatedSerialPropagationWithBatchRedistWithGroupTransactionEv
   }
 
   @Test
-  public void testReplicatedSerialPropagationWithMultipleDispatchers() throws Exception {
+  public void testReplicatedSerialPropagationWithGroupTransactionEventsDoesNotSendBatchesWithIncompleteTransactionsIfGatewaySenderIsStoppedWhileReceivingTrafficAndLaterStarted()
+      throws InterruptedException {
+    Integer lnPort = vm0.invoke(() -> WANTestBase.createFirstLocatorWithDSId(1));
+    Integer nyPort = vm1.invoke(() -> WANTestBase.createFirstRemoteLocator(2, lnPort));
+
+    String regionName = testName + "_RR";
+
+    createCacheInVMs(nyPort, vm2);
+    createReceiverInVMs(vm2);
+
+    createCacheInVMs(lnPort, vm4, vm5);
+    boolean groupTransactionEvents = true;
+    int batchSize = 10;
+    vm4.invoke(
+        () -> WANTestBase.createSender("ln", 2, false, 100, batchSize, false, true, null, true,
+            groupTransactionEvents));
+    vm5.invoke(
+        () -> WANTestBase.createSender("ln", 2, false, 100, batchSize, false, true, null, true,
+            groupTransactionEvents));
+
+    vm2.invoke(() -> WANTestBase.createReplicatedRegion(regionName, null, isOffHeap()));
+
+    vm4.invoke(() -> WANTestBase.createReplicatedRegion(regionName, "ln", isOffHeap()));
+    vm5.invoke(() -> WANTestBase.createReplicatedRegion(regionName, "ln", isOffHeap()));
+
+    startSenderInVMs("ln", vm4, vm5);
+
+    final Map<Object, Object> keyValues = new LinkedHashMap<>();
+    int entries = 2200;
+    for (int i = 0; i < entries; i++) {
+      keyValues.put(i, i + "_Value");
+    }
+
+    int eventsPerTransaction = 11;
+    System.out.println("Starting puts");
+    AsyncInvocation<Void> inv1 =
+        vm4.invokeAsync(
+            () -> WANTestBase.doPutsInsideTransactions(regionName, keyValues,
+                eventsPerTransaction));
+
+    // wait for batches to be distributed and then stop the sender
+    System.out.println("Waiting for some batches to be distributed");
+    vm4.invoke(() -> await()
+        .until(() -> WANTestBase.getSenderStats("ln", -1).get(4) > 0));
+    System.out
+        .println("Some batches distributed: " + vm4.invoke(() -> getSenderStats("ln", -1).get(4)));
+
+    addIgnoredException("Exception occurred in CacheListener");

Review comment:
       ok




----------------------------------------------------------------
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] albertogpz commented on a change in pull request #6052: GEODE-8971: Add grace period when stopping gateway sender with group-…

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



##########
File path: geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/WANTestBase.java
##########
@@ -1320,6 +1320,23 @@ public static void checkGatewayReceiverStats(int processBatches, int eventsRecei
     assertEquals(creates, gatewayReceiverStats.getCreateRequest());
   }
 
+  public static List<Integer> getReceiverStats() {

Review comment:
       I agree but the problem is that `GatewayReceiverStats` must be Serializable and it is not. That's why I implemented something similar to `gatewaySenderStats()`.




----------------------------------------------------------------
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] albertogpz commented on a change in pull request #6052: GEODE-8971: Add grace period when stopping gateway sender with group-…

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



##########
File path: geode-core/src/main/java/org/apache/geode/cache/wan/GatewaySender.java
##########
@@ -174,7 +174,32 @@
    */
   int GET_TRANSACTION_EVENTS_FROM_QUEUE_RETRIES =
       Integer.getInteger(GeodeGlossary.GEMFIRE_PREFIX + "get-transaction-events-from-queue-retries",
-          10);
+          2);
+  /**
+   * Milliseconds to wait before retrying to get events for a transaction from the
+   * gateway sender queue when group-transaction-events is true.
+   */
+  int GET_TRANSACTION_EVENTS_FROM_QUEUE_WAIT_TIME_MS =
+      Integer.getInteger(
+          GeodeGlossary.GEMFIRE_PREFIX + "get-transaction-events-from-queue-wait-time-ms",
+          1);
+
+  /**
+   * When group-transaction-events is set to true and the gateway sender is stopped,
+   * there is a possibility that the stopping occurs such that for a transaction,
+   * not all events belonging to it reach the queue. The reason would be that
+   * some reach the queue right before the sender is stopped and the rest do not make
+   * it to the queue because the sender is just stopped.
+   * In order to prevent that the queue contains incomplete transactions
+   * due to the above circumstance, this parameter allows for a grace period
+   * of the number of milliseconds set in it before the gateway sender is
+   * actually stopped, in which only events to complete transactions are put in the queue.
+   * Other events received in this period would be dropped.

Review comment:
       Thanks Dave. Very good suggestion. I made a slight change in the last sentence of your proposal but other than that, I took it as is.




----------------------------------------------------------------
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] albertogpz commented on a change in pull request #6052: GEODE-8971: Add grace period when stopping gateway sender with group-…

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



##########
File path: geode-core/src/main/java/org/apache/geode/internal/cache/wan/parallel/ParallelGatewaySenderQueue.java
##########
@@ -1797,6 +1817,22 @@ public static String getSenderId(String regionName) {
     return regionName.substring(1, queueStringStart);
   }
 
+  public boolean isThereEventsMatching(GatewaySenderEventImpl event,
+      Predicate<InternalGatewayQueueEvent> condition) {
+    boolean isDREvent = isDREvent(sender.getCache(), event);
+
+    String regionPath = event.getRegionPath();

Review comment:
       ok




----------------------------------------------------------------
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] albertogpz merged pull request #6052: GEODE-8971: Add grace period when stopping gateway sender with group-…

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


   


-- 
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] DonalEvans commented on a change in pull request #6052: GEODE-8971: Add grace period when stopping gateway sender with group-…

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



##########
File path: geode-core/src/main/java/org/apache/geode/internal/cache/wan/parallel/ConcurrentParallelGatewaySenderEventProcessor.java
##########
@@ -132,22 +134,24 @@ public int eventQueueSize() {
   }
 
   @Override
-  public void enqueueEvent(EnumListenerEvent operation, EntryEvent event, Object substituteValue)
+  public boolean enqueueEvent(EnumListenerEvent operation, EntryEvent event, Object substituteValue)
       throws IOException, CacheException {
-    enqueueEvent(operation, event, substituteValue, false);
+    return enqueueEvent(operation, event, substituteValue, false, null);
   }
 
   @Override
-  public void enqueueEvent(EnumListenerEvent operation, EntryEvent event, Object substituteValue,
-      boolean isLastEventInTransaction) throws IOException, CacheException {
+  public boolean enqueueEvent(EnumListenerEvent operation, EntryEvent event, Object substituteValue,
+      boolean isLastEventInTransaction, Predicate<InternalGatewayQueueEvent> condition)
+      throws IOException, CacheException {
     Region region = event.getRegion();
     // int bucketId = PartitionedRegionHelper.getHashKey((EntryOperation)event);

Review comment:
       The variable `region` is never used, and this commented-out line should be removed.

##########
File path: geode-core/src/main/java/org/apache/geode/internal/cache/wan/parallel/ParallelGatewaySenderQueue.java
##########
@@ -702,44 +704,15 @@ public boolean put(Object object) throws InterruptedException, CacheException {
     boolean isDREvent = isDREvent(sender.getCache(), value);
 
     String regionPath = value.getRegionPath();

Review comment:
       This line should be moved into the `getRegionPathForEventAndType()` method and the `regionPath` argument should be removed from the method signature.

##########
File path: geode-core/src/main/java/org/apache/geode/internal/cache/wan/serial/SerialGatewaySenderQueue.java
##########
@@ -847,7 +855,14 @@ private EventsAndLastKey peekEventsWithTransactionId(TransactionId transactionId
     }
   }
 
-  EventsAndLastKey getElementsMatching(Predicate condition, Predicate stopCondition, long lastKey) {
+  /**
+   * This method returns a list of objects that fulfill the matchingPredicate
+   * If a matching object also fulfills the endPredicate then the method
+   * stops looking for more matching objects.
+   */
+  List<KeyAndEventPair> getElementsMatching(Predicate<InternalGatewayQueueEvent> condition,

Review comment:
       With the change to this method's return type, `EventsAndLastKey` is no longer used, so that class should be removed.

##########
File path: geode-core/src/main/java/org/apache/geode/internal/cache/wan/serial/SerialGatewaySenderQueue.java
##########
@@ -862,19 +877,35 @@ EventsAndLastKey getElementsMatching(Predicate condition, Predicate stopConditio
         continue;
       }
 
-      if (condition.test(object)) {
-        elementsMatching.add(object);
-        peekedIds.add(currentKey);
-        extraPeekedIds.add(currentKey);
-        lastKey = currentKey;
+      if (condition.test((InternalGatewayQueueEvent) object)) {

Review comment:
       The casts here can be cleaned up a bit by declaring `object` as a `InternalGatewayQueueEvent` on line 866 and then casting to `InternalGatewayQueueEvent` when assigned from `optimalGet()`. It would also be good to rename `object` to `event`.

##########
File path: geode-core/src/main/java/org/apache/geode/internal/cache/BucketRegionQueue.java
##########
@@ -453,21 +454,22 @@ public Object peek() {
    * If a matching object also fulfills the endPredicate then the method
    * stops looking for more matching objects.
    */
-  public List<Object> getElementsMatching(Predicate matchingPredicate, Predicate endPredicate) {
+  public List<Object> getElementsMatching(Predicate<InternalGatewayQueueEvent> matchingPredicate,
+      Predicate<InternalGatewayQueueEvent> endPredicate) {
     getInitializationLock().readLock().lock();
     try {
       if (this.getPartitionedRegion().isDestroyed()) {
         throw new BucketRegionQueueUnavailableException();
       }
-      List<Object> elementsMatching = new ArrayList<Object>();
+      List<Object> elementsMatching = new ArrayList();

Review comment:
       The compiler warning here can be fixed by using `new ArrayList<>();`

##########
File path: geode-core/src/main/java/org/apache/geode/internal/cache/BucketRegionQueue.java
##########
@@ -453,21 +454,22 @@ public Object peek() {
    * If a matching object also fulfills the endPredicate then the method
    * stops looking for more matching objects.
    */
-  public List<Object> getElementsMatching(Predicate matchingPredicate, Predicate endPredicate) {
+  public List<Object> getElementsMatching(Predicate<InternalGatewayQueueEvent> matchingPredicate,
+      Predicate<InternalGatewayQueueEvent> endPredicate) {
     getInitializationLock().readLock().lock();
     try {
       if (this.getPartitionedRegion().isDestroyed()) {
         throw new BucketRegionQueueUnavailableException();
       }
-      List<Object> elementsMatching = new ArrayList<Object>();
+      List<Object> elementsMatching = new ArrayList();
       Iterator<Object> it = this.eventSeqNumDeque.iterator();
       while (it.hasNext()) {
         Object key = it.next();
         Object object = optimalGet(key);
-        if (matchingPredicate.test(object)) {
+        if (matchingPredicate.test((InternalGatewayQueueEvent) object)) {

Review comment:
       I believe there's a possibility of a ClassCastException here, since there's no guarantee that the Objects returned from `optimalGet()` in this class can be cast to `InternalGatewayQueueEvent`. It would be best to check that `object` is an instance of InternalGatewayQueueEvent prior to casting it here.

##########
File path: geode-core/src/test/java/org/apache/geode/internal/cache/wan/serial/SerialGatewaySenderQueueJUnitTest.java
##########
@@ -137,8 +137,8 @@ public void peekGetsExtraEventsWhenMustGroupTransactionEventsAndNotAllEventsForT
         .doAnswer(invocation -> null)
         .when(queue).peekAhead();
 
-    doAnswer(invocation -> new SerialGatewaySenderQueue.EventsAndLastKey(
-        Arrays.asList(new Object[] {event4}), 2L))
+    doAnswer(invocation -> Arrays

Review comment:
       Compiler warnings here can be fixed by changing this to `invocation -> Collections.singletonList(new SerialGatewaySenderQueue.KeyAndEventPair(1L, event4)))`

##########
File path: geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/parallel/ParallelWANStatsDUnitTest.java
##########
@@ -472,6 +473,322 @@ public void testPRParallelPropagationWithGroupTransactionEventsSendsBatchesWithC
 
   }
 
+  @Test
+  public void testPRParallelPropagationWithGroupTransactionEventsDoesNotSendBatchesWithIncompleteTransactionsIfGatewaySenderIsStoppedWhileReceivingTrafficAndLaterStarted()
+      throws InterruptedException {
+    Integer lnPort = vm0.invoke(() -> WANTestBase.createFirstLocatorWithDSId(1));
+    Integer nyPort = vm1.invoke(() -> WANTestBase.createFirstRemoteLocator(2, lnPort));
+
+    createCacheInVMs(nyPort, vm2);
+    createReceiverInVMs(vm2);
+
+    createCacheInVMs(lnPort, vm4, vm5);
+    vm4.invoke(() -> WANTestBase.createSender("ln", 2, true, 100, 10, false, true, null, true,
+        true));
+    vm5.invoke(() -> WANTestBase.createSender("ln", 2, true, 100, 10, false, true, null, true,
+        true));
+
+    createReceiverCustomerOrderShipmentPR(vm2);
+
+    createSenderCustomerOrderShipmentPRs(vm4);
+    createSenderCustomerOrderShipmentPRs(vm5);
+
+    startSenderInVMs("ln", vm4, vm5);
+
+    int customers = 4;
+    int transactionsPerCustomer = 100;
+    int shipmentsPerTransaction = 10;
+    final LinkedHashMap<Object, Object> keyValuesInTransactions = new LinkedHashMap<>();
+    for (int custId = 0; custId < customers; custId++) {
+      for (int i = 0; i < transactionsPerCustomer; i++) {
+        CustId custIdObject = new CustId(custId);
+        OrderId orderId = new OrderId(i, custIdObject);
+        keyValuesInTransactions.put(orderId, new Order());
+        for (int j = 0; j < shipmentsPerTransaction; j++) {
+          ShipmentId shipmentId = new ShipmentId(i + j, orderId);
+          keyValuesInTransactions.put(shipmentId, new Shipment());
+        }
+      }
+    }
+    int eventsPerTransaction = 1 + shipmentsPerTransaction;
+    AsyncInvocation<Void> inv1 =
+        vm4.invokeAsync(
+            () -> WANTestBase.doOrderAndShipmentPutsInsideTransactions(keyValuesInTransactions,
+                eventsPerTransaction));
+
+    // wait for some batches to be distributed and then stop the sender
+    vm4.invoke(() -> await()
+        .until(() -> WANTestBase.getSenderStats("ln", -1).get(4) > 0));
+
+    System.out.println("Stopping sender");

Review comment:
       Print statements like this shouldn't be used in tests. If logging is needed for assisting with debugging, either a logger should be added, or a descriptive message added to the `invoke()` method call that is being traced (in this case, in  `WANTestBase.stopSenderInVMsAsync()`).

##########
File path: geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/parallel/ParallelWANStatsDUnitTest.java
##########
@@ -472,6 +473,322 @@ public void testPRParallelPropagationWithGroupTransactionEventsSendsBatchesWithC
 
   }
 
+  @Test
+  public void testPRParallelPropagationWithGroupTransactionEventsDoesNotSendBatchesWithIncompleteTransactionsIfGatewaySenderIsStoppedWhileReceivingTrafficAndLaterStarted()
+      throws InterruptedException {
+    Integer lnPort = vm0.invoke(() -> WANTestBase.createFirstLocatorWithDSId(1));
+    Integer nyPort = vm1.invoke(() -> WANTestBase.createFirstRemoteLocator(2, lnPort));
+
+    createCacheInVMs(nyPort, vm2);
+    createReceiverInVMs(vm2);
+
+    createCacheInVMs(lnPort, vm4, vm5);
+    vm4.invoke(() -> WANTestBase.createSender("ln", 2, true, 100, 10, false, true, null, true,
+        true));
+    vm5.invoke(() -> WANTestBase.createSender("ln", 2, true, 100, 10, false, true, null, true,
+        true));
+
+    createReceiverCustomerOrderShipmentPR(vm2);
+
+    createSenderCustomerOrderShipmentPRs(vm4);
+    createSenderCustomerOrderShipmentPRs(vm5);
+
+    startSenderInVMs("ln", vm4, vm5);
+
+    int customers = 4;
+    int transactionsPerCustomer = 100;
+    int shipmentsPerTransaction = 10;
+    final LinkedHashMap<Object, Object> keyValuesInTransactions = new LinkedHashMap<>();
+    for (int custId = 0; custId < customers; custId++) {
+      for (int i = 0; i < transactionsPerCustomer; i++) {
+        CustId custIdObject = new CustId(custId);
+        OrderId orderId = new OrderId(i, custIdObject);
+        keyValuesInTransactions.put(orderId, new Order());
+        for (int j = 0; j < shipmentsPerTransaction; j++) {
+          ShipmentId shipmentId = new ShipmentId(i + j, orderId);
+          keyValuesInTransactions.put(shipmentId, new Shipment());
+        }
+      }
+    }
+    int eventsPerTransaction = 1 + shipmentsPerTransaction;
+    AsyncInvocation<Void> inv1 =
+        vm4.invokeAsync(
+            () -> WANTestBase.doOrderAndShipmentPutsInsideTransactions(keyValuesInTransactions,
+                eventsPerTransaction));
+
+    // wait for some batches to be distributed and then stop the sender
+    vm4.invoke(() -> await()
+        .until(() -> WANTestBase.getSenderStats("ln", -1).get(4) > 0));
+
+    System.out.println("Stopping sender");
+    stopSenderInVMsAsync("ln", vm4, vm5);
+    System.out.println("Stopped sender");
+
+    inv1.await();
+    int entries =
+        transactionsPerCustomer * customers;
+
+    vm4.invoke(() -> WANTestBase.validateRegionSize(orderRegionName, entries));
+    vm5.invoke(() -> WANTestBase.validateRegionSize(orderRegionName, entries));
+
+    // Wait for events to replicate: when batches received does not change
+    // we can assume that replication has finished.
+    int batchesReceived = (vm2.invoke(() -> getReceiverStats())).get(0);
+    while (true) {
+      int oldBatchesReceived = batchesReceived;
+      Thread.sleep(1000);
+      batchesReceived = (vm2.invoke(() -> getReceiverStats())).get(0);
+      if (batchesReceived == oldBatchesReceived) {
+        break;
+      }
+    }
+
+    System.out.println("batchesReceived after stop: " + batchesReceived);

Review comment:
       See earlier comment about not using `System.out` print statements in tests.

##########
File path: geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/parallel/ParallelWANStatsDUnitTest.java
##########
@@ -472,6 +473,322 @@ public void testPRParallelPropagationWithGroupTransactionEventsSendsBatchesWithC
 
   }
 
+  @Test
+  public void testPRParallelPropagationWithGroupTransactionEventsDoesNotSendBatchesWithIncompleteTransactionsIfGatewaySenderIsStoppedWhileReceivingTrafficAndLaterStarted()
+      throws InterruptedException {
+    Integer lnPort = vm0.invoke(() -> WANTestBase.createFirstLocatorWithDSId(1));
+    Integer nyPort = vm1.invoke(() -> WANTestBase.createFirstRemoteLocator(2, lnPort));
+
+    createCacheInVMs(nyPort, vm2);
+    createReceiverInVMs(vm2);
+
+    createCacheInVMs(lnPort, vm4, vm5);
+    vm4.invoke(() -> WANTestBase.createSender("ln", 2, true, 100, 10, false, true, null, true,
+        true));
+    vm5.invoke(() -> WANTestBase.createSender("ln", 2, true, 100, 10, false, true, null, true,
+        true));
+
+    createReceiverCustomerOrderShipmentPR(vm2);
+
+    createSenderCustomerOrderShipmentPRs(vm4);
+    createSenderCustomerOrderShipmentPRs(vm5);
+
+    startSenderInVMs("ln", vm4, vm5);
+
+    int customers = 4;
+    int transactionsPerCustomer = 100;
+    int shipmentsPerTransaction = 10;
+    final LinkedHashMap<Object, Object> keyValuesInTransactions = new LinkedHashMap<>();
+    for (int custId = 0; custId < customers; custId++) {
+      for (int i = 0; i < transactionsPerCustomer; i++) {
+        CustId custIdObject = new CustId(custId);
+        OrderId orderId = new OrderId(i, custIdObject);
+        keyValuesInTransactions.put(orderId, new Order());
+        for (int j = 0; j < shipmentsPerTransaction; j++) {
+          ShipmentId shipmentId = new ShipmentId(i + j, orderId);
+          keyValuesInTransactions.put(shipmentId, new Shipment());
+        }
+      }
+    }
+    int eventsPerTransaction = 1 + shipmentsPerTransaction;
+    AsyncInvocation<Void> inv1 =
+        vm4.invokeAsync(
+            () -> WANTestBase.doOrderAndShipmentPutsInsideTransactions(keyValuesInTransactions,
+                eventsPerTransaction));
+
+    // wait for some batches to be distributed and then stop the sender
+    vm4.invoke(() -> await()
+        .until(() -> WANTestBase.getSenderStats("ln", -1).get(4) > 0));
+
+    System.out.println("Stopping sender");
+    stopSenderInVMsAsync("ln", vm4, vm5);
+    System.out.println("Stopped sender");
+
+    inv1.await();
+    int entries =
+        transactionsPerCustomer * customers;
+
+    vm4.invoke(() -> WANTestBase.validateRegionSize(orderRegionName, entries));
+    vm5.invoke(() -> WANTestBase.validateRegionSize(orderRegionName, entries));
+
+    // Wait for events to replicate: when batches received does not change
+    // we can assume that replication has finished.
+    int batchesReceived = (vm2.invoke(() -> getReceiverStats())).get(0);
+    while (true) {
+      int oldBatchesReceived = batchesReceived;
+      Thread.sleep(1000);
+      batchesReceived = (vm2.invoke(() -> getReceiverStats())).get(0);
+      if (batchesReceived == oldBatchesReceived) {
+        break;
+      }
+    }
+
+    System.out.println("batchesReceived after stop: " + batchesReceived);
+
+    ArrayList<Integer> v4List =
+        (ArrayList<Integer>) vm4.invoke(() -> WANTestBase.getSenderStats("ln", -1));
+    ArrayList<Integer> v5List =
+        (ArrayList<Integer>) vm5.invoke(() -> WANTestBase.getSenderStats("ln", -1));
+
+    // Wait for events to replicate: when region size does not change
+    // we can assume that replication has finished.
+    int orderRegionSize = vm2.invoke(() -> getRegionSize(orderRegionName));
+    while (true) {
+      Thread.sleep(1000);
+      int newRegionSize = vm2.invoke(() -> getRegionSize(orderRegionName));
+      if (orderRegionSize == newRegionSize) {
+        break;
+      }
+    }
+
+    // Wait for events to replicate: when region size does not change
+    // we can assume that replication has finished.
+    int shipmentRegionSize = vm2.invoke(() -> getRegionSize(shipmentRegionName));
+    while (true) {
+      Thread.sleep(1000);
+      int newRegionSize = vm2.invoke(() -> getRegionSize(shipmentRegionName));
+      if (shipmentRegionSize == newRegionSize) {
+        break;
+      }
+    }
+
+    System.out.println("orderRegionSize: " + shipmentRegionSize);

Review comment:
       More print statements that should either be removed entirely or replaced with logger output.

##########
File path: geode-core/src/main/java/org/apache/geode/cache/wan/GatewaySender.java
##########
@@ -174,7 +174,32 @@
    */
   int GET_TRANSACTION_EVENTS_FROM_QUEUE_RETRIES =
       Integer.getInteger(GeodeGlossary.GEMFIRE_PREFIX + "get-transaction-events-from-queue-retries",
-          10);
+          2);

Review comment:
       What is the reason for this being changed from 10 to 2?

##########
File path: geode-core/src/main/java/org/apache/geode/internal/cache/wan/parallel/ParallelGatewaySenderQueue.java
##########
@@ -1797,6 +1817,22 @@ public static String getSenderId(String regionName) {
     return regionName.substring(1, queueStringStart);
   }
 
+  public boolean isThereEventsMatching(GatewaySenderEventImpl event,
+      Predicate<InternalGatewayQueueEvent> condition) {
+    boolean isDREvent = isDREvent(sender.getCache(), event);
+
+    String regionPath = event.getRegionPath();

Review comment:
       See earlier comment about moving this line into the `getRegionPathForEventAndType()` method.

##########
File path: geode-core/src/main/java/org/apache/geode/internal/cache/wan/parallel/ParallelGatewaySenderQueue.java
##########
@@ -1797,6 +1817,22 @@ public static String getSenderId(String regionName) {
     return regionName.substring(1, queueStringStart);
   }
 
+  public boolean isThereEventsMatching(GatewaySenderEventImpl event,

Review comment:
       This method (and other methods with the same name) might be better named "hasEventsMatching".

##########
File path: geode-core/src/main/java/org/apache/geode/internal/cache/wan/parallel/ParallelGatewaySenderQueue.java
##########
@@ -702,44 +704,15 @@ public boolean put(Object object) throws InterruptedException, CacheException {
     boolean isDREvent = isDREvent(sender.getCache(), value);
 
     String regionPath = value.getRegionPath();
-    if (!isDREvent) {
-      Region region = sender.getCache().getRegion(regionPath, true);
-      regionPath = ColocationHelper.getLeaderRegion((PartitionedRegion) region).getFullPath();
-    }
-    if (isDebugEnabled) {
-      logger.debug("Put is for the region {}", regionPath);
-    }
-    if (!this.userRegionNameToShadowPRMap.containsKey(regionPath)) {
-      if (isDebugEnabled) {
-        logger.debug("The userRegionNameToshadowPRMap is {}", userRegionNameToShadowPRMap);
-      }
-      logger.warn(
-          "GatewaySender: Not queuing the event {}, as the region for which this event originated is not yet configured in the GatewaySender",
-          value);
-      // does not put into queue
+    regionPath = getRegionPathForEventAndType(value, isDREvent, regionPath);
+    if (regionPath == null)

Review comment:
       Per the Geode style guide (https://cwiki.apache.org/confluence/display/GEODE/Code+Style+Guide), curly braces should be used for single-line if statements.

##########
File path: geode-core/src/main/java/org/apache/geode/internal/cache/wan/parallel/ParallelGatewaySenderQueue.java
##########
@@ -702,44 +704,15 @@ public boolean put(Object object) throws InterruptedException, CacheException {
     boolean isDREvent = isDREvent(sender.getCache(), value);
 
     String regionPath = value.getRegionPath();
-    if (!isDREvent) {
-      Region region = sender.getCache().getRegion(regionPath, true);
-      regionPath = ColocationHelper.getLeaderRegion((PartitionedRegion) region).getFullPath();
-    }
-    if (isDebugEnabled) {
-      logger.debug("Put is for the region {}", regionPath);
-    }
-    if (!this.userRegionNameToShadowPRMap.containsKey(regionPath)) {
-      if (isDebugEnabled) {
-        logger.debug("The userRegionNameToshadowPRMap is {}", userRegionNameToShadowPRMap);
-      }
-      logger.warn(
-          "GatewaySender: Not queuing the event {}, as the region for which this event originated is not yet configured in the GatewaySender",
-          value);
-      // does not put into queue
+    regionPath = getRegionPathForEventAndType(value, isDREvent, regionPath);
+    if (regionPath == null)
       return false;
-    }
 
     PartitionedRegion prQ = this.userRegionNameToShadowPRMap.get(regionPath);
     int bucketId = value.getBucketId();
-    Object key = null;
-    if (!isDREvent) {
-      key = value.getShadowKey();
-
-      if ((Long) key == -1) {
-        // In case of parallel we don't expect
-        // the key to be not set. If it is the case then the event must be coming
-        // through listener, so return.
-        if (isDebugEnabled) {
-          logger.debug("ParallelGatewaySenderOrderedQueue not putting key {} : Value : {}", key,
-              value);
-        }
-        // does not put into queue
-        return false;
-      }
-    } else {
-      key = value.getEventId();
-    }
+    Object key = getKeyForEventAndType(value, isDREvent);
+    if (key == null)

Review comment:
       Another single-line if statement that should have curly braces.

##########
File path: geode-core/src/main/java/org/apache/geode/internal/cache/wan/serial/SerialGatewaySenderQueue.java
##########
@@ -862,19 +877,35 @@ EventsAndLastKey getElementsMatching(Predicate condition, Predicate stopConditio
         continue;
       }
 
-      if (condition.test(object)) {
-        elementsMatching.add(object);
-        peekedIds.add(currentKey);
-        extraPeekedIds.add(currentKey);
-        lastKey = currentKey;
+      if (condition.test((InternalGatewayQueueEvent) object)) {
+        elementsMatching.add(new KeyAndEventPair(currentKey, (GatewaySenderEventImpl) object));
 
-        if (stopCondition.test(object)) {
+        if (stopCondition.test((InternalGatewayQueueEvent) object)) {
           break;
         }
       }
     }
 
-    return new EventsAndLastKey(elementsMatching, lastKey);
+    return elementsMatching;
+  }
+
+  public boolean isThereEventsMatching(Predicate<InternalGatewayQueueEvent> condition) {
+    InternalGatewayQueueEvent object;

Review comment:
       `object` would be better named `event`.

##########
File path: geode-core/src/main/java/org/apache/geode/internal/cache/wan/serial/ConcurrentSerialGatewaySenderEventProcessor.java
##########
@@ -418,10 +424,13 @@ protected void registerEventDroppedInPrimaryQueue(EntryEventImpl droppedEvent) {
   }
 
   @Override
-  protected void enqueueEvent(GatewayQueueEvent event) {
+  protected boolean enqueueEvent(GatewayQueueEvent event,
+      Predicate<InternalGatewayQueueEvent> condition) {
     for (SerialGatewaySenderEventProcessor serialProcessor : this.processors) {
-      serialProcessor.enqueueEvent(event);
+      // TODO revisit handling of "condition" when the following enqueueEvent() method is supported:

Review comment:
       Rather than adding a new TODO here, it might be okay to just rely on the exiting TODO in `SerialGatewaySenderEventProcessor.enqueueEvent()`

##########
File path: geode-core/src/main/java/org/apache/geode/internal/cache/wan/parallel/ParallelGatewaySenderQueue.java
##########
@@ -870,6 +843,49 @@ public boolean put(Object object) throws InterruptedException, CacheException {
     return putDone;
   }
 
+  private String getRegionPathForEventAndType(GatewaySenderEventImpl event, boolean isDREvent,
+      String regionPath) {
+    boolean isDebugEnabled = logger.isDebugEnabled();
+    if (!isDREvent) {
+      Region region = sender.getCache().getRegion(regionPath, true);

Review comment:
       As the variable `region` is only used once, and always as a `PartitionedRegion`, it can be declared as and cast to a PartitionedRegion on this line, which prevents the "Raw Types" compiler warning here.

##########
File path: geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/parallel/ParallelWANStatsDUnitTest.java
##########
@@ -472,6 +473,322 @@ public void testPRParallelPropagationWithGroupTransactionEventsSendsBatchesWithC
 
   }
 
+  @Test
+  public void testPRParallelPropagationWithGroupTransactionEventsDoesNotSendBatchesWithIncompleteTransactionsIfGatewaySenderIsStoppedWhileReceivingTrafficAndLaterStarted()
+      throws InterruptedException {
+    Integer lnPort = vm0.invoke(() -> WANTestBase.createFirstLocatorWithDSId(1));
+    Integer nyPort = vm1.invoke(() -> WANTestBase.createFirstRemoteLocator(2, lnPort));
+
+    createCacheInVMs(nyPort, vm2);
+    createReceiverInVMs(vm2);
+
+    createCacheInVMs(lnPort, vm4, vm5);
+    vm4.invoke(() -> WANTestBase.createSender("ln", 2, true, 100, 10, false, true, null, true,
+        true));
+    vm5.invoke(() -> WANTestBase.createSender("ln", 2, true, 100, 10, false, true, null, true,
+        true));
+
+    createReceiverCustomerOrderShipmentPR(vm2);
+
+    createSenderCustomerOrderShipmentPRs(vm4);
+    createSenderCustomerOrderShipmentPRs(vm5);
+
+    startSenderInVMs("ln", vm4, vm5);
+
+    int customers = 4;
+    int transactionsPerCustomer = 100;
+    int shipmentsPerTransaction = 10;
+    final LinkedHashMap<Object, Object> keyValuesInTransactions = new LinkedHashMap<>();
+    for (int custId = 0; custId < customers; custId++) {
+      for (int i = 0; i < transactionsPerCustomer; i++) {
+        CustId custIdObject = new CustId(custId);
+        OrderId orderId = new OrderId(i, custIdObject);
+        keyValuesInTransactions.put(orderId, new Order());
+        for (int j = 0; j < shipmentsPerTransaction; j++) {
+          ShipmentId shipmentId = new ShipmentId(i + j, orderId);
+          keyValuesInTransactions.put(shipmentId, new Shipment());
+        }
+      }
+    }
+    int eventsPerTransaction = 1 + shipmentsPerTransaction;
+    AsyncInvocation<Void> inv1 =
+        vm4.invokeAsync(
+            () -> WANTestBase.doOrderAndShipmentPutsInsideTransactions(keyValuesInTransactions,
+                eventsPerTransaction));
+
+    // wait for some batches to be distributed and then stop the sender
+    vm4.invoke(() -> await()
+        .until(() -> WANTestBase.getSenderStats("ln", -1).get(4) > 0));
+
+    System.out.println("Stopping sender");
+    stopSenderInVMsAsync("ln", vm4, vm5);
+    System.out.println("Stopped sender");
+
+    inv1.await();
+    int entries =
+        transactionsPerCustomer * customers;
+
+    vm4.invoke(() -> WANTestBase.validateRegionSize(orderRegionName, entries));
+    vm5.invoke(() -> WANTestBase.validateRegionSize(orderRegionName, entries));
+
+    // Wait for events to replicate: when batches received does not change
+    // we can assume that replication has finished.
+    int batchesReceived = (vm2.invoke(() -> getReceiverStats())).get(0);
+    while (true) {
+      int oldBatchesReceived = batchesReceived;
+      Thread.sleep(1000);
+      batchesReceived = (vm2.invoke(() -> getReceiverStats())).get(0);
+      if (batchesReceived == oldBatchesReceived) {
+        break;
+      }
+    }
+
+    System.out.println("batchesReceived after stop: " + batchesReceived);
+
+    ArrayList<Integer> v4List =
+        (ArrayList<Integer>) vm4.invoke(() -> WANTestBase.getSenderStats("ln", -1));
+    ArrayList<Integer> v5List =
+        (ArrayList<Integer>) vm5.invoke(() -> WANTestBase.getSenderStats("ln", -1));
+
+    // Wait for events to replicate: when region size does not change
+    // we can assume that replication has finished.
+    int orderRegionSize = vm2.invoke(() -> getRegionSize(orderRegionName));
+    while (true) {
+      Thread.sleep(1000);
+      int newRegionSize = vm2.invoke(() -> getRegionSize(orderRegionName));
+      if (orderRegionSize == newRegionSize) {
+        break;
+      }
+    }
+
+    // Wait for events to replicate: when region size does not change
+    // we can assume that replication has finished.
+    int shipmentRegionSize = vm2.invoke(() -> getRegionSize(shipmentRegionName));
+    while (true) {
+      Thread.sleep(1000);
+      int newRegionSize = vm2.invoke(() -> getRegionSize(shipmentRegionName));
+      if (shipmentRegionSize == newRegionSize) {
+        break;
+      }
+    }
+
+    System.out.println("orderRegionSize: " + shipmentRegionSize);
+    System.out.println("shipmentRegionSize: " + shipmentRegionSize);
+
+    System.out.println("v4List.get(0): " + v4List.get(0));
+    System.out.println("v5List.get(0): " + v5List.get(0));
+
+    // batches with incomplete transactions must be 0
+    assertEquals(0, (int) v4List.get(13));
+    assertEquals(0, (int) v5List.get(13));
+
+    // Only complete transactions (1 order + 10 shipments) must be replicated
+    assertEquals(10, shipmentRegionSize / orderRegionSize);
+    assertEquals(0, shipmentRegionSize % orderRegionSize);
+
+    System.out.println("Starting sender");
+    startSenderInVMsAsync("ln", vm4, vm5);
+
+    vm4.invoke(() -> WANTestBase.validateParallelSenderQueueAllBucketsDrained("ln"));
+    vm5.invoke(() -> WANTestBase.validateParallelSenderQueueAllBucketsDrained("ln"));
+
+    v4List =
+        (ArrayList<Integer>) vm4.invoke(() -> WANTestBase.getSenderStats("ln", 0));
+    v5List =
+        (ArrayList<Integer>) vm5.invoke(() -> WANTestBase.getSenderStats("ln", 0));
+
+    assertEquals(0, v4List.get(0) + v5List.get(0));
+
+    // Wait for events to replicate: when batches received does not change
+    // we can assume that replication has finished.
+    batchesReceived = (vm2.invoke(() -> getReceiverStats())).get(0);
+    while (true) {
+      int oldBatchesReceived = batchesReceived;
+      Thread.sleep(1000);
+      batchesReceived = (vm2.invoke(() -> getReceiverStats())).get(0);
+      if (batchesReceived == oldBatchesReceived) {
+        break;
+      }
+    }
+
+    System.out.println("batchesReceived after restart: " + batchesReceived);
+
+    // Wait for events to replicate: when region size does not change
+    // we can assume that replication has finished.
+    orderRegionSize = vm2.invoke(() -> getRegionSize(orderRegionName));
+    while (true) {
+      Thread.sleep(1000);
+      int newRegionSize = vm2.invoke(() -> getRegionSize(orderRegionName));
+      if (orderRegionSize == newRegionSize) {
+        break;
+      }
+    }
+
+    // Wait for events to replicate: when region size does not change
+    // we can assume that replication has finished.
+    shipmentRegionSize = vm2.invoke(() -> getRegionSize(shipmentRegionName));
+    while (true) {

Review comment:
       See earlier comments about not using `Thread.sleep` in tests.

##########
File path: geode-core/src/main/java/org/apache/geode/internal/cache/wan/serial/SerialGatewaySenderQueue.java
##########
@@ -862,19 +877,35 @@ EventsAndLastKey getElementsMatching(Predicate condition, Predicate stopConditio
         continue;
       }
 
-      if (condition.test(object)) {
-        elementsMatching.add(object);
-        peekedIds.add(currentKey);
-        extraPeekedIds.add(currentKey);
-        lastKey = currentKey;
+      if (condition.test((InternalGatewayQueueEvent) object)) {
+        elementsMatching.add(new KeyAndEventPair(currentKey, (GatewaySenderEventImpl) object));
 
-        if (stopCondition.test(object)) {
+        if (stopCondition.test((InternalGatewayQueueEvent) object)) {
           break;
         }
       }
     }
 
-    return new EventsAndLastKey(elementsMatching, lastKey);
+    return elementsMatching;
+  }
+
+  public boolean isThereEventsMatching(Predicate<InternalGatewayQueueEvent> condition) {

Review comment:
       This method would be better named "hasEventsMatching".

##########
File path: geode-core/src/main/java/org/apache/geode/internal/cache/wan/AbstractGatewaySender.java
##########
@@ -1100,7 +1108,21 @@ public void distribute(EnumListenerEvent operation, EntryEventImpl event,
           // Get substitution value to enqueue if necessary
           Object substituteValue = getSubstituteValue(clonedEvent, operation);
 
-          ev.enqueueEvent(operation, clonedEvent, substituteValue, isLastEventInTransaction);
+          Predicate<InternalGatewayQueueEvent> hasSameTransactionIdPredicate = null;
+          // In case the sender is about to be stopped, the event will only
+          // be queued if there is any event in the queue with the same
+          // transactionId as the one of this event
+          if (isStopping && mustGroupTransactionEvents()
+              && clonedEvent.getTransactionId() != null) {
+            hasSameTransactionIdPredicate =
+                x -> x instanceof GatewaySenderEventImpl && clonedEvent.getTransactionId() != null

Review comment:
       The `instanceof` check here can be safely replaced with a null check, as the predicate is typed such that `x` is an instance of `InternalGatewayQueueEvent`, which is all that is required for the call to `getTransactionId()`.

##########
File path: geode-core/src/main/java/org/apache/geode/internal/cache/wan/AbstractGatewaySender.java
##########
@@ -1100,7 +1108,21 @@ public void distribute(EnumListenerEvent operation, EntryEventImpl event,
           // Get substitution value to enqueue if necessary
           Object substituteValue = getSubstituteValue(clonedEvent, operation);
 
-          ev.enqueueEvent(operation, clonedEvent, substituteValue, isLastEventInTransaction);
+          Predicate<InternalGatewayQueueEvent> hasSameTransactionIdPredicate = null;
+          // In case the sender is about to be stopped, the event will only
+          // be queued if there is any event in the queue with the same
+          // transactionId as the one of this event
+          if (isStopping && mustGroupTransactionEvents()
+              && clonedEvent.getTransactionId() != null) {
+            hasSameTransactionIdPredicate =
+                x -> x instanceof GatewaySenderEventImpl && clonedEvent.getTransactionId() != null
+                    && clonedEvent.getTransactionId()
+                        .equals(((GatewaySenderEventImpl) x).getTransactionId());

Review comment:
       The cast to `GatewaySenderEventImpl` here is unnecessary

##########
File path: geode-core/src/main/java/org/apache/geode/internal/cache/wan/serial/SerialGatewaySenderQueue.java
##########
@@ -847,7 +855,14 @@ private EventsAndLastKey peekEventsWithTransactionId(TransactionId transactionId
     }
   }
 
-  EventsAndLastKey getElementsMatching(Predicate condition, Predicate stopCondition, long lastKey) {
+  /**
+   * This method returns a list of objects that fulfill the matchingPredicate
+   * If a matching object also fulfills the endPredicate then the method
+   * stops looking for more matching objects.
+   */
+  List<KeyAndEventPair> getElementsMatching(Predicate<InternalGatewayQueueEvent> condition,
+      Predicate<InternalGatewayQueueEvent> stopCondition,
+      long lastKey) {
     Object object;
     List elementsMatching = new ArrayList<>();

Review comment:
       The compiler warnings associated with `elementsMatching` here can be fixed by using `List<KeyAndEventPair>`

##########
File path: geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/parallel/ParallelWANStatsDUnitTest.java
##########
@@ -472,6 +473,322 @@ public void testPRParallelPropagationWithGroupTransactionEventsSendsBatchesWithC
 
   }
 
+  @Test
+  public void testPRParallelPropagationWithGroupTransactionEventsDoesNotSendBatchesWithIncompleteTransactionsIfGatewaySenderIsStoppedWhileReceivingTrafficAndLaterStarted()
+      throws InterruptedException {
+    Integer lnPort = vm0.invoke(() -> WANTestBase.createFirstLocatorWithDSId(1));
+    Integer nyPort = vm1.invoke(() -> WANTestBase.createFirstRemoteLocator(2, lnPort));
+
+    createCacheInVMs(nyPort, vm2);
+    createReceiverInVMs(vm2);
+
+    createCacheInVMs(lnPort, vm4, vm5);
+    vm4.invoke(() -> WANTestBase.createSender("ln", 2, true, 100, 10, false, true, null, true,
+        true));
+    vm5.invoke(() -> WANTestBase.createSender("ln", 2, true, 100, 10, false, true, null, true,
+        true));
+
+    createReceiverCustomerOrderShipmentPR(vm2);
+
+    createSenderCustomerOrderShipmentPRs(vm4);
+    createSenderCustomerOrderShipmentPRs(vm5);
+
+    startSenderInVMs("ln", vm4, vm5);
+
+    int customers = 4;
+    int transactionsPerCustomer = 100;
+    int shipmentsPerTransaction = 10;
+    final LinkedHashMap<Object, Object> keyValuesInTransactions = new LinkedHashMap<>();
+    for (int custId = 0; custId < customers; custId++) {
+      for (int i = 0; i < transactionsPerCustomer; i++) {
+        CustId custIdObject = new CustId(custId);
+        OrderId orderId = new OrderId(i, custIdObject);
+        keyValuesInTransactions.put(orderId, new Order());
+        for (int j = 0; j < shipmentsPerTransaction; j++) {
+          ShipmentId shipmentId = new ShipmentId(i + j, orderId);
+          keyValuesInTransactions.put(shipmentId, new Shipment());
+        }
+      }
+    }
+    int eventsPerTransaction = 1 + shipmentsPerTransaction;
+    AsyncInvocation<Void> inv1 =
+        vm4.invokeAsync(
+            () -> WANTestBase.doOrderAndShipmentPutsInsideTransactions(keyValuesInTransactions,
+                eventsPerTransaction));
+
+    // wait for some batches to be distributed and then stop the sender
+    vm4.invoke(() -> await()
+        .until(() -> WANTestBase.getSenderStats("ln", -1).get(4) > 0));
+
+    System.out.println("Stopping sender");
+    stopSenderInVMsAsync("ln", vm4, vm5);
+    System.out.println("Stopped sender");
+
+    inv1.await();
+    int entries =
+        transactionsPerCustomer * customers;
+
+    vm4.invoke(() -> WANTestBase.validateRegionSize(orderRegionName, entries));
+    vm5.invoke(() -> WANTestBase.validateRegionSize(orderRegionName, entries));
+
+    // Wait for events to replicate: when batches received does not change
+    // we can assume that replication has finished.
+    int batchesReceived = (vm2.invoke(() -> getReceiverStats())).get(0);
+    while (true) {
+      int oldBatchesReceived = batchesReceived;
+      Thread.sleep(1000);
+      batchesReceived = (vm2.invoke(() -> getReceiverStats())).get(0);
+      if (batchesReceived == oldBatchesReceived) {
+        break;
+      }
+    }
+
+    System.out.println("batchesReceived after stop: " + batchesReceived);
+
+    ArrayList<Integer> v4List =
+        (ArrayList<Integer>) vm4.invoke(() -> WANTestBase.getSenderStats("ln", -1));
+    ArrayList<Integer> v5List =
+        (ArrayList<Integer>) vm5.invoke(() -> WANTestBase.getSenderStats("ln", -1));
+
+    // Wait for events to replicate: when region size does not change
+    // we can assume that replication has finished.
+    int orderRegionSize = vm2.invoke(() -> getRegionSize(orderRegionName));
+    while (true) {
+      Thread.sleep(1000);
+      int newRegionSize = vm2.invoke(() -> getRegionSize(orderRegionName));
+      if (orderRegionSize == newRegionSize) {
+        break;
+      }
+    }
+
+    // Wait for events to replicate: when region size does not change
+    // we can assume that replication has finished.
+    int shipmentRegionSize = vm2.invoke(() -> getRegionSize(shipmentRegionName));
+    while (true) {
+      Thread.sleep(1000);
+      int newRegionSize = vm2.invoke(() -> getRegionSize(shipmentRegionName));
+      if (shipmentRegionSize == newRegionSize) {
+        break;
+      }
+    }
+
+    System.out.println("orderRegionSize: " + shipmentRegionSize);
+    System.out.println("shipmentRegionSize: " + shipmentRegionSize);
+
+    System.out.println("v4List.get(0): " + v4List.get(0));
+    System.out.println("v5List.get(0): " + v5List.get(0));
+
+    // batches with incomplete transactions must be 0
+    assertEquals(0, (int) v4List.get(13));
+    assertEquals(0, (int) v5List.get(13));
+
+    // Only complete transactions (1 order + 10 shipments) must be replicated
+    assertEquals(10, shipmentRegionSize / orderRegionSize);
+    assertEquals(0, shipmentRegionSize % orderRegionSize);
+
+    System.out.println("Starting sender");
+    startSenderInVMsAsync("ln", vm4, vm5);
+
+    vm4.invoke(() -> WANTestBase.validateParallelSenderQueueAllBucketsDrained("ln"));
+    vm5.invoke(() -> WANTestBase.validateParallelSenderQueueAllBucketsDrained("ln"));
+
+    v4List =
+        (ArrayList<Integer>) vm4.invoke(() -> WANTestBase.getSenderStats("ln", 0));
+    v5List =
+        (ArrayList<Integer>) vm5.invoke(() -> WANTestBase.getSenderStats("ln", 0));
+
+    assertEquals(0, v4List.get(0) + v5List.get(0));
+
+    // Wait for events to replicate: when batches received does not change
+    // we can assume that replication has finished.
+    batchesReceived = (vm2.invoke(() -> getReceiverStats())).get(0);
+    while (true) {
+      int oldBatchesReceived = batchesReceived;
+      Thread.sleep(1000);
+      batchesReceived = (vm2.invoke(() -> getReceiverStats())).get(0);
+      if (batchesReceived == oldBatchesReceived) {
+        break;
+      }
+    }
+
+    System.out.println("batchesReceived after restart: " + batchesReceived);

Review comment:
       Another print statement.

##########
File path: geode-core/src/main/java/org/apache/geode/internal/cache/wan/AbstractGatewaySender.java
##########
@@ -686,6 +689,22 @@ public boolean beforeEnqueue(GatewayQueueEvent gatewayEvent) {
     return enqueue;
   }
 
+  protected void preStop() {
+    if (!mustGroupTransactionEvents() || isStopping) {

Review comment:
       If two calls to `preStop()` are made in succession (for instance if two threads call `stop()` at almost the same time), the second thread will immediately return from `preStop()` and continue with the `stop()` method while the first thread sits and waits. I think that this would bypass the intended behaviour and not cause the shutdown to pause while waiting for incomplete transactions.

##########
File path: geode-core/src/main/java/org/apache/geode/internal/cache/wan/AbstractGatewaySenderEventProcessor.java
##########
@@ -195,6 +207,9 @@ protected void setIsStopped(boolean isStopped) {
     } else {
       this.isStopped = isStopped;
     }
+    if (isStopped) {

Review comment:
       The `setIstStopped()` method can be simplified to:
   ```
     protected void setIsStopped(boolean isStopped) {
       this.isStopped = isStopped;
       if (isStopped) {
         this.failureLogInterval.clear();
         sender.postStop();
       }
     }
   ```

##########
File path: geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/serial/SerialWANStatsDUnitTest.java
##########
@@ -348,7 +349,276 @@ public void testReplicatedSerialPropagationWithBatchRedistWithGroupTransactionEv
   }
 
   @Test
-  public void testReplicatedSerialPropagationWithMultipleDispatchers() throws Exception {
+  public void testReplicatedSerialPropagationWithGroupTransactionEventsDoesNotSendBatchesWithIncompleteTransactionsIfGatewaySenderIsStoppedWhileReceivingTrafficAndLaterStarted()
+      throws InterruptedException {
+    Integer lnPort = vm0.invoke(() -> WANTestBase.createFirstLocatorWithDSId(1));
+    Integer nyPort = vm1.invoke(() -> WANTestBase.createFirstRemoteLocator(2, lnPort));
+
+    String regionName = testName + "_RR";
+
+    createCacheInVMs(nyPort, vm2);
+    createReceiverInVMs(vm2);
+
+    createCacheInVMs(lnPort, vm4, vm5);
+    boolean groupTransactionEvents = true;
+    int batchSize = 10;
+    vm4.invoke(
+        () -> WANTestBase.createSender("ln", 2, false, 100, batchSize, false, true, null, true,
+            groupTransactionEvents));
+    vm5.invoke(
+        () -> WANTestBase.createSender("ln", 2, false, 100, batchSize, false, true, null, true,
+            groupTransactionEvents));
+
+    vm2.invoke(() -> WANTestBase.createReplicatedRegion(regionName, null, isOffHeap()));
+
+    vm4.invoke(() -> WANTestBase.createReplicatedRegion(regionName, "ln", isOffHeap()));
+    vm5.invoke(() -> WANTestBase.createReplicatedRegion(regionName, "ln", isOffHeap()));
+
+    startSenderInVMs("ln", vm4, vm5);
+
+    final Map<Object, Object> keyValues = new LinkedHashMap<>();
+    int entries = 2200;

Review comment:
       Is there a reason for this specific value being used for `entries`?

##########
File path: geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/parallel/ParallelWANStatsDUnitTest.java
##########
@@ -472,6 +473,322 @@ public void testPRParallelPropagationWithGroupTransactionEventsSendsBatchesWithC
 
   }
 
+  @Test
+  public void testPRParallelPropagationWithGroupTransactionEventsDoesNotSendBatchesWithIncompleteTransactionsIfGatewaySenderIsStoppedWhileReceivingTrafficAndLaterStarted()
+      throws InterruptedException {
+    Integer lnPort = vm0.invoke(() -> WANTestBase.createFirstLocatorWithDSId(1));
+    Integer nyPort = vm1.invoke(() -> WANTestBase.createFirstRemoteLocator(2, lnPort));
+
+    createCacheInVMs(nyPort, vm2);
+    createReceiverInVMs(vm2);
+
+    createCacheInVMs(lnPort, vm4, vm5);
+    vm4.invoke(() -> WANTestBase.createSender("ln", 2, true, 100, 10, false, true, null, true,
+        true));
+    vm5.invoke(() -> WANTestBase.createSender("ln", 2, true, 100, 10, false, true, null, true,
+        true));
+
+    createReceiverCustomerOrderShipmentPR(vm2);
+
+    createSenderCustomerOrderShipmentPRs(vm4);
+    createSenderCustomerOrderShipmentPRs(vm5);
+
+    startSenderInVMs("ln", vm4, vm5);
+
+    int customers = 4;
+    int transactionsPerCustomer = 100;
+    int shipmentsPerTransaction = 10;
+    final LinkedHashMap<Object, Object> keyValuesInTransactions = new LinkedHashMap<>();
+    for (int custId = 0; custId < customers; custId++) {
+      for (int i = 0; i < transactionsPerCustomer; i++) {
+        CustId custIdObject = new CustId(custId);
+        OrderId orderId = new OrderId(i, custIdObject);
+        keyValuesInTransactions.put(orderId, new Order());
+        for (int j = 0; j < shipmentsPerTransaction; j++) {
+          ShipmentId shipmentId = new ShipmentId(i + j, orderId);
+          keyValuesInTransactions.put(shipmentId, new Shipment());
+        }
+      }
+    }
+    int eventsPerTransaction = 1 + shipmentsPerTransaction;
+    AsyncInvocation<Void> inv1 =
+        vm4.invokeAsync(
+            () -> WANTestBase.doOrderAndShipmentPutsInsideTransactions(keyValuesInTransactions,
+                eventsPerTransaction));
+
+    // wait for some batches to be distributed and then stop the sender
+    vm4.invoke(() -> await()
+        .until(() -> WANTestBase.getSenderStats("ln", -1).get(4) > 0));
+
+    System.out.println("Stopping sender");
+    stopSenderInVMsAsync("ln", vm4, vm5);
+    System.out.println("Stopped sender");
+
+    inv1.await();
+    int entries =
+        transactionsPerCustomer * customers;
+
+    vm4.invoke(() -> WANTestBase.validateRegionSize(orderRegionName, entries));
+    vm5.invoke(() -> WANTestBase.validateRegionSize(orderRegionName, entries));
+
+    // Wait for events to replicate: when batches received does not change
+    // we can assume that replication has finished.
+    int batchesReceived = (vm2.invoke(() -> getReceiverStats())).get(0);
+    while (true) {
+      int oldBatchesReceived = batchesReceived;
+      Thread.sleep(1000);
+      batchesReceived = (vm2.invoke(() -> getReceiverStats())).get(0);
+      if (batchesReceived == oldBatchesReceived) {
+        break;
+      }
+    }
+
+    System.out.println("batchesReceived after stop: " + batchesReceived);
+
+    ArrayList<Integer> v4List =
+        (ArrayList<Integer>) vm4.invoke(() -> WANTestBase.getSenderStats("ln", -1));
+    ArrayList<Integer> v5List =
+        (ArrayList<Integer>) vm5.invoke(() -> WANTestBase.getSenderStats("ln", -1));
+
+    // Wait for events to replicate: when region size does not change
+    // we can assume that replication has finished.
+    int orderRegionSize = vm2.invoke(() -> getRegionSize(orderRegionName));
+    while (true) {
+      Thread.sleep(1000);
+      int newRegionSize = vm2.invoke(() -> getRegionSize(orderRegionName));
+      if (orderRegionSize == newRegionSize) {
+        break;
+      }
+    }
+
+    // Wait for events to replicate: when region size does not change
+    // we can assume that replication has finished.
+    int shipmentRegionSize = vm2.invoke(() -> getRegionSize(shipmentRegionName));
+    while (true) {
+      Thread.sleep(1000);
+      int newRegionSize = vm2.invoke(() -> getRegionSize(shipmentRegionName));
+      if (shipmentRegionSize == newRegionSize) {
+        break;
+      }
+    }
+
+    System.out.println("orderRegionSize: " + shipmentRegionSize);
+    System.out.println("shipmentRegionSize: " + shipmentRegionSize);
+
+    System.out.println("v4List.get(0): " + v4List.get(0));
+    System.out.println("v5List.get(0): " + v5List.get(0));
+
+    // batches with incomplete transactions must be 0
+    assertEquals(0, (int) v4List.get(13));
+    assertEquals(0, (int) v5List.get(13));
+
+    // Only complete transactions (1 order + 10 shipments) must be replicated
+    assertEquals(10, shipmentRegionSize / orderRegionSize);
+    assertEquals(0, shipmentRegionSize % orderRegionSize);
+
+    System.out.println("Starting sender");
+    startSenderInVMsAsync("ln", vm4, vm5);
+
+    vm4.invoke(() -> WANTestBase.validateParallelSenderQueueAllBucketsDrained("ln"));
+    vm5.invoke(() -> WANTestBase.validateParallelSenderQueueAllBucketsDrained("ln"));
+
+    v4List =
+        (ArrayList<Integer>) vm4.invoke(() -> WANTestBase.getSenderStats("ln", 0));
+    v5List =
+        (ArrayList<Integer>) vm5.invoke(() -> WANTestBase.getSenderStats("ln", 0));
+
+    assertEquals(0, v4List.get(0) + v5List.get(0));
+
+    // Wait for events to replicate: when batches received does not change
+    // we can assume that replication has finished.
+    batchesReceived = (vm2.invoke(() -> getReceiverStats())).get(0);
+    while (true) {
+      int oldBatchesReceived = batchesReceived;
+      Thread.sleep(1000);
+      batchesReceived = (vm2.invoke(() -> getReceiverStats())).get(0);
+      if (batchesReceived == oldBatchesReceived) {
+        break;
+      }
+    }
+
+    System.out.println("batchesReceived after restart: " + batchesReceived);
+
+    // Wait for events to replicate: when region size does not change
+    // we can assume that replication has finished.
+    orderRegionSize = vm2.invoke(() -> getRegionSize(orderRegionName));
+    while (true) {
+      Thread.sleep(1000);
+      int newRegionSize = vm2.invoke(() -> getRegionSize(orderRegionName));
+      if (orderRegionSize == newRegionSize) {
+        break;
+      }
+    }
+
+    // Wait for events to replicate: when region size does not change
+    // we can assume that replication has finished.
+    shipmentRegionSize = vm2.invoke(() -> getRegionSize(shipmentRegionName));
+    while (true) {
+      Thread.sleep(1000);
+      int newRegionSize = vm2.invoke(() -> getRegionSize(shipmentRegionName));
+      if (shipmentRegionSize == newRegionSize) {
+        break;
+      }
+    }
+
+    System.out.println("orderRegionSize after restart: " + shipmentRegionSize);
+    System.out.println("shipmentRegionSize after restart: " + shipmentRegionSize);
+
+    System.out.println("v4List.get(0): " + v4List.get(0));
+    System.out.println("v5List.get(0): " + v5List.get(0));
+
+    // batches with incomplete transactions must be 0
+    assertEquals(0, (int) v4List.get(13));
+    assertEquals(0, (int) v5List.get(13));
+
+    // Only complete transactions (1 order + 10 shipments) must be replicated
+    assertEquals(10, shipmentRegionSize / orderRegionSize);
+    assertEquals(0, shipmentRegionSize % orderRegionSize);
+  }
+
+  @Test
+  public void testPRParallelPropagationWithGroupTransactionEventsDoesNotSendBatchesWithIncompleteTransactionsIfGatewaySenderIsStoppedWhileReceivingTrafficAndLaterStartedReceiverStopped()

Review comment:
       As there is a lot of duplicated code between this test and the one above it, all comments applying to that test case likely also apply here.

##########
File path: geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/parallel/ParallelWANStatsDUnitTest.java
##########
@@ -472,6 +473,322 @@ public void testPRParallelPropagationWithGroupTransactionEventsSendsBatchesWithC
 
   }
 
+  @Test
+  public void testPRParallelPropagationWithGroupTransactionEventsDoesNotSendBatchesWithIncompleteTransactionsIfGatewaySenderIsStoppedWhileReceivingTrafficAndLaterStarted()
+      throws InterruptedException {
+    Integer lnPort = vm0.invoke(() -> WANTestBase.createFirstLocatorWithDSId(1));
+    Integer nyPort = vm1.invoke(() -> WANTestBase.createFirstRemoteLocator(2, lnPort));
+
+    createCacheInVMs(nyPort, vm2);
+    createReceiverInVMs(vm2);
+
+    createCacheInVMs(lnPort, vm4, vm5);
+    vm4.invoke(() -> WANTestBase.createSender("ln", 2, true, 100, 10, false, true, null, true,
+        true));
+    vm5.invoke(() -> WANTestBase.createSender("ln", 2, true, 100, 10, false, true, null, true,
+        true));
+
+    createReceiverCustomerOrderShipmentPR(vm2);
+
+    createSenderCustomerOrderShipmentPRs(vm4);
+    createSenderCustomerOrderShipmentPRs(vm5);
+
+    startSenderInVMs("ln", vm4, vm5);
+
+    int customers = 4;
+    int transactionsPerCustomer = 100;
+    int shipmentsPerTransaction = 10;
+    final LinkedHashMap<Object, Object> keyValuesInTransactions = new LinkedHashMap<>();
+    for (int custId = 0; custId < customers; custId++) {
+      for (int i = 0; i < transactionsPerCustomer; i++) {
+        CustId custIdObject = new CustId(custId);
+        OrderId orderId = new OrderId(i, custIdObject);
+        keyValuesInTransactions.put(orderId, new Order());
+        for (int j = 0; j < shipmentsPerTransaction; j++) {
+          ShipmentId shipmentId = new ShipmentId(i + j, orderId);
+          keyValuesInTransactions.put(shipmentId, new Shipment());
+        }
+      }
+    }
+    int eventsPerTransaction = 1 + shipmentsPerTransaction;
+    AsyncInvocation<Void> inv1 =
+        vm4.invokeAsync(
+            () -> WANTestBase.doOrderAndShipmentPutsInsideTransactions(keyValuesInTransactions,
+                eventsPerTransaction));
+
+    // wait for some batches to be distributed and then stop the sender
+    vm4.invoke(() -> await()
+        .until(() -> WANTestBase.getSenderStats("ln", -1).get(4) > 0));
+
+    System.out.println("Stopping sender");
+    stopSenderInVMsAsync("ln", vm4, vm5);
+    System.out.println("Stopped sender");
+
+    inv1.await();
+    int entries =
+        transactionsPerCustomer * customers;
+
+    vm4.invoke(() -> WANTestBase.validateRegionSize(orderRegionName, entries));
+    vm5.invoke(() -> WANTestBase.validateRegionSize(orderRegionName, entries));
+
+    // Wait for events to replicate: when batches received does not change
+    // we can assume that replication has finished.
+    int batchesReceived = (vm2.invoke(() -> getReceiverStats())).get(0);
+    while (true) {
+      int oldBatchesReceived = batchesReceived;
+      Thread.sleep(1000);
+      batchesReceived = (vm2.invoke(() -> getReceiverStats())).get(0);
+      if (batchesReceived == oldBatchesReceived) {
+        break;
+      }
+    }
+
+    System.out.println("batchesReceived after stop: " + batchesReceived);
+
+    ArrayList<Integer> v4List =
+        (ArrayList<Integer>) vm4.invoke(() -> WANTestBase.getSenderStats("ln", -1));
+    ArrayList<Integer> v5List =
+        (ArrayList<Integer>) vm5.invoke(() -> WANTestBase.getSenderStats("ln", -1));
+
+    // Wait for events to replicate: when region size does not change
+    // we can assume that replication has finished.
+    int orderRegionSize = vm2.invoke(() -> getRegionSize(orderRegionName));
+    while (true) {
+      Thread.sleep(1000);
+      int newRegionSize = vm2.invoke(() -> getRegionSize(orderRegionName));
+      if (orderRegionSize == newRegionSize) {
+        break;
+      }
+    }
+
+    // Wait for events to replicate: when region size does not change
+    // we can assume that replication has finished.
+    int shipmentRegionSize = vm2.invoke(() -> getRegionSize(shipmentRegionName));
+    while (true) {
+      Thread.sleep(1000);
+      int newRegionSize = vm2.invoke(() -> getRegionSize(shipmentRegionName));
+      if (shipmentRegionSize == newRegionSize) {
+        break;
+      }
+    }
+
+    System.out.println("orderRegionSize: " + shipmentRegionSize);
+    System.out.println("shipmentRegionSize: " + shipmentRegionSize);
+
+    System.out.println("v4List.get(0): " + v4List.get(0));
+    System.out.println("v5List.get(0): " + v5List.get(0));
+
+    // batches with incomplete transactions must be 0
+    assertEquals(0, (int) v4List.get(13));
+    assertEquals(0, (int) v5List.get(13));
+
+    // Only complete transactions (1 order + 10 shipments) must be replicated
+    assertEquals(10, shipmentRegionSize / orderRegionSize);
+    assertEquals(0, shipmentRegionSize % orderRegionSize);
+
+    System.out.println("Starting sender");
+    startSenderInVMsAsync("ln", vm4, vm5);
+
+    vm4.invoke(() -> WANTestBase.validateParallelSenderQueueAllBucketsDrained("ln"));
+    vm5.invoke(() -> WANTestBase.validateParallelSenderQueueAllBucketsDrained("ln"));
+
+    v4List =
+        (ArrayList<Integer>) vm4.invoke(() -> WANTestBase.getSenderStats("ln", 0));
+    v5List =
+        (ArrayList<Integer>) vm5.invoke(() -> WANTestBase.getSenderStats("ln", 0));
+
+    assertEquals(0, v4List.get(0) + v5List.get(0));
+
+    // Wait for events to replicate: when batches received does not change
+    // we can assume that replication has finished.
+    batchesReceived = (vm2.invoke(() -> getReceiverStats())).get(0);
+    while (true) {
+      int oldBatchesReceived = batchesReceived;
+      Thread.sleep(1000);
+      batchesReceived = (vm2.invoke(() -> getReceiverStats())).get(0);
+      if (batchesReceived == oldBatchesReceived) {
+        break;
+      }
+    }
+
+    System.out.println("batchesReceived after restart: " + batchesReceived);
+
+    // Wait for events to replicate: when region size does not change
+    // we can assume that replication has finished.
+    orderRegionSize = vm2.invoke(() -> getRegionSize(orderRegionName));
+    while (true) {
+      Thread.sleep(1000);
+      int newRegionSize = vm2.invoke(() -> getRegionSize(orderRegionName));
+      if (orderRegionSize == newRegionSize) {
+        break;
+      }
+    }
+
+    // Wait for events to replicate: when region size does not change
+    // we can assume that replication has finished.
+    shipmentRegionSize = vm2.invoke(() -> getRegionSize(shipmentRegionName));
+    while (true) {
+      Thread.sleep(1000);
+      int newRegionSize = vm2.invoke(() -> getRegionSize(shipmentRegionName));
+      if (shipmentRegionSize == newRegionSize) {
+        break;
+      }
+    }
+
+    System.out.println("orderRegionSize after restart: " + shipmentRegionSize);
+    System.out.println("shipmentRegionSize after restart: " + shipmentRegionSize);
+
+    System.out.println("v4List.get(0): " + v4List.get(0));
+    System.out.println("v5List.get(0): " + v5List.get(0));
+
+    // batches with incomplete transactions must be 0
+    assertEquals(0, (int) v4List.get(13));
+    assertEquals(0, (int) v5List.get(13));
+
+    // Only complete transactions (1 order + 10 shipments) must be replicated
+    assertEquals(10, shipmentRegionSize / orderRegionSize);

Review comment:
       I think these could be replaced with one assert, as described above.

##########
File path: geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/parallel/ParallelWANStatsDUnitTest.java
##########
@@ -472,6 +473,322 @@ public void testPRParallelPropagationWithGroupTransactionEventsSendsBatchesWithC
 
   }
 
+  @Test
+  public void testPRParallelPropagationWithGroupTransactionEventsDoesNotSendBatchesWithIncompleteTransactionsIfGatewaySenderIsStoppedWhileReceivingTrafficAndLaterStarted()
+      throws InterruptedException {
+    Integer lnPort = vm0.invoke(() -> WANTestBase.createFirstLocatorWithDSId(1));
+    Integer nyPort = vm1.invoke(() -> WANTestBase.createFirstRemoteLocator(2, lnPort));
+
+    createCacheInVMs(nyPort, vm2);
+    createReceiverInVMs(vm2);
+
+    createCacheInVMs(lnPort, vm4, vm5);
+    vm4.invoke(() -> WANTestBase.createSender("ln", 2, true, 100, 10, false, true, null, true,
+        true));
+    vm5.invoke(() -> WANTestBase.createSender("ln", 2, true, 100, 10, false, true, null, true,
+        true));
+
+    createReceiverCustomerOrderShipmentPR(vm2);
+
+    createSenderCustomerOrderShipmentPRs(vm4);
+    createSenderCustomerOrderShipmentPRs(vm5);
+
+    startSenderInVMs("ln", vm4, vm5);
+
+    int customers = 4;
+    int transactionsPerCustomer = 100;
+    int shipmentsPerTransaction = 10;
+    final LinkedHashMap<Object, Object> keyValuesInTransactions = new LinkedHashMap<>();
+    for (int custId = 0; custId < customers; custId++) {
+      for (int i = 0; i < transactionsPerCustomer; i++) {
+        CustId custIdObject = new CustId(custId);
+        OrderId orderId = new OrderId(i, custIdObject);
+        keyValuesInTransactions.put(orderId, new Order());
+        for (int j = 0; j < shipmentsPerTransaction; j++) {
+          ShipmentId shipmentId = new ShipmentId(i + j, orderId);
+          keyValuesInTransactions.put(shipmentId, new Shipment());
+        }
+      }
+    }
+    int eventsPerTransaction = 1 + shipmentsPerTransaction;
+    AsyncInvocation<Void> inv1 =
+        vm4.invokeAsync(
+            () -> WANTestBase.doOrderAndShipmentPutsInsideTransactions(keyValuesInTransactions,
+                eventsPerTransaction));
+
+    // wait for some batches to be distributed and then stop the sender
+    vm4.invoke(() -> await()
+        .until(() -> WANTestBase.getSenderStats("ln", -1).get(4) > 0));
+
+    System.out.println("Stopping sender");
+    stopSenderInVMsAsync("ln", vm4, vm5);
+    System.out.println("Stopped sender");
+
+    inv1.await();
+    int entries =
+        transactionsPerCustomer * customers;
+
+    vm4.invoke(() -> WANTestBase.validateRegionSize(orderRegionName, entries));
+    vm5.invoke(() -> WANTestBase.validateRegionSize(orderRegionName, entries));
+
+    // Wait for events to replicate: when batches received does not change
+    // we can assume that replication has finished.
+    int batchesReceived = (vm2.invoke(() -> getReceiverStats())).get(0);
+    while (true) {
+      int oldBatchesReceived = batchesReceived;
+      Thread.sleep(1000);
+      batchesReceived = (vm2.invoke(() -> getReceiverStats())).get(0);
+      if (batchesReceived == oldBatchesReceived) {
+        break;
+      }
+    }
+
+    System.out.println("batchesReceived after stop: " + batchesReceived);
+
+    ArrayList<Integer> v4List =
+        (ArrayList<Integer>) vm4.invoke(() -> WANTestBase.getSenderStats("ln", -1));
+    ArrayList<Integer> v5List =
+        (ArrayList<Integer>) vm5.invoke(() -> WANTestBase.getSenderStats("ln", -1));
+
+    // Wait for events to replicate: when region size does not change
+    // we can assume that replication has finished.
+    int orderRegionSize = vm2.invoke(() -> getRegionSize(orderRegionName));
+    while (true) {
+      Thread.sleep(1000);
+      int newRegionSize = vm2.invoke(() -> getRegionSize(orderRegionName));
+      if (orderRegionSize == newRegionSize) {
+        break;
+      }
+    }
+
+    // Wait for events to replicate: when region size does not change
+    // we can assume that replication has finished.
+    int shipmentRegionSize = vm2.invoke(() -> getRegionSize(shipmentRegionName));
+    while (true) {
+      Thread.sleep(1000);
+      int newRegionSize = vm2.invoke(() -> getRegionSize(shipmentRegionName));
+      if (shipmentRegionSize == newRegionSize) {
+        break;
+      }
+    }
+
+    System.out.println("orderRegionSize: " + shipmentRegionSize);
+    System.out.println("shipmentRegionSize: " + shipmentRegionSize);
+
+    System.out.println("v4List.get(0): " + v4List.get(0));
+    System.out.println("v5List.get(0): " + v5List.get(0));
+
+    // batches with incomplete transactions must be 0
+    assertEquals(0, (int) v4List.get(13));
+    assertEquals(0, (int) v5List.get(13));
+
+    // Only complete transactions (1 order + 10 shipments) must be replicated
+    assertEquals(10, shipmentRegionSize / orderRegionSize);
+    assertEquals(0, shipmentRegionSize % orderRegionSize);
+
+    System.out.println("Starting sender");
+    startSenderInVMsAsync("ln", vm4, vm5);
+
+    vm4.invoke(() -> WANTestBase.validateParallelSenderQueueAllBucketsDrained("ln"));
+    vm5.invoke(() -> WANTestBase.validateParallelSenderQueueAllBucketsDrained("ln"));
+
+    v4List =
+        (ArrayList<Integer>) vm4.invoke(() -> WANTestBase.getSenderStats("ln", 0));
+    v5List =
+        (ArrayList<Integer>) vm5.invoke(() -> WANTestBase.getSenderStats("ln", 0));
+
+    assertEquals(0, v4List.get(0) + v5List.get(0));
+
+    // Wait for events to replicate: when batches received does not change
+    // we can assume that replication has finished.
+    batchesReceived = (vm2.invoke(() -> getReceiverStats())).get(0);
+    while (true) {
+      int oldBatchesReceived = batchesReceived;
+      Thread.sleep(1000);
+      batchesReceived = (vm2.invoke(() -> getReceiverStats())).get(0);
+      if (batchesReceived == oldBatchesReceived) {
+        break;
+      }
+    }
+
+    System.out.println("batchesReceived after restart: " + batchesReceived);
+
+    // Wait for events to replicate: when region size does not change
+    // we can assume that replication has finished.
+    orderRegionSize = vm2.invoke(() -> getRegionSize(orderRegionName));
+    while (true) {

Review comment:
       See earlier comments about not using `Thread.sleep` in tests.

##########
File path: geode-core/src/main/java/org/apache/geode/internal/cache/BucketRegionQueue.java
##########
@@ -478,6 +480,24 @@ public Object peek() {
     }
   }
 
+  public boolean isThereEventsMatching(Predicate<InternalGatewayQueueEvent> matchingPredicate) {
+    getInitializationLock().readLock().lock();
+    try {
+      if (this.getPartitionedRegion().isDestroyed()) {
+        throw new BucketRegionQueueUnavailableException();
+      }
+      for (Object o : eventSeqNumDeque) {
+        Object object = optimalGet(o);
+        if (matchingPredicate.test((InternalGatewayQueueEvent) object)) {

Review comment:
       I believe there is a possibility of a ClassCastException here. There should be an `instanceof` check on `object` prior to casting.

##########
File path: geode-core/src/main/java/org/apache/geode/internal/cache/BucketRegionQueue.java
##########
@@ -478,6 +480,24 @@ public Object peek() {
     }
   }
 
+  public boolean isThereEventsMatching(Predicate<InternalGatewayQueueEvent> matchingPredicate) {

Review comment:
       This method would be better named "hasEventsMatching".

##########
File path: geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/serial/SerialWANStatsDUnitTest.java
##########
@@ -348,7 +349,276 @@ public void testReplicatedSerialPropagationWithBatchRedistWithGroupTransactionEv
   }
 
   @Test
-  public void testReplicatedSerialPropagationWithMultipleDispatchers() throws Exception {
+  public void testReplicatedSerialPropagationWithGroupTransactionEventsDoesNotSendBatchesWithIncompleteTransactionsIfGatewaySenderIsStoppedWhileReceivingTrafficAndLaterStarted()
+      throws InterruptedException {
+    Integer lnPort = vm0.invoke(() -> WANTestBase.createFirstLocatorWithDSId(1));
+    Integer nyPort = vm1.invoke(() -> WANTestBase.createFirstRemoteLocator(2, lnPort));
+
+    String regionName = testName + "_RR";
+
+    createCacheInVMs(nyPort, vm2);
+    createReceiverInVMs(vm2);
+
+    createCacheInVMs(lnPort, vm4, vm5);
+    boolean groupTransactionEvents = true;
+    int batchSize = 10;
+    vm4.invoke(
+        () -> WANTestBase.createSender("ln", 2, false, 100, batchSize, false, true, null, true,
+            groupTransactionEvents));
+    vm5.invoke(
+        () -> WANTestBase.createSender("ln", 2, false, 100, batchSize, false, true, null, true,
+            groupTransactionEvents));
+
+    vm2.invoke(() -> WANTestBase.createReplicatedRegion(regionName, null, isOffHeap()));
+
+    vm4.invoke(() -> WANTestBase.createReplicatedRegion(regionName, "ln", isOffHeap()));
+    vm5.invoke(() -> WANTestBase.createReplicatedRegion(regionName, "ln", isOffHeap()));
+
+    startSenderInVMs("ln", vm4, vm5);
+
+    final Map<Object, Object> keyValues = new LinkedHashMap<>();
+    int entries = 2200;
+    for (int i = 0; i < entries; i++) {
+      keyValues.put(i, i + "_Value");
+    }
+
+    int eventsPerTransaction = 11;

Review comment:
       I assume that this value is related to `batchSize`, so it would be good to make that explicit rather than just having it a seeminly arbitrary number.

##########
File path: geode-core/src/main/java/org/apache/geode/internal/cache/EntryEventImpl.java
##########
@@ -2231,6 +2231,9 @@ public String toString() {
     if (this.tailKey != -1) {
       buf.append(";tailKey=" + tailKey);
     }
+    if (this.isTransactional()) {
+      buf.append(";transactionId=" + getTransactionId());

Review comment:
       The compiler warning on this line (and on line 2232) can be fixed by chaining append calls rather than doing String concatenation on the argument.

##########
File path: geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/parallel/ParallelWANStatsDUnitTest.java
##########
@@ -472,6 +473,322 @@ public void testPRParallelPropagationWithGroupTransactionEventsSendsBatchesWithC
 
   }
 
+  @Test
+  public void testPRParallelPropagationWithGroupTransactionEventsDoesNotSendBatchesWithIncompleteTransactionsIfGatewaySenderIsStoppedWhileReceivingTrafficAndLaterStarted()
+      throws InterruptedException {
+    Integer lnPort = vm0.invoke(() -> WANTestBase.createFirstLocatorWithDSId(1));
+    Integer nyPort = vm1.invoke(() -> WANTestBase.createFirstRemoteLocator(2, lnPort));
+
+    createCacheInVMs(nyPort, vm2);
+    createReceiverInVMs(vm2);
+
+    createCacheInVMs(lnPort, vm4, vm5);
+    vm4.invoke(() -> WANTestBase.createSender("ln", 2, true, 100, 10, false, true, null, true,
+        true));
+    vm5.invoke(() -> WANTestBase.createSender("ln", 2, true, 100, 10, false, true, null, true,
+        true));
+
+    createReceiverCustomerOrderShipmentPR(vm2);
+
+    createSenderCustomerOrderShipmentPRs(vm4);
+    createSenderCustomerOrderShipmentPRs(vm5);
+
+    startSenderInVMs("ln", vm4, vm5);
+
+    int customers = 4;
+    int transactionsPerCustomer = 100;
+    int shipmentsPerTransaction = 10;
+    final LinkedHashMap<Object, Object> keyValuesInTransactions = new LinkedHashMap<>();

Review comment:
       This would be better as `final Map<Object, Object>` as it's preferable to declare and handle interfaces rather than implementations unless absolutely necessary.

##########
File path: geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/parallel/ParallelWANStatsDUnitTest.java
##########
@@ -472,6 +473,322 @@ public void testPRParallelPropagationWithGroupTransactionEventsSendsBatchesWithC
 
   }
 
+  @Test
+  public void testPRParallelPropagationWithGroupTransactionEventsDoesNotSendBatchesWithIncompleteTransactionsIfGatewaySenderIsStoppedWhileReceivingTrafficAndLaterStarted()

Review comment:
       There is a lot of duplicate code in this test and the other added in this class. It would be good to pull as much of it out into its own methods as possible.

##########
File path: geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/parallel/ParallelWANStatsDUnitTest.java
##########
@@ -472,6 +473,322 @@ public void testPRParallelPropagationWithGroupTransactionEventsSendsBatchesWithC
 
   }
 
+  @Test
+  public void testPRParallelPropagationWithGroupTransactionEventsDoesNotSendBatchesWithIncompleteTransactionsIfGatewaySenderIsStoppedWhileReceivingTrafficAndLaterStarted()
+      throws InterruptedException {
+    Integer lnPort = vm0.invoke(() -> WANTestBase.createFirstLocatorWithDSId(1));
+    Integer nyPort = vm1.invoke(() -> WANTestBase.createFirstRemoteLocator(2, lnPort));
+
+    createCacheInVMs(nyPort, vm2);
+    createReceiverInVMs(vm2);
+
+    createCacheInVMs(lnPort, vm4, vm5);
+    vm4.invoke(() -> WANTestBase.createSender("ln", 2, true, 100, 10, false, true, null, true,
+        true));
+    vm5.invoke(() -> WANTestBase.createSender("ln", 2, true, 100, 10, false, true, null, true,
+        true));
+
+    createReceiverCustomerOrderShipmentPR(vm2);
+
+    createSenderCustomerOrderShipmentPRs(vm4);
+    createSenderCustomerOrderShipmentPRs(vm5);
+
+    startSenderInVMs("ln", vm4, vm5);
+
+    int customers = 4;
+    int transactionsPerCustomer = 100;
+    int shipmentsPerTransaction = 10;

Review comment:
       I suspect that the values chosen for `transactionsPerCustomer` and `shipmentsPerTransaction` are related to the batch size for this test, so if that's the case, it would be good to make it explicit.

##########
File path: geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/WANTestBase.java
##########
@@ -1320,6 +1320,23 @@ public static void checkGatewayReceiverStats(int processBatches, int eventsRecei
     assertEquals(creates, gatewayReceiverStats.getCreateRequest());
   }
 
+  public static List<Integer> getReceiverStats() {

Review comment:
       It might be more user-friendly to have this method return the actual `GatewayReceiverStats` object, as otherwise, anyone who calls this method will have to come and check which stats correspond to which index in the returned list.

##########
File path: geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/parallel/ParallelWANStatsDUnitTest.java
##########
@@ -472,6 +473,322 @@ public void testPRParallelPropagationWithGroupTransactionEventsSendsBatchesWithC
 
   }
 
+  @Test
+  public void testPRParallelPropagationWithGroupTransactionEventsDoesNotSendBatchesWithIncompleteTransactionsIfGatewaySenderIsStoppedWhileReceivingTrafficAndLaterStarted()
+      throws InterruptedException {
+    Integer lnPort = vm0.invoke(() -> WANTestBase.createFirstLocatorWithDSId(1));
+    Integer nyPort = vm1.invoke(() -> WANTestBase.createFirstRemoteLocator(2, lnPort));
+
+    createCacheInVMs(nyPort, vm2);
+    createReceiverInVMs(vm2);
+
+    createCacheInVMs(lnPort, vm4, vm5);
+    vm4.invoke(() -> WANTestBase.createSender("ln", 2, true, 100, 10, false, true, null, true,
+        true));
+    vm5.invoke(() -> WANTestBase.createSender("ln", 2, true, 100, 10, false, true, null, true,
+        true));
+
+    createReceiverCustomerOrderShipmentPR(vm2);
+
+    createSenderCustomerOrderShipmentPRs(vm4);
+    createSenderCustomerOrderShipmentPRs(vm5);
+
+    startSenderInVMs("ln", vm4, vm5);
+
+    int customers = 4;
+    int transactionsPerCustomer = 100;
+    int shipmentsPerTransaction = 10;
+    final LinkedHashMap<Object, Object> keyValuesInTransactions = new LinkedHashMap<>();
+    for (int custId = 0; custId < customers; custId++) {
+      for (int i = 0; i < transactionsPerCustomer; i++) {
+        CustId custIdObject = new CustId(custId);
+        OrderId orderId = new OrderId(i, custIdObject);
+        keyValuesInTransactions.put(orderId, new Order());
+        for (int j = 0; j < shipmentsPerTransaction; j++) {
+          ShipmentId shipmentId = new ShipmentId(i + j, orderId);
+          keyValuesInTransactions.put(shipmentId, new Shipment());
+        }
+      }
+    }
+    int eventsPerTransaction = 1 + shipmentsPerTransaction;
+    AsyncInvocation<Void> inv1 =
+        vm4.invokeAsync(
+            () -> WANTestBase.doOrderAndShipmentPutsInsideTransactions(keyValuesInTransactions,
+                eventsPerTransaction));
+
+    // wait for some batches to be distributed and then stop the sender
+    vm4.invoke(() -> await()
+        .until(() -> WANTestBase.getSenderStats("ln", -1).get(4) > 0));
+
+    System.out.println("Stopping sender");
+    stopSenderInVMsAsync("ln", vm4, vm5);
+    System.out.println("Stopped sender");
+
+    inv1.await();
+    int entries =
+        transactionsPerCustomer * customers;
+
+    vm4.invoke(() -> WANTestBase.validateRegionSize(orderRegionName, entries));
+    vm5.invoke(() -> WANTestBase.validateRegionSize(orderRegionName, entries));
+
+    // Wait for events to replicate: when batches received does not change
+    // we can assume that replication has finished.
+    int batchesReceived = (vm2.invoke(() -> getReceiverStats())).get(0);

Review comment:
       This lambda can be replaced with a metod reference

##########
File path: geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/parallel/ParallelWANStatsDUnitTest.java
##########
@@ -472,6 +473,322 @@ public void testPRParallelPropagationWithGroupTransactionEventsSendsBatchesWithC
 
   }
 
+  @Test
+  public void testPRParallelPropagationWithGroupTransactionEventsDoesNotSendBatchesWithIncompleteTransactionsIfGatewaySenderIsStoppedWhileReceivingTrafficAndLaterStarted()
+      throws InterruptedException {
+    Integer lnPort = vm0.invoke(() -> WANTestBase.createFirstLocatorWithDSId(1));
+    Integer nyPort = vm1.invoke(() -> WANTestBase.createFirstRemoteLocator(2, lnPort));
+
+    createCacheInVMs(nyPort, vm2);
+    createReceiverInVMs(vm2);
+
+    createCacheInVMs(lnPort, vm4, vm5);
+    vm4.invoke(() -> WANTestBase.createSender("ln", 2, true, 100, 10, false, true, null, true,
+        true));
+    vm5.invoke(() -> WANTestBase.createSender("ln", 2, true, 100, 10, false, true, null, true,
+        true));
+
+    createReceiverCustomerOrderShipmentPR(vm2);
+
+    createSenderCustomerOrderShipmentPRs(vm4);
+    createSenderCustomerOrderShipmentPRs(vm5);
+
+    startSenderInVMs("ln", vm4, vm5);
+
+    int customers = 4;
+    int transactionsPerCustomer = 100;
+    int shipmentsPerTransaction = 10;
+    final LinkedHashMap<Object, Object> keyValuesInTransactions = new LinkedHashMap<>();
+    for (int custId = 0; custId < customers; custId++) {
+      for (int i = 0; i < transactionsPerCustomer; i++) {
+        CustId custIdObject = new CustId(custId);
+        OrderId orderId = new OrderId(i, custIdObject);
+        keyValuesInTransactions.put(orderId, new Order());
+        for (int j = 0; j < shipmentsPerTransaction; j++) {
+          ShipmentId shipmentId = new ShipmentId(i + j, orderId);
+          keyValuesInTransactions.put(shipmentId, new Shipment());
+        }
+      }
+    }
+    int eventsPerTransaction = 1 + shipmentsPerTransaction;
+    AsyncInvocation<Void> inv1 =
+        vm4.invokeAsync(
+            () -> WANTestBase.doOrderAndShipmentPutsInsideTransactions(keyValuesInTransactions,
+                eventsPerTransaction));
+
+    // wait for some batches to be distributed and then stop the sender
+    vm4.invoke(() -> await()
+        .until(() -> WANTestBase.getSenderStats("ln", -1).get(4) > 0));
+
+    System.out.println("Stopping sender");
+    stopSenderInVMsAsync("ln", vm4, vm5);
+    System.out.println("Stopped sender");
+
+    inv1.await();
+    int entries =
+        transactionsPerCustomer * customers;
+
+    vm4.invoke(() -> WANTestBase.validateRegionSize(orderRegionName, entries));
+    vm5.invoke(() -> WANTestBase.validateRegionSize(orderRegionName, entries));
+
+    // Wait for events to replicate: when batches received does not change
+    // we can assume that replication has finished.
+    int batchesReceived = (vm2.invoke(() -> getReceiverStats())).get(0);
+    while (true) {
+      int oldBatchesReceived = batchesReceived;
+      Thread.sleep(1000);
+      batchesReceived = (vm2.invoke(() -> getReceiverStats())).get(0);
+      if (batchesReceived == oldBatchesReceived) {
+        break;
+      }
+    }
+
+    System.out.println("batchesReceived after stop: " + batchesReceived);
+
+    ArrayList<Integer> v4List =

Review comment:
       These should be declared as `List<Integer>`, which removes the need to cast them here and elsewhere.

##########
File path: geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/parallel/ParallelWANStatsDUnitTest.java
##########
@@ -472,6 +473,322 @@ public void testPRParallelPropagationWithGroupTransactionEventsSendsBatchesWithC
 
   }
 
+  @Test
+  public void testPRParallelPropagationWithGroupTransactionEventsDoesNotSendBatchesWithIncompleteTransactionsIfGatewaySenderIsStoppedWhileReceivingTrafficAndLaterStarted()
+      throws InterruptedException {
+    Integer lnPort = vm0.invoke(() -> WANTestBase.createFirstLocatorWithDSId(1));
+    Integer nyPort = vm1.invoke(() -> WANTestBase.createFirstRemoteLocator(2, lnPort));
+
+    createCacheInVMs(nyPort, vm2);
+    createReceiverInVMs(vm2);
+
+    createCacheInVMs(lnPort, vm4, vm5);
+    vm4.invoke(() -> WANTestBase.createSender("ln", 2, true, 100, 10, false, true, null, true,
+        true));
+    vm5.invoke(() -> WANTestBase.createSender("ln", 2, true, 100, 10, false, true, null, true,
+        true));
+
+    createReceiverCustomerOrderShipmentPR(vm2);
+
+    createSenderCustomerOrderShipmentPRs(vm4);
+    createSenderCustomerOrderShipmentPRs(vm5);
+
+    startSenderInVMs("ln", vm4, vm5);
+
+    int customers = 4;
+    int transactionsPerCustomer = 100;
+    int shipmentsPerTransaction = 10;
+    final LinkedHashMap<Object, Object> keyValuesInTransactions = new LinkedHashMap<>();
+    for (int custId = 0; custId < customers; custId++) {
+      for (int i = 0; i < transactionsPerCustomer; i++) {
+        CustId custIdObject = new CustId(custId);
+        OrderId orderId = new OrderId(i, custIdObject);
+        keyValuesInTransactions.put(orderId, new Order());
+        for (int j = 0; j < shipmentsPerTransaction; j++) {
+          ShipmentId shipmentId = new ShipmentId(i + j, orderId);
+          keyValuesInTransactions.put(shipmentId, new Shipment());
+        }
+      }
+    }
+    int eventsPerTransaction = 1 + shipmentsPerTransaction;
+    AsyncInvocation<Void> inv1 =
+        vm4.invokeAsync(
+            () -> WANTestBase.doOrderAndShipmentPutsInsideTransactions(keyValuesInTransactions,
+                eventsPerTransaction));
+
+    // wait for some batches to be distributed and then stop the sender
+    vm4.invoke(() -> await()
+        .until(() -> WANTestBase.getSenderStats("ln", -1).get(4) > 0));
+
+    System.out.println("Stopping sender");
+    stopSenderInVMsAsync("ln", vm4, vm5);
+    System.out.println("Stopped sender");
+
+    inv1.await();
+    int entries =
+        transactionsPerCustomer * customers;
+
+    vm4.invoke(() -> WANTestBase.validateRegionSize(orderRegionName, entries));
+    vm5.invoke(() -> WANTestBase.validateRegionSize(orderRegionName, entries));
+
+    // Wait for events to replicate: when batches received does not change
+    // we can assume that replication has finished.
+    int batchesReceived = (vm2.invoke(() -> getReceiverStats())).get(0);
+    while (true) {

Review comment:
       Is there a better way to determine that replication has finished? While loops with `Thread.sleep` are notorious for causing flakiness in tests. Perhaps checking/comparing the stats in each VM using `await()`?

##########
File path: geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/parallel/ParallelWANStatsDUnitTest.java
##########
@@ -472,6 +473,322 @@ public void testPRParallelPropagationWithGroupTransactionEventsSendsBatchesWithC
 
   }
 
+  @Test
+  public void testPRParallelPropagationWithGroupTransactionEventsDoesNotSendBatchesWithIncompleteTransactionsIfGatewaySenderIsStoppedWhileReceivingTrafficAndLaterStarted()
+      throws InterruptedException {
+    Integer lnPort = vm0.invoke(() -> WANTestBase.createFirstLocatorWithDSId(1));
+    Integer nyPort = vm1.invoke(() -> WANTestBase.createFirstRemoteLocator(2, lnPort));
+
+    createCacheInVMs(nyPort, vm2);
+    createReceiverInVMs(vm2);
+
+    createCacheInVMs(lnPort, vm4, vm5);
+    vm4.invoke(() -> WANTestBase.createSender("ln", 2, true, 100, 10, false, true, null, true,
+        true));
+    vm5.invoke(() -> WANTestBase.createSender("ln", 2, true, 100, 10, false, true, null, true,
+        true));
+
+    createReceiverCustomerOrderShipmentPR(vm2);
+
+    createSenderCustomerOrderShipmentPRs(vm4);
+    createSenderCustomerOrderShipmentPRs(vm5);
+
+    startSenderInVMs("ln", vm4, vm5);
+
+    int customers = 4;
+    int transactionsPerCustomer = 100;
+    int shipmentsPerTransaction = 10;
+    final LinkedHashMap<Object, Object> keyValuesInTransactions = new LinkedHashMap<>();
+    for (int custId = 0; custId < customers; custId++) {
+      for (int i = 0; i < transactionsPerCustomer; i++) {
+        CustId custIdObject = new CustId(custId);
+        OrderId orderId = new OrderId(i, custIdObject);
+        keyValuesInTransactions.put(orderId, new Order());
+        for (int j = 0; j < shipmentsPerTransaction; j++) {
+          ShipmentId shipmentId = new ShipmentId(i + j, orderId);
+          keyValuesInTransactions.put(shipmentId, new Shipment());
+        }
+      }
+    }
+    int eventsPerTransaction = 1 + shipmentsPerTransaction;
+    AsyncInvocation<Void> inv1 =
+        vm4.invokeAsync(
+            () -> WANTestBase.doOrderAndShipmentPutsInsideTransactions(keyValuesInTransactions,
+                eventsPerTransaction));
+
+    // wait for some batches to be distributed and then stop the sender
+    vm4.invoke(() -> await()
+        .until(() -> WANTestBase.getSenderStats("ln", -1).get(4) > 0));
+
+    System.out.println("Stopping sender");
+    stopSenderInVMsAsync("ln", vm4, vm5);
+    System.out.println("Stopped sender");
+
+    inv1.await();
+    int entries =
+        transactionsPerCustomer * customers;
+
+    vm4.invoke(() -> WANTestBase.validateRegionSize(orderRegionName, entries));
+    vm5.invoke(() -> WANTestBase.validateRegionSize(orderRegionName, entries));
+
+    // Wait for events to replicate: when batches received does not change
+    // we can assume that replication has finished.
+    int batchesReceived = (vm2.invoke(() -> getReceiverStats())).get(0);
+    while (true) {
+      int oldBatchesReceived = batchesReceived;
+      Thread.sleep(1000);
+      batchesReceived = (vm2.invoke(() -> getReceiverStats())).get(0);
+      if (batchesReceived == oldBatchesReceived) {
+        break;
+      }
+    }
+
+    System.out.println("batchesReceived after stop: " + batchesReceived);
+
+    ArrayList<Integer> v4List =
+        (ArrayList<Integer>) vm4.invoke(() -> WANTestBase.getSenderStats("ln", -1));
+    ArrayList<Integer> v5List =
+        (ArrayList<Integer>) vm5.invoke(() -> WANTestBase.getSenderStats("ln", -1));
+
+    // Wait for events to replicate: when region size does not change
+    // we can assume that replication has finished.
+    int orderRegionSize = vm2.invoke(() -> getRegionSize(orderRegionName));
+    while (true) {
+      Thread.sleep(1000);
+      int newRegionSize = vm2.invoke(() -> getRegionSize(orderRegionName));
+      if (orderRegionSize == newRegionSize) {
+        break;
+      }
+    }
+
+    // Wait for events to replicate: when region size does not change
+    // we can assume that replication has finished.
+    int shipmentRegionSize = vm2.invoke(() -> getRegionSize(shipmentRegionName));
+    while (true) {

Review comment:
       See earlier comment about avoiding using `Thread.sleep` in tests.

##########
File path: geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/parallel/ParallelWANStatsDUnitTest.java
##########
@@ -472,6 +473,322 @@ public void testPRParallelPropagationWithGroupTransactionEventsSendsBatchesWithC
 
   }
 
+  @Test
+  public void testPRParallelPropagationWithGroupTransactionEventsDoesNotSendBatchesWithIncompleteTransactionsIfGatewaySenderIsStoppedWhileReceivingTrafficAndLaterStarted()
+      throws InterruptedException {
+    Integer lnPort = vm0.invoke(() -> WANTestBase.createFirstLocatorWithDSId(1));
+    Integer nyPort = vm1.invoke(() -> WANTestBase.createFirstRemoteLocator(2, lnPort));
+
+    createCacheInVMs(nyPort, vm2);
+    createReceiverInVMs(vm2);
+
+    createCacheInVMs(lnPort, vm4, vm5);
+    vm4.invoke(() -> WANTestBase.createSender("ln", 2, true, 100, 10, false, true, null, true,
+        true));
+    vm5.invoke(() -> WANTestBase.createSender("ln", 2, true, 100, 10, false, true, null, true,
+        true));
+
+    createReceiverCustomerOrderShipmentPR(vm2);
+
+    createSenderCustomerOrderShipmentPRs(vm4);
+    createSenderCustomerOrderShipmentPRs(vm5);
+
+    startSenderInVMs("ln", vm4, vm5);
+
+    int customers = 4;
+    int transactionsPerCustomer = 100;
+    int shipmentsPerTransaction = 10;
+    final LinkedHashMap<Object, Object> keyValuesInTransactions = new LinkedHashMap<>();
+    for (int custId = 0; custId < customers; custId++) {
+      for (int i = 0; i < transactionsPerCustomer; i++) {
+        CustId custIdObject = new CustId(custId);
+        OrderId orderId = new OrderId(i, custIdObject);
+        keyValuesInTransactions.put(orderId, new Order());
+        for (int j = 0; j < shipmentsPerTransaction; j++) {
+          ShipmentId shipmentId = new ShipmentId(i + j, orderId);
+          keyValuesInTransactions.put(shipmentId, new Shipment());
+        }
+      }
+    }
+    int eventsPerTransaction = 1 + shipmentsPerTransaction;
+    AsyncInvocation<Void> inv1 =
+        vm4.invokeAsync(
+            () -> WANTestBase.doOrderAndShipmentPutsInsideTransactions(keyValuesInTransactions,
+                eventsPerTransaction));
+
+    // wait for some batches to be distributed and then stop the sender
+    vm4.invoke(() -> await()
+        .until(() -> WANTestBase.getSenderStats("ln", -1).get(4) > 0));
+
+    System.out.println("Stopping sender");
+    stopSenderInVMsAsync("ln", vm4, vm5);
+    System.out.println("Stopped sender");
+
+    inv1.await();
+    int entries =
+        transactionsPerCustomer * customers;
+
+    vm4.invoke(() -> WANTestBase.validateRegionSize(orderRegionName, entries));
+    vm5.invoke(() -> WANTestBase.validateRegionSize(orderRegionName, entries));
+
+    // Wait for events to replicate: when batches received does not change
+    // we can assume that replication has finished.
+    int batchesReceived = (vm2.invoke(() -> getReceiverStats())).get(0);
+    while (true) {
+      int oldBatchesReceived = batchesReceived;
+      Thread.sleep(1000);
+      batchesReceived = (vm2.invoke(() -> getReceiverStats())).get(0);
+      if (batchesReceived == oldBatchesReceived) {
+        break;
+      }
+    }
+
+    System.out.println("batchesReceived after stop: " + batchesReceived);
+
+    ArrayList<Integer> v4List =
+        (ArrayList<Integer>) vm4.invoke(() -> WANTestBase.getSenderStats("ln", -1));
+    ArrayList<Integer> v5List =
+        (ArrayList<Integer>) vm5.invoke(() -> WANTestBase.getSenderStats("ln", -1));
+
+    // Wait for events to replicate: when region size does not change
+    // we can assume that replication has finished.
+    int orderRegionSize = vm2.invoke(() -> getRegionSize(orderRegionName));
+    while (true) {

Review comment:
       See earlier comment about avoiding using `Thread.sleep` in tests.

##########
File path: geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/parallel/ParallelWANStatsDUnitTest.java
##########
@@ -472,6 +473,322 @@ public void testPRParallelPropagationWithGroupTransactionEventsSendsBatchesWithC
 
   }
 
+  @Test
+  public void testPRParallelPropagationWithGroupTransactionEventsDoesNotSendBatchesWithIncompleteTransactionsIfGatewaySenderIsStoppedWhileReceivingTrafficAndLaterStarted()
+      throws InterruptedException {
+    Integer lnPort = vm0.invoke(() -> WANTestBase.createFirstLocatorWithDSId(1));
+    Integer nyPort = vm1.invoke(() -> WANTestBase.createFirstRemoteLocator(2, lnPort));
+
+    createCacheInVMs(nyPort, vm2);
+    createReceiverInVMs(vm2);
+
+    createCacheInVMs(lnPort, vm4, vm5);
+    vm4.invoke(() -> WANTestBase.createSender("ln", 2, true, 100, 10, false, true, null, true,
+        true));
+    vm5.invoke(() -> WANTestBase.createSender("ln", 2, true, 100, 10, false, true, null, true,
+        true));
+
+    createReceiverCustomerOrderShipmentPR(vm2);
+
+    createSenderCustomerOrderShipmentPRs(vm4);
+    createSenderCustomerOrderShipmentPRs(vm5);
+
+    startSenderInVMs("ln", vm4, vm5);
+
+    int customers = 4;
+    int transactionsPerCustomer = 100;
+    int shipmentsPerTransaction = 10;
+    final LinkedHashMap<Object, Object> keyValuesInTransactions = new LinkedHashMap<>();
+    for (int custId = 0; custId < customers; custId++) {
+      for (int i = 0; i < transactionsPerCustomer; i++) {
+        CustId custIdObject = new CustId(custId);
+        OrderId orderId = new OrderId(i, custIdObject);
+        keyValuesInTransactions.put(orderId, new Order());
+        for (int j = 0; j < shipmentsPerTransaction; j++) {
+          ShipmentId shipmentId = new ShipmentId(i + j, orderId);
+          keyValuesInTransactions.put(shipmentId, new Shipment());
+        }
+      }
+    }
+    int eventsPerTransaction = 1 + shipmentsPerTransaction;
+    AsyncInvocation<Void> inv1 =
+        vm4.invokeAsync(
+            () -> WANTestBase.doOrderAndShipmentPutsInsideTransactions(keyValuesInTransactions,
+                eventsPerTransaction));
+
+    // wait for some batches to be distributed and then stop the sender
+    vm4.invoke(() -> await()
+        .until(() -> WANTestBase.getSenderStats("ln", -1).get(4) > 0));
+
+    System.out.println("Stopping sender");
+    stopSenderInVMsAsync("ln", vm4, vm5);
+    System.out.println("Stopped sender");
+
+    inv1.await();
+    int entries =
+        transactionsPerCustomer * customers;
+
+    vm4.invoke(() -> WANTestBase.validateRegionSize(orderRegionName, entries));
+    vm5.invoke(() -> WANTestBase.validateRegionSize(orderRegionName, entries));
+
+    // Wait for events to replicate: when batches received does not change
+    // we can assume that replication has finished.
+    int batchesReceived = (vm2.invoke(() -> getReceiverStats())).get(0);
+    while (true) {
+      int oldBatchesReceived = batchesReceived;
+      Thread.sleep(1000);
+      batchesReceived = (vm2.invoke(() -> getReceiverStats())).get(0);
+      if (batchesReceived == oldBatchesReceived) {
+        break;
+      }
+    }
+
+    System.out.println("batchesReceived after stop: " + batchesReceived);
+
+    ArrayList<Integer> v4List =
+        (ArrayList<Integer>) vm4.invoke(() -> WANTestBase.getSenderStats("ln", -1));
+    ArrayList<Integer> v5List =
+        (ArrayList<Integer>) vm5.invoke(() -> WANTestBase.getSenderStats("ln", -1));
+
+    // Wait for events to replicate: when region size does not change
+    // we can assume that replication has finished.
+    int orderRegionSize = vm2.invoke(() -> getRegionSize(orderRegionName));
+    while (true) {
+      Thread.sleep(1000);
+      int newRegionSize = vm2.invoke(() -> getRegionSize(orderRegionName));
+      if (orderRegionSize == newRegionSize) {
+        break;
+      }
+    }
+
+    // Wait for events to replicate: when region size does not change
+    // we can assume that replication has finished.
+    int shipmentRegionSize = vm2.invoke(() -> getRegionSize(shipmentRegionName));
+    while (true) {
+      Thread.sleep(1000);
+      int newRegionSize = vm2.invoke(() -> getRegionSize(shipmentRegionName));
+      if (shipmentRegionSize == newRegionSize) {
+        break;
+      }
+    }
+
+    System.out.println("orderRegionSize: " + shipmentRegionSize);
+    System.out.println("shipmentRegionSize: " + shipmentRegionSize);
+
+    System.out.println("v4List.get(0): " + v4List.get(0));
+    System.out.println("v5List.get(0): " + v5List.get(0));
+
+    // batches with incomplete transactions must be 0
+    assertEquals(0, (int) v4List.get(13));
+    assertEquals(0, (int) v5List.get(13));
+
+    // Only complete transactions (1 order + 10 shipments) must be replicated
+    assertEquals(10, shipmentRegionSize / orderRegionSize);
+    assertEquals(0, shipmentRegionSize % orderRegionSize);
+
+    System.out.println("Starting sender");

Review comment:
       Another print statement.

##########
File path: geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/parallel/ParallelWANStatsDUnitTest.java
##########
@@ -472,6 +473,322 @@ public void testPRParallelPropagationWithGroupTransactionEventsSendsBatchesWithC
 
   }
 
+  @Test
+  public void testPRParallelPropagationWithGroupTransactionEventsDoesNotSendBatchesWithIncompleteTransactionsIfGatewaySenderIsStoppedWhileReceivingTrafficAndLaterStarted()
+      throws InterruptedException {
+    Integer lnPort = vm0.invoke(() -> WANTestBase.createFirstLocatorWithDSId(1));
+    Integer nyPort = vm1.invoke(() -> WANTestBase.createFirstRemoteLocator(2, lnPort));
+
+    createCacheInVMs(nyPort, vm2);
+    createReceiverInVMs(vm2);
+
+    createCacheInVMs(lnPort, vm4, vm5);
+    vm4.invoke(() -> WANTestBase.createSender("ln", 2, true, 100, 10, false, true, null, true,
+        true));
+    vm5.invoke(() -> WANTestBase.createSender("ln", 2, true, 100, 10, false, true, null, true,
+        true));
+
+    createReceiverCustomerOrderShipmentPR(vm2);
+
+    createSenderCustomerOrderShipmentPRs(vm4);
+    createSenderCustomerOrderShipmentPRs(vm5);
+
+    startSenderInVMs("ln", vm4, vm5);
+
+    int customers = 4;
+    int transactionsPerCustomer = 100;
+    int shipmentsPerTransaction = 10;
+    final LinkedHashMap<Object, Object> keyValuesInTransactions = new LinkedHashMap<>();
+    for (int custId = 0; custId < customers; custId++) {
+      for (int i = 0; i < transactionsPerCustomer; i++) {
+        CustId custIdObject = new CustId(custId);
+        OrderId orderId = new OrderId(i, custIdObject);
+        keyValuesInTransactions.put(orderId, new Order());
+        for (int j = 0; j < shipmentsPerTransaction; j++) {
+          ShipmentId shipmentId = new ShipmentId(i + j, orderId);
+          keyValuesInTransactions.put(shipmentId, new Shipment());
+        }
+      }
+    }
+    int eventsPerTransaction = 1 + shipmentsPerTransaction;
+    AsyncInvocation<Void> inv1 =
+        vm4.invokeAsync(
+            () -> WANTestBase.doOrderAndShipmentPutsInsideTransactions(keyValuesInTransactions,
+                eventsPerTransaction));
+
+    // wait for some batches to be distributed and then stop the sender
+    vm4.invoke(() -> await()
+        .until(() -> WANTestBase.getSenderStats("ln", -1).get(4) > 0));
+
+    System.out.println("Stopping sender");
+    stopSenderInVMsAsync("ln", vm4, vm5);
+    System.out.println("Stopped sender");
+
+    inv1.await();
+    int entries =
+        transactionsPerCustomer * customers;
+
+    vm4.invoke(() -> WANTestBase.validateRegionSize(orderRegionName, entries));
+    vm5.invoke(() -> WANTestBase.validateRegionSize(orderRegionName, entries));
+
+    // Wait for events to replicate: when batches received does not change
+    // we can assume that replication has finished.
+    int batchesReceived = (vm2.invoke(() -> getReceiverStats())).get(0);
+    while (true) {
+      int oldBatchesReceived = batchesReceived;
+      Thread.sleep(1000);
+      batchesReceived = (vm2.invoke(() -> getReceiverStats())).get(0);
+      if (batchesReceived == oldBatchesReceived) {
+        break;
+      }
+    }
+
+    System.out.println("batchesReceived after stop: " + batchesReceived);
+
+    ArrayList<Integer> v4List =
+        (ArrayList<Integer>) vm4.invoke(() -> WANTestBase.getSenderStats("ln", -1));
+    ArrayList<Integer> v5List =
+        (ArrayList<Integer>) vm5.invoke(() -> WANTestBase.getSenderStats("ln", -1));
+
+    // Wait for events to replicate: when region size does not change
+    // we can assume that replication has finished.
+    int orderRegionSize = vm2.invoke(() -> getRegionSize(orderRegionName));
+    while (true) {
+      Thread.sleep(1000);
+      int newRegionSize = vm2.invoke(() -> getRegionSize(orderRegionName));
+      if (orderRegionSize == newRegionSize) {
+        break;
+      }
+    }
+
+    // Wait for events to replicate: when region size does not change
+    // we can assume that replication has finished.
+    int shipmentRegionSize = vm2.invoke(() -> getRegionSize(shipmentRegionName));
+    while (true) {
+      Thread.sleep(1000);
+      int newRegionSize = vm2.invoke(() -> getRegionSize(shipmentRegionName));
+      if (shipmentRegionSize == newRegionSize) {
+        break;
+      }
+    }
+
+    System.out.println("orderRegionSize: " + shipmentRegionSize);
+    System.out.println("shipmentRegionSize: " + shipmentRegionSize);
+
+    System.out.println("v4List.get(0): " + v4List.get(0));
+    System.out.println("v5List.get(0): " + v5List.get(0));
+
+    // batches with incomplete transactions must be 0
+    assertEquals(0, (int) v4List.get(13));
+    assertEquals(0, (int) v5List.get(13));
+
+    // Only complete transactions (1 order + 10 shipments) must be replicated
+    assertEquals(10, shipmentRegionSize / orderRegionSize);

Review comment:
       Could these two asserts be replaced with one, asserting that `shipmentRegionSize` is equal to `10 * orderRegionSize`?

##########
File path: geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/parallel/ParallelWANStatsDUnitTest.java
##########
@@ -472,6 +473,322 @@ public void testPRParallelPropagationWithGroupTransactionEventsSendsBatchesWithC
 
   }
 
+  @Test
+  public void testPRParallelPropagationWithGroupTransactionEventsDoesNotSendBatchesWithIncompleteTransactionsIfGatewaySenderIsStoppedWhileReceivingTrafficAndLaterStarted()
+      throws InterruptedException {
+    Integer lnPort = vm0.invoke(() -> WANTestBase.createFirstLocatorWithDSId(1));
+    Integer nyPort = vm1.invoke(() -> WANTestBase.createFirstRemoteLocator(2, lnPort));
+
+    createCacheInVMs(nyPort, vm2);
+    createReceiverInVMs(vm2);
+
+    createCacheInVMs(lnPort, vm4, vm5);
+    vm4.invoke(() -> WANTestBase.createSender("ln", 2, true, 100, 10, false, true, null, true,
+        true));
+    vm5.invoke(() -> WANTestBase.createSender("ln", 2, true, 100, 10, false, true, null, true,
+        true));
+
+    createReceiverCustomerOrderShipmentPR(vm2);
+
+    createSenderCustomerOrderShipmentPRs(vm4);
+    createSenderCustomerOrderShipmentPRs(vm5);
+
+    startSenderInVMs("ln", vm4, vm5);
+
+    int customers = 4;
+    int transactionsPerCustomer = 100;
+    int shipmentsPerTransaction = 10;
+    final LinkedHashMap<Object, Object> keyValuesInTransactions = new LinkedHashMap<>();
+    for (int custId = 0; custId < customers; custId++) {
+      for (int i = 0; i < transactionsPerCustomer; i++) {
+        CustId custIdObject = new CustId(custId);
+        OrderId orderId = new OrderId(i, custIdObject);
+        keyValuesInTransactions.put(orderId, new Order());
+        for (int j = 0; j < shipmentsPerTransaction; j++) {
+          ShipmentId shipmentId = new ShipmentId(i + j, orderId);
+          keyValuesInTransactions.put(shipmentId, new Shipment());
+        }
+      }
+    }
+    int eventsPerTransaction = 1 + shipmentsPerTransaction;
+    AsyncInvocation<Void> inv1 =
+        vm4.invokeAsync(
+            () -> WANTestBase.doOrderAndShipmentPutsInsideTransactions(keyValuesInTransactions,
+                eventsPerTransaction));
+
+    // wait for some batches to be distributed and then stop the sender
+    vm4.invoke(() -> await()
+        .until(() -> WANTestBase.getSenderStats("ln", -1).get(4) > 0));
+
+    System.out.println("Stopping sender");
+    stopSenderInVMsAsync("ln", vm4, vm5);
+    System.out.println("Stopped sender");
+
+    inv1.await();
+    int entries =
+        transactionsPerCustomer * customers;
+
+    vm4.invoke(() -> WANTestBase.validateRegionSize(orderRegionName, entries));
+    vm5.invoke(() -> WANTestBase.validateRegionSize(orderRegionName, entries));
+
+    // Wait for events to replicate: when batches received does not change
+    // we can assume that replication has finished.
+    int batchesReceived = (vm2.invoke(() -> getReceiverStats())).get(0);
+    while (true) {
+      int oldBatchesReceived = batchesReceived;
+      Thread.sleep(1000);
+      batchesReceived = (vm2.invoke(() -> getReceiverStats())).get(0);
+      if (batchesReceived == oldBatchesReceived) {
+        break;
+      }
+    }
+
+    System.out.println("batchesReceived after stop: " + batchesReceived);
+
+    ArrayList<Integer> v4List =
+        (ArrayList<Integer>) vm4.invoke(() -> WANTestBase.getSenderStats("ln", -1));
+    ArrayList<Integer> v5List =
+        (ArrayList<Integer>) vm5.invoke(() -> WANTestBase.getSenderStats("ln", -1));
+
+    // Wait for events to replicate: when region size does not change
+    // we can assume that replication has finished.
+    int orderRegionSize = vm2.invoke(() -> getRegionSize(orderRegionName));
+    while (true) {
+      Thread.sleep(1000);
+      int newRegionSize = vm2.invoke(() -> getRegionSize(orderRegionName));
+      if (orderRegionSize == newRegionSize) {
+        break;
+      }
+    }
+
+    // Wait for events to replicate: when region size does not change
+    // we can assume that replication has finished.
+    int shipmentRegionSize = vm2.invoke(() -> getRegionSize(shipmentRegionName));
+    while (true) {
+      Thread.sleep(1000);
+      int newRegionSize = vm2.invoke(() -> getRegionSize(shipmentRegionName));
+      if (shipmentRegionSize == newRegionSize) {
+        break;
+      }
+    }
+
+    System.out.println("orderRegionSize: " + shipmentRegionSize);
+    System.out.println("shipmentRegionSize: " + shipmentRegionSize);
+
+    System.out.println("v4List.get(0): " + v4List.get(0));
+    System.out.println("v5List.get(0): " + v5List.get(0));
+
+    // batches with incomplete transactions must be 0
+    assertEquals(0, (int) v4List.get(13));
+    assertEquals(0, (int) v5List.get(13));
+
+    // Only complete transactions (1 order + 10 shipments) must be replicated
+    assertEquals(10, shipmentRegionSize / orderRegionSize);
+    assertEquals(0, shipmentRegionSize % orderRegionSize);
+
+    System.out.println("Starting sender");
+    startSenderInVMsAsync("ln", vm4, vm5);
+
+    vm4.invoke(() -> WANTestBase.validateParallelSenderQueueAllBucketsDrained("ln"));
+    vm5.invoke(() -> WANTestBase.validateParallelSenderQueueAllBucketsDrained("ln"));
+
+    v4List =
+        (ArrayList<Integer>) vm4.invoke(() -> WANTestBase.getSenderStats("ln", 0));
+    v5List =
+        (ArrayList<Integer>) vm5.invoke(() -> WANTestBase.getSenderStats("ln", 0));
+
+    assertEquals(0, v4List.get(0) + v5List.get(0));
+
+    // Wait for events to replicate: when batches received does not change
+    // we can assume that replication has finished.
+    batchesReceived = (vm2.invoke(() -> getReceiverStats())).get(0);
+    while (true) {

Review comment:
       See earlier comments about not using `Thread.sleep` in tests.

##########
File path: geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/serial/SerialWANStatsDUnitTest.java
##########
@@ -348,7 +349,276 @@ public void testReplicatedSerialPropagationWithBatchRedistWithGroupTransactionEv
   }
 
   @Test
-  public void testReplicatedSerialPropagationWithMultipleDispatchers() throws Exception {
+  public void testReplicatedSerialPropagationWithGroupTransactionEventsDoesNotSendBatchesWithIncompleteTransactionsIfGatewaySenderIsStoppedWhileReceivingTrafficAndLaterStarted()

Review comment:
       There is a lot of duplicated code between this test and the other added to this class. It should be pulled out into methods to avoid duplication.

##########
File path: geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/parallel/ParallelWANStatsDUnitTest.java
##########
@@ -472,6 +473,322 @@ public void testPRParallelPropagationWithGroupTransactionEventsSendsBatchesWithC
 
   }
 
+  @Test
+  public void testPRParallelPropagationWithGroupTransactionEventsDoesNotSendBatchesWithIncompleteTransactionsIfGatewaySenderIsStoppedWhileReceivingTrafficAndLaterStarted()
+      throws InterruptedException {
+    Integer lnPort = vm0.invoke(() -> WANTestBase.createFirstLocatorWithDSId(1));
+    Integer nyPort = vm1.invoke(() -> WANTestBase.createFirstRemoteLocator(2, lnPort));
+
+    createCacheInVMs(nyPort, vm2);
+    createReceiverInVMs(vm2);
+
+    createCacheInVMs(lnPort, vm4, vm5);
+    vm4.invoke(() -> WANTestBase.createSender("ln", 2, true, 100, 10, false, true, null, true,
+        true));
+    vm5.invoke(() -> WANTestBase.createSender("ln", 2, true, 100, 10, false, true, null, true,
+        true));
+
+    createReceiverCustomerOrderShipmentPR(vm2);
+
+    createSenderCustomerOrderShipmentPRs(vm4);
+    createSenderCustomerOrderShipmentPRs(vm5);
+
+    startSenderInVMs("ln", vm4, vm5);
+
+    int customers = 4;
+    int transactionsPerCustomer = 100;
+    int shipmentsPerTransaction = 10;
+    final LinkedHashMap<Object, Object> keyValuesInTransactions = new LinkedHashMap<>();
+    for (int custId = 0; custId < customers; custId++) {
+      for (int i = 0; i < transactionsPerCustomer; i++) {
+        CustId custIdObject = new CustId(custId);
+        OrderId orderId = new OrderId(i, custIdObject);
+        keyValuesInTransactions.put(orderId, new Order());
+        for (int j = 0; j < shipmentsPerTransaction; j++) {
+          ShipmentId shipmentId = new ShipmentId(i + j, orderId);
+          keyValuesInTransactions.put(shipmentId, new Shipment());
+        }
+      }
+    }
+    int eventsPerTransaction = 1 + shipmentsPerTransaction;
+    AsyncInvocation<Void> inv1 =
+        vm4.invokeAsync(
+            () -> WANTestBase.doOrderAndShipmentPutsInsideTransactions(keyValuesInTransactions,
+                eventsPerTransaction));
+
+    // wait for some batches to be distributed and then stop the sender
+    vm4.invoke(() -> await()
+        .until(() -> WANTestBase.getSenderStats("ln", -1).get(4) > 0));
+
+    System.out.println("Stopping sender");
+    stopSenderInVMsAsync("ln", vm4, vm5);
+    System.out.println("Stopped sender");
+
+    inv1.await();
+    int entries =
+        transactionsPerCustomer * customers;
+
+    vm4.invoke(() -> WANTestBase.validateRegionSize(orderRegionName, entries));
+    vm5.invoke(() -> WANTestBase.validateRegionSize(orderRegionName, entries));
+
+    // Wait for events to replicate: when batches received does not change
+    // we can assume that replication has finished.
+    int batchesReceived = (vm2.invoke(() -> getReceiverStats())).get(0);
+    while (true) {
+      int oldBatchesReceived = batchesReceived;
+      Thread.sleep(1000);
+      batchesReceived = (vm2.invoke(() -> getReceiverStats())).get(0);
+      if (batchesReceived == oldBatchesReceived) {
+        break;
+      }
+    }
+
+    System.out.println("batchesReceived after stop: " + batchesReceived);
+
+    ArrayList<Integer> v4List =
+        (ArrayList<Integer>) vm4.invoke(() -> WANTestBase.getSenderStats("ln", -1));
+    ArrayList<Integer> v5List =
+        (ArrayList<Integer>) vm5.invoke(() -> WANTestBase.getSenderStats("ln", -1));
+
+    // Wait for events to replicate: when region size does not change
+    // we can assume that replication has finished.
+    int orderRegionSize = vm2.invoke(() -> getRegionSize(orderRegionName));
+    while (true) {
+      Thread.sleep(1000);
+      int newRegionSize = vm2.invoke(() -> getRegionSize(orderRegionName));
+      if (orderRegionSize == newRegionSize) {
+        break;
+      }
+    }
+
+    // Wait for events to replicate: when region size does not change
+    // we can assume that replication has finished.
+    int shipmentRegionSize = vm2.invoke(() -> getRegionSize(shipmentRegionName));
+    while (true) {
+      Thread.sleep(1000);
+      int newRegionSize = vm2.invoke(() -> getRegionSize(shipmentRegionName));
+      if (shipmentRegionSize == newRegionSize) {
+        break;
+      }
+    }
+
+    System.out.println("orderRegionSize: " + shipmentRegionSize);
+    System.out.println("shipmentRegionSize: " + shipmentRegionSize);
+
+    System.out.println("v4List.get(0): " + v4List.get(0));
+    System.out.println("v5List.get(0): " + v5List.get(0));
+
+    // batches with incomplete transactions must be 0
+    assertEquals(0, (int) v4List.get(13));
+    assertEquals(0, (int) v5List.get(13));
+
+    // Only complete transactions (1 order + 10 shipments) must be replicated
+    assertEquals(10, shipmentRegionSize / orderRegionSize);
+    assertEquals(0, shipmentRegionSize % orderRegionSize);
+
+    System.out.println("Starting sender");
+    startSenderInVMsAsync("ln", vm4, vm5);
+
+    vm4.invoke(() -> WANTestBase.validateParallelSenderQueueAllBucketsDrained("ln"));
+    vm5.invoke(() -> WANTestBase.validateParallelSenderQueueAllBucketsDrained("ln"));
+
+    v4List =
+        (ArrayList<Integer>) vm4.invoke(() -> WANTestBase.getSenderStats("ln", 0));
+    v5List =
+        (ArrayList<Integer>) vm5.invoke(() -> WANTestBase.getSenderStats("ln", 0));
+
+    assertEquals(0, v4List.get(0) + v5List.get(0));
+
+    // Wait for events to replicate: when batches received does not change
+    // we can assume that replication has finished.
+    batchesReceived = (vm2.invoke(() -> getReceiverStats())).get(0);
+    while (true) {
+      int oldBatchesReceived = batchesReceived;
+      Thread.sleep(1000);
+      batchesReceived = (vm2.invoke(() -> getReceiverStats())).get(0);
+      if (batchesReceived == oldBatchesReceived) {
+        break;
+      }
+    }
+
+    System.out.println("batchesReceived after restart: " + batchesReceived);
+
+    // Wait for events to replicate: when region size does not change
+    // we can assume that replication has finished.
+    orderRegionSize = vm2.invoke(() -> getRegionSize(orderRegionName));
+    while (true) {
+      Thread.sleep(1000);
+      int newRegionSize = vm2.invoke(() -> getRegionSize(orderRegionName));
+      if (orderRegionSize == newRegionSize) {
+        break;
+      }
+    }
+
+    // Wait for events to replicate: when region size does not change
+    // we can assume that replication has finished.
+    shipmentRegionSize = vm2.invoke(() -> getRegionSize(shipmentRegionName));
+    while (true) {
+      Thread.sleep(1000);
+      int newRegionSize = vm2.invoke(() -> getRegionSize(shipmentRegionName));
+      if (shipmentRegionSize == newRegionSize) {
+        break;
+      }
+    }
+
+    System.out.println("orderRegionSize after restart: " + shipmentRegionSize);

Review comment:
       More print statements.

##########
File path: geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/serial/SerialWANStatsDUnitTest.java
##########
@@ -348,7 +349,276 @@ public void testReplicatedSerialPropagationWithBatchRedistWithGroupTransactionEv
   }
 
   @Test
-  public void testReplicatedSerialPropagationWithMultipleDispatchers() throws Exception {
+  public void testReplicatedSerialPropagationWithGroupTransactionEventsDoesNotSendBatchesWithIncompleteTransactionsIfGatewaySenderIsStoppedWhileReceivingTrafficAndLaterStarted()
+      throws InterruptedException {
+    Integer lnPort = vm0.invoke(() -> WANTestBase.createFirstLocatorWithDSId(1));
+    Integer nyPort = vm1.invoke(() -> WANTestBase.createFirstRemoteLocator(2, lnPort));
+
+    String regionName = testName + "_RR";
+
+    createCacheInVMs(nyPort, vm2);
+    createReceiverInVMs(vm2);
+
+    createCacheInVMs(lnPort, vm4, vm5);
+    boolean groupTransactionEvents = true;
+    int batchSize = 10;
+    vm4.invoke(
+        () -> WANTestBase.createSender("ln", 2, false, 100, batchSize, false, true, null, true,
+            groupTransactionEvents));
+    vm5.invoke(
+        () -> WANTestBase.createSender("ln", 2, false, 100, batchSize, false, true, null, true,
+            groupTransactionEvents));
+
+    vm2.invoke(() -> WANTestBase.createReplicatedRegion(regionName, null, isOffHeap()));
+
+    vm4.invoke(() -> WANTestBase.createReplicatedRegion(regionName, "ln", isOffHeap()));
+    vm5.invoke(() -> WANTestBase.createReplicatedRegion(regionName, "ln", isOffHeap()));
+
+    startSenderInVMs("ln", vm4, vm5);
+
+    final Map<Object, Object> keyValues = new LinkedHashMap<>();
+    int entries = 2200;
+    for (int i = 0; i < entries; i++) {
+      keyValues.put(i, i + "_Value");
+    }
+
+    int eventsPerTransaction = 11;
+    System.out.println("Starting puts");

Review comment:
       Another print statement that should be removed. In this case, a description can be added as the first argument to the `invokeAsync()` call below to help with debugging.

##########
File path: geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/serial/SerialWANStatsDUnitTest.java
##########
@@ -348,7 +349,276 @@ public void testReplicatedSerialPropagationWithBatchRedistWithGroupTransactionEv
   }
 
   @Test
-  public void testReplicatedSerialPropagationWithMultipleDispatchers() throws Exception {
+  public void testReplicatedSerialPropagationWithGroupTransactionEventsDoesNotSendBatchesWithIncompleteTransactionsIfGatewaySenderIsStoppedWhileReceivingTrafficAndLaterStarted()
+      throws InterruptedException {
+    Integer lnPort = vm0.invoke(() -> WANTestBase.createFirstLocatorWithDSId(1));
+    Integer nyPort = vm1.invoke(() -> WANTestBase.createFirstRemoteLocator(2, lnPort));
+
+    String regionName = testName + "_RR";
+
+    createCacheInVMs(nyPort, vm2);
+    createReceiverInVMs(vm2);
+
+    createCacheInVMs(lnPort, vm4, vm5);
+    boolean groupTransactionEvents = true;
+    int batchSize = 10;
+    vm4.invoke(
+        () -> WANTestBase.createSender("ln", 2, false, 100, batchSize, false, true, null, true,
+            groupTransactionEvents));
+    vm5.invoke(
+        () -> WANTestBase.createSender("ln", 2, false, 100, batchSize, false, true, null, true,
+            groupTransactionEvents));
+
+    vm2.invoke(() -> WANTestBase.createReplicatedRegion(regionName, null, isOffHeap()));
+
+    vm4.invoke(() -> WANTestBase.createReplicatedRegion(regionName, "ln", isOffHeap()));
+    vm5.invoke(() -> WANTestBase.createReplicatedRegion(regionName, "ln", isOffHeap()));
+
+    startSenderInVMs("ln", vm4, vm5);
+
+    final Map<Object, Object> keyValues = new LinkedHashMap<>();
+    int entries = 2200;
+    for (int i = 0; i < entries; i++) {
+      keyValues.put(i, i + "_Value");
+    }
+
+    int eventsPerTransaction = 11;
+    System.out.println("Starting puts");
+    AsyncInvocation<Void> inv1 =
+        vm4.invokeAsync(
+            () -> WANTestBase.doPutsInsideTransactions(regionName, keyValues,
+                eventsPerTransaction));
+
+    // wait for batches to be distributed and then stop the sender
+    System.out.println("Waiting for some batches to be distributed");
+    vm4.invoke(() -> await()
+        .until(() -> WANTestBase.getSenderStats("ln", -1).get(4) > 0));
+    System.out
+        .println("Some batches distributed: " + vm4.invoke(() -> getSenderStats("ln", -1).get(4)));
+
+    addIgnoredException("Exception occurred in CacheListener");

Review comment:
       It would be helpful to add a comment explaining why these exceptions can be ignored here.

##########
File path: geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/serial/SerialWANStatsDUnitTest.java
##########
@@ -348,7 +349,276 @@ public void testReplicatedSerialPropagationWithBatchRedistWithGroupTransactionEv
   }
 
   @Test
-  public void testReplicatedSerialPropagationWithMultipleDispatchers() throws Exception {
+  public void testReplicatedSerialPropagationWithGroupTransactionEventsDoesNotSendBatchesWithIncompleteTransactionsIfGatewaySenderIsStoppedWhileReceivingTrafficAndLaterStarted()
+      throws InterruptedException {
+    Integer lnPort = vm0.invoke(() -> WANTestBase.createFirstLocatorWithDSId(1));
+    Integer nyPort = vm1.invoke(() -> WANTestBase.createFirstRemoteLocator(2, lnPort));
+
+    String regionName = testName + "_RR";
+
+    createCacheInVMs(nyPort, vm2);
+    createReceiverInVMs(vm2);
+
+    createCacheInVMs(lnPort, vm4, vm5);
+    boolean groupTransactionEvents = true;
+    int batchSize = 10;
+    vm4.invoke(
+        () -> WANTestBase.createSender("ln", 2, false, 100, batchSize, false, true, null, true,
+            groupTransactionEvents));
+    vm5.invoke(
+        () -> WANTestBase.createSender("ln", 2, false, 100, batchSize, false, true, null, true,
+            groupTransactionEvents));
+
+    vm2.invoke(() -> WANTestBase.createReplicatedRegion(regionName, null, isOffHeap()));
+
+    vm4.invoke(() -> WANTestBase.createReplicatedRegion(regionName, "ln", isOffHeap()));
+    vm5.invoke(() -> WANTestBase.createReplicatedRegion(regionName, "ln", isOffHeap()));
+
+    startSenderInVMs("ln", vm4, vm5);
+
+    final Map<Object, Object> keyValues = new LinkedHashMap<>();
+    int entries = 2200;
+    for (int i = 0; i < entries; i++) {
+      keyValues.put(i, i + "_Value");
+    }
+
+    int eventsPerTransaction = 11;
+    System.out.println("Starting puts");
+    AsyncInvocation<Void> inv1 =
+        vm4.invokeAsync(
+            () -> WANTestBase.doPutsInsideTransactions(regionName, keyValues,
+                eventsPerTransaction));
+
+    // wait for batches to be distributed and then stop the sender
+    System.out.println("Waiting for some batches to be distributed");

Review comment:
       Another print statement that can be moved to being the first argument of the `invoke()` call.

##########
File path: geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/serial/SerialWANStatsDUnitTest.java
##########
@@ -348,7 +349,276 @@ public void testReplicatedSerialPropagationWithBatchRedistWithGroupTransactionEv
   }
 
   @Test
-  public void testReplicatedSerialPropagationWithMultipleDispatchers() throws Exception {
+  public void testReplicatedSerialPropagationWithGroupTransactionEventsDoesNotSendBatchesWithIncompleteTransactionsIfGatewaySenderIsStoppedWhileReceivingTrafficAndLaterStarted()
+      throws InterruptedException {
+    Integer lnPort = vm0.invoke(() -> WANTestBase.createFirstLocatorWithDSId(1));
+    Integer nyPort = vm1.invoke(() -> WANTestBase.createFirstRemoteLocator(2, lnPort));
+
+    String regionName = testName + "_RR";
+
+    createCacheInVMs(nyPort, vm2);
+    createReceiverInVMs(vm2);
+
+    createCacheInVMs(lnPort, vm4, vm5);
+    boolean groupTransactionEvents = true;
+    int batchSize = 10;
+    vm4.invoke(
+        () -> WANTestBase.createSender("ln", 2, false, 100, batchSize, false, true, null, true,
+            groupTransactionEvents));
+    vm5.invoke(
+        () -> WANTestBase.createSender("ln", 2, false, 100, batchSize, false, true, null, true,
+            groupTransactionEvents));
+
+    vm2.invoke(() -> WANTestBase.createReplicatedRegion(regionName, null, isOffHeap()));
+
+    vm4.invoke(() -> WANTestBase.createReplicatedRegion(regionName, "ln", isOffHeap()));
+    vm5.invoke(() -> WANTestBase.createReplicatedRegion(regionName, "ln", isOffHeap()));
+
+    startSenderInVMs("ln", vm4, vm5);
+
+    final Map<Object, Object> keyValues = new LinkedHashMap<>();
+    int entries = 2200;
+    for (int i = 0; i < entries; i++) {
+      keyValues.put(i, i + "_Value");
+    }
+
+    int eventsPerTransaction = 11;
+    System.out.println("Starting puts");
+    AsyncInvocation<Void> inv1 =
+        vm4.invokeAsync(
+            () -> WANTestBase.doPutsInsideTransactions(regionName, keyValues,
+                eventsPerTransaction));
+
+    // wait for batches to be distributed and then stop the sender
+    System.out.println("Waiting for some batches to be distributed");
+    vm4.invoke(() -> await()
+        .until(() -> WANTestBase.getSenderStats("ln", -1).get(4) > 0));
+    System.out
+        .println("Some batches distributed: " + vm4.invoke(() -> getSenderStats("ln", -1).get(4)));
+
+    addIgnoredException("Exception occurred in CacheListener");
+    addIgnoredException(RejectedExecutionException.class);
+
+    System.out.println("Stopping sender");

Review comment:
       More print statements.

##########
File path: geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/serial/SerialWANStatsDUnitTest.java
##########
@@ -348,7 +349,276 @@ public void testReplicatedSerialPropagationWithBatchRedistWithGroupTransactionEv
   }
 
   @Test
-  public void testReplicatedSerialPropagationWithMultipleDispatchers() throws Exception {
+  public void testReplicatedSerialPropagationWithGroupTransactionEventsDoesNotSendBatchesWithIncompleteTransactionsIfGatewaySenderIsStoppedWhileReceivingTrafficAndLaterStarted()
+      throws InterruptedException {
+    Integer lnPort = vm0.invoke(() -> WANTestBase.createFirstLocatorWithDSId(1));
+    Integer nyPort = vm1.invoke(() -> WANTestBase.createFirstRemoteLocator(2, lnPort));
+
+    String regionName = testName + "_RR";
+
+    createCacheInVMs(nyPort, vm2);
+    createReceiverInVMs(vm2);
+
+    createCacheInVMs(lnPort, vm4, vm5);
+    boolean groupTransactionEvents = true;
+    int batchSize = 10;
+    vm4.invoke(
+        () -> WANTestBase.createSender("ln", 2, false, 100, batchSize, false, true, null, true,
+            groupTransactionEvents));
+    vm5.invoke(
+        () -> WANTestBase.createSender("ln", 2, false, 100, batchSize, false, true, null, true,
+            groupTransactionEvents));
+
+    vm2.invoke(() -> WANTestBase.createReplicatedRegion(regionName, null, isOffHeap()));
+
+    vm4.invoke(() -> WANTestBase.createReplicatedRegion(regionName, "ln", isOffHeap()));
+    vm5.invoke(() -> WANTestBase.createReplicatedRegion(regionName, "ln", isOffHeap()));
+
+    startSenderInVMs("ln", vm4, vm5);
+
+    final Map<Object, Object> keyValues = new LinkedHashMap<>();
+    int entries = 2200;
+    for (int i = 0; i < entries; i++) {
+      keyValues.put(i, i + "_Value");
+    }
+
+    int eventsPerTransaction = 11;
+    System.out.println("Starting puts");
+    AsyncInvocation<Void> inv1 =
+        vm4.invokeAsync(
+            () -> WANTestBase.doPutsInsideTransactions(regionName, keyValues,
+                eventsPerTransaction));
+
+    // wait for batches to be distributed and then stop the sender
+    System.out.println("Waiting for some batches to be distributed");
+    vm4.invoke(() -> await()
+        .until(() -> WANTestBase.getSenderStats("ln", -1).get(4) > 0));
+    System.out
+        .println("Some batches distributed: " + vm4.invoke(() -> getSenderStats("ln", -1).get(4)));
+
+    addIgnoredException("Exception occurred in CacheListener");
+    addIgnoredException(RejectedExecutionException.class);
+
+    System.out.println("Stopping sender");
+    stopSenderInVMsAsync("ln", vm4, vm5);
+    System.out.println("Stopped sender");
+
+    inv1.await();
+    System.out.println("Puts completed");
+
+    vm4.invoke(() -> WANTestBase.validateRegionSize(regionName, entries));
+    vm5.invoke(() -> WANTestBase.validateRegionSize(regionName, entries));
+
+    // Wait for events to replicate: when batches received does not change
+    // we can assume that replication has finished.
+    int batchesReceived = (vm2.invoke(() -> getReceiverStats())).get(0);
+    while (true) {

Review comment:
       See comments about not using `Thread.sleep` in tests.

##########
File path: geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/serial/SerialWANStatsDUnitTest.java
##########
@@ -348,7 +349,276 @@ public void testReplicatedSerialPropagationWithBatchRedistWithGroupTransactionEv
   }
 
   @Test
-  public void testReplicatedSerialPropagationWithMultipleDispatchers() throws Exception {
+  public void testReplicatedSerialPropagationWithGroupTransactionEventsDoesNotSendBatchesWithIncompleteTransactionsIfGatewaySenderIsStoppedWhileReceivingTrafficAndLaterStarted()
+      throws InterruptedException {
+    Integer lnPort = vm0.invoke(() -> WANTestBase.createFirstLocatorWithDSId(1));
+    Integer nyPort = vm1.invoke(() -> WANTestBase.createFirstRemoteLocator(2, lnPort));
+
+    String regionName = testName + "_RR";
+
+    createCacheInVMs(nyPort, vm2);
+    createReceiverInVMs(vm2);
+
+    createCacheInVMs(lnPort, vm4, vm5);
+    boolean groupTransactionEvents = true;
+    int batchSize = 10;
+    vm4.invoke(
+        () -> WANTestBase.createSender("ln", 2, false, 100, batchSize, false, true, null, true,
+            groupTransactionEvents));
+    vm5.invoke(
+        () -> WANTestBase.createSender("ln", 2, false, 100, batchSize, false, true, null, true,
+            groupTransactionEvents));
+
+    vm2.invoke(() -> WANTestBase.createReplicatedRegion(regionName, null, isOffHeap()));
+
+    vm4.invoke(() -> WANTestBase.createReplicatedRegion(regionName, "ln", isOffHeap()));
+    vm5.invoke(() -> WANTestBase.createReplicatedRegion(regionName, "ln", isOffHeap()));
+
+    startSenderInVMs("ln", vm4, vm5);
+
+    final Map<Object, Object> keyValues = new LinkedHashMap<>();
+    int entries = 2200;
+    for (int i = 0; i < entries; i++) {
+      keyValues.put(i, i + "_Value");
+    }
+
+    int eventsPerTransaction = 11;
+    System.out.println("Starting puts");
+    AsyncInvocation<Void> inv1 =
+        vm4.invokeAsync(
+            () -> WANTestBase.doPutsInsideTransactions(regionName, keyValues,
+                eventsPerTransaction));
+
+    // wait for batches to be distributed and then stop the sender
+    System.out.println("Waiting for some batches to be distributed");
+    vm4.invoke(() -> await()
+        .until(() -> WANTestBase.getSenderStats("ln", -1).get(4) > 0));
+    System.out
+        .println("Some batches distributed: " + vm4.invoke(() -> getSenderStats("ln", -1).get(4)));
+
+    addIgnoredException("Exception occurred in CacheListener");
+    addIgnoredException(RejectedExecutionException.class);
+
+    System.out.println("Stopping sender");
+    stopSenderInVMsAsync("ln", vm4, vm5);
+    System.out.println("Stopped sender");
+
+    inv1.await();
+    System.out.println("Puts completed");
+
+    vm4.invoke(() -> WANTestBase.validateRegionSize(regionName, entries));
+    vm5.invoke(() -> WANTestBase.validateRegionSize(regionName, entries));
+
+    // Wait for events to replicate: when batches received does not change
+    // we can assume that replication has finished.
+    int batchesReceived = (vm2.invoke(() -> getReceiverStats())).get(0);
+    while (true) {
+      int oldBatchesReceived = batchesReceived;
+      Thread.sleep(1000);
+      batchesReceived = (vm2.invoke(() -> getReceiverStats())).get(0);
+      if (batchesReceived == oldBatchesReceived) {
+        break;
+      }
+    }
+
+    ArrayList<Integer> v4List =
+        (ArrayList<Integer>) vm4.invoke(() -> WANTestBase.getSenderStats("ln", -1));
+    ArrayList<Integer> v5List =
+        (ArrayList<Integer>) vm5.invoke(() -> WANTestBase.getSenderStats("ln", -1));
+
+    System.out.println("v4List.get(0):" + v4List.get(0));
+    System.out.println("v5List.get(0):" + v5List.get(0));
+
+    // Wait for events to replicate: when region size does not change
+    // we can assume that replication has finished.
+    int regionSize = vm2.invoke(() -> getRegionSize(regionName));
+    while (true) {
+      int oldRegionSize = regionSize;
+      Thread.sleep(1000);
+      regionSize = vm2.invoke(() -> getRegionSize(regionName));
+      if (regionSize == oldRegionSize) {
+        break;
+      }
+    }
+
+    // Only complete transactions (11 entries each) must be replicated
+    assertEquals(0, regionSize % eventsPerTransaction);
+
+    // batches with incomplete transactions must be 0
+    assertEquals(0, (int) v4List.get(13));
+    assertEquals(0, (int) v5List.get(13));
+
+    System.out.println("regionSize: " + regionSize);
+
+    System.out.println("Starting sender");
+    startSenderInVMsAsync("ln", vm4, vm5);
+
+    v4List =
+        (ArrayList<Integer>) vm4.invoke(() -> WANTestBase.getSenderStats("ln", 0));
+    v5List =
+        (ArrayList<Integer>) vm5.invoke(() -> WANTestBase.getSenderStats("ln", 0));
+
+    assertEquals(0, v4List.get(0) + v5List.get(0));
+
+    // Wait for events to replicate: when batches received does not change
+    // we can assume that replication has finished.
+    batchesReceived = (vm2.invoke(() -> getReceiverStats())).get(0);
+    while (true) {

Review comment:
       See comments about not using `Thread.sleep` in tests.

##########
File path: geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/serial/SerialWANStatsDUnitTest.java
##########
@@ -348,7 +349,276 @@ public void testReplicatedSerialPropagationWithBatchRedistWithGroupTransactionEv
   }
 
   @Test
-  public void testReplicatedSerialPropagationWithMultipleDispatchers() throws Exception {
+  public void testReplicatedSerialPropagationWithGroupTransactionEventsDoesNotSendBatchesWithIncompleteTransactionsIfGatewaySenderIsStoppedWhileReceivingTrafficAndLaterStarted()
+      throws InterruptedException {
+    Integer lnPort = vm0.invoke(() -> WANTestBase.createFirstLocatorWithDSId(1));
+    Integer nyPort = vm1.invoke(() -> WANTestBase.createFirstRemoteLocator(2, lnPort));
+
+    String regionName = testName + "_RR";
+
+    createCacheInVMs(nyPort, vm2);
+    createReceiverInVMs(vm2);
+
+    createCacheInVMs(lnPort, vm4, vm5);
+    boolean groupTransactionEvents = true;
+    int batchSize = 10;
+    vm4.invoke(
+        () -> WANTestBase.createSender("ln", 2, false, 100, batchSize, false, true, null, true,
+            groupTransactionEvents));
+    vm5.invoke(
+        () -> WANTestBase.createSender("ln", 2, false, 100, batchSize, false, true, null, true,
+            groupTransactionEvents));
+
+    vm2.invoke(() -> WANTestBase.createReplicatedRegion(regionName, null, isOffHeap()));
+
+    vm4.invoke(() -> WANTestBase.createReplicatedRegion(regionName, "ln", isOffHeap()));
+    vm5.invoke(() -> WANTestBase.createReplicatedRegion(regionName, "ln", isOffHeap()));
+
+    startSenderInVMs("ln", vm4, vm5);
+
+    final Map<Object, Object> keyValues = new LinkedHashMap<>();
+    int entries = 2200;
+    for (int i = 0; i < entries; i++) {
+      keyValues.put(i, i + "_Value");
+    }
+
+    int eventsPerTransaction = 11;
+    System.out.println("Starting puts");
+    AsyncInvocation<Void> inv1 =
+        vm4.invokeAsync(
+            () -> WANTestBase.doPutsInsideTransactions(regionName, keyValues,
+                eventsPerTransaction));
+
+    // wait for batches to be distributed and then stop the sender
+    System.out.println("Waiting for some batches to be distributed");
+    vm4.invoke(() -> await()
+        .until(() -> WANTestBase.getSenderStats("ln", -1).get(4) > 0));
+    System.out
+        .println("Some batches distributed: " + vm4.invoke(() -> getSenderStats("ln", -1).get(4)));
+
+    addIgnoredException("Exception occurred in CacheListener");
+    addIgnoredException(RejectedExecutionException.class);
+
+    System.out.println("Stopping sender");
+    stopSenderInVMsAsync("ln", vm4, vm5);
+    System.out.println("Stopped sender");
+
+    inv1.await();
+    System.out.println("Puts completed");
+
+    vm4.invoke(() -> WANTestBase.validateRegionSize(regionName, entries));
+    vm5.invoke(() -> WANTestBase.validateRegionSize(regionName, entries));
+
+    // Wait for events to replicate: when batches received does not change
+    // we can assume that replication has finished.
+    int batchesReceived = (vm2.invoke(() -> getReceiverStats())).get(0);
+    while (true) {
+      int oldBatchesReceived = batchesReceived;
+      Thread.sleep(1000);
+      batchesReceived = (vm2.invoke(() -> getReceiverStats())).get(0);
+      if (batchesReceived == oldBatchesReceived) {
+        break;
+      }
+    }
+
+    ArrayList<Integer> v4List =
+        (ArrayList<Integer>) vm4.invoke(() -> WANTestBase.getSenderStats("ln", -1));
+    ArrayList<Integer> v5List =
+        (ArrayList<Integer>) vm5.invoke(() -> WANTestBase.getSenderStats("ln", -1));
+
+    System.out.println("v4List.get(0):" + v4List.get(0));

Review comment:
       More print statements.

##########
File path: geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/serial/SerialWANStatsDUnitTest.java
##########
@@ -348,7 +349,276 @@ public void testReplicatedSerialPropagationWithBatchRedistWithGroupTransactionEv
   }
 
   @Test
-  public void testReplicatedSerialPropagationWithMultipleDispatchers() throws Exception {
+  public void testReplicatedSerialPropagationWithGroupTransactionEventsDoesNotSendBatchesWithIncompleteTransactionsIfGatewaySenderIsStoppedWhileReceivingTrafficAndLaterStarted()
+      throws InterruptedException {
+    Integer lnPort = vm0.invoke(() -> WANTestBase.createFirstLocatorWithDSId(1));
+    Integer nyPort = vm1.invoke(() -> WANTestBase.createFirstRemoteLocator(2, lnPort));
+
+    String regionName = testName + "_RR";
+
+    createCacheInVMs(nyPort, vm2);
+    createReceiverInVMs(vm2);
+
+    createCacheInVMs(lnPort, vm4, vm5);
+    boolean groupTransactionEvents = true;
+    int batchSize = 10;
+    vm4.invoke(
+        () -> WANTestBase.createSender("ln", 2, false, 100, batchSize, false, true, null, true,
+            groupTransactionEvents));
+    vm5.invoke(
+        () -> WANTestBase.createSender("ln", 2, false, 100, batchSize, false, true, null, true,
+            groupTransactionEvents));
+
+    vm2.invoke(() -> WANTestBase.createReplicatedRegion(regionName, null, isOffHeap()));
+
+    vm4.invoke(() -> WANTestBase.createReplicatedRegion(regionName, "ln", isOffHeap()));
+    vm5.invoke(() -> WANTestBase.createReplicatedRegion(regionName, "ln", isOffHeap()));
+
+    startSenderInVMs("ln", vm4, vm5);
+
+    final Map<Object, Object> keyValues = new LinkedHashMap<>();
+    int entries = 2200;
+    for (int i = 0; i < entries; i++) {
+      keyValues.put(i, i + "_Value");
+    }
+
+    int eventsPerTransaction = 11;
+    System.out.println("Starting puts");
+    AsyncInvocation<Void> inv1 =
+        vm4.invokeAsync(
+            () -> WANTestBase.doPutsInsideTransactions(regionName, keyValues,
+                eventsPerTransaction));
+
+    // wait for batches to be distributed and then stop the sender
+    System.out.println("Waiting for some batches to be distributed");
+    vm4.invoke(() -> await()
+        .until(() -> WANTestBase.getSenderStats("ln", -1).get(4) > 0));
+    System.out
+        .println("Some batches distributed: " + vm4.invoke(() -> getSenderStats("ln", -1).get(4)));
+
+    addIgnoredException("Exception occurred in CacheListener");
+    addIgnoredException(RejectedExecutionException.class);
+
+    System.out.println("Stopping sender");
+    stopSenderInVMsAsync("ln", vm4, vm5);
+    System.out.println("Stopped sender");
+
+    inv1.await();
+    System.out.println("Puts completed");
+
+    vm4.invoke(() -> WANTestBase.validateRegionSize(regionName, entries));
+    vm5.invoke(() -> WANTestBase.validateRegionSize(regionName, entries));
+
+    // Wait for events to replicate: when batches received does not change
+    // we can assume that replication has finished.
+    int batchesReceived = (vm2.invoke(() -> getReceiverStats())).get(0);
+    while (true) {
+      int oldBatchesReceived = batchesReceived;
+      Thread.sleep(1000);
+      batchesReceived = (vm2.invoke(() -> getReceiverStats())).get(0);
+      if (batchesReceived == oldBatchesReceived) {
+        break;
+      }
+    }
+
+    ArrayList<Integer> v4List =
+        (ArrayList<Integer>) vm4.invoke(() -> WANTestBase.getSenderStats("ln", -1));
+    ArrayList<Integer> v5List =
+        (ArrayList<Integer>) vm5.invoke(() -> WANTestBase.getSenderStats("ln", -1));
+
+    System.out.println("v4List.get(0):" + v4List.get(0));
+    System.out.println("v5List.get(0):" + v5List.get(0));
+
+    // Wait for events to replicate: when region size does not change
+    // we can assume that replication has finished.
+    int regionSize = vm2.invoke(() -> getRegionSize(regionName));
+    while (true) {

Review comment:
       See comments about not using `Thread.sleep` in tests.

##########
File path: geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/serial/SerialWANStatsDUnitTest.java
##########
@@ -348,7 +349,276 @@ public void testReplicatedSerialPropagationWithBatchRedistWithGroupTransactionEv
   }
 
   @Test
-  public void testReplicatedSerialPropagationWithMultipleDispatchers() throws Exception {
+  public void testReplicatedSerialPropagationWithGroupTransactionEventsDoesNotSendBatchesWithIncompleteTransactionsIfGatewaySenderIsStoppedWhileReceivingTrafficAndLaterStarted()
+      throws InterruptedException {
+    Integer lnPort = vm0.invoke(() -> WANTestBase.createFirstLocatorWithDSId(1));
+    Integer nyPort = vm1.invoke(() -> WANTestBase.createFirstRemoteLocator(2, lnPort));
+
+    String regionName = testName + "_RR";
+
+    createCacheInVMs(nyPort, vm2);
+    createReceiverInVMs(vm2);
+
+    createCacheInVMs(lnPort, vm4, vm5);
+    boolean groupTransactionEvents = true;
+    int batchSize = 10;
+    vm4.invoke(
+        () -> WANTestBase.createSender("ln", 2, false, 100, batchSize, false, true, null, true,
+            groupTransactionEvents));
+    vm5.invoke(
+        () -> WANTestBase.createSender("ln", 2, false, 100, batchSize, false, true, null, true,
+            groupTransactionEvents));
+
+    vm2.invoke(() -> WANTestBase.createReplicatedRegion(regionName, null, isOffHeap()));
+
+    vm4.invoke(() -> WANTestBase.createReplicatedRegion(regionName, "ln", isOffHeap()));
+    vm5.invoke(() -> WANTestBase.createReplicatedRegion(regionName, "ln", isOffHeap()));
+
+    startSenderInVMs("ln", vm4, vm5);
+
+    final Map<Object, Object> keyValues = new LinkedHashMap<>();
+    int entries = 2200;
+    for (int i = 0; i < entries; i++) {
+      keyValues.put(i, i + "_Value");
+    }
+
+    int eventsPerTransaction = 11;
+    System.out.println("Starting puts");
+    AsyncInvocation<Void> inv1 =
+        vm4.invokeAsync(
+            () -> WANTestBase.doPutsInsideTransactions(regionName, keyValues,
+                eventsPerTransaction));
+
+    // wait for batches to be distributed and then stop the sender
+    System.out.println("Waiting for some batches to be distributed");
+    vm4.invoke(() -> await()
+        .until(() -> WANTestBase.getSenderStats("ln", -1).get(4) > 0));
+    System.out
+        .println("Some batches distributed: " + vm4.invoke(() -> getSenderStats("ln", -1).get(4)));
+
+    addIgnoredException("Exception occurred in CacheListener");
+    addIgnoredException(RejectedExecutionException.class);
+
+    System.out.println("Stopping sender");
+    stopSenderInVMsAsync("ln", vm4, vm5);
+    System.out.println("Stopped sender");
+
+    inv1.await();
+    System.out.println("Puts completed");
+
+    vm4.invoke(() -> WANTestBase.validateRegionSize(regionName, entries));
+    vm5.invoke(() -> WANTestBase.validateRegionSize(regionName, entries));
+
+    // Wait for events to replicate: when batches received does not change
+    // we can assume that replication has finished.
+    int batchesReceived = (vm2.invoke(() -> getReceiverStats())).get(0);
+    while (true) {
+      int oldBatchesReceived = batchesReceived;
+      Thread.sleep(1000);
+      batchesReceived = (vm2.invoke(() -> getReceiverStats())).get(0);
+      if (batchesReceived == oldBatchesReceived) {
+        break;
+      }
+    }
+
+    ArrayList<Integer> v4List =
+        (ArrayList<Integer>) vm4.invoke(() -> WANTestBase.getSenderStats("ln", -1));
+    ArrayList<Integer> v5List =
+        (ArrayList<Integer>) vm5.invoke(() -> WANTestBase.getSenderStats("ln", -1));
+
+    System.out.println("v4List.get(0):" + v4List.get(0));
+    System.out.println("v5List.get(0):" + v5List.get(0));
+
+    // Wait for events to replicate: when region size does not change
+    // we can assume that replication has finished.
+    int regionSize = vm2.invoke(() -> getRegionSize(regionName));
+    while (true) {
+      int oldRegionSize = regionSize;
+      Thread.sleep(1000);
+      regionSize = vm2.invoke(() -> getRegionSize(regionName));
+      if (regionSize == oldRegionSize) {
+        break;
+      }
+    }
+
+    // Only complete transactions (11 entries each) must be replicated
+    assertEquals(0, regionSize % eventsPerTransaction);

Review comment:
       This assertion is not strictly checking what the comment above it is describing. It's possible that the region size could be a multiple of `eventsPerTransaction` but still contain incomplete transactions. Given that we check that the incomplete transactions stat is 0 immediately after, this assertion can probably be removed.

##########
File path: geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/serial/SerialWANStatsDUnitTest.java
##########
@@ -348,7 +349,276 @@ public void testReplicatedSerialPropagationWithBatchRedistWithGroupTransactionEv
   }
 
   @Test
-  public void testReplicatedSerialPropagationWithMultipleDispatchers() throws Exception {
+  public void testReplicatedSerialPropagationWithGroupTransactionEventsDoesNotSendBatchesWithIncompleteTransactionsIfGatewaySenderIsStoppedWhileReceivingTrafficAndLaterStarted()
+      throws InterruptedException {
+    Integer lnPort = vm0.invoke(() -> WANTestBase.createFirstLocatorWithDSId(1));
+    Integer nyPort = vm1.invoke(() -> WANTestBase.createFirstRemoteLocator(2, lnPort));
+
+    String regionName = testName + "_RR";
+
+    createCacheInVMs(nyPort, vm2);
+    createReceiverInVMs(vm2);
+
+    createCacheInVMs(lnPort, vm4, vm5);
+    boolean groupTransactionEvents = true;
+    int batchSize = 10;
+    vm4.invoke(
+        () -> WANTestBase.createSender("ln", 2, false, 100, batchSize, false, true, null, true,
+            groupTransactionEvents));
+    vm5.invoke(
+        () -> WANTestBase.createSender("ln", 2, false, 100, batchSize, false, true, null, true,
+            groupTransactionEvents));
+
+    vm2.invoke(() -> WANTestBase.createReplicatedRegion(regionName, null, isOffHeap()));
+
+    vm4.invoke(() -> WANTestBase.createReplicatedRegion(regionName, "ln", isOffHeap()));
+    vm5.invoke(() -> WANTestBase.createReplicatedRegion(regionName, "ln", isOffHeap()));
+
+    startSenderInVMs("ln", vm4, vm5);
+
+    final Map<Object, Object> keyValues = new LinkedHashMap<>();
+    int entries = 2200;
+    for (int i = 0; i < entries; i++) {
+      keyValues.put(i, i + "_Value");
+    }
+
+    int eventsPerTransaction = 11;
+    System.out.println("Starting puts");
+    AsyncInvocation<Void> inv1 =
+        vm4.invokeAsync(
+            () -> WANTestBase.doPutsInsideTransactions(regionName, keyValues,
+                eventsPerTransaction));
+
+    // wait for batches to be distributed and then stop the sender
+    System.out.println("Waiting for some batches to be distributed");
+    vm4.invoke(() -> await()
+        .until(() -> WANTestBase.getSenderStats("ln", -1).get(4) > 0));
+    System.out
+        .println("Some batches distributed: " + vm4.invoke(() -> getSenderStats("ln", -1).get(4)));
+
+    addIgnoredException("Exception occurred in CacheListener");
+    addIgnoredException(RejectedExecutionException.class);
+
+    System.out.println("Stopping sender");
+    stopSenderInVMsAsync("ln", vm4, vm5);
+    System.out.println("Stopped sender");
+
+    inv1.await();
+    System.out.println("Puts completed");
+
+    vm4.invoke(() -> WANTestBase.validateRegionSize(regionName, entries));
+    vm5.invoke(() -> WANTestBase.validateRegionSize(regionName, entries));
+
+    // Wait for events to replicate: when batches received does not change
+    // we can assume that replication has finished.
+    int batchesReceived = (vm2.invoke(() -> getReceiverStats())).get(0);
+    while (true) {
+      int oldBatchesReceived = batchesReceived;
+      Thread.sleep(1000);
+      batchesReceived = (vm2.invoke(() -> getReceiverStats())).get(0);
+      if (batchesReceived == oldBatchesReceived) {
+        break;
+      }
+    }
+
+    ArrayList<Integer> v4List =
+        (ArrayList<Integer>) vm4.invoke(() -> WANTestBase.getSenderStats("ln", -1));
+    ArrayList<Integer> v5List =
+        (ArrayList<Integer>) vm5.invoke(() -> WANTestBase.getSenderStats("ln", -1));
+
+    System.out.println("v4List.get(0):" + v4List.get(0));
+    System.out.println("v5List.get(0):" + v5List.get(0));
+
+    // Wait for events to replicate: when region size does not change
+    // we can assume that replication has finished.
+    int regionSize = vm2.invoke(() -> getRegionSize(regionName));
+    while (true) {
+      int oldRegionSize = regionSize;
+      Thread.sleep(1000);
+      regionSize = vm2.invoke(() -> getRegionSize(regionName));
+      if (regionSize == oldRegionSize) {
+        break;
+      }
+    }
+
+    // Only complete transactions (11 entries each) must be replicated
+    assertEquals(0, regionSize % eventsPerTransaction);
+
+    // batches with incomplete transactions must be 0
+    assertEquals(0, (int) v4List.get(13));
+    assertEquals(0, (int) v5List.get(13));
+
+    System.out.println("regionSize: " + regionSize);

Review comment:
       More print statements.

##########
File path: geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/serial/SerialWANStatsDUnitTest.java
##########
@@ -348,7 +349,276 @@ public void testReplicatedSerialPropagationWithBatchRedistWithGroupTransactionEv
   }
 
   @Test
-  public void testReplicatedSerialPropagationWithMultipleDispatchers() throws Exception {
+  public void testReplicatedSerialPropagationWithGroupTransactionEventsDoesNotSendBatchesWithIncompleteTransactionsIfGatewaySenderIsStoppedWhileReceivingTrafficAndLaterStarted()
+      throws InterruptedException {
+    Integer lnPort = vm0.invoke(() -> WANTestBase.createFirstLocatorWithDSId(1));
+    Integer nyPort = vm1.invoke(() -> WANTestBase.createFirstRemoteLocator(2, lnPort));
+
+    String regionName = testName + "_RR";
+
+    createCacheInVMs(nyPort, vm2);
+    createReceiverInVMs(vm2);
+
+    createCacheInVMs(lnPort, vm4, vm5);
+    boolean groupTransactionEvents = true;
+    int batchSize = 10;
+    vm4.invoke(
+        () -> WANTestBase.createSender("ln", 2, false, 100, batchSize, false, true, null, true,
+            groupTransactionEvents));
+    vm5.invoke(
+        () -> WANTestBase.createSender("ln", 2, false, 100, batchSize, false, true, null, true,
+            groupTransactionEvents));
+
+    vm2.invoke(() -> WANTestBase.createReplicatedRegion(regionName, null, isOffHeap()));
+
+    vm4.invoke(() -> WANTestBase.createReplicatedRegion(regionName, "ln", isOffHeap()));
+    vm5.invoke(() -> WANTestBase.createReplicatedRegion(regionName, "ln", isOffHeap()));
+
+    startSenderInVMs("ln", vm4, vm5);
+
+    final Map<Object, Object> keyValues = new LinkedHashMap<>();
+    int entries = 2200;
+    for (int i = 0; i < entries; i++) {
+      keyValues.put(i, i + "_Value");
+    }
+
+    int eventsPerTransaction = 11;
+    System.out.println("Starting puts");
+    AsyncInvocation<Void> inv1 =
+        vm4.invokeAsync(
+            () -> WANTestBase.doPutsInsideTransactions(regionName, keyValues,
+                eventsPerTransaction));
+
+    // wait for batches to be distributed and then stop the sender
+    System.out.println("Waiting for some batches to be distributed");
+    vm4.invoke(() -> await()
+        .until(() -> WANTestBase.getSenderStats("ln", -1).get(4) > 0));
+    System.out
+        .println("Some batches distributed: " + vm4.invoke(() -> getSenderStats("ln", -1).get(4)));
+
+    addIgnoredException("Exception occurred in CacheListener");
+    addIgnoredException(RejectedExecutionException.class);
+
+    System.out.println("Stopping sender");
+    stopSenderInVMsAsync("ln", vm4, vm5);
+    System.out.println("Stopped sender");
+
+    inv1.await();
+    System.out.println("Puts completed");
+
+    vm4.invoke(() -> WANTestBase.validateRegionSize(regionName, entries));
+    vm5.invoke(() -> WANTestBase.validateRegionSize(regionName, entries));
+
+    // Wait for events to replicate: when batches received does not change
+    // we can assume that replication has finished.
+    int batchesReceived = (vm2.invoke(() -> getReceiverStats())).get(0);
+    while (true) {
+      int oldBatchesReceived = batchesReceived;
+      Thread.sleep(1000);
+      batchesReceived = (vm2.invoke(() -> getReceiverStats())).get(0);
+      if (batchesReceived == oldBatchesReceived) {
+        break;
+      }
+    }
+
+    ArrayList<Integer> v4List =
+        (ArrayList<Integer>) vm4.invoke(() -> WANTestBase.getSenderStats("ln", -1));
+    ArrayList<Integer> v5List =
+        (ArrayList<Integer>) vm5.invoke(() -> WANTestBase.getSenderStats("ln", -1));
+
+    System.out.println("v4List.get(0):" + v4List.get(0));
+    System.out.println("v5List.get(0):" + v5List.get(0));
+
+    // Wait for events to replicate: when region size does not change
+    // we can assume that replication has finished.
+    int regionSize = vm2.invoke(() -> getRegionSize(regionName));
+    while (true) {
+      int oldRegionSize = regionSize;
+      Thread.sleep(1000);
+      regionSize = vm2.invoke(() -> getRegionSize(regionName));
+      if (regionSize == oldRegionSize) {
+        break;
+      }
+    }
+
+    // Only complete transactions (11 entries each) must be replicated
+    assertEquals(0, regionSize % eventsPerTransaction);
+
+    // batches with incomplete transactions must be 0
+    assertEquals(0, (int) v4List.get(13));
+    assertEquals(0, (int) v5List.get(13));
+
+    System.out.println("regionSize: " + regionSize);
+
+    System.out.println("Starting sender");
+    startSenderInVMsAsync("ln", vm4, vm5);
+
+    v4List =
+        (ArrayList<Integer>) vm4.invoke(() -> WANTestBase.getSenderStats("ln", 0));
+    v5List =
+        (ArrayList<Integer>) vm5.invoke(() -> WANTestBase.getSenderStats("ln", 0));
+
+    assertEquals(0, v4List.get(0) + v5List.get(0));
+
+    // Wait for events to replicate: when batches received does not change
+    // we can assume that replication has finished.
+    batchesReceived = (vm2.invoke(() -> getReceiverStats())).get(0);
+    while (true) {
+      int oldBatchesReceived = batchesReceived;
+      Thread.sleep(1000);
+      batchesReceived = (vm2.invoke(() -> getReceiverStats())).get(0);
+      if (batchesReceived == oldBatchesReceived) {
+        break;
+      }
+    }
+
+    System.out.println("batchesReceived: " + batchesReceived);
+
+    // Wait for events to replicate: when region size does not change
+    // we can assume that replication has finished.
+    regionSize = vm2.invoke(() -> getRegionSize(regionName));
+    while (true) {

Review comment:
       See comments about not using `Thread.sleep` in tests.

##########
File path: geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/serial/SerialWANStatsDUnitTest.java
##########
@@ -348,7 +349,276 @@ public void testReplicatedSerialPropagationWithBatchRedistWithGroupTransactionEv
   }
 
   @Test
-  public void testReplicatedSerialPropagationWithMultipleDispatchers() throws Exception {
+  public void testReplicatedSerialPropagationWithGroupTransactionEventsDoesNotSendBatchesWithIncompleteTransactionsIfGatewaySenderIsStoppedWhileReceivingTrafficAndLaterStarted()
+      throws InterruptedException {
+    Integer lnPort = vm0.invoke(() -> WANTestBase.createFirstLocatorWithDSId(1));
+    Integer nyPort = vm1.invoke(() -> WANTestBase.createFirstRemoteLocator(2, lnPort));
+
+    String regionName = testName + "_RR";
+
+    createCacheInVMs(nyPort, vm2);
+    createReceiverInVMs(vm2);
+
+    createCacheInVMs(lnPort, vm4, vm5);
+    boolean groupTransactionEvents = true;
+    int batchSize = 10;
+    vm4.invoke(
+        () -> WANTestBase.createSender("ln", 2, false, 100, batchSize, false, true, null, true,
+            groupTransactionEvents));
+    vm5.invoke(
+        () -> WANTestBase.createSender("ln", 2, false, 100, batchSize, false, true, null, true,
+            groupTransactionEvents));
+
+    vm2.invoke(() -> WANTestBase.createReplicatedRegion(regionName, null, isOffHeap()));
+
+    vm4.invoke(() -> WANTestBase.createReplicatedRegion(regionName, "ln", isOffHeap()));
+    vm5.invoke(() -> WANTestBase.createReplicatedRegion(regionName, "ln", isOffHeap()));
+
+    startSenderInVMs("ln", vm4, vm5);
+
+    final Map<Object, Object> keyValues = new LinkedHashMap<>();
+    int entries = 2200;
+    for (int i = 0; i < entries; i++) {
+      keyValues.put(i, i + "_Value");
+    }
+
+    int eventsPerTransaction = 11;
+    System.out.println("Starting puts");
+    AsyncInvocation<Void> inv1 =
+        vm4.invokeAsync(
+            () -> WANTestBase.doPutsInsideTransactions(regionName, keyValues,
+                eventsPerTransaction));
+
+    // wait for batches to be distributed and then stop the sender
+    System.out.println("Waiting for some batches to be distributed");
+    vm4.invoke(() -> await()
+        .until(() -> WANTestBase.getSenderStats("ln", -1).get(4) > 0));
+    System.out
+        .println("Some batches distributed: " + vm4.invoke(() -> getSenderStats("ln", -1).get(4)));
+
+    addIgnoredException("Exception occurred in CacheListener");
+    addIgnoredException(RejectedExecutionException.class);
+
+    System.out.println("Stopping sender");
+    stopSenderInVMsAsync("ln", vm4, vm5);
+    System.out.println("Stopped sender");
+
+    inv1.await();
+    System.out.println("Puts completed");
+
+    vm4.invoke(() -> WANTestBase.validateRegionSize(regionName, entries));
+    vm5.invoke(() -> WANTestBase.validateRegionSize(regionName, entries));
+
+    // Wait for events to replicate: when batches received does not change
+    // we can assume that replication has finished.
+    int batchesReceived = (vm2.invoke(() -> getReceiverStats())).get(0);
+    while (true) {
+      int oldBatchesReceived = batchesReceived;
+      Thread.sleep(1000);
+      batchesReceived = (vm2.invoke(() -> getReceiverStats())).get(0);
+      if (batchesReceived == oldBatchesReceived) {
+        break;
+      }
+    }
+
+    ArrayList<Integer> v4List =
+        (ArrayList<Integer>) vm4.invoke(() -> WANTestBase.getSenderStats("ln", -1));
+    ArrayList<Integer> v5List =
+        (ArrayList<Integer>) vm5.invoke(() -> WANTestBase.getSenderStats("ln", -1));
+
+    System.out.println("v4List.get(0):" + v4List.get(0));
+    System.out.println("v5List.get(0):" + v5List.get(0));
+
+    // Wait for events to replicate: when region size does not change
+    // we can assume that replication has finished.
+    int regionSize = vm2.invoke(() -> getRegionSize(regionName));
+    while (true) {
+      int oldRegionSize = regionSize;
+      Thread.sleep(1000);
+      regionSize = vm2.invoke(() -> getRegionSize(regionName));
+      if (regionSize == oldRegionSize) {
+        break;
+      }
+    }
+
+    // Only complete transactions (11 entries each) must be replicated
+    assertEquals(0, regionSize % eventsPerTransaction);
+
+    // batches with incomplete transactions must be 0
+    assertEquals(0, (int) v4List.get(13));
+    assertEquals(0, (int) v5List.get(13));
+
+    System.out.println("regionSize: " + regionSize);
+
+    System.out.println("Starting sender");
+    startSenderInVMsAsync("ln", vm4, vm5);
+
+    v4List =
+        (ArrayList<Integer>) vm4.invoke(() -> WANTestBase.getSenderStats("ln", 0));
+    v5List =
+        (ArrayList<Integer>) vm5.invoke(() -> WANTestBase.getSenderStats("ln", 0));
+
+    assertEquals(0, v4List.get(0) + v5List.get(0));
+
+    // Wait for events to replicate: when batches received does not change
+    // we can assume that replication has finished.
+    batchesReceived = (vm2.invoke(() -> getReceiverStats())).get(0);
+    while (true) {
+      int oldBatchesReceived = batchesReceived;
+      Thread.sleep(1000);
+      batchesReceived = (vm2.invoke(() -> getReceiverStats())).get(0);
+      if (batchesReceived == oldBatchesReceived) {
+        break;
+      }
+    }
+
+    System.out.println("batchesReceived: " + batchesReceived);

Review comment:
       Another print statement.

##########
File path: geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/serial/SerialWANStatsDUnitTest.java
##########
@@ -348,7 +349,276 @@ public void testReplicatedSerialPropagationWithBatchRedistWithGroupTransactionEv
   }
 
   @Test
-  public void testReplicatedSerialPropagationWithMultipleDispatchers() throws Exception {
+  public void testReplicatedSerialPropagationWithGroupTransactionEventsDoesNotSendBatchesWithIncompleteTransactionsIfGatewaySenderIsStoppedWhileReceivingTrafficAndLaterStarted()
+      throws InterruptedException {
+    Integer lnPort = vm0.invoke(() -> WANTestBase.createFirstLocatorWithDSId(1));
+    Integer nyPort = vm1.invoke(() -> WANTestBase.createFirstRemoteLocator(2, lnPort));
+
+    String regionName = testName + "_RR";
+
+    createCacheInVMs(nyPort, vm2);
+    createReceiverInVMs(vm2);
+
+    createCacheInVMs(lnPort, vm4, vm5);
+    boolean groupTransactionEvents = true;
+    int batchSize = 10;
+    vm4.invoke(
+        () -> WANTestBase.createSender("ln", 2, false, 100, batchSize, false, true, null, true,
+            groupTransactionEvents));
+    vm5.invoke(
+        () -> WANTestBase.createSender("ln", 2, false, 100, batchSize, false, true, null, true,
+            groupTransactionEvents));
+
+    vm2.invoke(() -> WANTestBase.createReplicatedRegion(regionName, null, isOffHeap()));
+
+    vm4.invoke(() -> WANTestBase.createReplicatedRegion(regionName, "ln", isOffHeap()));
+    vm5.invoke(() -> WANTestBase.createReplicatedRegion(regionName, "ln", isOffHeap()));
+
+    startSenderInVMs("ln", vm4, vm5);
+
+    final Map<Object, Object> keyValues = new LinkedHashMap<>();
+    int entries = 2200;
+    for (int i = 0; i < entries; i++) {
+      keyValues.put(i, i + "_Value");
+    }
+
+    int eventsPerTransaction = 11;
+    System.out.println("Starting puts");
+    AsyncInvocation<Void> inv1 =
+        vm4.invokeAsync(
+            () -> WANTestBase.doPutsInsideTransactions(regionName, keyValues,
+                eventsPerTransaction));
+
+    // wait for batches to be distributed and then stop the sender
+    System.out.println("Waiting for some batches to be distributed");
+    vm4.invoke(() -> await()
+        .until(() -> WANTestBase.getSenderStats("ln", -1).get(4) > 0));
+    System.out
+        .println("Some batches distributed: " + vm4.invoke(() -> getSenderStats("ln", -1).get(4)));
+
+    addIgnoredException("Exception occurred in CacheListener");
+    addIgnoredException(RejectedExecutionException.class);
+
+    System.out.println("Stopping sender");
+    stopSenderInVMsAsync("ln", vm4, vm5);
+    System.out.println("Stopped sender");
+
+    inv1.await();
+    System.out.println("Puts completed");
+
+    vm4.invoke(() -> WANTestBase.validateRegionSize(regionName, entries));
+    vm5.invoke(() -> WANTestBase.validateRegionSize(regionName, entries));
+
+    // Wait for events to replicate: when batches received does not change
+    // we can assume that replication has finished.
+    int batchesReceived = (vm2.invoke(() -> getReceiverStats())).get(0);
+    while (true) {
+      int oldBatchesReceived = batchesReceived;
+      Thread.sleep(1000);
+      batchesReceived = (vm2.invoke(() -> getReceiverStats())).get(0);
+      if (batchesReceived == oldBatchesReceived) {
+        break;
+      }
+    }
+
+    ArrayList<Integer> v4List =

Review comment:
       These should be declared as `List<Integer>`, which removes the need for casting them here and elsewhere.

##########
File path: geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/serial/SerialWANStatsDUnitTest.java
##########
@@ -348,7 +349,276 @@ public void testReplicatedSerialPropagationWithBatchRedistWithGroupTransactionEv
   }
 
   @Test
-  public void testReplicatedSerialPropagationWithMultipleDispatchers() throws Exception {
+  public void testReplicatedSerialPropagationWithGroupTransactionEventsDoesNotSendBatchesWithIncompleteTransactionsIfGatewaySenderIsStoppedWhileReceivingTrafficAndLaterStarted()
+      throws InterruptedException {
+    Integer lnPort = vm0.invoke(() -> WANTestBase.createFirstLocatorWithDSId(1));
+    Integer nyPort = vm1.invoke(() -> WANTestBase.createFirstRemoteLocator(2, lnPort));
+
+    String regionName = testName + "_RR";
+
+    createCacheInVMs(nyPort, vm2);
+    createReceiverInVMs(vm2);
+
+    createCacheInVMs(lnPort, vm4, vm5);
+    boolean groupTransactionEvents = true;
+    int batchSize = 10;
+    vm4.invoke(
+        () -> WANTestBase.createSender("ln", 2, false, 100, batchSize, false, true, null, true,
+            groupTransactionEvents));
+    vm5.invoke(
+        () -> WANTestBase.createSender("ln", 2, false, 100, batchSize, false, true, null, true,
+            groupTransactionEvents));
+
+    vm2.invoke(() -> WANTestBase.createReplicatedRegion(regionName, null, isOffHeap()));
+
+    vm4.invoke(() -> WANTestBase.createReplicatedRegion(regionName, "ln", isOffHeap()));
+    vm5.invoke(() -> WANTestBase.createReplicatedRegion(regionName, "ln", isOffHeap()));
+
+    startSenderInVMs("ln", vm4, vm5);
+
+    final Map<Object, Object> keyValues = new LinkedHashMap<>();
+    int entries = 2200;
+    for (int i = 0; i < entries; i++) {
+      keyValues.put(i, i + "_Value");
+    }
+
+    int eventsPerTransaction = 11;
+    System.out.println("Starting puts");
+    AsyncInvocation<Void> inv1 =
+        vm4.invokeAsync(
+            () -> WANTestBase.doPutsInsideTransactions(regionName, keyValues,
+                eventsPerTransaction));
+
+    // wait for batches to be distributed and then stop the sender
+    System.out.println("Waiting for some batches to be distributed");
+    vm4.invoke(() -> await()
+        .until(() -> WANTestBase.getSenderStats("ln", -1).get(4) > 0));
+    System.out
+        .println("Some batches distributed: " + vm4.invoke(() -> getSenderStats("ln", -1).get(4)));
+
+    addIgnoredException("Exception occurred in CacheListener");
+    addIgnoredException(RejectedExecutionException.class);
+
+    System.out.println("Stopping sender");
+    stopSenderInVMsAsync("ln", vm4, vm5);
+    System.out.println("Stopped sender");
+
+    inv1.await();
+    System.out.println("Puts completed");
+
+    vm4.invoke(() -> WANTestBase.validateRegionSize(regionName, entries));
+    vm5.invoke(() -> WANTestBase.validateRegionSize(regionName, entries));
+
+    // Wait for events to replicate: when batches received does not change
+    // we can assume that replication has finished.
+    int batchesReceived = (vm2.invoke(() -> getReceiverStats())).get(0);
+    while (true) {
+      int oldBatchesReceived = batchesReceived;
+      Thread.sleep(1000);
+      batchesReceived = (vm2.invoke(() -> getReceiverStats())).get(0);
+      if (batchesReceived == oldBatchesReceived) {
+        break;
+      }
+    }
+
+    ArrayList<Integer> v4List =
+        (ArrayList<Integer>) vm4.invoke(() -> WANTestBase.getSenderStats("ln", -1));
+    ArrayList<Integer> v5List =
+        (ArrayList<Integer>) vm5.invoke(() -> WANTestBase.getSenderStats("ln", -1));
+
+    System.out.println("v4List.get(0):" + v4List.get(0));
+    System.out.println("v5List.get(0):" + v5List.get(0));
+
+    // Wait for events to replicate: when region size does not change
+    // we can assume that replication has finished.
+    int regionSize = vm2.invoke(() -> getRegionSize(regionName));
+    while (true) {
+      int oldRegionSize = regionSize;
+      Thread.sleep(1000);
+      regionSize = vm2.invoke(() -> getRegionSize(regionName));
+      if (regionSize == oldRegionSize) {
+        break;
+      }
+    }
+
+    // Only complete transactions (11 entries each) must be replicated
+    assertEquals(0, regionSize % eventsPerTransaction);
+
+    // batches with incomplete transactions must be 0
+    assertEquals(0, (int) v4List.get(13));
+    assertEquals(0, (int) v5List.get(13));
+
+    System.out.println("regionSize: " + regionSize);
+
+    System.out.println("Starting sender");
+    startSenderInVMsAsync("ln", vm4, vm5);
+
+    v4List =
+        (ArrayList<Integer>) vm4.invoke(() -> WANTestBase.getSenderStats("ln", 0));
+    v5List =
+        (ArrayList<Integer>) vm5.invoke(() -> WANTestBase.getSenderStats("ln", 0));
+
+    assertEquals(0, v4List.get(0) + v5List.get(0));
+
+    // Wait for events to replicate: when batches received does not change
+    // we can assume that replication has finished.
+    batchesReceived = (vm2.invoke(() -> getReceiverStats())).get(0);
+    while (true) {
+      int oldBatchesReceived = batchesReceived;
+      Thread.sleep(1000);
+      batchesReceived = (vm2.invoke(() -> getReceiverStats())).get(0);
+      if (batchesReceived == oldBatchesReceived) {
+        break;
+      }
+    }
+
+    System.out.println("batchesReceived: " + batchesReceived);
+
+    // Wait for events to replicate: when region size does not change
+    // we can assume that replication has finished.
+    regionSize = vm2.invoke(() -> getRegionSize(regionName));
+    while (true) {
+      int oldRegionSize = regionSize;
+      Thread.sleep(1000);
+      regionSize = vm2.invoke(() -> getRegionSize(regionName));
+      if (regionSize == oldRegionSize) {
+        break;
+      }
+    }
+
+    System.out.println("regionSize: " + regionSize);

Review comment:
       More print statements.

##########
File path: geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/serial/SerialWANStatsDUnitTest.java
##########
@@ -348,7 +349,276 @@ public void testReplicatedSerialPropagationWithBatchRedistWithGroupTransactionEv
   }
 
   @Test
-  public void testReplicatedSerialPropagationWithMultipleDispatchers() throws Exception {
+  public void testReplicatedSerialPropagationWithGroupTransactionEventsDoesNotSendBatchesWithIncompleteTransactionsIfGatewaySenderIsStoppedWhileReceivingTrafficAndLaterStarted()
+      throws InterruptedException {
+    Integer lnPort = vm0.invoke(() -> WANTestBase.createFirstLocatorWithDSId(1));
+    Integer nyPort = vm1.invoke(() -> WANTestBase.createFirstRemoteLocator(2, lnPort));
+
+    String regionName = testName + "_RR";
+
+    createCacheInVMs(nyPort, vm2);
+    createReceiverInVMs(vm2);
+
+    createCacheInVMs(lnPort, vm4, vm5);
+    boolean groupTransactionEvents = true;
+    int batchSize = 10;
+    vm4.invoke(
+        () -> WANTestBase.createSender("ln", 2, false, 100, batchSize, false, true, null, true,
+            groupTransactionEvents));
+    vm5.invoke(
+        () -> WANTestBase.createSender("ln", 2, false, 100, batchSize, false, true, null, true,
+            groupTransactionEvents));
+
+    vm2.invoke(() -> WANTestBase.createReplicatedRegion(regionName, null, isOffHeap()));
+
+    vm4.invoke(() -> WANTestBase.createReplicatedRegion(regionName, "ln", isOffHeap()));
+    vm5.invoke(() -> WANTestBase.createReplicatedRegion(regionName, "ln", isOffHeap()));
+
+    startSenderInVMs("ln", vm4, vm5);
+
+    final Map<Object, Object> keyValues = new LinkedHashMap<>();
+    int entries = 2200;
+    for (int i = 0; i < entries; i++) {
+      keyValues.put(i, i + "_Value");
+    }
+
+    int eventsPerTransaction = 11;
+    System.out.println("Starting puts");
+    AsyncInvocation<Void> inv1 =
+        vm4.invokeAsync(
+            () -> WANTestBase.doPutsInsideTransactions(regionName, keyValues,
+                eventsPerTransaction));
+
+    // wait for batches to be distributed and then stop the sender
+    System.out.println("Waiting for some batches to be distributed");
+    vm4.invoke(() -> await()
+        .until(() -> WANTestBase.getSenderStats("ln", -1).get(4) > 0));
+    System.out
+        .println("Some batches distributed: " + vm4.invoke(() -> getSenderStats("ln", -1).get(4)));
+
+    addIgnoredException("Exception occurred in CacheListener");
+    addIgnoredException(RejectedExecutionException.class);
+
+    System.out.println("Stopping sender");
+    stopSenderInVMsAsync("ln", vm4, vm5);
+    System.out.println("Stopped sender");
+
+    inv1.await();
+    System.out.println("Puts completed");
+
+    vm4.invoke(() -> WANTestBase.validateRegionSize(regionName, entries));
+    vm5.invoke(() -> WANTestBase.validateRegionSize(regionName, entries));
+
+    // Wait for events to replicate: when batches received does not change
+    // we can assume that replication has finished.
+    int batchesReceived = (vm2.invoke(() -> getReceiverStats())).get(0);
+    while (true) {
+      int oldBatchesReceived = batchesReceived;
+      Thread.sleep(1000);
+      batchesReceived = (vm2.invoke(() -> getReceiverStats())).get(0);
+      if (batchesReceived == oldBatchesReceived) {
+        break;
+      }
+    }
+
+    ArrayList<Integer> v4List =
+        (ArrayList<Integer>) vm4.invoke(() -> WANTestBase.getSenderStats("ln", -1));
+    ArrayList<Integer> v5List =
+        (ArrayList<Integer>) vm5.invoke(() -> WANTestBase.getSenderStats("ln", -1));
+
+    System.out.println("v4List.get(0):" + v4List.get(0));
+    System.out.println("v5List.get(0):" + v5List.get(0));
+
+    // Wait for events to replicate: when region size does not change
+    // we can assume that replication has finished.
+    int regionSize = vm2.invoke(() -> getRegionSize(regionName));
+    while (true) {
+      int oldRegionSize = regionSize;
+      Thread.sleep(1000);
+      regionSize = vm2.invoke(() -> getRegionSize(regionName));
+      if (regionSize == oldRegionSize) {
+        break;
+      }
+    }
+
+    // Only complete transactions (11 entries each) must be replicated
+    assertEquals(0, regionSize % eventsPerTransaction);
+
+    // batches with incomplete transactions must be 0
+    assertEquals(0, (int) v4List.get(13));
+    assertEquals(0, (int) v5List.get(13));
+
+    System.out.println("regionSize: " + regionSize);
+
+    System.out.println("Starting sender");
+    startSenderInVMsAsync("ln", vm4, vm5);
+
+    v4List =
+        (ArrayList<Integer>) vm4.invoke(() -> WANTestBase.getSenderStats("ln", 0));
+    v5List =
+        (ArrayList<Integer>) vm5.invoke(() -> WANTestBase.getSenderStats("ln", 0));
+
+    assertEquals(0, v4List.get(0) + v5List.get(0));
+
+    // Wait for events to replicate: when batches received does not change
+    // we can assume that replication has finished.
+    batchesReceived = (vm2.invoke(() -> getReceiverStats())).get(0);
+    while (true) {
+      int oldBatchesReceived = batchesReceived;
+      Thread.sleep(1000);
+      batchesReceived = (vm2.invoke(() -> getReceiverStats())).get(0);
+      if (batchesReceived == oldBatchesReceived) {
+        break;
+      }
+    }
+
+    System.out.println("batchesReceived: " + batchesReceived);
+
+    // Wait for events to replicate: when region size does not change
+    // we can assume that replication has finished.
+    regionSize = vm2.invoke(() -> getRegionSize(regionName));
+    while (true) {
+      int oldRegionSize = regionSize;
+      Thread.sleep(1000);
+      regionSize = vm2.invoke(() -> getRegionSize(regionName));
+      if (regionSize == oldRegionSize) {
+        break;
+      }
+    }
+
+    System.out.println("regionSize: " + regionSize);
+
+    System.out.println("v4List.get(0):" + v4List.get(0));
+    System.out.println("v5List.get(0):" + v5List.get(0));
+
+    // batches with incomplete transactions must be 0
+    assertEquals(0, (int) v4List.get(13));
+    assertEquals(0, (int) v5List.get(13));
+
+    // Only complete transactions (11 events) must be replicated
+    assertEquals(0, regionSize % eventsPerTransaction);

Review comment:
       See above comment about this assertion not necessarily checking what is being described in the comment.

##########
File path: geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/serial/SerialWANStatsDUnitTest.java
##########
@@ -348,7 +349,276 @@ public void testReplicatedSerialPropagationWithBatchRedistWithGroupTransactionEv
   }
 
   @Test
-  public void testReplicatedSerialPropagationWithMultipleDispatchers() throws Exception {
+  public void testReplicatedSerialPropagationWithGroupTransactionEventsDoesNotSendBatchesWithIncompleteTransactionsIfGatewaySenderIsStoppedWhileReceivingTrafficAndLaterStarted()
+      throws InterruptedException {
+    Integer lnPort = vm0.invoke(() -> WANTestBase.createFirstLocatorWithDSId(1));
+    Integer nyPort = vm1.invoke(() -> WANTestBase.createFirstRemoteLocator(2, lnPort));
+
+    String regionName = testName + "_RR";
+
+    createCacheInVMs(nyPort, vm2);
+    createReceiverInVMs(vm2);
+
+    createCacheInVMs(lnPort, vm4, vm5);
+    boolean groupTransactionEvents = true;
+    int batchSize = 10;
+    vm4.invoke(
+        () -> WANTestBase.createSender("ln", 2, false, 100, batchSize, false, true, null, true,
+            groupTransactionEvents));
+    vm5.invoke(
+        () -> WANTestBase.createSender("ln", 2, false, 100, batchSize, false, true, null, true,
+            groupTransactionEvents));
+
+    vm2.invoke(() -> WANTestBase.createReplicatedRegion(regionName, null, isOffHeap()));
+
+    vm4.invoke(() -> WANTestBase.createReplicatedRegion(regionName, "ln", isOffHeap()));
+    vm5.invoke(() -> WANTestBase.createReplicatedRegion(regionName, "ln", isOffHeap()));
+
+    startSenderInVMs("ln", vm4, vm5);
+
+    final Map<Object, Object> keyValues = new LinkedHashMap<>();
+    int entries = 2200;
+    for (int i = 0; i < entries; i++) {
+      keyValues.put(i, i + "_Value");
+    }
+
+    int eventsPerTransaction = 11;
+    System.out.println("Starting puts");
+    AsyncInvocation<Void> inv1 =
+        vm4.invokeAsync(
+            () -> WANTestBase.doPutsInsideTransactions(regionName, keyValues,
+                eventsPerTransaction));
+
+    // wait for batches to be distributed and then stop the sender
+    System.out.println("Waiting for some batches to be distributed");
+    vm4.invoke(() -> await()
+        .until(() -> WANTestBase.getSenderStats("ln", -1).get(4) > 0));
+    System.out
+        .println("Some batches distributed: " + vm4.invoke(() -> getSenderStats("ln", -1).get(4)));
+
+    addIgnoredException("Exception occurred in CacheListener");
+    addIgnoredException(RejectedExecutionException.class);
+
+    System.out.println("Stopping sender");
+    stopSenderInVMsAsync("ln", vm4, vm5);
+    System.out.println("Stopped sender");
+
+    inv1.await();
+    System.out.println("Puts completed");
+
+    vm4.invoke(() -> WANTestBase.validateRegionSize(regionName, entries));
+    vm5.invoke(() -> WANTestBase.validateRegionSize(regionName, entries));
+
+    // Wait for events to replicate: when batches received does not change
+    // we can assume that replication has finished.
+    int batchesReceived = (vm2.invoke(() -> getReceiverStats())).get(0);
+    while (true) {
+      int oldBatchesReceived = batchesReceived;
+      Thread.sleep(1000);
+      batchesReceived = (vm2.invoke(() -> getReceiverStats())).get(0);
+      if (batchesReceived == oldBatchesReceived) {
+        break;
+      }
+    }
+
+    ArrayList<Integer> v4List =
+        (ArrayList<Integer>) vm4.invoke(() -> WANTestBase.getSenderStats("ln", -1));
+    ArrayList<Integer> v5List =
+        (ArrayList<Integer>) vm5.invoke(() -> WANTestBase.getSenderStats("ln", -1));
+
+    System.out.println("v4List.get(0):" + v4List.get(0));
+    System.out.println("v5List.get(0):" + v5List.get(0));
+
+    // Wait for events to replicate: when region size does not change
+    // we can assume that replication has finished.
+    int regionSize = vm2.invoke(() -> getRegionSize(regionName));
+    while (true) {
+      int oldRegionSize = regionSize;
+      Thread.sleep(1000);
+      regionSize = vm2.invoke(() -> getRegionSize(regionName));
+      if (regionSize == oldRegionSize) {
+        break;
+      }
+    }
+
+    // Only complete transactions (11 entries each) must be replicated
+    assertEquals(0, regionSize % eventsPerTransaction);
+
+    // batches with incomplete transactions must be 0
+    assertEquals(0, (int) v4List.get(13));
+    assertEquals(0, (int) v5List.get(13));
+
+    System.out.println("regionSize: " + regionSize);
+
+    System.out.println("Starting sender");
+    startSenderInVMsAsync("ln", vm4, vm5);
+
+    v4List =
+        (ArrayList<Integer>) vm4.invoke(() -> WANTestBase.getSenderStats("ln", 0));
+    v5List =
+        (ArrayList<Integer>) vm5.invoke(() -> WANTestBase.getSenderStats("ln", 0));
+
+    assertEquals(0, v4List.get(0) + v5List.get(0));
+
+    // Wait for events to replicate: when batches received does not change
+    // we can assume that replication has finished.
+    batchesReceived = (vm2.invoke(() -> getReceiverStats())).get(0);
+    while (true) {
+      int oldBatchesReceived = batchesReceived;
+      Thread.sleep(1000);
+      batchesReceived = (vm2.invoke(() -> getReceiverStats())).get(0);
+      if (batchesReceived == oldBatchesReceived) {
+        break;
+      }
+    }
+
+    System.out.println("batchesReceived: " + batchesReceived);
+
+    // Wait for events to replicate: when region size does not change
+    // we can assume that replication has finished.
+    regionSize = vm2.invoke(() -> getRegionSize(regionName));
+    while (true) {
+      int oldRegionSize = regionSize;
+      Thread.sleep(1000);
+      regionSize = vm2.invoke(() -> getRegionSize(regionName));
+      if (regionSize == oldRegionSize) {
+        break;
+      }
+    }
+
+    System.out.println("regionSize: " + regionSize);
+
+    System.out.println("v4List.get(0):" + v4List.get(0));
+    System.out.println("v5List.get(0):" + v5List.get(0));
+
+    // batches with incomplete transactions must be 0
+    assertEquals(0, (int) v4List.get(13));
+    assertEquals(0, (int) v5List.get(13));
+
+    // Only complete transactions (11 events) must be replicated
+    assertEquals(0, regionSize % eventsPerTransaction);
+  }
+
+  @Test
+  public void testReplicatedSerialPropagationWithGroupTransactionEventsDoesNotSendBatchesWithIncompleteTransactionsIfGatewaySenderIsStoppedWhileReceivingTrafficAndLaterStartedReceiverStopped()

Review comment:
       Given that there is a large amount of duplicat code between this test case and the one above it, all comments that apply to the first test case also likely apply to this one.




----------------------------------------------------------------
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] albertogpz commented on a change in pull request #6052: GEODE-8971: Add grace period when stopping gateway sender with group-…

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



##########
File path: geode-core/src/main/java/org/apache/geode/internal/cache/BucketRegionQueue.java
##########
@@ -478,6 +480,24 @@ public Object peek() {
     }
   }
 
+  public boolean isThereEventsMatching(Predicate<InternalGatewayQueueEvent> matchingPredicate) {
+    getInitializationLock().readLock().lock();
+    try {
+      if (this.getPartitionedRegion().isDestroyed()) {
+        throw new BucketRegionQueueUnavailableException();
+      }
+      for (Object o : eventSeqNumDeque) {
+        Object object = optimalGet(o);
+        if (matchingPredicate.test((InternalGatewayQueueEvent) object)) {

Review comment:
       ok




----------------------------------------------------------------
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] kirklund commented on a change in pull request #6052: GEODE-8971: Add grace period when stopping gateway sender with group-…

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



##########
File path: geode-core/src/main/java/org/apache/geode/internal/cache/BucketRegionQueue.java
##########
@@ -453,21 +454,22 @@ public Object peek() {
    * If a matching object also fulfills the endPredicate then the method
    * stops looking for more matching objects.
    */
-  public List<Object> getElementsMatching(Predicate matchingPredicate, Predicate endPredicate) {
+  public List<Object> getElementsMatching(Predicate<GatewaySenderEventImpl> matchingPredicate,

Review comment:
       You might want to change from `GatewaySenderEventImpl` to your new `InternalGatewaySenderEvent` everywhere you've added new code. I don't see anything that would obviously break.




----------------------------------------------------------------
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] albertogpz commented on a change in pull request #6052: GEODE-8971: Add grace period when stopping gateway sender with group-…

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



##########
File path: geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/parallel/ParallelWANStatsDUnitTest.java
##########
@@ -472,6 +473,322 @@ public void testPRParallelPropagationWithGroupTransactionEventsSendsBatchesWithC
 
   }
 
+  @Test
+  public void testPRParallelPropagationWithGroupTransactionEventsDoesNotSendBatchesWithIncompleteTransactionsIfGatewaySenderIsStoppedWhileReceivingTrafficAndLaterStarted()
+      throws InterruptedException {
+    Integer lnPort = vm0.invoke(() -> WANTestBase.createFirstLocatorWithDSId(1));
+    Integer nyPort = vm1.invoke(() -> WANTestBase.createFirstRemoteLocator(2, lnPort));
+
+    createCacheInVMs(nyPort, vm2);
+    createReceiverInVMs(vm2);
+
+    createCacheInVMs(lnPort, vm4, vm5);
+    vm4.invoke(() -> WANTestBase.createSender("ln", 2, true, 100, 10, false, true, null, true,
+        true));
+    vm5.invoke(() -> WANTestBase.createSender("ln", 2, true, 100, 10, false, true, null, true,
+        true));
+
+    createReceiverCustomerOrderShipmentPR(vm2);
+
+    createSenderCustomerOrderShipmentPRs(vm4);
+    createSenderCustomerOrderShipmentPRs(vm5);
+
+    startSenderInVMs("ln", vm4, vm5);
+
+    int customers = 4;
+    int transactionsPerCustomer = 100;
+    int shipmentsPerTransaction = 10;
+    final LinkedHashMap<Object, Object> keyValuesInTransactions = new LinkedHashMap<>();
+    for (int custId = 0; custId < customers; custId++) {
+      for (int i = 0; i < transactionsPerCustomer; i++) {
+        CustId custIdObject = new CustId(custId);
+        OrderId orderId = new OrderId(i, custIdObject);
+        keyValuesInTransactions.put(orderId, new Order());
+        for (int j = 0; j < shipmentsPerTransaction; j++) {
+          ShipmentId shipmentId = new ShipmentId(i + j, orderId);
+          keyValuesInTransactions.put(shipmentId, new Shipment());
+        }
+      }
+    }
+    int eventsPerTransaction = 1 + shipmentsPerTransaction;
+    AsyncInvocation<Void> inv1 =
+        vm4.invokeAsync(
+            () -> WANTestBase.doOrderAndShipmentPutsInsideTransactions(keyValuesInTransactions,
+                eventsPerTransaction));
+
+    // wait for some batches to be distributed and then stop the sender
+    vm4.invoke(() -> await()
+        .until(() -> WANTestBase.getSenderStats("ln", -1).get(4) > 0));
+
+    System.out.println("Stopping sender");

Review comment:
       ok




----------------------------------------------------------------
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] albertogpz commented on a change in pull request #6052: GEODE-8971: Add grace period when stopping gateway sender with group-…

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



##########
File path: geode-core/src/main/java/org/apache/geode/internal/cache/wan/AbstractGatewaySender.java
##########
@@ -1100,7 +1108,21 @@ public void distribute(EnumListenerEvent operation, EntryEventImpl event,
           // Get substitution value to enqueue if necessary
           Object substituteValue = getSubstituteValue(clonedEvent, operation);
 
-          ev.enqueueEvent(operation, clonedEvent, substituteValue, isLastEventInTransaction);
+          Predicate<InternalGatewayQueueEvent> hasSameTransactionIdPredicate = null;
+          // In case the sender is about to be stopped, the event will only
+          // be queued if there is any event in the queue with the same
+          // transactionId as the one of this event
+          if (isStopping && mustGroupTransactionEvents()
+              && clonedEvent.getTransactionId() != null) {
+            hasSameTransactionIdPredicate =
+                x -> x instanceof GatewaySenderEventImpl && clonedEvent.getTransactionId() != null

Review comment:
       ok




----------------------------------------------------------------
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] albertogpz commented on a change in pull request #6052: GEODE-8971: Add grace period when stopping gateway sender with group-…

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



##########
File path: geode-core/src/main/java/org/apache/geode/internal/cache/wan/serial/ConcurrentSerialGatewaySenderEventProcessor.java
##########
@@ -418,10 +424,13 @@ protected void registerEventDroppedInPrimaryQueue(EntryEventImpl droppedEvent) {
   }
 
   @Override
-  protected void enqueueEvent(GatewayQueueEvent event) {
+  protected boolean enqueueEvent(GatewayQueueEvent event,
+      Predicate<InternalGatewayQueueEvent> condition) {
     for (SerialGatewaySenderEventProcessor serialProcessor : this.processors) {
-      serialProcessor.enqueueEvent(event);
+      // TODO revisit handling of "condition" when the following enqueueEvent() method is supported:

Review comment:
       ok




----------------------------------------------------------------
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] albertogpz commented on a change in pull request #6052: GEODE-8971: Add grace period when stopping gateway sender with group-…

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



##########
File path: geode-core/src/main/java/org/apache/geode/internal/cache/wan/serial/SerialGatewaySenderQueue.java
##########
@@ -847,7 +855,14 @@ private EventsAndLastKey peekEventsWithTransactionId(TransactionId transactionId
     }
   }
 
-  EventsAndLastKey getElementsMatching(Predicate condition, Predicate stopCondition, long lastKey) {
+  /**
+   * This method returns a list of objects that fulfill the matchingPredicate
+   * If a matching object also fulfills the endPredicate then the method
+   * stops looking for more matching objects.
+   */
+  List<KeyAndEventPair> getElementsMatching(Predicate<InternalGatewayQueueEvent> condition,
+      Predicate<InternalGatewayQueueEvent> stopCondition,
+      long lastKey) {
     Object object;
     List elementsMatching = new ArrayList<>();

Review comment:
       ok




----------------------------------------------------------------
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] albertogpz commented on a change in pull request #6052: GEODE-8971: Add grace period when stopping gateway sender with group-…

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



##########
File path: geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/parallel/ParallelWANStatsDUnitTest.java
##########
@@ -472,6 +473,322 @@ public void testPRParallelPropagationWithGroupTransactionEventsSendsBatchesWithC
 
   }
 
+  @Test
+  public void testPRParallelPropagationWithGroupTransactionEventsDoesNotSendBatchesWithIncompleteTransactionsIfGatewaySenderIsStoppedWhileReceivingTrafficAndLaterStarted()
+      throws InterruptedException {
+    Integer lnPort = vm0.invoke(() -> WANTestBase.createFirstLocatorWithDSId(1));
+    Integer nyPort = vm1.invoke(() -> WANTestBase.createFirstRemoteLocator(2, lnPort));
+
+    createCacheInVMs(nyPort, vm2);
+    createReceiverInVMs(vm2);
+
+    createCacheInVMs(lnPort, vm4, vm5);
+    vm4.invoke(() -> WANTestBase.createSender("ln", 2, true, 100, 10, false, true, null, true,
+        true));
+    vm5.invoke(() -> WANTestBase.createSender("ln", 2, true, 100, 10, false, true, null, true,
+        true));
+
+    createReceiverCustomerOrderShipmentPR(vm2);
+
+    createSenderCustomerOrderShipmentPRs(vm4);
+    createSenderCustomerOrderShipmentPRs(vm5);
+
+    startSenderInVMs("ln", vm4, vm5);
+
+    int customers = 4;
+    int transactionsPerCustomer = 100;
+    int shipmentsPerTransaction = 10;
+    final LinkedHashMap<Object, Object> keyValuesInTransactions = new LinkedHashMap<>();
+    for (int custId = 0; custId < customers; custId++) {
+      for (int i = 0; i < transactionsPerCustomer; i++) {
+        CustId custIdObject = new CustId(custId);
+        OrderId orderId = new OrderId(i, custIdObject);
+        keyValuesInTransactions.put(orderId, new Order());
+        for (int j = 0; j < shipmentsPerTransaction; j++) {
+          ShipmentId shipmentId = new ShipmentId(i + j, orderId);
+          keyValuesInTransactions.put(shipmentId, new Shipment());
+        }
+      }
+    }
+    int eventsPerTransaction = 1 + shipmentsPerTransaction;
+    AsyncInvocation<Void> inv1 =
+        vm4.invokeAsync(
+            () -> WANTestBase.doOrderAndShipmentPutsInsideTransactions(keyValuesInTransactions,
+                eventsPerTransaction));
+
+    // wait for some batches to be distributed and then stop the sender
+    vm4.invoke(() -> await()
+        .until(() -> WANTestBase.getSenderStats("ln", -1).get(4) > 0));
+
+    System.out.println("Stopping sender");
+    stopSenderInVMsAsync("ln", vm4, vm5);
+    System.out.println("Stopped sender");
+
+    inv1.await();
+    int entries =
+        transactionsPerCustomer * customers;
+
+    vm4.invoke(() -> WANTestBase.validateRegionSize(orderRegionName, entries));
+    vm5.invoke(() -> WANTestBase.validateRegionSize(orderRegionName, entries));
+
+    // Wait for events to replicate: when batches received does not change
+    // we can assume that replication has finished.
+    int batchesReceived = (vm2.invoke(() -> getReceiverStats())).get(0);

Review comment:
       ok




----------------------------------------------------------------
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] albertogpz commented on pull request #6052: GEODE-8971: Add grace period when stopping gateway sender with group-…

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


   > Unless we have a real need to expose `TransactionId getTransactionId()` and `boolean isLastEventInTransaction()` to the User (it's in a non-internal package), then I would **_not_** add them to the `GatewayQueueEvent`.
   > 
   > I would create a new `InternalGatewayQueueEvent` interface that extends `GatewayQueueEvent` and add the two tx related methods to that internal interface instead. Then you could define the predicate as `Predicate<InternalGatewayQueueEvent>`.
   > 
   > Adding `InternalGatewayQueueEvent` would probably require some follow-up PRs that update the WAN code to use it internally. This would them be part of longer term refactoring to improve the codebase with interfaces and is similar to the many other internal interfaces that we've been adding including `InternalRegion` and `InternalCache`. Note that we tend to not parameterize these internal interfaces.
   
   Thanks for the comments. I have done as suggested.


----------------------------------------------------------------
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] albertogpz commented on a change in pull request #6052: GEODE-8971: Add grace period when stopping gateway sender with group-…

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



##########
File path: geode-core/src/main/java/org/apache/geode/internal/cache/wan/AbstractGatewaySender.java
##########
@@ -1100,7 +1108,21 @@ public void distribute(EnumListenerEvent operation, EntryEventImpl event,
           // Get substitution value to enqueue if necessary
           Object substituteValue = getSubstituteValue(clonedEvent, operation);
 
-          ev.enqueueEvent(operation, clonedEvent, substituteValue, isLastEventInTransaction);
+          Predicate<InternalGatewayQueueEvent> hasSameTransactionIdPredicate = null;
+          // In case the sender is about to be stopped, the event will only
+          // be queued if there is any event in the queue with the same
+          // transactionId as the one of this event
+          if (isStopping && mustGroupTransactionEvents()
+              && clonedEvent.getTransactionId() != null) {
+            hasSameTransactionIdPredicate =
+                x -> x instanceof GatewaySenderEventImpl && clonedEvent.getTransactionId() != null
+                    && clonedEvent.getTransactionId()
+                        .equals(((GatewaySenderEventImpl) x).getTransactionId());

Review comment:
       ok




----------------------------------------------------------------
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] albertogpz commented on a change in pull request #6052: GEODE-8971: Add grace period when stopping gateway sender with group-…

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



##########
File path: geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/serial/SerialWANStatsDUnitTest.java
##########
@@ -348,7 +349,276 @@ public void testReplicatedSerialPropagationWithBatchRedistWithGroupTransactionEv
   }
 
   @Test
-  public void testReplicatedSerialPropagationWithMultipleDispatchers() throws Exception {
+  public void testReplicatedSerialPropagationWithGroupTransactionEventsDoesNotSendBatchesWithIncompleteTransactionsIfGatewaySenderIsStoppedWhileReceivingTrafficAndLaterStarted()
+      throws InterruptedException {
+    Integer lnPort = vm0.invoke(() -> WANTestBase.createFirstLocatorWithDSId(1));
+    Integer nyPort = vm1.invoke(() -> WANTestBase.createFirstRemoteLocator(2, lnPort));
+
+    String regionName = testName + "_RR";
+
+    createCacheInVMs(nyPort, vm2);
+    createReceiverInVMs(vm2);
+
+    createCacheInVMs(lnPort, vm4, vm5);
+    boolean groupTransactionEvents = true;
+    int batchSize = 10;
+    vm4.invoke(
+        () -> WANTestBase.createSender("ln", 2, false, 100, batchSize, false, true, null, true,
+            groupTransactionEvents));
+    vm5.invoke(
+        () -> WANTestBase.createSender("ln", 2, false, 100, batchSize, false, true, null, true,
+            groupTransactionEvents));
+
+    vm2.invoke(() -> WANTestBase.createReplicatedRegion(regionName, null, isOffHeap()));
+
+    vm4.invoke(() -> WANTestBase.createReplicatedRegion(regionName, "ln", isOffHeap()));
+    vm5.invoke(() -> WANTestBase.createReplicatedRegion(regionName, "ln", isOffHeap()));
+
+    startSenderInVMs("ln", vm4, vm5);
+
+    final Map<Object, Object> keyValues = new LinkedHashMap<>();
+    int entries = 2200;

Review comment:
       Not really. I picked a number divisible by 11 (as transactions are that size) and big enough to allow for some entries to be replicated by the gateway sender before it was stopped 




----------------------------------------------------------------
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] albertogpz commented on a change in pull request #6052: GEODE-8971: Add grace period when stopping gateway sender with group-…

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



##########
File path: geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/parallel/ParallelWANStatsDUnitTest.java
##########
@@ -472,6 +473,322 @@ public void testPRParallelPropagationWithGroupTransactionEventsSendsBatchesWithC
 
   }
 
+  @Test
+  public void testPRParallelPropagationWithGroupTransactionEventsDoesNotSendBatchesWithIncompleteTransactionsIfGatewaySenderIsStoppedWhileReceivingTrafficAndLaterStarted()
+      throws InterruptedException {
+    Integer lnPort = vm0.invoke(() -> WANTestBase.createFirstLocatorWithDSId(1));
+    Integer nyPort = vm1.invoke(() -> WANTestBase.createFirstRemoteLocator(2, lnPort));
+
+    createCacheInVMs(nyPort, vm2);
+    createReceiverInVMs(vm2);
+
+    createCacheInVMs(lnPort, vm4, vm5);
+    vm4.invoke(() -> WANTestBase.createSender("ln", 2, true, 100, 10, false, true, null, true,
+        true));
+    vm5.invoke(() -> WANTestBase.createSender("ln", 2, true, 100, 10, false, true, null, true,
+        true));
+
+    createReceiverCustomerOrderShipmentPR(vm2);
+
+    createSenderCustomerOrderShipmentPRs(vm4);
+    createSenderCustomerOrderShipmentPRs(vm5);
+
+    startSenderInVMs("ln", vm4, vm5);
+
+    int customers = 4;
+    int transactionsPerCustomer = 100;
+    int shipmentsPerTransaction = 10;
+    final LinkedHashMap<Object, Object> keyValuesInTransactions = new LinkedHashMap<>();
+    for (int custId = 0; custId < customers; custId++) {
+      for (int i = 0; i < transactionsPerCustomer; i++) {
+        CustId custIdObject = new CustId(custId);
+        OrderId orderId = new OrderId(i, custIdObject);
+        keyValuesInTransactions.put(orderId, new Order());
+        for (int j = 0; j < shipmentsPerTransaction; j++) {
+          ShipmentId shipmentId = new ShipmentId(i + j, orderId);
+          keyValuesInTransactions.put(shipmentId, new Shipment());
+        }
+      }
+    }
+    int eventsPerTransaction = 1 + shipmentsPerTransaction;
+    AsyncInvocation<Void> inv1 =
+        vm4.invokeAsync(
+            () -> WANTestBase.doOrderAndShipmentPutsInsideTransactions(keyValuesInTransactions,
+                eventsPerTransaction));
+
+    // wait for some batches to be distributed and then stop the sender
+    vm4.invoke(() -> await()
+        .until(() -> WANTestBase.getSenderStats("ln", -1).get(4) > 0));
+
+    System.out.println("Stopping sender");
+    stopSenderInVMsAsync("ln", vm4, vm5);
+    System.out.println("Stopped sender");
+
+    inv1.await();
+    int entries =
+        transactionsPerCustomer * customers;
+
+    vm4.invoke(() -> WANTestBase.validateRegionSize(orderRegionName, entries));
+    vm5.invoke(() -> WANTestBase.validateRegionSize(orderRegionName, entries));
+
+    // Wait for events to replicate: when batches received does not change
+    // we can assume that replication has finished.
+    int batchesReceived = (vm2.invoke(() -> getReceiverStats())).get(0);
+    while (true) {
+      int oldBatchesReceived = batchesReceived;
+      Thread.sleep(1000);
+      batchesReceived = (vm2.invoke(() -> getReceiverStats())).get(0);
+      if (batchesReceived == oldBatchesReceived) {
+        break;
+      }
+    }
+
+    System.out.println("batchesReceived after stop: " + batchesReceived);
+
+    ArrayList<Integer> v4List =
+        (ArrayList<Integer>) vm4.invoke(() -> WANTestBase.getSenderStats("ln", -1));
+    ArrayList<Integer> v5List =
+        (ArrayList<Integer>) vm5.invoke(() -> WANTestBase.getSenderStats("ln", -1));
+
+    // Wait for events to replicate: when region size does not change
+    // we can assume that replication has finished.
+    int orderRegionSize = vm2.invoke(() -> getRegionSize(orderRegionName));
+    while (true) {
+      Thread.sleep(1000);
+      int newRegionSize = vm2.invoke(() -> getRegionSize(orderRegionName));
+      if (orderRegionSize == newRegionSize) {
+        break;
+      }
+    }
+
+    // Wait for events to replicate: when region size does not change
+    // we can assume that replication has finished.
+    int shipmentRegionSize = vm2.invoke(() -> getRegionSize(shipmentRegionName));
+    while (true) {
+      Thread.sleep(1000);
+      int newRegionSize = vm2.invoke(() -> getRegionSize(shipmentRegionName));
+      if (shipmentRegionSize == newRegionSize) {
+        break;
+      }
+    }
+
+    System.out.println("orderRegionSize: " + shipmentRegionSize);
+    System.out.println("shipmentRegionSize: " + shipmentRegionSize);
+
+    System.out.println("v4List.get(0): " + v4List.get(0));
+    System.out.println("v5List.get(0): " + v5List.get(0));
+
+    // batches with incomplete transactions must be 0
+    assertEquals(0, (int) v4List.get(13));
+    assertEquals(0, (int) v5List.get(13));
+
+    // Only complete transactions (1 order + 10 shipments) must be replicated
+    assertEquals(10, shipmentRegionSize / orderRegionSize);

Review comment:
       Indeed




----------------------------------------------------------------
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] albertogpz commented on pull request #6052: GEODE-8971: Add grace period when stopping gateway sender with group-…

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


   > Quite a lot of changes here to clean up the code. I'm also concerned that there's not enough test coverage for the changes in terms of possible failure conditions, such as if two threads call `stop()` on the same gateway sender within quick succession. More unit tests should also be added for the new/modified behaviour.
   
   Thanks a lot for your thorough review. Impressive, as usual.
   
   I have added a couple of tests in `SerialGatewaySenderImplTest` and `ParallelGatewalSenderImplTest`to verify that two threads calling `stop()` on the same gateway sender within quick succession does not provoke any harm and the `preStop` and `postStop` methods do their job.


----------------------------------------------------------------
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] gesterzhou commented on a change in pull request #6052: GEODE-8971: Add grace period when stopping gateway sender with group-…

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



##########
File path: geode-core/src/main/java/org/apache/geode/cache/wan/GatewaySender.java
##########
@@ -174,7 +174,32 @@
    */
   int GET_TRANSACTION_EVENTS_FROM_QUEUE_RETRIES =
       Integer.getInteger(GeodeGlossary.GEMFIRE_PREFIX + "get-transaction-events-from-queue-retries",
-          10);
+          2);

Review comment:
       But 2 is still too less. How about 3~5?
   




-- 
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] albertogpz commented on a change in pull request #6052: GEODE-8971: Add grace period when stopping gateway sender with group-…

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



##########
File path: geode-core/src/main/java/org/apache/geode/internal/cache/wan/serial/SerialGatewaySenderQueue.java
##########
@@ -862,19 +877,35 @@ EventsAndLastKey getElementsMatching(Predicate condition, Predicate stopConditio
         continue;
       }
 
-      if (condition.test(object)) {
-        elementsMatching.add(object);
-        peekedIds.add(currentKey);
-        extraPeekedIds.add(currentKey);
-        lastKey = currentKey;
+      if (condition.test((InternalGatewayQueueEvent) object)) {
+        elementsMatching.add(new KeyAndEventPair(currentKey, (GatewaySenderEventImpl) object));
 
-        if (stopCondition.test(object)) {
+        if (stopCondition.test((InternalGatewayQueueEvent) object)) {
           break;
         }
       }
     }
 
-    return new EventsAndLastKey(elementsMatching, lastKey);
+    return elementsMatching;
+  }
+
+  public boolean isThereEventsMatching(Predicate<InternalGatewayQueueEvent> condition) {
+    InternalGatewayQueueEvent object;

Review comment:
       ok




----------------------------------------------------------------
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] albertogpz commented on a change in pull request #6052: GEODE-8971: Add grace period when stopping gateway sender with group-…

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



##########
File path: geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/parallel/ParallelWANStatsDUnitTest.java
##########
@@ -472,6 +473,200 @@ public void testPRParallelPropagationWithGroupTransactionEventsSendsBatchesWithC
 
   }
 
+  @Test
+  public void testPRParallelPropagationWithGroupTransactionEventsDoesNotSendBatchesWithIncompleteTransactionsIfGatewaySenderIsStoppedWhileReceivingTrafficAndLaterStarted()
+      throws InterruptedException {
+    Integer lnPort = vm0.invoke(() -> WANTestBase.createFirstLocatorWithDSId(1));
+    Integer nyPort = vm1.invoke(() -> WANTestBase.createFirstRemoteLocator(2, lnPort));
+
+    createCacheInVMs(nyPort, vm2);
+    createReceiverCustomerOrderShipmentPR(vm2);
+    createReceiverInVMs(vm2);
+
+    createCacheInVMs(lnPort, vm4, vm5);
+    createSenderCustomerOrderShipmentPRs(vm4);
+    createSenderCustomerOrderShipmentPRs(vm5);
+
+    int batchSize = 10;
+    vm4.invoke(
+        () -> WANTestBase.createSender("ln", 2, true, 100, batchSize, false, true, null, false,
+            true));
+    vm5.invoke(
+        () -> WANTestBase.createSender("ln", 2, true, 100, batchSize, false, true, null, false,
+            true));
+
+    int customers = 4;
+    int transactionsPerCustomer = 100;
+    // Each transaction will contain one order plus the following shipments
+    int shipmentsPerTransaction = batchSize;
+    AsyncInvocation<Void> inv1 = asyncExecuteCustomerTransactions(vm4, customers,
+        transactionsPerCustomer, shipmentsPerTransaction);
+
+    // wait for some batches to be distributed and then stop the sender
+    vm4.invoke(() -> await()
+        .until(() -> WANTestBase.getSenderStats("ln", -1).get(4) > 0));
+
+    stopSenderInVMsAsync("ln", vm4, vm5);
+
+    // Wait for customer transactions to finish
+    inv1.await();
+    int orderEntries = transactionsPerCustomer * customers;
+    int shipmentEntries = orderEntries * 10;

Review comment:
       Totally agree




----------------------------------------------------------------
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] DonalEvans commented on a change in pull request #6052: GEODE-8971: Add grace period when stopping gateway sender with group-…

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



##########
File path: geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/parallel/ParallelWANStatsDUnitTest.java
##########
@@ -472,6 +473,200 @@ public void testPRParallelPropagationWithGroupTransactionEventsSendsBatchesWithC
 
   }
 
+  @Test
+  public void testPRParallelPropagationWithGroupTransactionEventsDoesNotSendBatchesWithIncompleteTransactionsIfGatewaySenderIsStoppedWhileReceivingTrafficAndLaterStarted()
+      throws InterruptedException {
+    Integer lnPort = vm0.invoke(() -> WANTestBase.createFirstLocatorWithDSId(1));
+    Integer nyPort = vm1.invoke(() -> WANTestBase.createFirstRemoteLocator(2, lnPort));
+
+    createCacheInVMs(nyPort, vm2);
+    createReceiverCustomerOrderShipmentPR(vm2);
+    createReceiverInVMs(vm2);
+
+    createCacheInVMs(lnPort, vm4, vm5);
+    createSenderCustomerOrderShipmentPRs(vm4);
+    createSenderCustomerOrderShipmentPRs(vm5);
+
+    int batchSize = 10;
+    vm4.invoke(
+        () -> WANTestBase.createSender("ln", 2, true, 100, batchSize, false, true, null, false,
+            true));
+    vm5.invoke(
+        () -> WANTestBase.createSender("ln", 2, true, 100, batchSize, false, true, null, false,
+            true));
+
+    int customers = 4;
+    int transactionsPerCustomer = 100;
+    // Each transaction will contain one order plus the following shipments
+    int shipmentsPerTransaction = batchSize;
+    AsyncInvocation<Void> inv1 = asyncExecuteCustomerTransactions(vm4, customers,
+        transactionsPerCustomer, shipmentsPerTransaction);
+
+    // wait for some batches to be distributed and then stop the sender
+    vm4.invoke(() -> await()
+        .until(() -> WANTestBase.getSenderStats("ln", -1).get(4) > 0));
+
+    stopSenderInVMsAsync("ln", vm4, vm5);
+
+    // Wait for customer transactions to finish
+    inv1.await();
+    int orderEntries = transactionsPerCustomer * customers;
+    int shipmentEntries = orderEntries * 10;

Review comment:
       I think this could be `int shipmentEntries = orderEntries * shipmentsPerTransaction;` for additional clarity.

##########
File path: geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/parallel/ParallelWANStatsDUnitTest.java
##########
@@ -472,6 +473,200 @@ public void testPRParallelPropagationWithGroupTransactionEventsSendsBatchesWithC
 
   }
 
+  @Test
+  public void testPRParallelPropagationWithGroupTransactionEventsDoesNotSendBatchesWithIncompleteTransactionsIfGatewaySenderIsStoppedWhileReceivingTrafficAndLaterStarted()
+      throws InterruptedException {
+    Integer lnPort = vm0.invoke(() -> WANTestBase.createFirstLocatorWithDSId(1));
+    Integer nyPort = vm1.invoke(() -> WANTestBase.createFirstRemoteLocator(2, lnPort));
+
+    createCacheInVMs(nyPort, vm2);
+    createReceiverCustomerOrderShipmentPR(vm2);
+    createReceiverInVMs(vm2);
+
+    createCacheInVMs(lnPort, vm4, vm5);
+    createSenderCustomerOrderShipmentPRs(vm4);
+    createSenderCustomerOrderShipmentPRs(vm5);
+
+    int batchSize = 10;
+    vm4.invoke(
+        () -> WANTestBase.createSender("ln", 2, true, 100, batchSize, false, true, null, false,
+            true));
+    vm5.invoke(
+        () -> WANTestBase.createSender("ln", 2, true, 100, batchSize, false, true, null, false,
+            true));
+
+    int customers = 4;
+    int transactionsPerCustomer = 100;
+    // Each transaction will contain one order plus the following shipments
+    int shipmentsPerTransaction = batchSize;
+    AsyncInvocation<Void> inv1 = asyncExecuteCustomerTransactions(vm4, customers,
+        transactionsPerCustomer, shipmentsPerTransaction);
+
+    // wait for some batches to be distributed and then stop the sender
+    vm4.invoke(() -> await()
+        .until(() -> WANTestBase.getSenderStats("ln", -1).get(4) > 0));
+
+    stopSenderInVMsAsync("ln", vm4, vm5);
+
+    // Wait for customer transactions to finish
+    inv1.await();
+    int orderEntries = transactionsPerCustomer * customers;
+    int shipmentEntries = orderEntries * 10;
+    vm4.invoke(() -> WANTestBase.validateRegionSize(orderRegionName, orderEntries));
+    vm5.invoke(() -> WANTestBase.validateRegionSize(orderRegionName, orderEntries));
+    vm4.invoke(() -> WANTestBase.validateRegionSize(shipmentRegionName, shipmentEntries));
+    vm5.invoke(() -> WANTestBase.validateRegionSize(shipmentRegionName, shipmentEntries));
+
+    checkOnlyCompleteTransactionsAreReplicatedAfterSenderStopped(
+        shipmentsPerTransaction);
+
+    // Start sender to validate that queued events do not contain incomplete transactions after
+    // restart
+    startSenderInVMsAsync("ln", vm4, vm5);
+
+    checkOnlyCompleteTransactionsAreReplicatedWithSenderRestarted(shipmentsPerTransaction);
+  }
+
+  @Test
+  public void testPRParallelPropagationWithGroupTransactionEventsDoesNotSendBatchesWithIncompleteTransactionsIfGatewaySenderIsStoppedWhileReceivingTrafficAndLaterStartedReceiverStopped()
+      throws InterruptedException {
+    Integer lnPort = vm0.invoke(() -> WANTestBase.createFirstLocatorWithDSId(1));
+    Integer nyPort = vm1.invoke(() -> WANTestBase.createFirstRemoteLocator(2, lnPort));
+
+    createCacheInVMs(nyPort, vm2);
+    createReceiverCustomerOrderShipmentPR(vm2);
+    createReceiverInVMs(vm2);
+    vm2.invoke(WANTestBase::stopReceivers);
+
+    createCacheInVMs(lnPort, vm4, vm5);
+    createSenderCustomerOrderShipmentPRs(vm4);
+    createSenderCustomerOrderShipmentPRs(vm5);
+
+    int batchSize = 10;
+    vm4.invoke(() -> WANTestBase.createSender("ln", 2, true, 100, 10, false, true, null, false,
+        true));
+    vm5.invoke(() -> WANTestBase.createSender("ln", 2, true, 100, 10, false, true, null, false,
+        true));
+
+    int customers = 4;
+    int transactionsPerCustomer = 100;
+    // Each transaction will contain one order plus the following shipments
+    int shipmentsPerTransaction = batchSize;
+
+    AsyncInvocation<Void> inv1 = asyncExecuteCustomerTransactions(vm4, customers,
+        transactionsPerCustomer, shipmentsPerTransaction);
+
+    // wait for some batches to be redistributed and then stop the sender
+    vm4.invoke(() -> await()
+        .until(() -> WANTestBase.getSenderStats("ln", -1).get(5) > 0));
+
+    stopSenderInVMsAsync("ln", vm4, vm5);
+
+    // Wait for the customer transactions to finish
+    inv1.await();
+    int orderEntries = transactionsPerCustomer * customers;
+    int shipmentEntries = orderEntries * 10;
+    vm4.invoke(() -> WANTestBase.validateRegionSize(orderRegionName, orderEntries));
+    vm5.invoke(() -> WANTestBase.validateRegionSize(orderRegionName, orderEntries));
+    vm4.invoke(() -> WANTestBase.validateRegionSize(shipmentRegionName, shipmentEntries));
+    vm5.invoke(() -> WANTestBase.validateRegionSize(shipmentRegionName, shipmentEntries));
+
+    // Start receiver and sender
+    vm2.invoke(WANTestBase::startReceivers);
+    startSenderInVMsAsync("ln", vm4, vm5);
+
+    checkOnlyCompleteTransactionsAreReplicatedWithSenderRestarted(shipmentsPerTransaction);
+  }
+
+  private AsyncInvocation<Void> asyncExecuteCustomerTransactions(VM vm, int customers,
+      int transactionsPerCustomer, int shipmentsPerTransaction) {
+    final Map<Object, Object> keyValuesInTransactions = new LinkedHashMap<>();
+    for (int custId = 0; custId < customers; custId++) {
+      for (int i = 0; i < transactionsPerCustomer; i++) {
+        CustId custIdObject = new CustId(custId);
+        OrderId orderId = new OrderId(i, custIdObject);
+        keyValuesInTransactions.put(orderId, new Order());
+        for (int j = 0; j < shipmentsPerTransaction; j++) {
+          ShipmentId shipmentId = new ShipmentId(i + j, orderId);
+          keyValuesInTransactions.put(shipmentId, new Shipment());
+        }
+      }
+    }
+    int eventsPerTransaction = 1 + shipmentsPerTransaction;
+    return vm.invokeAsync(
+        () -> WANTestBase.doOrderAndShipmentPutsInsideTransactions(keyValuesInTransactions,
+            eventsPerTransaction));
+  }
+
+  private void checkOnlyCompleteTransactionsAreReplicatedAfterSenderStopped(
+      int shipmentsPerTransaction) {
+    waitForBatchesToBeAppliedInTheReceiver(shipmentsPerTransaction);
+
+    List<Integer> v4List =
+        vm4.invoke(() -> WANTestBase.getSenderStats("ln", -1));
+    List<Integer> v5List =
+        vm5.invoke(() -> WANTestBase.getSenderStats("ln", -1));
+
+    // batches with incomplete transactions must be 0
+    assertEquals(0, (int) v4List.get(13));
+    assertEquals(0, (int) v5List.get(13));
+
+    // Check the entries replicated against the number of batches distributed
+    int batchesDistributed = v4List.get(4) + v5List.get(4);
+    checkOnlyCompleteTransactionsAreReplicated(shipmentsPerTransaction, batchesDistributed);
+  }
+
+  private void checkOnlyCompleteTransactionsAreReplicatedWithSenderRestarted(
+      int shipmentsPerTransaction) {
+    // Wait for sender queues to be drained
+    vm4.invoke(() -> WANTestBase.validateParallelSenderQueueAllBucketsDrained("ln"));
+    vm5.invoke(() -> WANTestBase.validateParallelSenderQueueAllBucketsDrained("ln"));
+
+    List<Integer> v4List =
+        vm4.invoke(() -> WANTestBase.getSenderStats("ln", 0));
+    List<Integer> v5List =
+        vm5.invoke(() -> WANTestBase.getSenderStats("ln", 0));
+    assertEquals(0, v4List.get(0) + v5List.get(0));
+
+    // batches with incomplete transactions must be 0
+    assertEquals(0, (int) v4List.get(13));
+    assertEquals(0, (int) v5List.get(13));
+
+    waitForBatchesToBeAppliedInTheReceiver(shipmentsPerTransaction);
+
+    // Check the entries replicated against the number of batches distributed
+    int batchesDistributed = v4List.get(4) + v5List.get(4);
+    checkOnlyCompleteTransactionsAreReplicated(shipmentsPerTransaction, batchesDistributed);
+  }
+
+  private void checkOnlyCompleteTransactionsAreReplicated(int shipmentsPerTransaction,
+      int batchesDistributed) {
+    // Only complete transactions (1 order + 10 shipments) must be replicated
+    int orderRegionSize = vm2.invoke(() -> getRegionSize(orderRegionName));
+    int shipmentRegionSize = vm2.invoke(() -> getRegionSize(shipmentRegionName));
+    assertEquals(shipmentRegionSize, 10 * orderRegionSize);
+
+    vm2.invoke(() -> WANTestBase.validateRegionSize(orderRegionName, batchesDistributed));
+    vm2.invoke(() -> WANTestBase.validateRegionSize(shipmentRegionName,
+        batchesDistributed * shipmentsPerTransaction));
+  }
+
+  private void waitForBatchesToBeAppliedInTheReceiver(int shipmentsPerTransaction) {
+    int batchesSentTotal = vm4.invoke(() -> WANTestBase.getSenderStats("ln", -1)).get(4) +
+        vm5.invoke(() -> WANTestBase.getSenderStats("ln", -1)).get(4);
+
+    // Wait for all batches to be received by the sender

Review comment:
       I think this comment is supposed to be "received by the receiver"

##########
File path: geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/parallel/ParallelWANStatsDUnitTest.java
##########
@@ -472,6 +473,200 @@ public void testPRParallelPropagationWithGroupTransactionEventsSendsBatchesWithC
 
   }
 
+  @Test
+  public void testPRParallelPropagationWithGroupTransactionEventsDoesNotSendBatchesWithIncompleteTransactionsIfGatewaySenderIsStoppedWhileReceivingTrafficAndLaterStarted()
+      throws InterruptedException {
+    Integer lnPort = vm0.invoke(() -> WANTestBase.createFirstLocatorWithDSId(1));
+    Integer nyPort = vm1.invoke(() -> WANTestBase.createFirstRemoteLocator(2, lnPort));
+
+    createCacheInVMs(nyPort, vm2);
+    createReceiverCustomerOrderShipmentPR(vm2);
+    createReceiverInVMs(vm2);
+
+    createCacheInVMs(lnPort, vm4, vm5);
+    createSenderCustomerOrderShipmentPRs(vm4);
+    createSenderCustomerOrderShipmentPRs(vm5);
+
+    int batchSize = 10;
+    vm4.invoke(
+        () -> WANTestBase.createSender("ln", 2, true, 100, batchSize, false, true, null, false,
+            true));
+    vm5.invoke(
+        () -> WANTestBase.createSender("ln", 2, true, 100, batchSize, false, true, null, false,
+            true));
+
+    int customers = 4;
+    int transactionsPerCustomer = 100;
+    // Each transaction will contain one order plus the following shipments
+    int shipmentsPerTransaction = batchSize;
+    AsyncInvocation<Void> inv1 = asyncExecuteCustomerTransactions(vm4, customers,
+        transactionsPerCustomer, shipmentsPerTransaction);
+
+    // wait for some batches to be distributed and then stop the sender
+    vm4.invoke(() -> await()
+        .until(() -> WANTestBase.getSenderStats("ln", -1).get(4) > 0));
+
+    stopSenderInVMsAsync("ln", vm4, vm5);
+
+    // Wait for customer transactions to finish
+    inv1.await();
+    int orderEntries = transactionsPerCustomer * customers;
+    int shipmentEntries = orderEntries * 10;
+    vm4.invoke(() -> WANTestBase.validateRegionSize(orderRegionName, orderEntries));
+    vm5.invoke(() -> WANTestBase.validateRegionSize(orderRegionName, orderEntries));
+    vm4.invoke(() -> WANTestBase.validateRegionSize(shipmentRegionName, shipmentEntries));
+    vm5.invoke(() -> WANTestBase.validateRegionSize(shipmentRegionName, shipmentEntries));
+
+    checkOnlyCompleteTransactionsAreReplicatedAfterSenderStopped(
+        shipmentsPerTransaction);
+
+    // Start sender to validate that queued events do not contain incomplete transactions after
+    // restart
+    startSenderInVMsAsync("ln", vm4, vm5);
+
+    checkOnlyCompleteTransactionsAreReplicatedWithSenderRestarted(shipmentsPerTransaction);
+  }
+
+  @Test
+  public void testPRParallelPropagationWithGroupTransactionEventsDoesNotSendBatchesWithIncompleteTransactionsIfGatewaySenderIsStoppedWhileReceivingTrafficAndLaterStartedReceiverStopped()
+      throws InterruptedException {
+    Integer lnPort = vm0.invoke(() -> WANTestBase.createFirstLocatorWithDSId(1));
+    Integer nyPort = vm1.invoke(() -> WANTestBase.createFirstRemoteLocator(2, lnPort));
+
+    createCacheInVMs(nyPort, vm2);
+    createReceiverCustomerOrderShipmentPR(vm2);
+    createReceiverInVMs(vm2);
+    vm2.invoke(WANTestBase::stopReceivers);
+
+    createCacheInVMs(lnPort, vm4, vm5);
+    createSenderCustomerOrderShipmentPRs(vm4);
+    createSenderCustomerOrderShipmentPRs(vm5);
+
+    int batchSize = 10;
+    vm4.invoke(() -> WANTestBase.createSender("ln", 2, true, 100, 10, false, true, null, false,
+        true));
+    vm5.invoke(() -> WANTestBase.createSender("ln", 2, true, 100, 10, false, true, null, false,
+        true));
+
+    int customers = 4;
+    int transactionsPerCustomer = 100;
+    // Each transaction will contain one order plus the following shipments
+    int shipmentsPerTransaction = batchSize;
+
+    AsyncInvocation<Void> inv1 = asyncExecuteCustomerTransactions(vm4, customers,
+        transactionsPerCustomer, shipmentsPerTransaction);
+
+    // wait for some batches to be redistributed and then stop the sender
+    vm4.invoke(() -> await()
+        .until(() -> WANTestBase.getSenderStats("ln", -1).get(5) > 0));
+
+    stopSenderInVMsAsync("ln", vm4, vm5);
+
+    // Wait for the customer transactions to finish
+    inv1.await();
+    int orderEntries = transactionsPerCustomer * customers;
+    int shipmentEntries = orderEntries * 10;
+    vm4.invoke(() -> WANTestBase.validateRegionSize(orderRegionName, orderEntries));
+    vm5.invoke(() -> WANTestBase.validateRegionSize(orderRegionName, orderEntries));
+    vm4.invoke(() -> WANTestBase.validateRegionSize(shipmentRegionName, shipmentEntries));
+    vm5.invoke(() -> WANTestBase.validateRegionSize(shipmentRegionName, shipmentEntries));
+
+    // Start receiver and sender
+    vm2.invoke(WANTestBase::startReceivers);
+    startSenderInVMsAsync("ln", vm4, vm5);
+
+    checkOnlyCompleteTransactionsAreReplicatedWithSenderRestarted(shipmentsPerTransaction);
+  }
+
+  private AsyncInvocation<Void> asyncExecuteCustomerTransactions(VM vm, int customers,
+      int transactionsPerCustomer, int shipmentsPerTransaction) {
+    final Map<Object, Object> keyValuesInTransactions = new LinkedHashMap<>();
+    for (int custId = 0; custId < customers; custId++) {
+      for (int i = 0; i < transactionsPerCustomer; i++) {
+        CustId custIdObject = new CustId(custId);
+        OrderId orderId = new OrderId(i, custIdObject);
+        keyValuesInTransactions.put(orderId, new Order());
+        for (int j = 0; j < shipmentsPerTransaction; j++) {
+          ShipmentId shipmentId = new ShipmentId(i + j, orderId);
+          keyValuesInTransactions.put(shipmentId, new Shipment());
+        }
+      }
+    }
+    int eventsPerTransaction = 1 + shipmentsPerTransaction;
+    return vm.invokeAsync(
+        () -> WANTestBase.doOrderAndShipmentPutsInsideTransactions(keyValuesInTransactions,
+            eventsPerTransaction));
+  }
+
+  private void checkOnlyCompleteTransactionsAreReplicatedAfterSenderStopped(
+      int shipmentsPerTransaction) {
+    waitForBatchesToBeAppliedInTheReceiver(shipmentsPerTransaction);
+
+    List<Integer> v4List =
+        vm4.invoke(() -> WANTestBase.getSenderStats("ln", -1));
+    List<Integer> v5List =
+        vm5.invoke(() -> WANTestBase.getSenderStats("ln", -1));
+
+    // batches with incomplete transactions must be 0
+    assertEquals(0, (int) v4List.get(13));
+    assertEquals(0, (int) v5List.get(13));
+
+    // Check the entries replicated against the number of batches distributed
+    int batchesDistributed = v4List.get(4) + v5List.get(4);
+    checkOnlyCompleteTransactionsAreReplicated(shipmentsPerTransaction, batchesDistributed);
+  }
+
+  private void checkOnlyCompleteTransactionsAreReplicatedWithSenderRestarted(
+      int shipmentsPerTransaction) {
+    // Wait for sender queues to be drained
+    vm4.invoke(() -> WANTestBase.validateParallelSenderQueueAllBucketsDrained("ln"));
+    vm5.invoke(() -> WANTestBase.validateParallelSenderQueueAllBucketsDrained("ln"));
+
+    List<Integer> v4List =
+        vm4.invoke(() -> WANTestBase.getSenderStats("ln", 0));
+    List<Integer> v5List =
+        vm5.invoke(() -> WANTestBase.getSenderStats("ln", 0));
+    assertEquals(0, v4List.get(0) + v5List.get(0));
+
+    // batches with incomplete transactions must be 0
+    assertEquals(0, (int) v4List.get(13));
+    assertEquals(0, (int) v5List.get(13));
+
+    waitForBatchesToBeAppliedInTheReceiver(shipmentsPerTransaction);
+
+    // Check the entries replicated against the number of batches distributed
+    int batchesDistributed = v4List.get(4) + v5List.get(4);
+    checkOnlyCompleteTransactionsAreReplicated(shipmentsPerTransaction, batchesDistributed);
+  }
+
+  private void checkOnlyCompleteTransactionsAreReplicated(int shipmentsPerTransaction,
+      int batchesDistributed) {
+    // Only complete transactions (1 order + 10 shipments) must be replicated
+    int orderRegionSize = vm2.invoke(() -> getRegionSize(orderRegionName));
+    int shipmentRegionSize = vm2.invoke(() -> getRegionSize(shipmentRegionName));
+    assertEquals(shipmentRegionSize, 10 * orderRegionSize);

Review comment:
       I think that this could be `assertEquals(shipmentRegionSize, shipmentsPerTransaction * orderRegionSize);` for added clarity.

##########
File path: geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/serial/SerialWANStatsDUnitTest.java
##########
@@ -348,7 +349,276 @@ public void testReplicatedSerialPropagationWithBatchRedistWithGroupTransactionEv
   }
 
   @Test
-  public void testReplicatedSerialPropagationWithMultipleDispatchers() throws Exception {
+  public void testReplicatedSerialPropagationWithGroupTransactionEventsDoesNotSendBatchesWithIncompleteTransactionsIfGatewaySenderIsStoppedWhileReceivingTrafficAndLaterStarted()
+      throws InterruptedException {
+    Integer lnPort = vm0.invoke(() -> WANTestBase.createFirstLocatorWithDSId(1));
+    Integer nyPort = vm1.invoke(() -> WANTestBase.createFirstRemoteLocator(2, lnPort));
+
+    String regionName = testName + "_RR";
+
+    createCacheInVMs(nyPort, vm2);
+    createReceiverInVMs(vm2);
+
+    createCacheInVMs(lnPort, vm4, vm5);
+    boolean groupTransactionEvents = true;
+    int batchSize = 10;
+    vm4.invoke(
+        () -> WANTestBase.createSender("ln", 2, false, 100, batchSize, false, true, null, true,
+            groupTransactionEvents));
+    vm5.invoke(
+        () -> WANTestBase.createSender("ln", 2, false, 100, batchSize, false, true, null, true,
+            groupTransactionEvents));
+
+    vm2.invoke(() -> WANTestBase.createReplicatedRegion(regionName, null, isOffHeap()));
+
+    vm4.invoke(() -> WANTestBase.createReplicatedRegion(regionName, "ln", isOffHeap()));
+    vm5.invoke(() -> WANTestBase.createReplicatedRegion(regionName, "ln", isOffHeap()));
+
+    startSenderInVMs("ln", vm4, vm5);
+
+    final Map<Object, Object> keyValues = new LinkedHashMap<>();
+    int entries = 2200;

Review comment:
       In that case, it might be good to make the connection to transaction size explicit and make it `int entries = eventsPerTransaction * 200;`




----------------------------------------------------------------
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] albertogpz commented on a change in pull request #6052: GEODE-8971: Add grace period when stopping gateway sender with group-…

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



##########
File path: geode-core/src/main/java/org/apache/geode/cache/wan/GatewaySender.java
##########
@@ -174,7 +174,32 @@
    */
   int GET_TRANSACTION_EVENTS_FROM_QUEUE_RETRIES =
       Integer.getInteger(GeodeGlossary.GEMFIRE_PREFIX + "get-transaction-events-from-queue-retries",
-          10);
+          2);

Review comment:
       > But 2 is still too less. How about 3~5?
   
   Given that prior to this change we had 10 retries and no timeout, and these retries were done really fast (in less than 1 ms) I did not want to add a much higher waiting time because it does not seem necessary if the events will eventually reach the queue so I thought it would be better to have some small waiting time (2ms) and have a smaller number of retries. These numbers seem reasonable and have not caused flakyneess in the test cases.
   
   If we put a higher waiting time and more retries as you are suggesting and, for some reason, the configuration of the gateway senders or the traffic received contains transactions but they cannot be completed (even with infinite time), then you are going to be delaying replication for a long time: each batch for waiting time*number of retries which could be a lot.
   
   That's why I prefer to leave these values low and, if in some case it is necessary, they can be changed with the system properties.




-- 
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] albertogpz commented on a change in pull request #6052: GEODE-8971: Add grace period when stopping gateway sender with group-…

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



##########
File path: geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/parallel/ParallelWANStatsDUnitTest.java
##########
@@ -472,6 +473,322 @@ public void testPRParallelPropagationWithGroupTransactionEventsSendsBatchesWithC
 
   }
 
+  @Test
+  public void testPRParallelPropagationWithGroupTransactionEventsDoesNotSendBatchesWithIncompleteTransactionsIfGatewaySenderIsStoppedWhileReceivingTrafficAndLaterStarted()
+      throws InterruptedException {
+    Integer lnPort = vm0.invoke(() -> WANTestBase.createFirstLocatorWithDSId(1));
+    Integer nyPort = vm1.invoke(() -> WANTestBase.createFirstRemoteLocator(2, lnPort));
+
+    createCacheInVMs(nyPort, vm2);
+    createReceiverInVMs(vm2);
+
+    createCacheInVMs(lnPort, vm4, vm5);
+    vm4.invoke(() -> WANTestBase.createSender("ln", 2, true, 100, 10, false, true, null, true,
+        true));
+    vm5.invoke(() -> WANTestBase.createSender("ln", 2, true, 100, 10, false, true, null, true,
+        true));
+
+    createReceiverCustomerOrderShipmentPR(vm2);
+
+    createSenderCustomerOrderShipmentPRs(vm4);
+    createSenderCustomerOrderShipmentPRs(vm5);
+
+    startSenderInVMs("ln", vm4, vm5);
+
+    int customers = 4;
+    int transactionsPerCustomer = 100;
+    int shipmentsPerTransaction = 10;
+    final LinkedHashMap<Object, Object> keyValuesInTransactions = new LinkedHashMap<>();
+    for (int custId = 0; custId < customers; custId++) {
+      for (int i = 0; i < transactionsPerCustomer; i++) {
+        CustId custIdObject = new CustId(custId);
+        OrderId orderId = new OrderId(i, custIdObject);
+        keyValuesInTransactions.put(orderId, new Order());
+        for (int j = 0; j < shipmentsPerTransaction; j++) {
+          ShipmentId shipmentId = new ShipmentId(i + j, orderId);
+          keyValuesInTransactions.put(shipmentId, new Shipment());
+        }
+      }
+    }
+    int eventsPerTransaction = 1 + shipmentsPerTransaction;
+    AsyncInvocation<Void> inv1 =
+        vm4.invokeAsync(
+            () -> WANTestBase.doOrderAndShipmentPutsInsideTransactions(keyValuesInTransactions,
+                eventsPerTransaction));
+
+    // wait for some batches to be distributed and then stop the sender
+    vm4.invoke(() -> await()
+        .until(() -> WANTestBase.getSenderStats("ln", -1).get(4) > 0));
+
+    System.out.println("Stopping sender");
+    stopSenderInVMsAsync("ln", vm4, vm5);
+    System.out.println("Stopped sender");
+
+    inv1.await();
+    int entries =
+        transactionsPerCustomer * customers;
+
+    vm4.invoke(() -> WANTestBase.validateRegionSize(orderRegionName, entries));
+    vm5.invoke(() -> WANTestBase.validateRegionSize(orderRegionName, entries));
+
+    // Wait for events to replicate: when batches received does not change
+    // we can assume that replication has finished.
+    int batchesReceived = (vm2.invoke(() -> getReceiverStats())).get(0);
+    while (true) {

Review comment:
       I have tried by checking that the number of batches sent by the sender is equal to the number of batches received by the receiver.




----------------------------------------------------------------
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] kirklund commented on a change in pull request #6052: GEODE-8971: Add grace period when stopping gateway sender with group-…

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



##########
File path: geode-core/src/main/java/org/apache/geode/internal/cache/BucketRegionQueue.java
##########
@@ -453,21 +454,22 @@ public Object peek() {
    * If a matching object also fulfills the endPredicate then the method
    * stops looking for more matching objects.
    */
-  public List<Object> getElementsMatching(Predicate matchingPredicate, Predicate endPredicate) {
+  public List<Object> getElementsMatching(Predicate<GatewaySenderEventImpl> matchingPredicate,

Review comment:
       You might want to change from `GatewaySenderEventImpl` to your new `InternalGatewaySenderEvent` everywhere you've added new code. I don't see anything that would obviously break, and it would be great to start using that new interface in newer code.




----------------------------------------------------------------
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] DonalEvans commented on a change in pull request #6052: GEODE-8971: Add grace period when stopping gateway sender with group-…

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



##########
File path: geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/serial/SerialWANStatsDUnitTest.java
##########
@@ -348,7 +349,201 @@ public void testReplicatedSerialPropagationWithBatchRedistWithGroupTransactionEv
   }
 
   @Test
-  public void testReplicatedSerialPropagationWithMultipleDispatchers() throws Exception {
+  public void testReplicatedSerialPropagationWithGroupTransactionEventsDoesNotSendBatchesWithIncompleteTransactionsIfGatewaySenderIsStoppedWhileReceivingTrafficAndLaterStarted()
+      throws InterruptedException {
+    Integer lnPort = vm0.invoke(() -> WANTestBase.createFirstLocatorWithDSId(1));
+    Integer nyPort = vm1.invoke(() -> WANTestBase.createFirstRemoteLocator(2, lnPort));
+
+    String regionName = testName + "_RR";
+
+    createCacheInVMs(nyPort, vm2);
+    createReceiverInVMs(vm2);
+    vm2.invoke(() -> WANTestBase.createReplicatedRegion(regionName, null, isOffHeap()));
+
+    createCacheInVMs(lnPort, vm4, vm5);
+    vm4.invoke(() -> WANTestBase.createReplicatedRegion(regionName, "ln", isOffHeap()));
+    vm5.invoke(() -> WANTestBase.createReplicatedRegion(regionName, "ln", isOffHeap()));
+
+    boolean groupTransactionEvents = true;
+    int batchSize = 10;
+    vm4.invoke(
+        () -> WANTestBase.createSender("ln", 2, false, 100, batchSize, false, true, null, false,
+            groupTransactionEvents));
+    vm5.invoke(
+        () -> WANTestBase.createSender("ln", 2, false, 100, batchSize, false, true, null, false,
+            groupTransactionEvents));
+
+    int eventsPerTransaction = batchSize + 1;
+    // The number of entries must be big enough so that not all entries
+    // are replicated before the sender is stopped and also divisible by eventsPerTransaction
+    int entries = 2200;
+    // Execute some transactions
+    AsyncInvocation<Void> inv1 =
+        asyncExecuteTransactions(regionName, eventsPerTransaction, entries);
+
+    // wait for batches to be distributed and then stop the sender
+    vm4.invoke(() -> await()
+        .until(() -> WANTestBase.getSenderStats("ln", -1).get(4) > 0));
+
+    // These exceptions are ignored here because it could happen that when an event
+    // is to be handled, the sender is stopped. The sender, when stopped, shuts down
+    // the thread pool that would handle the event and this could provoke the exception.
+    addIgnoredException("Exception occurred in CacheListener");
+    addIgnoredException(RejectedExecutionException.class);
+
+    // Stop the sender
+    stopSenderInVMsAsync("ln", vm4, vm5);
+
+    // Wait for transactions to finish
+    inv1.await();
+    vm4.invoke(() -> WANTestBase.validateRegionSize(regionName, entries));
+    vm5.invoke(() -> WANTestBase.validateRegionSize(regionName, entries));
+
+    // Check
+    checkOnlyCompleteTransactionsAreReplicatedAfterSenderStopped(regionName,
+        eventsPerTransaction);
+
+    // Start the sender
+    startSenderInVMsAsync("ln", vm4, vm5);
+
+    // Check
+    checkOnlyCompleteTransactionsAreReplicatedAfterSenderRestarted(regionName,
+        eventsPerTransaction);
+  }
+
+  @Test
+  public void testReplicatedSerialPropagationWithGroupTransactionEventsDoesNotSendBatchesWithIncompleteTransactionsIfGatewaySenderIsStoppedWhileReceivingTrafficAndLaterStartedReceiverStopped()
+      throws InterruptedException {
+    Integer lnPort = vm0.invoke(() -> WANTestBase.createFirstLocatorWithDSId(1));
+    Integer nyPort = vm1.invoke(() -> WANTestBase.createFirstRemoteLocator(2, lnPort));
+
+    String regionName = testName + "_RR";
+
+    createCacheInVMs(nyPort, vm2);
+    createReceiverInVMs(vm2);
+    vm2.invoke(() -> WANTestBase.createReplicatedRegion(regionName, null, isOffHeap()));
+    vm2.invoke(WANTestBase::stopReceivers);
+
+    createCacheInVMs(lnPort, vm4, vm5);
+    vm4.invoke(() -> WANTestBase.createReplicatedRegion(regionName, "ln", isOffHeap()));
+    vm5.invoke(() -> WANTestBase.createReplicatedRegion(regionName, "ln", isOffHeap()));
+
+    boolean groupTransactionEvents = true;
+    int batchSize = 10;
+    vm4.invoke(
+        () -> WANTestBase.createSender("ln", 2, false, 100, batchSize, false, true, null, false,
+            groupTransactionEvents));
+    vm5.invoke(
+        () -> WANTestBase.createSender("ln", 2, false, 100, batchSize, false, true, null, false,
+            groupTransactionEvents));
+
+    int eventsPerTransaction = batchSize + 1;
+    // The number of entries must be big enough so that not all entries
+    // are replicated before the sender is stopped and also divisible by eventsPerTransaction
+    int entries = 2200;
+    // Execute some transactions
+    AsyncInvocation<Void> inv1 =
+        asyncExecuteTransactions(regionName, eventsPerTransaction, entries);
+
+    // wait for batches to be redistributed and then stop the sender
+    vm4.invoke(() -> await()
+        .until(() -> WANTestBase.getSenderStats("ln", -1).get(5) > 0));
+
+    // Stop the sender
+    stopSenderInVMsAsync("ln", vm4, vm5);
+
+    // Wait for transactions to finish
+    inv1.await();
+    vm4.invoke(() -> WANTestBase.validateRegionSize(regionName, entries));
+    vm5.invoke(() -> WANTestBase.validateRegionSize(regionName, entries));
+
+    // Start the receiver and the sender
+    vm2.invoke(WANTestBase::startReceivers);
+    startSenderInVMsAsync("ln", vm4, vm5);
+
+    // Check
+    checkOnlyCompleteTransactionsAreReplicatedAfterSenderRestarted(regionName,
+        eventsPerTransaction);
+  }
+
+  private void checkOnlyCompleteTransactionsAreReplicatedAfterSenderStopped(String regionName,
+      int eventsPerTransaction) {
+    waitForBatchesToBeAppliedInTheReceiver(regionName, eventsPerTransaction);
+
+    List<Integer> v4List =
+        vm4.invoke(() -> WANTestBase.getSenderStats("ln", -1));
+    List<Integer> v5List =
+        vm5.invoke(() -> WANTestBase.getSenderStats("ln", -1));
+
+    // batches with incomplete transactions must be 0
+    assertEquals(0, (int) v4List.get(13));
+    assertEquals(0, (int) v5List.get(13));
+
+    int batchesDistributed = v4List.get(4) + v5List.get(4);
+    checkOnlyCompleteTransactionsAreReplicated(regionName, eventsPerTransaction,
+        batchesDistributed);
+  }
+
+  private void checkOnlyCompleteTransactionsAreReplicatedAfterSenderRestarted(String regionName,
+      int eventsPerTransaction) {
+    // Wait for sender queues to be empty
+    List<Integer> v4List =
+        vm4.invoke(() -> WANTestBase.getSenderStats("ln", 0));
+    List<Integer> v5List =
+        vm5.invoke(() -> WANTestBase.getSenderStats("ln", 0));
+
+    assertEquals(0, v4List.get(0) + v5List.get(0));
+
+    // batches with incomplete transactions must be 0
+    assertEquals(0, (int) v4List.get(13));
+    assertEquals(0, (int) v5List.get(13));
+
+    waitForBatchesToBeAppliedInTheReceiver(regionName, eventsPerTransaction);
+
+    int batchesDistributed = v4List.get(4) + v5List.get(4);
+    checkOnlyCompleteTransactionsAreReplicated(regionName, eventsPerTransaction,
+        batchesDistributed);
+  }
+
+  private void checkOnlyCompleteTransactionsAreReplicated(String regionName,
+      int eventsPerTransaction, int batchesDistributed) {
+    int regionSize = vm2.invoke(() -> getRegionSize(regionName));
+
+    // The number of entries must be divisible by the number of events per transaction
+    assertEquals(0, regionSize % eventsPerTransaction);
+
+    // Check the entries replicated against the number of batches distributed
+    vm2.invoke(() -> WANTestBase.validateRegionSize(regionName,
+        batchesDistributed * eventsPerTransaction));
+  }
+
+  private AsyncInvocation<Void> asyncExecuteTransactions(String regionName,
+      int eventsPerTransaction, int entries) {
+    final Map<Object, Object> keyValues = new LinkedHashMap<>();
+    for (int i = 0; i < entries; i++) {
+      keyValues.put(i, i + "_Value");
+    }
+
+    return vm4.invokeAsync(
+        () -> WANTestBase.doPutsInsideTransactions(regionName, keyValues,
+            eventsPerTransaction));
+  }
+
+  private void waitForBatchesToBeAppliedInTheReceiver(String regionName, int eventsPerTransaction) {
+    int batchesSentTotal = vm4.invoke(() -> WANTestBase.getSenderStats("ln", -1)).get(4) +
+        vm5.invoke(() -> WANTestBase.getSenderStats("ln", -1)).get(4);
+
+    // Wait for all batches to be received by the sender

Review comment:
       It looks like this got missed in the last set of changed, but other than that, everything looks good.




----------------------------------------------------------------
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] albertogpz commented on a change in pull request #6052: GEODE-8971: Add grace period when stopping gateway sender with group-…

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



##########
File path: geode-core/src/main/java/org/apache/geode/internal/cache/wan/parallel/ConcurrentParallelGatewaySenderEventProcessor.java
##########
@@ -132,22 +134,24 @@ public int eventQueueSize() {
   }
 
   @Override
-  public void enqueueEvent(EnumListenerEvent operation, EntryEvent event, Object substituteValue)
+  public boolean enqueueEvent(EnumListenerEvent operation, EntryEvent event, Object substituteValue)
       throws IOException, CacheException {
-    enqueueEvent(operation, event, substituteValue, false);
+    return enqueueEvent(operation, event, substituteValue, false, null);
   }
 
   @Override
-  public void enqueueEvent(EnumListenerEvent operation, EntryEvent event, Object substituteValue,
-      boolean isLastEventInTransaction) throws IOException, CacheException {
+  public boolean enqueueEvent(EnumListenerEvent operation, EntryEvent event, Object substituteValue,
+      boolean isLastEventInTransaction, Predicate<InternalGatewayQueueEvent> condition)
+      throws IOException, CacheException {
     Region region = event.getRegion();
     // int bucketId = PartitionedRegionHelper.getHashKey((EntryOperation)event);

Review comment:
       ok




----------------------------------------------------------------
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] DonalEvans commented on a change in pull request #6052: GEODE-8971: Add grace period when stopping gateway sender with group-…

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



##########
File path: geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/WANTestBase.java
##########
@@ -1320,6 +1320,23 @@ public static void checkGatewayReceiverStats(int processBatches, int eventsRecei
     assertEquals(creates, gatewayReceiverStats.getCreateRequest());
   }
 
+  public static List<Integer> getReceiverStats() {

Review comment:
       I see, that's unfortunate but unavoidable then.




----------------------------------------------------------------
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] albertogpz commented on a change in pull request #6052: GEODE-8971: Add grace period when stopping gateway sender with group-…

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



##########
File path: geode-core/src/main/java/org/apache/geode/internal/cache/wan/parallel/ParallelGatewaySenderQueue.java
##########
@@ -702,44 +704,15 @@ public boolean put(Object object) throws InterruptedException, CacheException {
     boolean isDREvent = isDREvent(sender.getCache(), value);
 
     String regionPath = value.getRegionPath();
-    if (!isDREvent) {
-      Region region = sender.getCache().getRegion(regionPath, true);
-      regionPath = ColocationHelper.getLeaderRegion((PartitionedRegion) region).getFullPath();
-    }
-    if (isDebugEnabled) {
-      logger.debug("Put is for the region {}", regionPath);
-    }
-    if (!this.userRegionNameToShadowPRMap.containsKey(regionPath)) {
-      if (isDebugEnabled) {
-        logger.debug("The userRegionNameToshadowPRMap is {}", userRegionNameToShadowPRMap);
-      }
-      logger.warn(
-          "GatewaySender: Not queuing the event {}, as the region for which this event originated is not yet configured in the GatewaySender",
-          value);
-      // does not put into queue
+    regionPath = getRegionPathForEventAndType(value, isDREvent, regionPath);
+    if (regionPath == null)

Review comment:
       ok

##########
File path: geode-core/src/main/java/org/apache/geode/internal/cache/wan/parallel/ParallelGatewaySenderQueue.java
##########
@@ -702,44 +704,15 @@ public boolean put(Object object) throws InterruptedException, CacheException {
     boolean isDREvent = isDREvent(sender.getCache(), value);
 
     String regionPath = value.getRegionPath();
-    if (!isDREvent) {
-      Region region = sender.getCache().getRegion(regionPath, true);
-      regionPath = ColocationHelper.getLeaderRegion((PartitionedRegion) region).getFullPath();
-    }
-    if (isDebugEnabled) {
-      logger.debug("Put is for the region {}", regionPath);
-    }
-    if (!this.userRegionNameToShadowPRMap.containsKey(regionPath)) {
-      if (isDebugEnabled) {
-        logger.debug("The userRegionNameToshadowPRMap is {}", userRegionNameToShadowPRMap);
-      }
-      logger.warn(
-          "GatewaySender: Not queuing the event {}, as the region for which this event originated is not yet configured in the GatewaySender",
-          value);
-      // does not put into queue
+    regionPath = getRegionPathForEventAndType(value, isDREvent, regionPath);
+    if (regionPath == null)
       return false;
-    }
 
     PartitionedRegion prQ = this.userRegionNameToShadowPRMap.get(regionPath);
     int bucketId = value.getBucketId();
-    Object key = null;
-    if (!isDREvent) {
-      key = value.getShadowKey();
-
-      if ((Long) key == -1) {
-        // In case of parallel we don't expect
-        // the key to be not set. If it is the case then the event must be coming
-        // through listener, so return.
-        if (isDebugEnabled) {
-          logger.debug("ParallelGatewaySenderOrderedQueue not putting key {} : Value : {}", key,
-              value);
-        }
-        // does not put into queue
-        return false;
-      }
-    } else {
-      key = value.getEventId();
-    }
+    Object key = getKeyForEventAndType(value, isDREvent);
+    if (key == null)

Review comment:
       ok




----------------------------------------------------------------
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] albertogpz commented on a change in pull request #6052: GEODE-8971: Add grace period when stopping gateway sender with group-…

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



##########
File path: geode-core/src/main/java/org/apache/geode/internal/cache/wan/serial/SerialGatewaySenderQueue.java
##########
@@ -847,7 +855,14 @@ private EventsAndLastKey peekEventsWithTransactionId(TransactionId transactionId
     }
   }
 
-  EventsAndLastKey getElementsMatching(Predicate condition, Predicate stopCondition, long lastKey) {
+  /**
+   * This method returns a list of objects that fulfill the matchingPredicate
+   * If a matching object also fulfills the endPredicate then the method
+   * stops looking for more matching objects.
+   */
+  List<KeyAndEventPair> getElementsMatching(Predicate<InternalGatewayQueueEvent> condition,

Review comment:
       Good catch.

##########
File path: geode-core/src/main/java/org/apache/geode/internal/cache/wan/serial/SerialGatewaySenderQueue.java
##########
@@ -862,19 +877,35 @@ EventsAndLastKey getElementsMatching(Predicate condition, Predicate stopConditio
         continue;
       }
 
-      if (condition.test(object)) {
-        elementsMatching.add(object);
-        peekedIds.add(currentKey);
-        extraPeekedIds.add(currentKey);
-        lastKey = currentKey;
+      if (condition.test((InternalGatewayQueueEvent) object)) {
+        elementsMatching.add(new KeyAndEventPair(currentKey, (GatewaySenderEventImpl) object));
 
-        if (stopCondition.test(object)) {
+        if (stopCondition.test((InternalGatewayQueueEvent) object)) {
           break;
         }
       }
     }
 
-    return new EventsAndLastKey(elementsMatching, lastKey);
+    return elementsMatching;
+  }
+
+  public boolean isThereEventsMatching(Predicate<InternalGatewayQueueEvent> condition) {

Review comment:
       ok




----------------------------------------------------------------
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] albertogpz commented on a change in pull request #6052: GEODE-8971: Add grace period when stopping gateway sender with group-…

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



##########
File path: geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/parallel/ParallelWANStatsDUnitTest.java
##########
@@ -472,6 +473,322 @@ public void testPRParallelPropagationWithGroupTransactionEventsSendsBatchesWithC
 
   }
 
+  @Test
+  public void testPRParallelPropagationWithGroupTransactionEventsDoesNotSendBatchesWithIncompleteTransactionsIfGatewaySenderIsStoppedWhileReceivingTrafficAndLaterStarted()
+      throws InterruptedException {
+    Integer lnPort = vm0.invoke(() -> WANTestBase.createFirstLocatorWithDSId(1));
+    Integer nyPort = vm1.invoke(() -> WANTestBase.createFirstRemoteLocator(2, lnPort));
+
+    createCacheInVMs(nyPort, vm2);
+    createReceiverInVMs(vm2);
+
+    createCacheInVMs(lnPort, vm4, vm5);
+    vm4.invoke(() -> WANTestBase.createSender("ln", 2, true, 100, 10, false, true, null, true,
+        true));
+    vm5.invoke(() -> WANTestBase.createSender("ln", 2, true, 100, 10, false, true, null, true,
+        true));
+
+    createReceiverCustomerOrderShipmentPR(vm2);
+
+    createSenderCustomerOrderShipmentPRs(vm4);
+    createSenderCustomerOrderShipmentPRs(vm5);
+
+    startSenderInVMs("ln", vm4, vm5);
+
+    int customers = 4;
+    int transactionsPerCustomer = 100;
+    int shipmentsPerTransaction = 10;

Review comment:
       ok




----------------------------------------------------------------
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] albertogpz commented on a change in pull request #6052: GEODE-8971: Add grace period when stopping gateway sender with group-…

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



##########
File path: geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/parallel/ParallelWANStatsDUnitTest.java
##########
@@ -472,6 +473,322 @@ public void testPRParallelPropagationWithGroupTransactionEventsSendsBatchesWithC
 
   }
 
+  @Test
+  public void testPRParallelPropagationWithGroupTransactionEventsDoesNotSendBatchesWithIncompleteTransactionsIfGatewaySenderIsStoppedWhileReceivingTrafficAndLaterStarted()
+      throws InterruptedException {
+    Integer lnPort = vm0.invoke(() -> WANTestBase.createFirstLocatorWithDSId(1));
+    Integer nyPort = vm1.invoke(() -> WANTestBase.createFirstRemoteLocator(2, lnPort));
+
+    createCacheInVMs(nyPort, vm2);
+    createReceiverInVMs(vm2);
+
+    createCacheInVMs(lnPort, vm4, vm5);
+    vm4.invoke(() -> WANTestBase.createSender("ln", 2, true, 100, 10, false, true, null, true,
+        true));
+    vm5.invoke(() -> WANTestBase.createSender("ln", 2, true, 100, 10, false, true, null, true,
+        true));
+
+    createReceiverCustomerOrderShipmentPR(vm2);
+
+    createSenderCustomerOrderShipmentPRs(vm4);
+    createSenderCustomerOrderShipmentPRs(vm5);
+
+    startSenderInVMs("ln", vm4, vm5);
+
+    int customers = 4;
+    int transactionsPerCustomer = 100;
+    int shipmentsPerTransaction = 10;
+    final LinkedHashMap<Object, Object> keyValuesInTransactions = new LinkedHashMap<>();

Review comment:
       I agree




----------------------------------------------------------------
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] albertogpz commented on a change in pull request #6052: GEODE-8971: Add grace period when stopping gateway sender with group-…

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



##########
File path: geode-core/src/main/java/org/apache/geode/internal/cache/wan/AbstractGatewaySender.java
##########
@@ -686,6 +689,22 @@ public boolean beforeEnqueue(GatewayQueueEvent gatewayEvent) {
     return enqueue;
   }
 
+  protected void preStop() {
+    if (!mustGroupTransactionEvents() || isStopping) {

Review comment:
       You are right. I have removed the `isStopping` variable from the condition. 




----------------------------------------------------------------
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] albertogpz commented on a change in pull request #6052: GEODE-8971: Add grace period when stopping gateway sender with group-…

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



##########
File path: geode-core/src/main/java/org/apache/geode/cache/wan/GatewaySender.java
##########
@@ -174,7 +174,32 @@
    */
   int GET_TRANSACTION_EVENTS_FROM_QUEUE_RETRIES =
       Integer.getInteger(GeodeGlossary.GEMFIRE_PREFIX + "get-transaction-events-from-queue-retries",
-          10);
+          2);

Review comment:
       Prior to this change, retries were done immediately if it was detected that not all events for a transaction were in the batch. In this change, I have added a configurable sleep before retrying with a default value of 1ms because I saw that retries were sometimes done so fast that it was not enough to get the expected events in the queue and it was more reasonable to have less retries and have some waiting before retrying.




----------------------------------------------------------------
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] gesterzhou commented on a change in pull request #6052: GEODE-8971: Add grace period when stopping gateway sender with group-…

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



##########
File path: geode-core/src/main/java/org/apache/geode/cache/wan/GatewaySender.java
##########
@@ -174,7 +174,32 @@
    */
   int GET_TRANSACTION_EVENTS_FROM_QUEUE_RETRIES =
       Integer.getInteger(GeodeGlossary.GEMFIRE_PREFIX + "get-transaction-events-from-queue-retries",
-          10);
+          2);

Review comment:
       maybe just keep it as 10. 
   
   10 ms is no big deal
   




-- 
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] albertogpz commented on a change in pull request #6052: GEODE-8971: Add grace period when stopping gateway sender with group-…

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



##########
File path: geode-core/src/main/java/org/apache/geode/internal/cache/wan/parallel/ParallelGatewaySenderQueue.java
##########
@@ -1797,6 +1817,22 @@ public static String getSenderId(String regionName) {
     return regionName.substring(1, queueStringStart);
   }
 
+  public boolean isThereEventsMatching(GatewaySenderEventImpl event,

Review comment:
       Agree




----------------------------------------------------------------
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] albertogpz commented on a change in pull request #6052: GEODE-8971: Add grace period when stopping gateway sender with group-…

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



##########
File path: geode-core/src/main/java/org/apache/geode/internal/cache/wan/AbstractGatewaySenderEventProcessor.java
##########
@@ -195,6 +207,9 @@ protected void setIsStopped(boolean isStopped) {
     } else {
       this.isStopped = isStopped;
     }
+    if (isStopped) {

Review comment:
       Yep




----------------------------------------------------------------
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] albertogpz commented on a change in pull request #6052: GEODE-8971: Add grace period when stopping gateway sender with group-…

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



##########
File path: geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/serial/SerialWANStatsDUnitTest.java
##########
@@ -348,7 +349,276 @@ public void testReplicatedSerialPropagationWithBatchRedistWithGroupTransactionEv
   }
 
   @Test
-  public void testReplicatedSerialPropagationWithMultipleDispatchers() throws Exception {
+  public void testReplicatedSerialPropagationWithGroupTransactionEventsDoesNotSendBatchesWithIncompleteTransactionsIfGatewaySenderIsStoppedWhileReceivingTrafficAndLaterStarted()
+      throws InterruptedException {
+    Integer lnPort = vm0.invoke(() -> WANTestBase.createFirstLocatorWithDSId(1));
+    Integer nyPort = vm1.invoke(() -> WANTestBase.createFirstRemoteLocator(2, lnPort));
+
+    String regionName = testName + "_RR";
+
+    createCacheInVMs(nyPort, vm2);
+    createReceiverInVMs(vm2);
+
+    createCacheInVMs(lnPort, vm4, vm5);
+    boolean groupTransactionEvents = true;
+    int batchSize = 10;
+    vm4.invoke(
+        () -> WANTestBase.createSender("ln", 2, false, 100, batchSize, false, true, null, true,
+            groupTransactionEvents));
+    vm5.invoke(
+        () -> WANTestBase.createSender("ln", 2, false, 100, batchSize, false, true, null, true,
+            groupTransactionEvents));
+
+    vm2.invoke(() -> WANTestBase.createReplicatedRegion(regionName, null, isOffHeap()));
+
+    vm4.invoke(() -> WANTestBase.createReplicatedRegion(regionName, "ln", isOffHeap()));
+    vm5.invoke(() -> WANTestBase.createReplicatedRegion(regionName, "ln", isOffHeap()));
+
+    startSenderInVMs("ln", vm4, vm5);
+
+    final Map<Object, Object> keyValues = new LinkedHashMap<>();
+    int entries = 2200;

Review comment:
       I picked a number divisible by 11 (as transactions are that size) and big enough so that not all entries are replicated by the gateway sender before it is stopped.




----------------------------------------------------------------
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] davebarnes97 commented on a change in pull request #6052: GEODE-8971: Add grace period when stopping gateway sender with group-…

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



##########
File path: geode-core/src/main/java/org/apache/geode/cache/wan/GatewaySender.java
##########
@@ -174,7 +174,32 @@
    */
   int GET_TRANSACTION_EVENTS_FROM_QUEUE_RETRIES =
       Integer.getInteger(GeodeGlossary.GEMFIRE_PREFIX + "get-transaction-events-from-queue-retries",
-          10);
+          2);
+  /**
+   * Milliseconds to wait before retrying to get events for a transaction from the
+   * gateway sender queue when group-transaction-events is true.
+   */
+  int GET_TRANSACTION_EVENTS_FROM_QUEUE_WAIT_TIME_MS =
+      Integer.getInteger(
+          GeodeGlossary.GEMFIRE_PREFIX + "get-transaction-events-from-queue-wait-time-ms",
+          1);
+
+  /**
+   * When group-transaction-events is set to true and the gateway sender is stopped,
+   * there is a possibility that the stopping occurs such that for a transaction,
+   * not all events belonging to it reach the queue. The reason would be that
+   * some reach the queue right before the sender is stopped and the rest do not make
+   * it to the queue because the sender is just stopped.
+   * In order to prevent that the queue contains incomplete transactions
+   * due to the above circumstance, this parameter allows for a grace period
+   * of the number of milliseconds set in it before the gateway sender is
+   * actually stopped, in which only events to complete transactions are put in the queue.
+   * Other events received in this period would be dropped.

Review comment:
       Suggested rewrite for economy of language:
   When group-transaction-events is true and the gateway sender is stopped,
   addition to the queue of a group of transaction events might be interrupted.
   To ensure that the queue does not contain incomplete transactions, this parameter
   allows for a grace period, specified in milliseconds, before the gateway sender is actually
   stopped, allowing complete transaction event groups to be queued. Any event received
   during the grace period that is not part of a transaction event group is dropped.
   




----------------------------------------------------------------
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] davebarnes97 commented on a change in pull request #6052: GEODE-8971: Add grace period when stopping gateway sender with group-…

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



##########
File path: geode-core/src/main/java/org/apache/geode/cache/wan/GatewaySender.java
##########
@@ -174,7 +174,32 @@
    */
   int GET_TRANSACTION_EVENTS_FROM_QUEUE_RETRIES =
       Integer.getInteger(GeodeGlossary.GEMFIRE_PREFIX + "get-transaction-events-from-queue-retries",
-          10);
+          2);
+  /**
+   * Milliseconds to wait before retrying to get events for a transaction from the
+   * gateway sender queue when group-transaction-events is true.
+   */
+  int GET_TRANSACTION_EVENTS_FROM_QUEUE_WAIT_TIME_MS =
+      Integer.getInteger(
+          GeodeGlossary.GEMFIRE_PREFIX + "get-transaction-events-from-queue-wait-time-ms",
+          1);
+
+  /**
+   * When group-transaction-events is set to true and the gateway sender is stopped,
+   * there is a possibility that the stopping occurs such that for a transaction,
+   * not all events belonging to it reach the queue. The reason would be that
+   * some reach the queue right before the sender is stopped and the rest do not make
+   * it to the queue because the sender is just stopped.
+   * In order to prevent that the queue contains incomplete transactions
+   * due to the above circumstance, this parameter allows for a grace period
+   * of the number of milliseconds set in it before the gateway sender is
+   * actually stopped, in which only events to complete transactions are put in the queue.
+   * Other events received in this period would be dropped.

Review comment:
       Suggested rewrite for economy of language:
   When group-transaction-events is true and the gateway sender is stopped,
   addition to the queue of a group of transaction events might be interrupted.
   To ensure that the queue does not contain incomplete transactions, this parameter
   allows for a grace period, specified in milliseconds, before the gateway sender is actually
   stopped, allowing complete transaction event groups to be queued. Any event received
   during the grace period that are not part of a transaction event group are dropped.
   




----------------------------------------------------------------
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] albertogpz commented on a change in pull request #6052: GEODE-8971: Add grace period when stopping gateway sender with group-…

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



##########
File path: geode-core/src/test/java/org/apache/geode/internal/cache/wan/serial/SerialGatewaySenderQueueJUnitTest.java
##########
@@ -137,8 +137,8 @@ public void peekGetsExtraEventsWhenMustGroupTransactionEventsAndNotAllEventsForT
         .doAnswer(invocation -> null)
         .when(queue).peekAhead();
 
-    doAnswer(invocation -> new SerialGatewaySenderQueue.EventsAndLastKey(
-        Arrays.asList(new Object[] {event4}), 2L))
+    doAnswer(invocation -> Arrays

Review comment:
       ok




----------------------------------------------------------------
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] albertogpz commented on a change in pull request #6052: GEODE-8971: Add grace period when stopping gateway sender with group-…

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



##########
File path: geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/serial/SerialWANStatsDUnitTest.java
##########
@@ -348,7 +349,276 @@ public void testReplicatedSerialPropagationWithBatchRedistWithGroupTransactionEv
   }
 
   @Test
-  public void testReplicatedSerialPropagationWithMultipleDispatchers() throws Exception {
+  public void testReplicatedSerialPropagationWithGroupTransactionEventsDoesNotSendBatchesWithIncompleteTransactionsIfGatewaySenderIsStoppedWhileReceivingTrafficAndLaterStarted()
+      throws InterruptedException {
+    Integer lnPort = vm0.invoke(() -> WANTestBase.createFirstLocatorWithDSId(1));
+    Integer nyPort = vm1.invoke(() -> WANTestBase.createFirstRemoteLocator(2, lnPort));
+
+    String regionName = testName + "_RR";
+
+    createCacheInVMs(nyPort, vm2);
+    createReceiverInVMs(vm2);
+
+    createCacheInVMs(lnPort, vm4, vm5);
+    boolean groupTransactionEvents = true;
+    int batchSize = 10;
+    vm4.invoke(
+        () -> WANTestBase.createSender("ln", 2, false, 100, batchSize, false, true, null, true,
+            groupTransactionEvents));
+    vm5.invoke(
+        () -> WANTestBase.createSender("ln", 2, false, 100, batchSize, false, true, null, true,
+            groupTransactionEvents));
+
+    vm2.invoke(() -> WANTestBase.createReplicatedRegion(regionName, null, isOffHeap()));
+
+    vm4.invoke(() -> WANTestBase.createReplicatedRegion(regionName, "ln", isOffHeap()));
+    vm5.invoke(() -> WANTestBase.createReplicatedRegion(regionName, "ln", isOffHeap()));
+
+    startSenderInVMs("ln", vm4, vm5);
+
+    final Map<Object, Object> keyValues = new LinkedHashMap<>();
+    int entries = 2200;

Review comment:
       Not really. I picked a number divisible by 11 (as transactions are that size) and big enough so that not all entries are replicated by the gateway sender before it is stopped.




----------------------------------------------------------------
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] albertogpz commented on a change in pull request #6052: GEODE-8971: Add grace period when stopping gateway sender with group-…

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



##########
File path: geode-core/src/main/java/org/apache/geode/internal/cache/wan/AbstractGatewaySender.java
##########
@@ -126,6 +127,8 @@
 
   protected boolean groupTransactionEvents;
 
+  protected volatile boolean isStopping = false;

Review comment:
       I would prefer to leave it as it is. The variable is just indicating that we are in the process of stopping the gateway sender.
   I have changed the `AbstractGatewaySender.preStop()` method to make this more clear.




-- 
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] kirklund commented on a change in pull request #6052: GEODE-8971: Add grace period when stopping gateway sender with group-…

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



##########
File path: geode-core/src/main/java/org/apache/geode/internal/cache/BucketRegionQueue.java
##########
@@ -478,6 +478,25 @@ public Object peek() {
     }
   }
 
+  public boolean isThereEventsMatching(Predicate matchingPredicate) {
+    getInitializationLock().readLock().lock();
+    try {
+      if (this.getPartitionedRegion().isDestroyed()) {
+        throw new BucketRegionQueueUnavailableException();
+      }
+      Iterator<Object> it = this.eventSeqNumDeque.iterator();

Review comment:
       [comment] Another syntax to consider if you're not familiar with it:
   ```
         for (Object o : eventSeqNumDeque) {
           Object object = optimalGet(o);
   ```

##########
File path: geode-core/src/main/java/org/apache/geode/internal/cache/BucketRegionQueue.java
##########
@@ -478,6 +478,25 @@ public Object peek() {
     }
   }
 
+  public boolean isThereEventsMatching(Predicate matchingPredicate) {

Review comment:
       Predicate is a parameterized:
   ```
   public boolean isThereEventsMatching(Predicate<Object> matchingPredicate) {
   ```
   Giving it a type (even just `Object`) will remove warnings (I'm seeing warnings about it in IntelliJ).
   
   In general, you should always try to avoid raw types.

##########
File path: geode-core/src/main/java/org/apache/geode/internal/cache/wan/parallel/ParallelGatewaySenderQueue.java
##########
@@ -1797,6 +1816,22 @@ public static String getSenderId(String regionName) {
     return regionName.substring(1, queueStringStart);
   }
 
+  public boolean isThereEventsMatching(Object object, Predicate condition) {

Review comment:
       If you know that `object` is always a `GatewaySenderEventImpl` then it would be better to use that in the method signature:
   ```
   public boolean isThereEventsMatching(GatewaySenderEventImpl value, Predicate<Object> condition) {
   ```
   ...rather than casting it on the 1st line of the method.




----------------------------------------------------------------
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] albertogpz commented on a change in pull request #6052: GEODE-8971: Add grace period when stopping gateway sender with group-…

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



##########
File path: geode-core/src/main/java/org/apache/geode/internal/cache/EntryEventImpl.java
##########
@@ -2231,6 +2231,9 @@ public String toString() {
     if (this.tailKey != -1) {
       buf.append(";tailKey=" + tailKey);
     }
+    if (this.isTransactional()) {
+      buf.append(";transactionId=" + getTransactionId());

Review comment:
       ok




----------------------------------------------------------------
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] gesterzhou commented on a change in pull request #6052: GEODE-8971: Add grace period when stopping gateway sender with group-…

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



##########
File path: geode-core/src/main/java/org/apache/geode/internal/cache/wan/AbstractGatewaySender.java
##########
@@ -126,6 +127,8 @@
 
   protected boolean groupTransactionEvents;
 
+  protected volatile boolean isStopping = false;

Review comment:
       can we have a better name, such as isStoppingForTX?




-- 
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] albertogpz commented on a change in pull request #6052: GEODE-8971: Add grace period when stopping gateway sender with group-…

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



##########
File path: geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/parallel/ParallelWANStatsDUnitTest.java
##########
@@ -472,6 +473,200 @@ public void testPRParallelPropagationWithGroupTransactionEventsSendsBatchesWithC
 
   }
 
+  @Test
+  public void testPRParallelPropagationWithGroupTransactionEventsDoesNotSendBatchesWithIncompleteTransactionsIfGatewaySenderIsStoppedWhileReceivingTrafficAndLaterStarted()
+      throws InterruptedException {
+    Integer lnPort = vm0.invoke(() -> WANTestBase.createFirstLocatorWithDSId(1));
+    Integer nyPort = vm1.invoke(() -> WANTestBase.createFirstRemoteLocator(2, lnPort));
+
+    createCacheInVMs(nyPort, vm2);
+    createReceiverCustomerOrderShipmentPR(vm2);
+    createReceiverInVMs(vm2);
+
+    createCacheInVMs(lnPort, vm4, vm5);
+    createSenderCustomerOrderShipmentPRs(vm4);
+    createSenderCustomerOrderShipmentPRs(vm5);
+
+    int batchSize = 10;
+    vm4.invoke(
+        () -> WANTestBase.createSender("ln", 2, true, 100, batchSize, false, true, null, false,
+            true));
+    vm5.invoke(
+        () -> WANTestBase.createSender("ln", 2, true, 100, batchSize, false, true, null, false,
+            true));
+
+    int customers = 4;
+    int transactionsPerCustomer = 100;
+    // Each transaction will contain one order plus the following shipments
+    int shipmentsPerTransaction = batchSize;
+    AsyncInvocation<Void> inv1 = asyncExecuteCustomerTransactions(vm4, customers,
+        transactionsPerCustomer, shipmentsPerTransaction);
+
+    // wait for some batches to be distributed and then stop the sender
+    vm4.invoke(() -> await()
+        .until(() -> WANTestBase.getSenderStats("ln", -1).get(4) > 0));
+
+    stopSenderInVMsAsync("ln", vm4, vm5);
+
+    // Wait for customer transactions to finish
+    inv1.await();
+    int orderEntries = transactionsPerCustomer * customers;
+    int shipmentEntries = orderEntries * 10;
+    vm4.invoke(() -> WANTestBase.validateRegionSize(orderRegionName, orderEntries));
+    vm5.invoke(() -> WANTestBase.validateRegionSize(orderRegionName, orderEntries));
+    vm4.invoke(() -> WANTestBase.validateRegionSize(shipmentRegionName, shipmentEntries));
+    vm5.invoke(() -> WANTestBase.validateRegionSize(shipmentRegionName, shipmentEntries));
+
+    checkOnlyCompleteTransactionsAreReplicatedAfterSenderStopped(
+        shipmentsPerTransaction);
+
+    // Start sender to validate that queued events do not contain incomplete transactions after
+    // restart
+    startSenderInVMsAsync("ln", vm4, vm5);
+
+    checkOnlyCompleteTransactionsAreReplicatedWithSenderRestarted(shipmentsPerTransaction);
+  }
+
+  @Test
+  public void testPRParallelPropagationWithGroupTransactionEventsDoesNotSendBatchesWithIncompleteTransactionsIfGatewaySenderIsStoppedWhileReceivingTrafficAndLaterStartedReceiverStopped()
+      throws InterruptedException {
+    Integer lnPort = vm0.invoke(() -> WANTestBase.createFirstLocatorWithDSId(1));
+    Integer nyPort = vm1.invoke(() -> WANTestBase.createFirstRemoteLocator(2, lnPort));
+
+    createCacheInVMs(nyPort, vm2);
+    createReceiverCustomerOrderShipmentPR(vm2);
+    createReceiverInVMs(vm2);
+    vm2.invoke(WANTestBase::stopReceivers);
+
+    createCacheInVMs(lnPort, vm4, vm5);
+    createSenderCustomerOrderShipmentPRs(vm4);
+    createSenderCustomerOrderShipmentPRs(vm5);
+
+    int batchSize = 10;
+    vm4.invoke(() -> WANTestBase.createSender("ln", 2, true, 100, 10, false, true, null, false,
+        true));
+    vm5.invoke(() -> WANTestBase.createSender("ln", 2, true, 100, 10, false, true, null, false,
+        true));
+
+    int customers = 4;
+    int transactionsPerCustomer = 100;
+    // Each transaction will contain one order plus the following shipments
+    int shipmentsPerTransaction = batchSize;
+
+    AsyncInvocation<Void> inv1 = asyncExecuteCustomerTransactions(vm4, customers,
+        transactionsPerCustomer, shipmentsPerTransaction);
+
+    // wait for some batches to be redistributed and then stop the sender
+    vm4.invoke(() -> await()
+        .until(() -> WANTestBase.getSenderStats("ln", -1).get(5) > 0));
+
+    stopSenderInVMsAsync("ln", vm4, vm5);
+
+    // Wait for the customer transactions to finish
+    inv1.await();
+    int orderEntries = transactionsPerCustomer * customers;
+    int shipmentEntries = orderEntries * 10;
+    vm4.invoke(() -> WANTestBase.validateRegionSize(orderRegionName, orderEntries));
+    vm5.invoke(() -> WANTestBase.validateRegionSize(orderRegionName, orderEntries));
+    vm4.invoke(() -> WANTestBase.validateRegionSize(shipmentRegionName, shipmentEntries));
+    vm5.invoke(() -> WANTestBase.validateRegionSize(shipmentRegionName, shipmentEntries));
+
+    // Start receiver and sender
+    vm2.invoke(WANTestBase::startReceivers);
+    startSenderInVMsAsync("ln", vm4, vm5);
+
+    checkOnlyCompleteTransactionsAreReplicatedWithSenderRestarted(shipmentsPerTransaction);
+  }
+
+  private AsyncInvocation<Void> asyncExecuteCustomerTransactions(VM vm, int customers,
+      int transactionsPerCustomer, int shipmentsPerTransaction) {
+    final Map<Object, Object> keyValuesInTransactions = new LinkedHashMap<>();
+    for (int custId = 0; custId < customers; custId++) {
+      for (int i = 0; i < transactionsPerCustomer; i++) {
+        CustId custIdObject = new CustId(custId);
+        OrderId orderId = new OrderId(i, custIdObject);
+        keyValuesInTransactions.put(orderId, new Order());
+        for (int j = 0; j < shipmentsPerTransaction; j++) {
+          ShipmentId shipmentId = new ShipmentId(i + j, orderId);
+          keyValuesInTransactions.put(shipmentId, new Shipment());
+        }
+      }
+    }
+    int eventsPerTransaction = 1 + shipmentsPerTransaction;
+    return vm.invokeAsync(
+        () -> WANTestBase.doOrderAndShipmentPutsInsideTransactions(keyValuesInTransactions,
+            eventsPerTransaction));
+  }
+
+  private void checkOnlyCompleteTransactionsAreReplicatedAfterSenderStopped(
+      int shipmentsPerTransaction) {
+    waitForBatchesToBeAppliedInTheReceiver(shipmentsPerTransaction);
+
+    List<Integer> v4List =
+        vm4.invoke(() -> WANTestBase.getSenderStats("ln", -1));
+    List<Integer> v5List =
+        vm5.invoke(() -> WANTestBase.getSenderStats("ln", -1));
+
+    // batches with incomplete transactions must be 0
+    assertEquals(0, (int) v4List.get(13));
+    assertEquals(0, (int) v5List.get(13));
+
+    // Check the entries replicated against the number of batches distributed
+    int batchesDistributed = v4List.get(4) + v5List.get(4);
+    checkOnlyCompleteTransactionsAreReplicated(shipmentsPerTransaction, batchesDistributed);
+  }
+
+  private void checkOnlyCompleteTransactionsAreReplicatedWithSenderRestarted(
+      int shipmentsPerTransaction) {
+    // Wait for sender queues to be drained
+    vm4.invoke(() -> WANTestBase.validateParallelSenderQueueAllBucketsDrained("ln"));
+    vm5.invoke(() -> WANTestBase.validateParallelSenderQueueAllBucketsDrained("ln"));
+
+    List<Integer> v4List =
+        vm4.invoke(() -> WANTestBase.getSenderStats("ln", 0));
+    List<Integer> v5List =
+        vm5.invoke(() -> WANTestBase.getSenderStats("ln", 0));
+    assertEquals(0, v4List.get(0) + v5List.get(0));
+
+    // batches with incomplete transactions must be 0
+    assertEquals(0, (int) v4List.get(13));
+    assertEquals(0, (int) v5List.get(13));
+
+    waitForBatchesToBeAppliedInTheReceiver(shipmentsPerTransaction);
+
+    // Check the entries replicated against the number of batches distributed
+    int batchesDistributed = v4List.get(4) + v5List.get(4);
+    checkOnlyCompleteTransactionsAreReplicated(shipmentsPerTransaction, batchesDistributed);
+  }
+
+  private void checkOnlyCompleteTransactionsAreReplicated(int shipmentsPerTransaction,
+      int batchesDistributed) {
+    // Only complete transactions (1 order + 10 shipments) must be replicated
+    int orderRegionSize = vm2.invoke(() -> getRegionSize(orderRegionName));
+    int shipmentRegionSize = vm2.invoke(() -> getRegionSize(shipmentRegionName));
+    assertEquals(shipmentRegionSize, 10 * orderRegionSize);

Review comment:
       Agree




----------------------------------------------------------------
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] DonalEvans commented on a change in pull request #6052: GEODE-8971: Add grace period when stopping gateway sender with group-…

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



##########
File path: geode-wan/src/distributedTest/java/org/apache/geode/internal/cache/wan/serial/SerialWANStatsDUnitTest.java
##########
@@ -348,7 +349,201 @@ public void testReplicatedSerialPropagationWithBatchRedistWithGroupTransactionEv
   }
 
   @Test
-  public void testReplicatedSerialPropagationWithMultipleDispatchers() throws Exception {
+  public void testReplicatedSerialPropagationWithGroupTransactionEventsDoesNotSendBatchesWithIncompleteTransactionsIfGatewaySenderIsStoppedWhileReceivingTrafficAndLaterStarted()
+      throws InterruptedException {
+    Integer lnPort = vm0.invoke(() -> WANTestBase.createFirstLocatorWithDSId(1));
+    Integer nyPort = vm1.invoke(() -> WANTestBase.createFirstRemoteLocator(2, lnPort));
+
+    String regionName = testName + "_RR";
+
+    createCacheInVMs(nyPort, vm2);
+    createReceiverInVMs(vm2);
+    vm2.invoke(() -> WANTestBase.createReplicatedRegion(regionName, null, isOffHeap()));
+
+    createCacheInVMs(lnPort, vm4, vm5);
+    vm4.invoke(() -> WANTestBase.createReplicatedRegion(regionName, "ln", isOffHeap()));
+    vm5.invoke(() -> WANTestBase.createReplicatedRegion(regionName, "ln", isOffHeap()));
+
+    boolean groupTransactionEvents = true;
+    int batchSize = 10;
+    vm4.invoke(
+        () -> WANTestBase.createSender("ln", 2, false, 100, batchSize, false, true, null, false,
+            groupTransactionEvents));
+    vm5.invoke(
+        () -> WANTestBase.createSender("ln", 2, false, 100, batchSize, false, true, null, false,
+            groupTransactionEvents));
+
+    int eventsPerTransaction = batchSize + 1;
+    // The number of entries must be big enough so that not all entries
+    // are replicated before the sender is stopped and also divisible by eventsPerTransaction
+    int entries = 2200;
+    // Execute some transactions
+    AsyncInvocation<Void> inv1 =
+        asyncExecuteTransactions(regionName, eventsPerTransaction, entries);
+
+    // wait for batches to be distributed and then stop the sender
+    vm4.invoke(() -> await()
+        .until(() -> WANTestBase.getSenderStats("ln", -1).get(4) > 0));
+
+    // These exceptions are ignored here because it could happen that when an event
+    // is to be handled, the sender is stopped. The sender, when stopped, shuts down
+    // the thread pool that would handle the event and this could provoke the exception.
+    addIgnoredException("Exception occurred in CacheListener");
+    addIgnoredException(RejectedExecutionException.class);
+
+    // Stop the sender
+    stopSenderInVMsAsync("ln", vm4, vm5);
+
+    // Wait for transactions to finish
+    inv1.await();
+    vm4.invoke(() -> WANTestBase.validateRegionSize(regionName, entries));
+    vm5.invoke(() -> WANTestBase.validateRegionSize(regionName, entries));
+
+    // Check
+    checkOnlyCompleteTransactionsAreReplicatedAfterSenderStopped(regionName,
+        eventsPerTransaction);
+
+    // Start the sender
+    startSenderInVMsAsync("ln", vm4, vm5);
+
+    // Check
+    checkOnlyCompleteTransactionsAreReplicatedAfterSenderRestarted(regionName,
+        eventsPerTransaction);
+  }
+
+  @Test
+  public void testReplicatedSerialPropagationWithGroupTransactionEventsDoesNotSendBatchesWithIncompleteTransactionsIfGatewaySenderIsStoppedWhileReceivingTrafficAndLaterStartedReceiverStopped()
+      throws InterruptedException {
+    Integer lnPort = vm0.invoke(() -> WANTestBase.createFirstLocatorWithDSId(1));
+    Integer nyPort = vm1.invoke(() -> WANTestBase.createFirstRemoteLocator(2, lnPort));
+
+    String regionName = testName + "_RR";
+
+    createCacheInVMs(nyPort, vm2);
+    createReceiverInVMs(vm2);
+    vm2.invoke(() -> WANTestBase.createReplicatedRegion(regionName, null, isOffHeap()));
+    vm2.invoke(WANTestBase::stopReceivers);
+
+    createCacheInVMs(lnPort, vm4, vm5);
+    vm4.invoke(() -> WANTestBase.createReplicatedRegion(regionName, "ln", isOffHeap()));
+    vm5.invoke(() -> WANTestBase.createReplicatedRegion(regionName, "ln", isOffHeap()));
+
+    boolean groupTransactionEvents = true;
+    int batchSize = 10;
+    vm4.invoke(
+        () -> WANTestBase.createSender("ln", 2, false, 100, batchSize, false, true, null, false,
+            groupTransactionEvents));
+    vm5.invoke(
+        () -> WANTestBase.createSender("ln", 2, false, 100, batchSize, false, true, null, false,
+            groupTransactionEvents));
+
+    int eventsPerTransaction = batchSize + 1;
+    // The number of entries must be big enough so that not all entries
+    // are replicated before the sender is stopped and also divisible by eventsPerTransaction
+    int entries = 2200;
+    // Execute some transactions
+    AsyncInvocation<Void> inv1 =
+        asyncExecuteTransactions(regionName, eventsPerTransaction, entries);
+
+    // wait for batches to be redistributed and then stop the sender
+    vm4.invoke(() -> await()
+        .until(() -> WANTestBase.getSenderStats("ln", -1).get(5) > 0));
+
+    // Stop the sender
+    stopSenderInVMsAsync("ln", vm4, vm5);
+
+    // Wait for transactions to finish
+    inv1.await();
+    vm4.invoke(() -> WANTestBase.validateRegionSize(regionName, entries));
+    vm5.invoke(() -> WANTestBase.validateRegionSize(regionName, entries));
+
+    // Start the receiver and the sender
+    vm2.invoke(WANTestBase::startReceivers);
+    startSenderInVMsAsync("ln", vm4, vm5);
+
+    // Check
+    checkOnlyCompleteTransactionsAreReplicatedAfterSenderRestarted(regionName,
+        eventsPerTransaction);
+  }
+
+  private void checkOnlyCompleteTransactionsAreReplicatedAfterSenderStopped(String regionName,
+      int eventsPerTransaction) {
+    waitForBatchesToBeAppliedInTheReceiver(regionName, eventsPerTransaction);
+
+    List<Integer> v4List =
+        vm4.invoke(() -> WANTestBase.getSenderStats("ln", -1));
+    List<Integer> v5List =
+        vm5.invoke(() -> WANTestBase.getSenderStats("ln", -1));
+
+    // batches with incomplete transactions must be 0
+    assertEquals(0, (int) v4List.get(13));
+    assertEquals(0, (int) v5List.get(13));
+
+    int batchesDistributed = v4List.get(4) + v5List.get(4);
+    checkOnlyCompleteTransactionsAreReplicated(regionName, eventsPerTransaction,
+        batchesDistributed);
+  }
+
+  private void checkOnlyCompleteTransactionsAreReplicatedAfterSenderRestarted(String regionName,
+      int eventsPerTransaction) {
+    // Wait for sender queues to be empty
+    List<Integer> v4List =
+        vm4.invoke(() -> WANTestBase.getSenderStats("ln", 0));
+    List<Integer> v5List =
+        vm5.invoke(() -> WANTestBase.getSenderStats("ln", 0));
+
+    assertEquals(0, v4List.get(0) + v5List.get(0));
+
+    // batches with incomplete transactions must be 0
+    assertEquals(0, (int) v4List.get(13));
+    assertEquals(0, (int) v5List.get(13));
+
+    waitForBatchesToBeAppliedInTheReceiver(regionName, eventsPerTransaction);
+
+    int batchesDistributed = v4List.get(4) + v5List.get(4);
+    checkOnlyCompleteTransactionsAreReplicated(regionName, eventsPerTransaction,
+        batchesDistributed);
+  }
+
+  private void checkOnlyCompleteTransactionsAreReplicated(String regionName,
+      int eventsPerTransaction, int batchesDistributed) {
+    int regionSize = vm2.invoke(() -> getRegionSize(regionName));
+
+    // The number of entries must be divisible by the number of events per transaction
+    assertEquals(0, regionSize % eventsPerTransaction);
+
+    // Check the entries replicated against the number of batches distributed
+    vm2.invoke(() -> WANTestBase.validateRegionSize(regionName,
+        batchesDistributed * eventsPerTransaction));
+  }
+
+  private AsyncInvocation<Void> asyncExecuteTransactions(String regionName,
+      int eventsPerTransaction, int entries) {
+    final Map<Object, Object> keyValues = new LinkedHashMap<>();
+    for (int i = 0; i < entries; i++) {
+      keyValues.put(i, i + "_Value");
+    }
+
+    return vm4.invokeAsync(
+        () -> WANTestBase.doPutsInsideTransactions(regionName, keyValues,
+            eventsPerTransaction));
+  }
+
+  private void waitForBatchesToBeAppliedInTheReceiver(String regionName, int eventsPerTransaction) {
+    int batchesSentTotal = vm4.invoke(() -> WANTestBase.getSenderStats("ln", -1)).get(4) +
+        vm5.invoke(() -> WANTestBase.getSenderStats("ln", -1)).get(4);
+
+    // Wait for all batches to be received by the sender

Review comment:
       I think that this should probably be "received by the receiver"

##########
File path: geode-wan/src/test/java/org/apache/geode/internal/cache/wan/parallel/ParallelGatewaySenderImplTest.java
##########
@@ -88,5 +97,36 @@ public void testStartWithCleanQueue() {
     assertTrue(((ConcurrentParallelGatewaySenderQueue) queue).getCleanQueues());
   }
 
+  @Test
+  public void whenStoppedTwiceCloseInTimeWithGroupTransactionEventsPreStopWaitsTwice() {
+    gatewaysender.start();
+
+    long start = System.currentTimeMillis();
+
+    Thread t1 = new Thread(this::stopGatewaySenderAndCheckTime);
+    Thread t2 = new Thread(this::stopGatewaySenderAndCheckTime);
+    t1.start();
+    t2.start();
+    try {
+      t1.join();
+      t2.join();
+    } catch (InterruptedException e) {
+      e.printStackTrace();
+    }
 
+    long finish = System.currentTimeMillis();
+    long timeElapsed = finish - start;
+    // Each call to preStop waits for 1 second but they are not serialized

Review comment:
       I think this comment would be clearer if it read "Each call to preStop waits for 1 second but these waits execute in parallel"

##########
File path: geode-wan/src/test/java/org/apache/geode/internal/cache/wan/serial/SerialGatewaySenderImplTest.java
##########
@@ -102,6 +102,8 @@ private SerialGatewaySenderImpl createSerialGatewaySenderImplSpy() {
 
     doReturn(null).when(spySerialGatewaySender).getQueues();
 
+    doReturn(true).when(spySerialGatewaySender).mustGroupTransactionEvents();

Review comment:
       The value returned here should only be `true` in tests that specifically want to test the behaviour of group transaction events, since the default value is `false`. Maybe the desired value could be passed as an argument to this `createSerialGatewaySenderImplSpy()` method

##########
File path: geode-wan/src/test/java/org/apache/geode/internal/cache/wan/parallel/ParallelGatewaySenderImplTest.java
##########
@@ -51,11 +54,17 @@ public void setUp() {
     attrs = new GatewaySenderAttributes();
     attrs.isParallel = true;
     attrs.id = "sender";
+    attrs.groupTransactionEvents = true;

Review comment:
       I think it would be best not to set this value in the `setUp()` method for this class, as it's not a default value, and could lead to unexpected behaviour if someone else tries to add tests to this class in the future. Only the test cases that specifically rely on `groupTransactionEvents` being `true` should have it set.

##########
File path: geode-wan/src/test/java/org/apache/geode/internal/cache/wan/serial/SerialGatewaySenderImplTest.java
##########
@@ -132,4 +134,37 @@ public void whenStoppedShouldResetTheEventProcessor() {
     assertThat(serialGatewaySender.getEventProcessor()).isNull();
   }
 
+  @Test
+  public void whenStoppedTwiceCloseInTimeWithGroupTransactionEventsPreStopWaitsTwice() {
+    serialGatewaySender = createSerialGatewaySenderImplSpy();
+
+    long start = System.currentTimeMillis();
+
+    Thread t1 = new Thread(this::stopGatewaySenderAndCheckTime);
+    Thread t2 = new Thread(this::stopGatewaySenderAndCheckTime);
+    t1.start();
+    t2.start();
+    try {
+      t1.join();
+      t2.join();
+    } catch (InterruptedException e) {
+      e.printStackTrace();
+    }
+
+    long finish = System.currentTimeMillis();
+    long timeElapsed = finish - start;
+
+    // Each call to preStop waits for 1 second but they are not serialized

Review comment:
       I think this comment would be clearer if it read "Each call to preStop waits for 1 second but these waits execute in parallel"




----------------------------------------------------------------
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] albertogpz commented on a change in pull request #6052: GEODE-8971: Add grace period when stopping gateway sender with group-…

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



##########
File path: geode-core/src/main/java/org/apache/geode/internal/cache/BucketRegionQueue.java
##########
@@ -453,21 +454,22 @@ public Object peek() {
    * If a matching object also fulfills the endPredicate then the method
    * stops looking for more matching objects.
    */
-  public List<Object> getElementsMatching(Predicate matchingPredicate, Predicate endPredicate) {
+  public List<Object> getElementsMatching(Predicate<InternalGatewayQueueEvent> matchingPredicate,
+      Predicate<InternalGatewayQueueEvent> endPredicate) {
     getInitializationLock().readLock().lock();
     try {
       if (this.getPartitionedRegion().isDestroyed()) {
         throw new BucketRegionQueueUnavailableException();
       }
-      List<Object> elementsMatching = new ArrayList<Object>();
+      List<Object> elementsMatching = new ArrayList();
       Iterator<Object> it = this.eventSeqNumDeque.iterator();
       while (it.hasNext()) {
         Object key = it.next();
         Object object = optimalGet(key);
-        if (matchingPredicate.test(object)) {
+        if (matchingPredicate.test((InternalGatewayQueueEvent) object)) {

Review comment:
       ok

##########
File path: geode-core/src/main/java/org/apache/geode/internal/cache/BucketRegionQueue.java
##########
@@ -478,6 +480,24 @@ public Object peek() {
     }
   }
 
+  public boolean isThereEventsMatching(Predicate<InternalGatewayQueueEvent> matchingPredicate) {

Review comment:
       ok




----------------------------------------------------------------
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] albertogpz commented on a change in pull request #6052: GEODE-8971: Add grace period when stopping gateway sender with group-…

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



##########
File path: geode-core/src/main/java/org/apache/geode/internal/cache/wan/parallel/ParallelGatewaySenderQueue.java
##########
@@ -702,44 +704,15 @@ public boolean put(Object object) throws InterruptedException, CacheException {
     boolean isDREvent = isDREvent(sender.getCache(), value);
 
     String regionPath = value.getRegionPath();

Review comment:
       Good catch.




----------------------------------------------------------------
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] albertogpz commented on a change in pull request #6052: GEODE-8971: Add grace period when stopping gateway sender with group-…

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



##########
File path: geode-core/src/main/java/org/apache/geode/internal/cache/BucketRegionQueue.java
##########
@@ -453,21 +454,22 @@ public Object peek() {
    * If a matching object also fulfills the endPredicate then the method
    * stops looking for more matching objects.
    */
-  public List<Object> getElementsMatching(Predicate matchingPredicate, Predicate endPredicate) {
+  public List<Object> getElementsMatching(Predicate<InternalGatewayQueueEvent> matchingPredicate,
+      Predicate<InternalGatewayQueueEvent> endPredicate) {
     getInitializationLock().readLock().lock();
     try {
       if (this.getPartitionedRegion().isDestroyed()) {
         throw new BucketRegionQueueUnavailableException();
       }
-      List<Object> elementsMatching = new ArrayList<Object>();
+      List<Object> elementsMatching = new ArrayList();

Review comment:
       ok




----------------------------------------------------------------
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] albertogpz commented on a change in pull request #6052: GEODE-8971: Add grace period when stopping gateway sender with group-…

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



##########
File path: geode-core/src/main/java/org/apache/geode/internal/cache/wan/parallel/ParallelGatewaySenderQueue.java
##########
@@ -870,6 +843,49 @@ public boolean put(Object object) throws InterruptedException, CacheException {
     return putDone;
   }
 
+  private String getRegionPathForEventAndType(GatewaySenderEventImpl event, boolean isDREvent,
+      String regionPath) {
+    boolean isDebugEnabled = logger.isDebugEnabled();
+    if (!isDREvent) {
+      Region region = sender.getCache().getRegion(regionPath, true);

Review comment:
       ok




----------------------------------------------------------------
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] albertogpz commented on pull request #6052: GEODE-8971: Add grace period when stopping gateway sender with group-…

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


   > I've requested a few changes in-line and a couple general comments that aren't requests for changes.
   > 
   > A good rule is to always avoid raw types (`Predicate condition`) by providing a type (`Predicate<Object> condition`).
   > 
   > I really want to see us moving away from passing around impl classes like `GatewaySenderEventImpl` if an interface like `GatewayQueueEvent` defines the methods you need to call. If you need to call an impl method that's NOT on the interface, then use the impl for now. We'll probably eventually create a new internal interface like `InternalGatewayQueueEvent` if we find that we need to pass around the impl to call non-interface-defined methods. This will help us avoid concrete dependencies which then helps produce higher quality code and unit tests.
   > 
   > Looks like the existing `throw` statements for `BucketRegionQueueUnavailableException` all have no message. It's always better to provide a message, but it's ok as is since it's at least consistent with the other lines that throw it.
   
   
   
   > I've requested a few changes in-line and a couple general comments that aren't requests for changes.
   > 
   > A good rule is to always avoid raw types (`Predicate condition`) by providing a type (`Predicate<Object> condition`).
   > 
   > I really want to see us moving away from passing around impl classes like `GatewaySenderEventImpl` if an interface like `GatewayQueueEvent` defines the methods you need to call. If you need to call an impl method that's NOT on the interface, then use the impl for now. We'll probably eventually create a new internal interface like `InternalGatewayQueueEvent` if we find that we need to pass around the impl to call non-interface-defined methods. This will help us avoid concrete dependencies which then helps produce higher quality code and unit tests.
   > 
   > Looks like the existing `throw` statements for `BucketRegionQueueUnavailableException` all have no message. It's always better to provide a message, but it's ok as is since it's at least consistent with the other lines that throw it.
   
   I have added the isLastEventInTransaction() and getTransactionId() to the GatewayQueueEvent interface. Do you see any drawback to it?
   


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